From be37730906230fcc4255a9802231cc4d3fe11679 Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 27 Jan 2023 16:23:27 +0300 Subject: [PATCH 001/515] upd --- src/Common/DateLUT.cpp | 10 +++++++++- src/Common/DateLUT.h | 24 ++++++++++++++++++++++-- src/Core/Settings.h | 1 + 3 files changed, 32 insertions(+), 3 deletions(-) diff --git a/src/Common/DateLUT.cpp b/src/Common/DateLUT.cpp index ae6127670e5..fd10b799b66 100644 --- a/src/Common/DateLUT.cpp +++ b/src/Common/DateLUT.cpp @@ -7,6 +7,8 @@ #include #include +#include +#include namespace @@ -147,7 +149,7 @@ DateLUT::DateLUT() } -const DateLUTImpl & DateLUT::getImplementation(const std::string & time_zone) const +const ALWAYS_INLINE DateLUTImpl & DateLUT::getImplementation(const std::string & time_zone) const { std::lock_guard lock(mutex); @@ -163,3 +165,9 @@ DateLUT & DateLUT::getInstance() static DateLUT ret; return ret; } + +std::string DateLUT::extractTimezoneFromContext(const DB::ContextPtr query_context) +{ + std::string ret = query_context->getSettingsRef().implicit_timezone.value; + return ret; +} diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index b7ba37c2bec..bd7969bffa6 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -5,6 +5,10 @@ #include #include +// +//#include "Interpreters/Context_fwd.h" +//#include "Interpreters/Context.h" +#include "Common/CurrentThread.h" #include #include @@ -20,16 +24,30 @@ public: static ALWAYS_INLINE const DateLUTImpl & instance() // -V1071 { const auto & date_lut = getInstance(); + + if (DB::CurrentThread::isInitialized()) + { + const auto query_context = DB::CurrentThread::get().getQueryContext(); + + if (query_context) + { + auto implicit_timezone = extractTimezoneFromContext(query_context); + + if (!implicit_timezone.empty()) + return instance(implicit_timezone); + } + } + return *date_lut.default_impl.load(std::memory_order_acquire); } /// Return singleton DateLUTImpl instance for a given time zone. static ALWAYS_INLINE const DateLUTImpl & instance(const std::string & time_zone) { - const auto & date_lut = getInstance(); if (time_zone.empty()) - return *date_lut.default_impl.load(std::memory_order_acquire); + return instance(); + const auto & date_lut = getInstance(); return date_lut.getImplementation(time_zone); } static void setDefaultTimezone(const std::string & time_zone) @@ -45,6 +63,8 @@ protected: private: static DateLUT & getInstance(); + static std::string extractTimezoneFromContext(const DB::ContextPtr query_context); + const DateLUTImpl & getImplementation(const std::string & time_zone) const; using DateLUTImplPtr = std::unique_ptr; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1948a6da012..2da5791ff81 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -688,6 +688,7 @@ class IColumn; M(Float, insert_keeper_fault_injection_probability, 0.0f, "Approximate probability of failure for a keeper request during insert. Valid value is in interval [0.0f, 1.0f]", 0) \ M(UInt64, insert_keeper_fault_injection_seed, 0, "0 - random seed, otherwise the setting value", 0) \ M(Bool, force_aggregation_in_order, false, "Force use of aggregation in order on remote nodes during distributed aggregation. PLEASE, NEVER CHANGE THIS SETTING VALUE MANUALLY!", IMPORTANT) \ + M(String, implicit_timezone, "", "Use specified timezone for interpreting Date and DateTime instead of server's timezone.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. From 537721e297e4ba94f30d5bd76ca0a7b01a080a5c Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 27 Jan 2023 23:40:43 +0300 Subject: [PATCH 002/515] re-add tests --- .../0_stateless/02538_implicit_timezone.reference | 3 +++ tests/queries/0_stateless/02538_implicit_timezone.sql | 9 +++++++++ 2 files changed, 12 insertions(+) create mode 100644 tests/queries/0_stateless/02538_implicit_timezone.reference create mode 100644 tests/queries/0_stateless/02538_implicit_timezone.sql diff --git a/tests/queries/0_stateless/02538_implicit_timezone.reference b/tests/queries/0_stateless/02538_implicit_timezone.reference new file mode 100644 index 00000000000..8ed8024f652 --- /dev/null +++ b/tests/queries/0_stateless/02538_implicit_timezone.reference @@ -0,0 +1,3 @@ +1999-12-12 18:23:23.123 +1999-12-12 23:23:23.123 +1999-12-13 04:23:23.123 diff --git a/tests/queries/0_stateless/02538_implicit_timezone.sql b/tests/queries/0_stateless/02538_implicit_timezone.sql new file mode 100644 index 00000000000..663b218d235 --- /dev/null +++ b/tests/queries/0_stateless/02538_implicit_timezone.sql @@ -0,0 +1,9 @@ +SET implicit_timezone = 'Asia/Novosibirsk'; + +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich'); + +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS implicit_timezone = 'Europe/Zurich'; + +SET implicit_timezone = 'Europe/Zurich'; + +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); \ No newline at end of file From f29700bd2fe1fc79cb6fd5bf2b31129119e6978c Mon Sep 17 00:00:00 2001 From: LiuYangkuan Date: Wed, 1 Feb 2023 18:50:43 +0800 Subject: [PATCH 003/515] use IDisk to do IO in Keeper's snapshots and logs --- programs/keeper-converter/KeeperConverter.cpp | 3 +- src/Coordination/Changelog.cpp | 151 +++++++++++------- src/Coordination/Changelog.h | 6 +- src/Coordination/KeeperLogStore.cpp | 11 +- src/Coordination/KeeperLogStore.h | 3 + src/Coordination/KeeperServer.cpp | 23 ++- src/Coordination/KeeperSnapshotManager.cpp | 91 ++++++----- src/Coordination/KeeperSnapshotManager.h | 24 ++- src/Coordination/KeeperStateMachine.cpp | 4 +- src/Coordination/KeeperStateMachine.h | 2 +- src/Coordination/KeeperStateManager.cpp | 71 ++++---- src/Coordination/KeeperStateManager.h | 8 +- src/IO/ZstdDeflatingAppendableWriteBuffer.cpp | 2 +- src/IO/ZstdDeflatingAppendableWriteBuffer.h | 4 +- 14 files changed, 244 insertions(+), 159 deletions(-) diff --git a/programs/keeper-converter/KeeperConverter.cpp b/programs/keeper-converter/KeeperConverter.cpp index 7d25c1d5017..c81e61685fd 100644 --- a/programs/keeper-converter/KeeperConverter.cpp +++ b/programs/keeper-converter/KeeperConverter.cpp @@ -9,6 +9,7 @@ #include #include #include +#include int mainEntryClickHouseKeeperConverter(int argc, char ** argv) @@ -51,7 +52,7 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv) DB::SnapshotMetadataPtr snapshot_meta = std::make_shared(storage.getZXID(), 1, std::make_shared()); DB::KeeperStorageSnapshot snapshot(&storage, snapshot_meta); - DB::KeeperSnapshotManager manager(options["output-dir"].as(), 1, keeper_context); + DB::KeeperSnapshotManager manager(std::make_shared("Keeper-snapshots", options["output-dir"].as(), 0), 1, keeper_context); auto snp = manager.serializeSnapshotToBuffer(snapshot); auto path = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID()); std::cout << "Snapshot serialized to path:" << path << std::endl; diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 142a7209b42..2d3fb9cba47 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -31,12 +31,9 @@ namespace constexpr auto DEFAULT_PREFIX = "changelog"; -std::string formatChangelogPath( - const std::string & prefix, const std::string & name_prefix, uint64_t from_index, uint64_t to_index, const std::string & extension) +inline std::string formatChangelogPath(const std::string & name_prefix, uint64_t from_index, uint64_t to_index, const std::string & extension) { - std::filesystem::path path(prefix); - path /= std::filesystem::path(fmt::format("{}_{}_{}.{}", name_prefix, from_index, to_index, extension)); - return path; + return fmt::format("{}_{}_{}.{}", name_prefix, from_index, to_index, extension); } ChangelogFileDescriptionPtr getChangelogFileDescription(const std::filesystem::path & path) @@ -88,11 +85,11 @@ class ChangelogWriter public: ChangelogWriter( std::map & existing_changelogs_, - const std::filesystem::path & changelogs_dir_, + DiskPtr disk_, LogFileSettings log_file_settings_) : existing_changelogs(existing_changelogs_) , log_file_settings(log_file_settings_) - , changelogs_dir(changelogs_dir_) + , disk(disk_) , log(&Poco::Logger::get("Changelog")) { } @@ -109,7 +106,7 @@ public: file_description->expectedEntriesCountInLog()); // we have a file we need to finalize first - if (tryGetFileBuffer() && prealloc_done) + if (tryGetFileBaseBuffer() && prealloc_done) { finalizeCurrentFile(); @@ -121,18 +118,16 @@ public: && *last_index_written != current_file_description->to_log_index) { auto new_path = formatChangelogPath( - changelogs_dir, current_file_description->prefix, current_file_description->from_log_index, *last_index_written, current_file_description->extension); - std::filesystem::rename(current_file_description->path, new_path); + disk->moveFile(current_file_description->path, new_path); current_file_description->path = std::move(new_path); } } - file_buf = std::make_unique( - file_description->path, DBMS_DEFAULT_BUFFER_SIZE, mode == WriteMode::Rewrite ? -1 : (O_APPEND | O_CREAT | O_WRONLY)); + file_buf = disk->writeFile(file_description->path, DBMS_DEFAULT_BUFFER_SIZE, mode); last_index_written.reset(); current_file_description = std::move(file_description); @@ -148,12 +143,15 @@ public: } } - bool isFileSet() const { return tryGetFileBuffer() != nullptr; } - + /// There is bug when compressed_buffer has value, file_buf's ownership transfer to compressed_buffer + bool isFileSet() const + { + return compressed_buffer.get() != nullptr || file_buf.get() != nullptr; + } bool appendRecord(ChangelogRecord && record) { - const auto * file_buffer = tryGetFileBuffer(); + const auto * file_buffer = tryGetFileBaseBuffer(); assert(file_buffer && current_file_description); assert(record.header.index - getStartIndex() <= current_file_description->expectedEntriesCountInLog()); @@ -207,7 +205,7 @@ public: void flush() { - auto * file_buffer = tryGetFileBuffer(); + auto * file_buffer = tryGetFileBaseBuffer(); /// Fsync file system if needed if (file_buffer && log_file_settings.force_sync) file_buffer->sync(); @@ -232,7 +230,6 @@ public: new_description->extension += "." + toContentEncodingName(CompressionMethod::Zstd); new_description->path = formatChangelogPath( - changelogs_dir, new_description->prefix, new_start_log_index, new_start_log_index + log_file_settings.rotate_interval - 1, @@ -254,14 +251,13 @@ private: void finalizeCurrentFile() { - const auto * file_buffer = tryGetFileBuffer(); - assert(file_buffer && prealloc_done); + assert(prealloc_done); assert(current_file_description); // compact can delete the file and we don't need to do anything if (current_file_description->deleted) { - LOG_WARNING(log, "Log {} is already deleted", file_buffer->getFileName()); + LOG_WARNING(log, "Log {} is already deleted", current_file_description->path); return; } @@ -270,7 +266,8 @@ private: flush(); - if (log_file_settings.max_size != 0) + const auto * file_buffer = tryGetFileBuffer(); + if (log_file_settings.max_size != 0 && file_buffer) ftruncate(file_buffer->getFD(), initial_file_size + file_buffer->count()); if (log_file_settings.compress_logs) @@ -281,6 +278,8 @@ private: WriteBuffer & getBuffer() { + /// TODO: unify compressed_buffer and file_buf, + /// compressed_buffer can use its NestedBuffer directly if compress_logs=false if (compressed_buffer) return *compressed_buffer; @@ -310,10 +309,15 @@ private: if (compressed_buffer) return dynamic_cast(compressed_buffer->getNestedBuffer()); - if (file_buf) - return file_buf.get(); + return dynamic_cast(file_buf.get()); + } - return nullptr; + WriteBufferFromFileBase * tryGetFileBaseBuffer() + { + if (compressed_buffer) + return dynamic_cast(compressed_buffer->getNestedBuffer()); + + return file_buf.get(); } void tryPreallocateForFile() @@ -325,13 +329,22 @@ private: return; } - const auto & file_buffer = getFileBuffer(); + const auto * file_buffer = tryGetFileBuffer(); + + if (!file_buffer) + { + initial_file_size = 0; + prealloc_done = true; + LOG_WARNING(log, "Could not preallocate space on disk {} using fallocate", disk->getName()); + return; + } + #ifdef OS_LINUX { int res = -1; do { - res = fallocate(file_buffer.getFD(), FALLOC_FL_KEEP_SIZE, 0, log_file_settings.max_size + log_file_settings.overallocate_size); + res = fallocate(file_buffer->getFD(), FALLOC_FL_KEEP_SIZE, 0, log_file_settings.max_size + log_file_settings.overallocate_size); } while (res < 0 && errno == EINTR); if (res != 0) @@ -346,7 +359,7 @@ private: } } #endif - initial_file_size = getSizeFromFileDescriptor(file_buffer.getFD()); + initial_file_size = getSizeFromFileDescriptor(file_buffer->getFD()); prealloc_done = true; } @@ -354,7 +367,7 @@ private: std::map & existing_changelogs; ChangelogFileDescriptionPtr current_file_description{nullptr}; - std::unique_ptr file_buf; + std::unique_ptr file_buf; std::optional last_index_written; size_t initial_file_size{0}; @@ -364,7 +377,7 @@ private: LogFileSettings log_file_settings; - const std::filesystem::path changelogs_dir; + DiskPtr disk; Poco::Logger * const log; }; @@ -394,10 +407,12 @@ struct ChangelogReadResult class ChangelogReader { public: - explicit ChangelogReader(const std::string & filepath_) : filepath(filepath_) + explicit ChangelogReader(DiskPtr disk_, const std::string & filepath_) + : disk(disk_) + , filepath(filepath_) { auto compression_method = chooseCompressionMethod(filepath, ""); - auto read_buffer_from_file = std::make_unique(filepath); + auto read_buffer_from_file = disk->readFile(filepath); read_buf = wrapReadBufferWithCompressionMethod(std::move(read_buffer_from_file), compression_method); } @@ -493,37 +508,35 @@ public: } private: + DiskPtr disk; std::string filepath; std::unique_ptr read_buf; }; Changelog::Changelog( - const std::string & changelogs_dir_, + DiskPtr disk_, Poco::Logger * log_, LogFileSettings log_file_settings) - : changelogs_dir(changelogs_dir_) - , changelogs_detached_dir(changelogs_dir / "detached") + : disk(disk_) + , changelogs_detached_dir("detached") , rotate_interval(log_file_settings.rotate_interval) , log(log_) , write_operations(std::numeric_limits::max()) , append_completion_queue(std::numeric_limits::max()) { /// Load all files in changelog directory - namespace fs = std::filesystem; - if (!fs::exists(changelogs_dir)) - fs::create_directories(changelogs_dir); - for (const auto & p : fs::directory_iterator(changelogs_dir)) + for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) { - if (p == changelogs_detached_dir) + if (it->name() == changelogs_detached_dir) continue; - auto file_description = getChangelogFileDescription(p.path()); + auto file_description = getChangelogFileDescription(it->path()); existing_changelogs[file_description->from_log_index] = std::move(file_description); } if (existing_changelogs.empty()) - LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", changelogs_dir.generic_string()); + LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", disk->getPath()); clean_log_thread = ThreadFromGlobalPool([this] { cleanLogThread(); }); @@ -532,7 +545,7 @@ Changelog::Changelog( append_completion_thread = ThreadFromGlobalPool([this] { appendCompletionThread(); }); current_writer = std::make_unique( - existing_changelogs, changelogs_dir, log_file_settings); + existing_changelogs, disk, log_file_settings); } void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uint64_t logs_to_keep) @@ -604,7 +617,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin break; } - ChangelogReader reader(changelog_description.path); + ChangelogReader reader(disk, changelog_description.path); last_log_read_result = reader.readChangelog(logs, start_to_read_from, log); last_log_read_result->log_start_index = changelog_description.from_log_index; @@ -671,7 +684,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin if (last_log_read_result->last_read_index == 0 || last_log_read_result->error) /// If it's broken log then remove it { LOG_INFO(log, "Removing chagelog {} because it's empty or read finished with error", description->path); - std::filesystem::remove(description->path); + disk->removeFile(description->path); existing_changelogs.erase(last_log_read_result->log_start_index); std::erase_if(logs, [last_log_read_result](const auto & item) { return item.first >= last_log_read_result->log_start_index; }); } @@ -691,6 +704,9 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin void Changelog::initWriter(ChangelogFileDescriptionPtr description) { + if (description->expectedEntriesCountInLog() != rotate_interval) + LOG_TRACE(log, "Looks like rotate_logs_interval was changed, current {}, expected entries in last log {}", rotate_interval, description->expectedEntriesCountInLog()); + LOG_TRACE(log, "Continue to write into {}", description->path); current_writer->setFile(std::move(description), WriteMode::Append); } @@ -715,20 +731,20 @@ std::string getCurrentTimestampFolder() void Changelog::removeExistingLogs(ChangelogIter begin, ChangelogIter end) { - const auto timestamp_folder = changelogs_detached_dir / getCurrentTimestampFolder(); + const auto timestamp_folder = (fs::path(changelogs_detached_dir) / getCurrentTimestampFolder()).generic_string(); for (auto itr = begin; itr != end;) { - if (!std::filesystem::exists(timestamp_folder)) + if (!disk->exists(timestamp_folder)) { - LOG_WARNING(log, "Moving broken logs to {}", timestamp_folder.generic_string()); - std::filesystem::create_directories(timestamp_folder); + LOG_WARNING(log, "Moving broken logs to {}", timestamp_folder); + disk->createDirectories(timestamp_folder); } LOG_WARNING(log, "Removing changelog {}", itr->second->path); const std::filesystem::path & path = itr->second->path; const auto new_path = timestamp_folder / path.filename(); - std::filesystem::rename(path, new_path); + disk->moveFile(path.generic_string(), new_path.generic_string()); itr = existing_changelogs.erase(itr); } } @@ -885,7 +901,7 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry) auto to_remove_itr = existing_changelogs.upper_bound(index); for (auto itr = to_remove_itr; itr != existing_changelogs.end();) { - std::filesystem::remove(itr->second->path); + disk->removeFile(itr->second->path); itr = existing_changelogs.erase(itr); } } @@ -937,12 +953,19 @@ void Changelog::compact(uint64_t up_to_log_index) /// If failed to push to queue for background removing, then we will remove it now if (!log_files_to_delete_queue.tryPush(changelog_description.path, 1)) { - std::error_code ec; - std::filesystem::remove(changelog_description.path, ec); - if (ec) - LOG_WARNING(log, "Failed to remove changelog {} in compaction, error message: {}", changelog_description.path, ec.message()); - else - LOG_INFO(log, "Removed changelog {} because of compaction", changelog_description.path); + try + { + disk->removeFile(itr->second->path); + LOG_INFO(log, "Removed changelog {} because of compaction.", itr->second->path); + } + catch (Exception & e) + { + LOG_WARNING(log, "Failed to remove changelog {} in compaction, error message: {}", itr->second->path, e.message()); + } + catch (...) + { + tryLogCurrentException(log); + } } changelog_description.deleted = true; @@ -1135,11 +1158,19 @@ void Changelog::cleanLogThread() std::string path; while (log_files_to_delete_queue.pop(path)) { - std::error_code ec; - if (std::filesystem::remove(path, ec)) + try + { + disk->removeFile(path); LOG_INFO(log, "Removed changelog {} because of compaction.", path); - else - LOG_WARNING(log, "Failed to remove changelog {} in compaction, error message: {}", path, ec.message()); + } + catch (Exception & e) + { + LOG_WARNING(log, "Failed to remove changelog {} in compaction, error message: {}", path, e.message()); + } + catch (...) + { + tryLogCurrentException(log); + } } } diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 288f71bb915..052eba45858 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -86,7 +86,7 @@ class Changelog { public: Changelog( - const std::string & changelogs_dir_, + DiskPtr disk_, Poco::Logger * log_, LogFileSettings log_file_settings); @@ -168,8 +168,8 @@ private: /// Clean useless log files in a background thread void cleanLogThread(); - const std::filesystem::path changelogs_dir; - const std::filesystem::path changelogs_detached_dir; + DiskPtr disk; + const String changelogs_detached_dir; const uint64_t rotate_interval; Poco::Logger * log; diff --git a/src/Coordination/KeeperLogStore.cpp b/src/Coordination/KeeperLogStore.cpp index d1bd2f9db18..fcf9400a1d9 100644 --- a/src/Coordination/KeeperLogStore.cpp +++ b/src/Coordination/KeeperLogStore.cpp @@ -1,13 +1,13 @@ #include #include +#include namespace DB { -KeeperLogStore::KeeperLogStore( - const std::string & changelogs_path, LogFileSettings log_file_settings) +KeeperLogStore::KeeperLogStore(DiskPtr disk_, LogFileSettings log_file_settings) : log(&Poco::Logger::get("KeeperLogStore")) - , changelog(changelogs_path, log, log_file_settings) + , changelog(disk_, log, log_file_settings) { if (log_file_settings.force_sync) LOG_INFO(log, "force_sync enabled"); @@ -15,6 +15,11 @@ KeeperLogStore::KeeperLogStore( LOG_INFO(log, "force_sync disabled"); } +KeeperLogStore::KeeperLogStore(const std::string & changelogs_path, LogFileSettings log_file_settings) + : KeeperLogStore(std::make_shared("Keeper-logs", changelogs_path, 0), log_file_settings) +{ +} + uint64_t KeeperLogStore::start_index() const { std::lock_guard lock(changelog_lock); diff --git a/src/Coordination/KeeperLogStore.h b/src/Coordination/KeeperLogStore.h index 108241e024e..c2428413753 100644 --- a/src/Coordination/KeeperLogStore.h +++ b/src/Coordination/KeeperLogStore.h @@ -14,6 +14,9 @@ namespace DB class KeeperLogStore : public nuraft::log_store { public: + KeeperLogStore(DiskPtr disk_, LogFileSettings log_file_settings); + + /// For gtest KeeperLogStore(const std::string & changelogs_path, LogFileSettings log_file_settings); /// Read log storage from filesystem starting from last_commited_log_index diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index b823fbc697c..4de194fe9ae 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -25,6 +25,7 @@ #include #include #include +#include namespace DB { @@ -121,20 +122,36 @@ KeeperServer::KeeperServer( keeper_context->digest_enabled = config.getBool("keeper_server.digest_enabled", false); keeper_context->ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); + if (!fs::exists(configuration_and_settings_->snapshot_storage_path)) + fs::create_directories(configuration_and_settings_->snapshot_storage_path); + auto snapshots_disk = std::make_shared("Keeper-snapshots", configuration_and_settings_->snapshot_storage_path, 0); + state_machine = nuraft::cs_new( responses_queue_, snapshots_queue_, - configuration_and_settings_->snapshot_storage_path, + snapshots_disk, coordination_settings, keeper_context, config.getBool("keeper_server.upload_snapshot_on_exit", true) ? &snapshot_manager_s3 : nullptr, checkAndGetSuperdigest(configuration_and_settings_->super_digest)); + auto state_path = fs::path(configuration_and_settings_->state_file_path).parent_path().generic_string(); + auto state_file_name = fs::path(configuration_and_settings_->state_file_path).filename().generic_string(); + + if (!fs::exists(state_path)) + fs::create_directories(state_path); + auto state_disk = std::make_shared("Keeper-state", state_path, 0); + + if (!fs::exists(configuration_and_settings_->log_storage_path)) + fs::create_directories(configuration_and_settings_->log_storage_path); + auto logs_disk = std::make_shared("Keeper-logs", configuration_and_settings_->log_storage_path, 0); + state_manager = nuraft::cs_new( server_id, "keeper_server", - configuration_and_settings_->log_storage_path, - configuration_and_settings_->state_file_path, + logs_disk, + state_disk, + state_file_name, config, coordination_settings); } diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index a3f7dbc2c6a..3d308fbabda 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB @@ -507,39 +508,45 @@ KeeperSnapshotManager::KeeperSnapshotManager( bool compress_snapshots_zstd_, const std::string & superdigest_, size_t storage_tick_time_) - : snapshots_path(snapshots_path_) + : KeeperSnapshotManager( + std::make_shared("Keeper-snapshots", snapshots_path_, 0), + snapshots_to_keep_, + keeper_context_, + compress_snapshots_zstd_, + superdigest_, + storage_tick_time_) +{ +} + +KeeperSnapshotManager::KeeperSnapshotManager( + DiskPtr disk_, + size_t snapshots_to_keep_, + const KeeperContextPtr & keeper_context_, + bool compress_snapshots_zstd_, + const std::string & superdigest_, + size_t storage_tick_time_) + : disk(disk_) , snapshots_to_keep(snapshots_to_keep_) , compress_snapshots_zstd(compress_snapshots_zstd_) , superdigest(superdigest_) , storage_tick_time(storage_tick_time_) , keeper_context(keeper_context_) { - namespace fs = std::filesystem; - - if (!fs::exists(snapshots_path)) - fs::create_directories(snapshots_path); - - for (const auto & p : fs::directory_iterator(snapshots_path)) + for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) { - const auto & path = p.path(); - - if (!path.has_filename()) + const auto & name = it->name(); + if (name.empty()) continue; - - if (startsWith(path.filename(), "tmp_")) /// Unfinished tmp files + if (startsWith(name, "tmp_")) { - std::filesystem::remove(p); + disk->removeFile(it->path()); continue; } - /// Not snapshot file - if (!startsWith(path.filename(), "snapshot_")) - { + if (!startsWith(name, "snapshot_")) continue; - } - - size_t snapshot_up_to = getSnapshotPathUpToLogIdx(p.path()); - existing_snapshots[snapshot_up_to] = p.path(); + size_t snapshot_up_to = getSnapshotPathUpToLogIdx(name); + existing_snapshots[snapshot_up_to] = it->path(); } removeOutdatedSnapshotsIfNeeded(); @@ -552,19 +559,17 @@ std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; - std::string tmp_snapshot_path = std::filesystem::path{snapshots_path} / tmp_snapshot_file_name; - std::string new_snapshot_path = std::filesystem::path{snapshots_path} / snapshot_file_name; - WriteBufferFromFile plain_buf(tmp_snapshot_path); - copyData(reader, plain_buf); - plain_buf.sync(); + auto plain_buf = disk->writeFile(tmp_snapshot_file_name); + copyData(reader, *plain_buf); + plain_buf->sync(); - std::filesystem::rename(tmp_snapshot_path, new_snapshot_path); + disk->moveFile(tmp_snapshot_file_name, snapshot_file_name); - existing_snapshots.emplace(up_to_log_idx, new_snapshot_path); + existing_snapshots.emplace(up_to_log_idx, snapshot_file_name); removeOutdatedSnapshotsIfNeeded(); - return new_snapshot_path; + return snapshot_file_name; } nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBufferFromDisk() @@ -578,7 +583,7 @@ nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBuff } catch (const DB::Exception &) { - std::filesystem::remove(latest_itr->second); + disk->removeFile(latest_itr->second); existing_snapshots.erase(latest_itr->first); tryLogCurrentException(__PRETTY_FUNCTION__); } @@ -591,8 +596,8 @@ nuraft::ptr KeeperSnapshotManager::deserializeSnapshotBufferFrom { const std::string & snapshot_path = existing_snapshots.at(up_to_log_idx); WriteBufferFromNuraftBuffer writer; - ReadBufferFromFile reader(snapshot_path); - copyData(reader, writer); + auto reader = disk->readFile(snapshot_path); + copyData(*reader, writer); return writer.getBuffer(); } @@ -664,7 +669,7 @@ void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) auto itr = existing_snapshots.find(log_idx); if (itr == existing_snapshots.end()) throw Exception(ErrorCodes::UNKNOWN_SNAPSHOT, "Unknown snapshot with log index {}", log_idx); - std::filesystem::remove(itr->second); + disk->removeFile(itr->second); existing_snapshots.erase(itr); } @@ -673,10 +678,8 @@ std::pair KeeperSnapshotManager::serializeSnapshot auto up_to_log_idx = snapshot.snapshot_meta->get_last_log_idx(); auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; - std::string tmp_snapshot_path = std::filesystem::path{snapshots_path} / tmp_snapshot_file_name; - std::string new_snapshot_path = std::filesystem::path{snapshots_path} / snapshot_file_name; - auto writer = std::make_unique(tmp_snapshot_path, O_WRONLY | O_TRUNC | O_CREAT | O_CLOEXEC | O_APPEND); + auto writer = disk->writeFile(tmp_snapshot_file_name); std::unique_ptr compressed_writer; if (compress_snapshots_zstd) compressed_writer = wrapWriteBufferWithCompressionMethod(std::move(writer), CompressionMethod::Zstd, 3); @@ -688,13 +691,21 @@ std::pair KeeperSnapshotManager::serializeSnapshot compressed_writer->sync(); std::error_code ec; - std::filesystem::rename(tmp_snapshot_path, new_snapshot_path, ec); - if (!ec) + + try { - existing_snapshots.emplace(up_to_log_idx, new_snapshot_path); - removeOutdatedSnapshotsIfNeeded(); + disk->moveFile(tmp_snapshot_file_name, snapshot_file_name); } - return {new_snapshot_path, ec}; + catch (fs::filesystem_error & e) + { + ec = e.code(); + return {snapshot_file_name, ec}; + } + + existing_snapshots.emplace(up_to_log_idx, snapshot_file_name); + removeOutdatedSnapshotsIfNeeded(); + + return {snapshot_file_name, ec}; } } diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 52647712083..9babad9ed98 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -97,6 +98,15 @@ using SnapshotMetaAndStorage = std::pair; class KeeperSnapshotManager { public: + KeeperSnapshotManager( + DiskPtr disk_, + size_t snapshots_to_keep_, + const KeeperContextPtr & keeper_context_, + bool compress_snapshots_zstd_ = true, + const std::string & superdigest_ = "", + size_t storage_tick_time_ = 500); + + /// For gtest KeeperSnapshotManager( const std::string & snapshots_path_, size_t snapshots_to_keep_, @@ -144,9 +154,15 @@ public: if (!existing_snapshots.empty()) { const auto & path = existing_snapshots.at(getLatestSnapshotIndex()); - std::error_code ec; - if (std::filesystem::exists(path, ec)) - return path; + + try + { + if (disk->exists(path)) + return path; + } + catch (...) + { + } } return ""; } @@ -158,7 +174,7 @@ private: /// ZSTD codec. static bool isZstdCompressed(nuraft::ptr buffer); - const std::string snapshots_path; + DiskPtr disk; /// How many snapshots to keep before remove const size_t snapshots_to_keep; /// All existing snapshots in our path (log_index -> path) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index e591f87c6f1..56b8c7e5f2e 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -41,14 +41,14 @@ namespace KeeperStateMachine::KeeperStateMachine( ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, - const std::string & snapshots_path_, + DiskPtr disk_, const CoordinationSettingsPtr & coordination_settings_, const KeeperContextPtr & keeper_context_, KeeperSnapshotManagerS3 * snapshot_manager_s3_, const std::string & superdigest_) : coordination_settings(coordination_settings_) , snapshot_manager( - snapshots_path_, + disk_, coordination_settings->snapshots_to_keep, keeper_context_, coordination_settings->compress_snapshots_with_zstd_format, diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index ffc7fce1cfe..ba209f63a0c 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -25,7 +25,7 @@ public: KeeperStateMachine( ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, - const std::string & snapshots_path_, + DiskPtr disk_, const CoordinationSettingsPtr & coordination_settings_, const KeeperContextPtr & keeper_context_, KeeperSnapshotManagerS3 * snapshot_manager_s3_, diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index cfb3519e597..36a88f5591f 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -214,8 +215,8 @@ KeeperStateManager::KeeperStateManager( int server_id_, const std::string & host, int port, const std::string & logs_path, const std::string & state_file_path) : my_server_id(server_id_) , secure(false) - , log_store(nuraft::cs_new(logs_path, LogFileSettings{.force_sync =false, .compress_logs = false, .rotate_interval = 5000})) - , server_state_path(state_file_path) + , log_store(nuraft::cs_new(std::make_shared("Keeper-logs", logs_path, 0), LogFileSettings{.force_sync =false, .compress_logs = false, .rotate_interval = 5000})) + , server_state_file_name(fs::path(state_file_path).filename().generic_string()) , logger(&Poco::Logger::get("KeeperStateManager")) { auto peer_config = nuraft::cs_new(my_server_id, host + ":" + std::to_string(port)); @@ -228,8 +229,9 @@ KeeperStateManager::KeeperStateManager( KeeperStateManager::KeeperStateManager( int my_server_id_, const std::string & config_prefix_, - const std::string & log_storage_path, - const std::string & state_file_path, + DiskPtr log_disk_, + DiskPtr state_disk_, + const std::string & server_state_file_name_, const Poco::Util::AbstractConfiguration & config, const CoordinationSettingsPtr & coordination_settings) : my_server_id(my_server_id_) @@ -237,7 +239,7 @@ KeeperStateManager::KeeperStateManager( , config_prefix(config_prefix_) , configuration_wrapper(parseServersConfiguration(config, false)) , log_store(nuraft::cs_new( - log_storage_path, + log_disk_, LogFileSettings { .force_sync = coordination_settings->force_sync, @@ -246,7 +248,8 @@ KeeperStateManager::KeeperStateManager( .max_size = coordination_settings->max_log_file_size, .overallocate_size = coordination_settings->log_file_overallocate_size })) - , server_state_path(state_file_path) + , disk(state_disk_) + , server_state_file_name(server_state_file_name_) , logger(&Poco::Logger::get("KeeperStateManager")) { } @@ -285,11 +288,11 @@ void KeeperStateManager::save_config(const nuraft::cluster_config & config) configuration_wrapper.cluster_config = nuraft::cluster_config::deserialize(*buf); } -const std::filesystem::path & KeeperStateManager::getOldServerStatePath() +const String & KeeperStateManager::getOldServerStatePath() { static auto old_path = [this] { - return server_state_path.parent_path() / (server_state_path.filename().generic_string() + "-OLD"); + return server_state_file_name + "-OLD"; }(); return old_path; @@ -310,25 +313,24 @@ void KeeperStateManager::save_state(const nuraft::srv_state & state) { const auto & old_path = getOldServerStatePath(); - if (std::filesystem::exists(server_state_path)) - std::filesystem::rename(server_state_path, old_path); + if (disk->exists(server_state_file_name)) + disk->moveFile(server_state_file_name, old_path); - WriteBufferFromFile server_state_file(server_state_path, DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY); + auto server_state_file = disk->writeFile(server_state_file_name); auto buf = state.serialize(); // calculate checksum SipHash hash; hash.update(current_server_state_version); hash.update(reinterpret_cast(buf->data_begin()), buf->size()); - writeIntBinary(hash.get64(), server_state_file); + writeIntBinary(hash.get64(), *server_state_file); - writeIntBinary(static_cast(current_server_state_version), server_state_file); + writeIntBinary(static_cast(current_server_state_version), *server_state_file); - server_state_file.write(reinterpret_cast(buf->data_begin()), buf->size()); - server_state_file.sync(); - server_state_file.close(); + server_state_file->write(reinterpret_cast(buf->data_begin()), buf->size()); + server_state_file->sync(); - std::filesystem::remove(old_path); + disk->removeFileIfExists(old_path); } nuraft::ptr KeeperStateManager::read_state() @@ -339,22 +341,22 @@ nuraft::ptr KeeperStateManager::read_state() { try { - ReadBufferFromFile read_buf(path); - auto content_size = read_buf.getFileSize(); + auto read_buf = disk->readFile(path); + auto content_size = read_buf->getFileSize(); if (content_size == 0) return nullptr; uint64_t read_checksum{0}; - readIntBinary(read_checksum, read_buf); + readIntBinary(read_checksum, *read_buf); uint8_t version; - readIntBinary(version, read_buf); + readIntBinary(version, *read_buf); auto buffer_size = content_size - sizeof read_checksum - sizeof version; auto state_buf = nuraft::buffer::alloc(buffer_size); - read_buf.readStrict(reinterpret_cast(state_buf->data_begin()), buffer_size); + read_buf->readStrict(reinterpret_cast(state_buf->data_begin()), buffer_size); SipHash hash; hash.update(version); @@ -364,15 +366,15 @@ nuraft::ptr KeeperStateManager::read_state() { constexpr auto error_format = "Invalid checksum while reading state from {}. Got {}, expected {}"; #ifdef NDEBUG - LOG_ERROR(logger, error_format, path.generic_string(), hash.get64(), read_checksum); + LOG_ERROR(logger, error_format, path, hash.get64(), read_checksum); return nullptr; #else - throw Exception(ErrorCodes::CORRUPTED_DATA, error_format, path.generic_string(), hash.get64(), read_checksum); + throw Exception(ErrorCodes::CORRUPTED_DATA, error_format, disk->getPath() + path, hash.get64(), read_checksum); #endif } auto state = nuraft::srv_state::deserialize(*state_buf); - LOG_INFO(logger, "Read state from {}", path.generic_string()); + LOG_INFO(logger, "Read state from {}", disk->getPath() + path); return state; } catch (const std::exception & e) @@ -383,37 +385,34 @@ nuraft::ptr KeeperStateManager::read_state() throw; } - LOG_ERROR(logger, "Failed to deserialize state from {}", path.generic_string()); + LOG_ERROR(logger, "Failed to deserialize state from {}", disk->getPath() + path); return nullptr; } }; - if (std::filesystem::exists(server_state_path)) + if (disk->exists(server_state_file_name)) { - auto state = try_read_file(server_state_path); + auto state = try_read_file(server_state_file_name); if (state) { - if (std::filesystem::exists(old_path)) - std::filesystem::remove(old_path); + disk->removeFileIfExists(old_path); return state; } - std::filesystem::remove(server_state_path); + disk->removeFile(server_state_file_name); } - if (std::filesystem::exists(old_path)) + if (disk->exists(old_path)) { auto state = try_read_file(old_path); - if (state) { - std::filesystem::rename(old_path, server_state_path); + disk->moveFile(old_path, server_state_file_name); return state; } - - std::filesystem::remove(old_path); + disk->removeFile(old_path); } LOG_WARNING(logger, "No state was read"); diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index 5d210f8c0ea..d8369100d1c 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -39,7 +39,8 @@ public: KeeperStateManager( int server_id_, const std::string & config_prefix_, - const std::string & log_storage_path, + DiskPtr logs_disk_, + DiskPtr state_disk_, const std::string & state_file_path, const Poco::Util::AbstractConfiguration & config, const CoordinationSettingsPtr & coordination_settings); @@ -111,7 +112,7 @@ public: ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const; private: - const std::filesystem::path & getOldServerStatePath(); + const String & getOldServerStatePath(); /// Wrapper struct for Keeper cluster config. We parse this /// info from XML files. @@ -136,7 +137,8 @@ private: nuraft::ptr log_store; - const std::filesystem::path server_state_path; + DiskPtr disk; + const String server_state_file_name; Poco::Logger * logger; diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp index f8c4d0e2bac..d079a0d230d 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp @@ -11,7 +11,7 @@ namespace ErrorCodes } ZstdDeflatingAppendableWriteBuffer::ZstdDeflatingAppendableWriteBuffer( - std::unique_ptr out_, + std::unique_ptr out_, int compression_level, bool append_to_existing_file_, size_t buf_size, diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.h b/src/IO/ZstdDeflatingAppendableWriteBuffer.h index a0715480737..b64e5d5c4cf 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.h +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.h @@ -29,7 +29,7 @@ public: static inline constexpr ZSTDLastBlock ZSTD_CORRECT_TERMINATION_LAST_BLOCK = {0x01, 0x00, 0x00}; ZstdDeflatingAppendableWriteBuffer( - std::unique_ptr out_, + std::unique_ptr out_, int compression_level, bool append_to_existing_file_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, @@ -68,7 +68,7 @@ private: /// Adding zstd empty block (ZSTD_CORRECT_TERMINATION_LAST_BLOCK) to out.working_buffer void addEmptyBlock(); - std::unique_ptr out; + std::unique_ptr out; bool append_to_existing_file = false; ZSTD_CCtx * cctx; From 010edbf2aad3508402e82b8ce62f90ce62fc9f09 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 22 Feb 2023 10:39:23 +0100 Subject: [PATCH 004/515] do another way and logs work --- src/Common/DateLUT.h | 38 ++++++++++++++++++-------------------- 1 file changed, 18 insertions(+), 20 deletions(-) diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index bd7969bffa6..6ec9cf1646d 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -5,9 +5,6 @@ #include #include -// -//#include "Interpreters/Context_fwd.h" -//#include "Interpreters/Context.h" #include "Common/CurrentThread.h" #include @@ -24,30 +21,31 @@ public: static ALWAYS_INLINE const DateLUTImpl & instance() // -V1071 { const auto & date_lut = getInstance(); - - if (DB::CurrentThread::isInitialized()) - { - const auto query_context = DB::CurrentThread::get().getQueryContext(); - - if (query_context) - { - auto implicit_timezone = extractTimezoneFromContext(query_context); - - if (!implicit_timezone.empty()) - return instance(implicit_timezone); - } - } - return *date_lut.default_impl.load(std::memory_order_acquire); } /// Return singleton DateLUTImpl instance for a given time zone. static ALWAYS_INLINE const DateLUTImpl & instance(const std::string & time_zone) { - if (time_zone.empty()) - return instance(); - const auto & date_lut = getInstance(); + + if (time_zone.empty()) + { + if (DB::CurrentThread::isInitialized()) + { + const auto query_context = DB::CurrentThread::get().getQueryContext(); + + if (query_context) + { + auto implicit_timezone = extractTimezoneFromContext(query_context); + + if (!implicit_timezone.empty()) + return instance(implicit_timezone); + } + } + return *date_lut.default_impl.load(std::memory_order_acquire); + } + return date_lut.getImplementation(time_zone); } static void setDefaultTimezone(const std::string & time_zone) From 1cf6c3a9c0fa4867684dee56c651d4131aa3b0fe Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 22 Feb 2023 10:51:32 +0100 Subject: [PATCH 005/515] update test names --- ...licit_timezone.reference => 02668_implicit_timezone.reference} | 0 .../{02538_implicit_timezone.sql => 02668_implicit_timezone.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{02538_implicit_timezone.reference => 02668_implicit_timezone.reference} (100%) rename tests/queries/0_stateless/{02538_implicit_timezone.sql => 02668_implicit_timezone.sql} (100%) diff --git a/tests/queries/0_stateless/02538_implicit_timezone.reference b/tests/queries/0_stateless/02668_implicit_timezone.reference similarity index 100% rename from tests/queries/0_stateless/02538_implicit_timezone.reference rename to tests/queries/0_stateless/02668_implicit_timezone.reference diff --git a/tests/queries/0_stateless/02538_implicit_timezone.sql b/tests/queries/0_stateless/02668_implicit_timezone.sql similarity index 100% rename from tests/queries/0_stateless/02538_implicit_timezone.sql rename to tests/queries/0_stateless/02668_implicit_timezone.sql From 393830ecdc78cd2745cc439d7ac95c3421fe9044 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 22 Feb 2023 16:30:46 +0100 Subject: [PATCH 006/515] add docs + tiny cleanup --- docs/en/operations/settings/settings.md | 27 ++++++++++++++++++++++++ docs/ru/operations/settings/settings.md | 28 +++++++++++++++++++++++++ src/Common/DateLUT.cpp | 3 +-- 3 files changed, 56 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 1060eae1b0e..4e105124086 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3939,3 +3939,30 @@ Default value: `0`. :::note Use this setting only for backward compatibility if your use cases depend on old syntax. ::: + +## implicit_timezone {#implicit_timezone} + +If specified, sets a implicit timezone (instead of server-default). All DateTime/DateTime64 values (and/or functions results) that have no explicit timezone specified are treated as having this timezone instead of default. +Examples: +``` +SELECT timeZone(), timeZoneOf(now()) +┌─timeZone()────┬─timeZoneOf(now())─┐ +│ Europe/Berlin │ Europe/Berlin │ +└───────────────┴───────────────────┘ + +:) SELECT timeZone(), timeZoneOf(now()) SETTINGS implicit_timezone = 'Asia/Novosibirsk' +┌─timeZone()────┬─timeZoneOf(now())─┐ +│ Europe/Berlin │ Asia/Novosibirsk │ +└───────────────┴───────────────────┘ + +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS implicit_timezone = 'America/Denver'; +┌─toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich')─┐ +│ 1999-12-13 07:23:23.123 │ +└──────────────────────────────────────────────────────────────────────────────┘ +``` + +Possible values: + +- Any valid timezone in `Region/Place` notation, e.g. `Europe/Berlin` + +Default value: `''`. \ No newline at end of file diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 4025966ac21..8d3f2706585 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4084,3 +4084,31 @@ ALTER TABLE test FREEZE SETTINGS alter_partition_verbose_result = 1; Задает символ, который интерпретируется как суффикс после результирующего набора данных формата [CustomSeparated](../../interfaces/formats.md#format-customseparated). Значение по умолчанию: `''`. + +## implicit_timezone {#implicit_timezone} + +Задаёт значение часового пояса (timezone) по умолчанию для текущей сессии вместо часового пояса сервера. То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. + +Примеры: +``` +SELECT timeZone(), timeZoneOf(now()) +┌─timeZone()────┬─timeZoneOf(now())─┐ +│ Europe/Berlin │ Europe/Berlin │ +└───────────────┴───────────────────┘ + +:) SELECT timeZone(), timeZoneOf(now()) SETTINGS implicit_timezone = 'Asia/Novosibirsk' +┌─timeZone()────┬─timeZoneOf(now())─┐ +│ Europe/Berlin │ Asia/Novosibirsk │ +└───────────────┴───────────────────┘ + +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS implicit_timezone = 'America/Denver'; +┌─toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich')─┐ +│ 1999-12-13 07:23:23.123 │ +└──────────────────────────────────────────────────────────────────────────────┘ +``` + +Возможные значения: + +- Строка вида `Регион/Город`, например `Europe/Zurich` + +Значение по умолчанию: `''`. \ No newline at end of file diff --git a/src/Common/DateLUT.cpp b/src/Common/DateLUT.cpp index fd10b799b66..e309b0cb28a 100644 --- a/src/Common/DateLUT.cpp +++ b/src/Common/DateLUT.cpp @@ -8,7 +8,6 @@ #include #include #include -#include namespace @@ -149,7 +148,7 @@ DateLUT::DateLUT() } -const ALWAYS_INLINE DateLUTImpl & DateLUT::getImplementation(const std::string & time_zone) const +const DateLUTImpl & DateLUT::getImplementation(const std::string & time_zone) const { std::lock_guard lock(mutex); From c61aff7cac2e5cc79dc4591d9228308e017e5b28 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 23 Feb 2023 13:38:13 +0100 Subject: [PATCH 007/515] Added standalone function to get server's own timezone Fix missing global_context --- src/Common/DateLUT.h | 10 ++++++++++ src/Functions/serverConstants.cpp | 19 +++++++++++++++++-- 2 files changed, 27 insertions(+), 2 deletions(-) diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index b66821989e3..18ef5ee2e30 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -42,6 +42,16 @@ public: if (!implicit_timezone.empty()) return instance(implicit_timezone); } + + const auto global_context = DB::CurrentThread::get().getGlobalContext(); + if (global_context) + { + auto implicit_timezone = extractTimezoneFromContext(global_context); + + if (!implicit_timezone.empty()) + return instance(implicit_timezone); + } + } return *date_lut.default_impl.load(std::memory_order_acquire); } diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index 96615d0a4c9..a89e1564f28 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -60,13 +60,22 @@ namespace }; - /// Returns the server time zone. + /// Returns default timezone for current session. class FunctionTimezone : public FunctionConstantBase { public: static constexpr auto name = "timezone"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance().getTimeZone()}, context->isDistributed()) {} + explicit FunctionTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance("").getTimeZone()}, context->isDistributed()) {} + }; + + /// Returns the server time zone (timezone in which server runs). + class FunctionServerTimezone : public FunctionConstantBase + { + public: + static constexpr auto name = "serverTimezone"; + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + explicit FunctionServerTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance().getTimeZone()}, context->isDistributed()) {} }; @@ -155,6 +164,12 @@ REGISTER_FUNCTION(Timezone) factory.registerAlias("timeZone", "timezone"); } +REGISTER_FUNCTION(ServerTimezone) +{ + factory.registerFunction(); + factory.registerAlias("serverTimeZone", "serverTimezone"); +} + REGISTER_FUNCTION(Uptime) { factory.registerFunction(); From a9d0f7e7dbb281def311b22e4ae6300c73b5e979 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 23 Feb 2023 19:14:49 +0100 Subject: [PATCH 008/515] Added docs for new serverTimeZone function Updated tests and docs --- docs/en/operations/settings/settings.md | 28 ++++++++++--------- .../functions/date-time-functions.md | 21 +++++++++++++- docs/ru/operations/settings/settings.md | 27 +++++++++--------- .../functions/date-time-functions.md | 21 +++++++++++++- src/Common/DateLUT.cpp | 3 +- src/Common/DateLUT.h | 8 ++++-- src/Core/Settings.h | 2 +- src/Functions/serverConstants.cpp | 23 +++++++++++++-- ...rence => 02668_timezone_setting.reference} | 0 ...imezone.sql => 02668_timezone_setting.sql} | 6 ++-- 10 files changed, 101 insertions(+), 38 deletions(-) rename tests/queries/0_stateless/{02668_implicit_timezone.reference => 02668_timezone_setting.reference} (100%) rename tests/queries/0_stateless/{02668_implicit_timezone.sql => 02668_timezone_setting.sql} (61%) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 4e105124086..c1c4483d341 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3940,25 +3940,27 @@ Default value: `0`. Use this setting only for backward compatibility if your use cases depend on old syntax. ::: -## implicit_timezone {#implicit_timezone} +## timezone {#timezone} If specified, sets a implicit timezone (instead of server-default). All DateTime/DateTime64 values (and/or functions results) that have no explicit timezone specified are treated as having this timezone instead of default. Examples: + +```clickhouse +SELECT timeZone(), serverTimezone() FORMAT TSV + +Europe/Berlin Europe/Berlin ``` -SELECT timeZone(), timeZoneOf(now()) -┌─timeZone()────┬─timeZoneOf(now())─┐ -│ Europe/Berlin │ Europe/Berlin │ -└───────────────┴───────────────────┘ -:) SELECT timeZone(), timeZoneOf(now()) SETTINGS implicit_timezone = 'Asia/Novosibirsk' -┌─timeZone()────┬─timeZoneOf(now())─┐ -│ Europe/Berlin │ Asia/Novosibirsk │ -└───────────────┴───────────────────┘ +```clickhouse +SELECT timeZone(), serverTimezone() SETTINGS timezone = 'Asia/Novosibirsk' FORMAT TSV -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS implicit_timezone = 'America/Denver'; -┌─toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich')─┐ -│ 1999-12-13 07:23:23.123 │ -└──────────────────────────────────────────────────────────────────────────────┘ +Asia/Novosibirsk Europe/Berlin +``` + +```clickhouse +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'America/Denver' FORMAT TSV + +1999-12-13 07:23:23.123 ``` Possible values: diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index f6af8abcbaf..8d31cb3872f 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -26,7 +26,7 @@ SELECT ## timeZone -Returns the timezone of the server. +Returns the default timezone of the server for current session. This can be modified using `SET timezone = 'New/Value'` If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. **Syntax** @@ -43,6 +43,25 @@ Alias: `timezone`. Type: [String](../../sql-reference/data-types/string.md). +## serverTimeZone + +Returns the actual timezone in which the server runs in. +If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. + +**Syntax** + +``` sql +timeZone() +``` + +Alias: `ServerTimezone`, `servertimezone`. + +**Returned value** + +- Timezone. + +Type: [String](../../sql-reference/data-types/string.md). + ## toTimeZone Converts time or date and time to the specified time zone. The time zone is an attribute of the `Date` and `DateTime` data types. The internal value (number of seconds) of the table field or of the resultset's column does not change, the column's type changes and its string representation changes accordingly. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 8d3f2706585..dd1e9d98427 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4085,26 +4085,27 @@ ALTER TABLE test FREEZE SETTINGS alter_partition_verbose_result = 1; Значение по умолчанию: `''`. -## implicit_timezone {#implicit_timezone} +## timezone {#timezone} Задаёт значение часового пояса (timezone) по умолчанию для текущей сессии вместо часового пояса сервера. То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. Примеры: +```clickhouse +SELECT timeZone(), serverTimezone() FORMAT TSV + +Europe/Berlin Europe/Berlin ``` -SELECT timeZone(), timeZoneOf(now()) -┌─timeZone()────┬─timeZoneOf(now())─┐ -│ Europe/Berlin │ Europe/Berlin │ -└───────────────┴───────────────────┘ -:) SELECT timeZone(), timeZoneOf(now()) SETTINGS implicit_timezone = 'Asia/Novosibirsk' -┌─timeZone()────┬─timeZoneOf(now())─┐ -│ Europe/Berlin │ Asia/Novosibirsk │ -└───────────────┴───────────────────┘ +```clickhouse +SELECT timeZone(), serverTimezone() SETTINGS timezone = 'Asia/Novosibirsk' FORMAT TSV -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS implicit_timezone = 'America/Denver'; -┌─toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich')─┐ -│ 1999-12-13 07:23:23.123 │ -└──────────────────────────────────────────────────────────────────────────────┘ +Asia/Novosibirsk Europe/Berlin +``` + +```clickhouse +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'America/Denver' FORMAT TSV + +1999-12-13 07:23:23.123 ``` Возможные значения: diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 8fbcaf9568b..77188ea2797 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -26,7 +26,7 @@ SELECT ## timeZone {#timezone} -Возвращает часовой пояс сервера. +Возвращает часовой пояс сервера, считающийся умолчанием для текущей сессии. Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями, актуальными для каждого шарда. Иначе возвращается константа. **Синтаксис** @@ -43,6 +43,25 @@ timeZone() Тип: [String](../../sql-reference/data-types/string.md). +## serverTimeZone {#servertimezone} + +Возвращает (истинный) часовой пояс сервера, в котором тот работает. +Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями, актуальными для каждого шарда. Иначе возвращается константа. + +**Синтаксис** + +``` sql +serverTimeZone() +``` + +Синонимы: `servertimezone`, `serverTimezone`. + +**Возвращаемое значение** + +- Часовой пояс. + +Тип: [String](../../sql-reference/data-types/string.md). + ## toTimeZone {#totimezone} Переводит дату или дату с временем в указанный часовой пояс. Часовой пояс - это атрибут типов `Date` и `DateTime`. Внутреннее значение (количество секунд) поля таблицы или результирующего столбца не изменяется, изменяется тип поля и, соответственно, его текстовое отображение. diff --git a/src/Common/DateLUT.cpp b/src/Common/DateLUT.cpp index e309b0cb28a..3698fe45aa7 100644 --- a/src/Common/DateLUT.cpp +++ b/src/Common/DateLUT.cpp @@ -167,6 +167,5 @@ DateLUT & DateLUT::getInstance() std::string DateLUT::extractTimezoneFromContext(const DB::ContextPtr query_context) { - std::string ret = query_context->getSettingsRef().implicit_timezone.value; - return ret; + return query_context->getSettingsRef().timezone.value; } diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index 18ef5ee2e30..a9ee61dc8ab 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -17,14 +17,18 @@ class DateLUT : private boost::noncopyable { public: - /// Return singleton DateLUTImpl instance for the default time zone. + /// Return singleton DateLUTImpl instance for server's timezone (the one which server has). static ALWAYS_INLINE const DateLUTImpl & instance() { const auto & date_lut = getInstance(); return *date_lut.default_impl.load(std::memory_order_acquire); } - /// Return singleton DateLUTImpl instance for a given time zone. + /* + Return singleton DateLUTImpl instance for a given time zone. If timezone is an empty string, + timezone set by `timezone` setting for current session is used. If it is not set, server's timezone is used, + and return is the same as calling instance(). + */ static ALWAYS_INLINE const DateLUTImpl & instance(const std::string & time_zone) { const auto & date_lut = getInstance(); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e3ed1b26269..e70b8c131b3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -712,7 +712,7 @@ class IColumn; M(Float, insert_keeper_fault_injection_probability, 0.0f, "Approximate probability of failure for a keeper request during insert. Valid value is in interval [0.0f, 1.0f]", 0) \ M(UInt64, insert_keeper_fault_injection_seed, 0, "0 - random seed, otherwise the setting value", 0) \ M(Bool, force_aggregation_in_order, false, "Force use of aggregation in order on remote nodes during distributed aggregation. PLEASE, NEVER CHANGE THIS SETTING VALUE MANUALLY!", IMPORTANT) \ - M(String, implicit_timezone, "", "Use specified timezone for interpreting Date and DateTime instead of server's timezone.", 0) \ + M(String, timezone, "", "Use specified timezone for interpreting Date and DateTime instead of server's timezone.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index a89e1564f28..b57f7e40e07 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -160,14 +160,33 @@ REGISTER_FUNCTION(TcpPort) REGISTER_FUNCTION(Timezone) { - factory.registerFunction(); + factory.registerFunction({ + R"( +Returns the default timezone for current session. +Used as default timezone for parsing DateTime|DateTime64 without explicitly specified timezone. +Can be changed with SET timezone = 'New/Tz' + +[example:timezone] +)", + Documentation::Examples{{"serverTimezone", "SELECT timezone();"}}, + Documentation::Categories{"Constant", "Miscellaneous"} + }); factory.registerAlias("timeZone", "timezone"); } REGISTER_FUNCTION(ServerTimezone) { - factory.registerFunction(); + factory.registerFunction({ + R"( +Returns the timezone name in which server operates. + +[example:serverTimezone] +)", + Documentation::Examples{{"serverTimezone", "SELECT serverTimezone();"}}, + Documentation::Categories{"Constant", "Miscellaneous"} + }); factory.registerAlias("serverTimeZone", "serverTimezone"); + factory.registerAlias("servertimezone", "serverTimezone"); } REGISTER_FUNCTION(Uptime) diff --git a/tests/queries/0_stateless/02668_implicit_timezone.reference b/tests/queries/0_stateless/02668_timezone_setting.reference similarity index 100% rename from tests/queries/0_stateless/02668_implicit_timezone.reference rename to tests/queries/0_stateless/02668_timezone_setting.reference diff --git a/tests/queries/0_stateless/02668_implicit_timezone.sql b/tests/queries/0_stateless/02668_timezone_setting.sql similarity index 61% rename from tests/queries/0_stateless/02668_implicit_timezone.sql rename to tests/queries/0_stateless/02668_timezone_setting.sql index 663b218d235..3748b536614 100644 --- a/tests/queries/0_stateless/02668_implicit_timezone.sql +++ b/tests/queries/0_stateless/02668_timezone_setting.sql @@ -1,9 +1,9 @@ -SET implicit_timezone = 'Asia/Novosibirsk'; +SET timezone = 'Asia/Novosibirsk'; SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich'); -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS implicit_timezone = 'Europe/Zurich'; +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'Europe/Zurich'; -SET implicit_timezone = 'Europe/Zurich'; +SET timezone = 'Europe/Zurich'; SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); \ No newline at end of file From c3a6efe0310ec23521eb97c1b53c6616f72ba7a0 Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 24 Feb 2023 20:51:12 +0100 Subject: [PATCH 009/515] update --- src/Client/ClientBase.cpp | 2 +- src/Common/DateLUT.h | 13 +++++++------ src/Functions/serverConstants.cpp | 2 +- src/IO/ReadHelpers.h | 8 ++++---- src/IO/WriteHelpers.h | 6 +++--- .../queries/0_stateless/02668_timezone_setting.sql | 6 +++--- 6 files changed, 19 insertions(+), 18 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index bc8c43af8c6..9ebe115d408 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2139,7 +2139,7 @@ void ClientBase::runInteractive() initQueryIdFormats(); /// Initialize DateLUT here to avoid counting time spent here as query execution time. - const auto local_tz = DateLUT::instance().getTimeZone(); + const auto local_tz = DateLUT::instance("").getTimeZone(); suggest.emplace(); if (load_suggestions) diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index a9ee61dc8ab..29a4ee13d87 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -32,6 +32,7 @@ public: static ALWAYS_INLINE const DateLUTImpl & instance(const std::string & time_zone) { const auto & date_lut = getInstance(); + std::string effective_time_zone; if (time_zone.empty()) { @@ -41,19 +42,19 @@ public: if (query_context) { - auto implicit_timezone = extractTimezoneFromContext(query_context); + effective_time_zone = extractTimezoneFromContext(query_context); - if (!implicit_timezone.empty()) - return instance(implicit_timezone); + if (!effective_time_zone.empty()) + return date_lut.getImplementation(effective_time_zone); } const auto global_context = DB::CurrentThread::get().getGlobalContext(); if (global_context) { - auto implicit_timezone = extractTimezoneFromContext(global_context); + effective_time_zone = extractTimezoneFromContext(global_context); - if (!implicit_timezone.empty()) - return instance(implicit_timezone); + if (!effective_time_zone.empty()) + return date_lut.getImplementation(effective_time_zone); } } diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index b57f7e40e07..ea74d7a89bb 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -168,7 +168,7 @@ Can be changed with SET timezone = 'New/Tz' [example:timezone] )", - Documentation::Examples{{"serverTimezone", "SELECT timezone();"}}, + Documentation::Examples{{"timezone", "SELECT timezone();"}}, Documentation::Categories{"Constant", "Miscellaneous"} }); factory.registerAlias("timeZone", "timezone"); diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index f8931a7f622..9f5358ee141 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1032,22 +1032,22 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re return ReturnType(is_ok); } -inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) +inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) { readDateTimeTextImpl(datetime, buf, time_zone); } -inline void readDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) +inline void readDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance("")) { readDateTimeTextImpl(datetime64, scale, buf, date_lut); } -inline bool tryReadDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) +inline bool tryReadDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) { return readDateTimeTextImpl(datetime, buf, time_zone); } -inline bool tryReadDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) +inline bool tryReadDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance("")) { return readDateTimeTextImpl(datetime64, scale, buf, date_lut); } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 1c0b48c53c3..d408e2bed42 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -755,14 +755,14 @@ inline void writeDateTimeText(const LocalDateTime & datetime, WriteBuffer & buf) /// In the format YYYY-MM-DD HH:MM:SS, according to the specified time zone. template -inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) +inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) { writeDateTimeText(LocalDateTime(datetime, time_zone), buf); } /// In the format YYYY-MM-DD HH:MM:SS.NNNNNNNNN, according to the specified time zone. template -inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) +inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) { static constexpr UInt32 MaxScale = DecimalUtils::max_precision; scale = scale > MaxScale ? MaxScale : scale; @@ -796,7 +796,7 @@ inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer & /// In the RFC 1123 format: "Tue, 03 Dec 2019 00:11:50 GMT". You must provide GMT DateLUT. /// This is needed for HTTP requests. -inline void writeDateTimeTextRFC1123(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) +inline void writeDateTimeTextRFC1123(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) { const auto & values = time_zone.getValues(datetime); diff --git a/tests/queries/0_stateless/02668_timezone_setting.sql b/tests/queries/0_stateless/02668_timezone_setting.sql index 3748b536614..f331ab58307 100644 --- a/tests/queries/0_stateless/02668_timezone_setting.sql +++ b/tests/queries/0_stateless/02668_timezone_setting.sql @@ -1,9 +1,9 @@ SET timezone = 'Asia/Novosibirsk'; - SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich'); - SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'Europe/Zurich'; SET timezone = 'Europe/Zurich'; +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); \ No newline at end of file +SET timezone = 'Абырвалг'; +select now(); -- { serverError POCO_EXCEPTION } \ No newline at end of file From 6a996f552b9cf70f88d2a6c7c8f1ef2780268666 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sun, 26 Feb 2023 10:06:27 +0100 Subject: [PATCH 010/515] update undocumented funcs reference --- .../02415_all_new_functions_must_be_documented.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index e41249af54c..ce14ee871f5 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -683,7 +683,6 @@ throwIf tid timeSlot timeSlots -timezone timezoneOf timezoneOffset toBool From a69425326de20dcf5814c39a0962023cabec27ec Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 27 Feb 2023 01:40:00 +0100 Subject: [PATCH 011/515] upd --- src/Client/ClientBase.cpp | 2 +- src/Common/DateLUT.h | 59 +++++++++++++++---------------- src/DataTypes/TimezoneMixin.h | 2 +- src/Functions/serverConstants.cpp | 4 +-- src/IO/ReadHelpers.h | 8 ++--- src/IO/WriteHelpers.h | 6 ++-- 6 files changed, 39 insertions(+), 42 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index a335dca0602..96aff9aa304 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2139,7 +2139,7 @@ void ClientBase::runInteractive() initQueryIdFormats(); /// Initialize DateLUT here to avoid counting time spent here as query execution time. - const auto local_tz = DateLUT::instance("").getTimeZone(); + const auto local_tz = DateLUT::instance().getTimeZone(); suggest.emplace(); if (load_suggestions) diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index 29a4ee13d87..efbf56b59b2 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -17,49 +17,46 @@ class DateLUT : private boost::noncopyable { public: - /// Return singleton DateLUTImpl instance for server's timezone (the one which server has). + /// Return singleton DateLUTImpl instance for timezone set by `timezone` setting for current session is used. + /// If it is not set, server's timezone (the one which server has) is being used. static ALWAYS_INLINE const DateLUTImpl & instance() { + std::string effective_time_zone; const auto & date_lut = getInstance(); + + if (DB::CurrentThread::isInitialized()) + { + const auto query_context = DB::CurrentThread::get().getQueryContext(); + + if (query_context) + { + effective_time_zone = extractTimezoneFromContext(query_context); + + if (!effective_time_zone.empty()) + return date_lut.getImplementation(effective_time_zone); + } + + const auto global_context = DB::CurrentThread::get().getGlobalContext(); + if (global_context) + { + effective_time_zone = extractTimezoneFromContext(global_context); + + if (!effective_time_zone.empty()) + return date_lut.getImplementation(effective_time_zone); + } + + } return *date_lut.default_impl.load(std::memory_order_acquire); } - /* - Return singleton DateLUTImpl instance for a given time zone. If timezone is an empty string, - timezone set by `timezone` setting for current session is used. If it is not set, server's timezone is used, - and return is the same as calling instance(). - */ + /// Return singleton DateLUTImpl instance for a given time zone. If timezone is an empty string, + /// server's timezone is used. The `timezone` setting is not considered here. static ALWAYS_INLINE const DateLUTImpl & instance(const std::string & time_zone) { const auto & date_lut = getInstance(); - std::string effective_time_zone; if (time_zone.empty()) - { - if (DB::CurrentThread::isInitialized()) - { - const auto query_context = DB::CurrentThread::get().getQueryContext(); - - if (query_context) - { - effective_time_zone = extractTimezoneFromContext(query_context); - - if (!effective_time_zone.empty()) - return date_lut.getImplementation(effective_time_zone); - } - - const auto global_context = DB::CurrentThread::get().getGlobalContext(); - if (global_context) - { - effective_time_zone = extractTimezoneFromContext(global_context); - - if (!effective_time_zone.empty()) - return date_lut.getImplementation(effective_time_zone); - } - - } return *date_lut.default_impl.load(std::memory_order_acquire); - } return date_lut.getImplementation(time_zone); } diff --git a/src/DataTypes/TimezoneMixin.h b/src/DataTypes/TimezoneMixin.h index 03ecde5dd0a..5b7870c7b9a 100644 --- a/src/DataTypes/TimezoneMixin.h +++ b/src/DataTypes/TimezoneMixin.h @@ -15,7 +15,7 @@ public: explicit TimezoneMixin(const String & time_zone_name = "") : has_explicit_time_zone(!time_zone_name.empty()) - , time_zone(DateLUT::instance(time_zone_name)) + , time_zone(time_zone_name.empty() ? DateLUT::instance() : DateLUT::instance(time_zone_name)) , utc_time_zone(DateLUT::instance("UTC")) { } diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index ea74d7a89bb..57a6279bd7a 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -66,7 +66,7 @@ namespace public: static constexpr auto name = "timezone"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance("").getTimeZone()}, context->isDistributed()) {} + explicit FunctionTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance().getTimeZone()}, context->isDistributed()) {} }; /// Returns the server time zone (timezone in which server runs). @@ -75,7 +75,7 @@ namespace public: static constexpr auto name = "serverTimezone"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionServerTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance().getTimeZone()}, context->isDistributed()) {} + explicit FunctionServerTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance("").getTimeZone()}, context->isDistributed()) {} }; diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 9f5358ee141..f8931a7f622 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1032,22 +1032,22 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re return ReturnType(is_ok); } -inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) +inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { readDateTimeTextImpl(datetime, buf, time_zone); } -inline void readDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance("")) +inline void readDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) { readDateTimeTextImpl(datetime64, scale, buf, date_lut); } -inline bool tryReadDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) +inline bool tryReadDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { return readDateTimeTextImpl(datetime, buf, time_zone); } -inline bool tryReadDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance("")) +inline bool tryReadDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) { return readDateTimeTextImpl(datetime64, scale, buf, date_lut); } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index d408e2bed42..1c0b48c53c3 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -755,14 +755,14 @@ inline void writeDateTimeText(const LocalDateTime & datetime, WriteBuffer & buf) /// In the format YYYY-MM-DD HH:MM:SS, according to the specified time zone. template -inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) +inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { writeDateTimeText(LocalDateTime(datetime, time_zone), buf); } /// In the format YYYY-MM-DD HH:MM:SS.NNNNNNNNN, according to the specified time zone. template -inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) +inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { static constexpr UInt32 MaxScale = DecimalUtils::max_precision; scale = scale > MaxScale ? MaxScale : scale; @@ -796,7 +796,7 @@ inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer & /// In the RFC 1123 format: "Tue, 03 Dec 2019 00:11:50 GMT". You must provide GMT DateLUT. /// This is needed for HTTP requests. -inline void writeDateTimeTextRFC1123(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance("")) +inline void writeDateTimeTextRFC1123(time_t datetime, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { const auto & values = time_zone.getValues(datetime); From f3e19144d81449c1e2bdec52ebc38e85ea1e8ee9 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 27 Feb 2023 14:38:15 +0100 Subject: [PATCH 012/515] update --- src/Interpreters/executeQuery.cpp | 3 ++- src/Loggers/OwnPatternFormatter.cpp | 2 +- src/Loggers/OwnPatternFormatter.h | 2 ++ 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 435401796a0..cda7ec2b0d3 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -49,6 +49,7 @@ #include #include #include +#include #include #include #include @@ -1279,7 +1280,7 @@ void executeQuery( QueryResultDetails result_details { .query_id = context->getClientInfo().current_query_id, - .timezone = DateLUT::instance().getTimeZone(), + .timezone = DateLUT::instance("").getTimeZone(), }; std::unique_ptr compressed_buffer; diff --git a/src/Loggers/OwnPatternFormatter.cpp b/src/Loggers/OwnPatternFormatter.cpp index 02a2c2e510b..54d2b995d15 100644 --- a/src/Loggers/OwnPatternFormatter.cpp +++ b/src/Loggers/OwnPatternFormatter.cpp @@ -22,7 +22,7 @@ void OwnPatternFormatter::formatExtended(const DB::ExtendedLogMessage & msg_ext, const Poco::Message & msg = msg_ext.base; /// Change delimiters in date for compatibility with old logs. - DB::writeDateTimeText<'.', ':'>(msg_ext.time_seconds, wb); + DB::writeDateTimeText<'.', ':'>(msg_ext.time_seconds, wb, server_timezone); DB::writeChar('.', wb); DB::writeChar('0' + ((msg_ext.time_microseconds / 100000) % 10), wb); diff --git a/src/Loggers/OwnPatternFormatter.h b/src/Loggers/OwnPatternFormatter.h index d776b097cb2..07d0409b0ae 100644 --- a/src/Loggers/OwnPatternFormatter.h +++ b/src/Loggers/OwnPatternFormatter.h @@ -2,6 +2,7 @@ #include +#include #include "ExtendedLogChannel.h" @@ -30,5 +31,6 @@ public: virtual void formatExtended(const DB::ExtendedLogMessage & msg_ext, std::string & text) const; private: + const DateLUTImpl & server_timezone = DateLUT::instance(""); bool color; }; From f2fbf2d61e8ede663ba37065d8ea8fe9b430de3e Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 6 Mar 2023 02:52:05 +0100 Subject: [PATCH 013/515] tcp protocol modification (min revision to be updated) --- src/Client/ClientBase.cpp | 4 ++++ src/Client/Connection.cpp | 5 +++++ src/Client/IServerConnection.h | 2 ++ src/Client/MultiplexedConnections.cpp | 2 ++ src/Client/Suggest.cpp | 1 + src/Common/DateLUT.h | 1 + src/Core/Protocol.h | 4 +++- src/Core/ProtocolDefines.h | 2 ++ src/Interpreters/executeQuery.cpp | 2 +- src/Server/TCPHandler.cpp | 12 ++++++++++++ src/Server/TCPHandler.h | 1 + 11 files changed, 34 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 96aff9aa304..65d04a6bb9d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1020,6 +1020,10 @@ bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_) onProfileEvents(packet.block); return true; + case Protocol::Server::TimezoneUpdate: + DateLUT::setDefaultTimezone(packet.server_timezone); + return true; + default: throw Exception( ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from server {}", packet.type, connection->getDescription()); diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index eea007a8608..87e9e20e8f7 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -972,6 +972,11 @@ Packet Connection::receivePacket() res.block = receiveProfileEvents(); return res; + case Protocol::Server::TimezoneUpdate: + readStringBinary(server_timezone, *in); + res.server_timezone = server_timezone; + return res; + default: /// In unknown state, disconnect - to not leave unsynchronised connection. disconnect(); diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index cd4db8f5258..52382ff9d45 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -38,6 +38,8 @@ struct Packet ParallelReadRequest request; ParallelReadResponse response; + std::string server_timezone; + Packet() : type(Protocol::Server::Hello) {} }; diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index cc260353339..668833b2a84 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -258,6 +258,7 @@ Packet MultiplexedConnections::drain() switch (packet.type) { + case Protocol::Server::TimezoneUpdate: case Protocol::Server::MergeTreeAllRangesAnnounecement: case Protocol::Server::MergeTreeReadTaskRequest: case Protocol::Server::ReadTaskRequest: @@ -339,6 +340,7 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac switch (packet.type) { + case Protocol::Server::TimezoneUpdate: case Protocol::Server::MergeTreeAllRangesAnnounecement: case Protocol::Server::MergeTreeReadTaskRequest: case Protocol::Server::ReadTaskRequest: diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 7027f35d21a..4a29bead540 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -158,6 +158,7 @@ void Suggest::fetch(IServerConnection & connection, const ConnectionTimeouts & t fillWordsFromBlock(packet.block); continue; + case Protocol::Server::TimezoneUpdate: case Protocol::Server::Progress: case Protocol::Server::ProfileInfo: case Protocol::Server::Totals: diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index efbf56b59b2..f17fe772dbc 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -60,6 +60,7 @@ public: return date_lut.getImplementation(time_zone); } + static void setDefaultTimezone(const std::string & time_zone) { auto & date_lut = getInstance(); diff --git a/src/Core/Protocol.h b/src/Core/Protocol.h index 86c0a851c60..97a2831ffe8 100644 --- a/src/Core/Protocol.h +++ b/src/Core/Protocol.h @@ -83,7 +83,8 @@ namespace Protocol ProfileEvents = 14, /// Packet with profile events from server. MergeTreeAllRangesAnnounecement = 15, MergeTreeReadTaskRequest = 16, /// Request from a MergeTree replica to a coordinator - MAX = MergeTreeReadTaskRequest, + TimezoneUpdate = 17, /// Receive server's (session-wide) default timezone + MAX = TimezoneUpdate, }; @@ -111,6 +112,7 @@ namespace Protocol "ProfileEvents", "MergeTreeAllRangesAnnounecement", "MergeTreeReadTaskRequest", + "TimezoneUpdate", }; return packet <= MAX ? data[packet] diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 3bbfb95f020..e56ae0305cc 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -72,3 +72,5 @@ #define DBMS_MIN_PROTOCOL_VERSION_WITH_SERVER_QUERY_TIME_IN_PROGRESS 54460 #define DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES 54461 + +#define DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES 54461 diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index cda7ec2b0d3..85e623dc17d 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1280,7 +1280,7 @@ void executeQuery( QueryResultDetails result_details { .query_id = context->getClientInfo().current_query_id, - .timezone = DateLUT::instance("").getTimeZone(), + .timezone = DateLUT::instance().getTimeZone(), }; std::unique_ptr compressed_buffer; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a307b472a64..9bb11f34916 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -446,6 +446,7 @@ void TCPHandler::runImpl() sendSelectProfileEvents(); sendLogs(); + return false; }; @@ -483,6 +484,9 @@ void TCPHandler::runImpl() { std::lock_guard lock(task_callback_mutex); sendLogs(); + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES + && client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) + sendTimezone(); sendEndOfStream(); } @@ -1035,6 +1039,14 @@ void TCPHandler::sendInsertProfileEvents() sendProfileEvents(); } +void TCPHandler::sendTimezone() +{ + writeVarUInt(Protocol::Server::TimezoneUpdate, *out); + writeStringBinary(DateLUT::instance().getTimeZone(), *out); + out->next(); +} + + bool TCPHandler::receiveProxyHeader() { if (in->eof()) diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index f06b0b060b3..b19f908bc27 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -262,6 +262,7 @@ private: void sendProfileEvents(); void sendSelectProfileEvents(); void sendInsertProfileEvents(); + void sendTimezone(); /// Creates state.block_in/block_out for blocks read/write, depending on whether compression is enabled. void initBlockInput(); From 0706108b683ab5d67885b81a16b24a76c4d59513 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 6 Mar 2023 11:16:53 +0100 Subject: [PATCH 014/515] typo fix --- src/Interpreters/executeQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 85e623dc17d..435401796a0 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -49,7 +49,6 @@ #include #include #include -#include #include #include #include From 57c5a637217779fbcc999cbaa5bd965f8892d092 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 6 Mar 2023 16:39:47 +0100 Subject: [PATCH 015/515] fix receive of timezone update on processing --- src/Client/ClientBase.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 65d04a6bb9d..7ca6bbed6ba 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1188,6 +1188,10 @@ bool ClientBase::receiveSampleBlock(Block & out, ColumnsDescription & columns_de columns_description = ColumnsDescription::parse(packet.multistring_message[1]); return receiveSampleBlock(out, columns_description, parsed_query); + case Protocol::Server::TimezoneUpdate: + DateLUT::setDefaultTimezone(packet.server_timezone); + break; + default: throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER, "Unexpected packet from server (expected Data, Exception or Log, got {})", @@ -1533,6 +1537,10 @@ bool ClientBase::receiveEndOfQuery() onProfileEvents(packet.block); break; + case Protocol::Server::TimezoneUpdate: + DateLUT::setDefaultTimezone(packet.server_timezone); + break; + default: throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER, "Unexpected packet from server (expected Exception, EndOfStream, Log, Progress or ProfileEvents. Got {})", From d93937cc5e92ae4612259e9e57bca15489aabc8f Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 6 Mar 2023 16:45:38 +0100 Subject: [PATCH 016/515] increment protocol version --- src/Core/ProtocolDefines.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index e56ae0305cc..5483489d5c4 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -54,7 +54,7 @@ /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -#define DBMS_TCP_PROTOCOL_VERSION 54461 +#define DBMS_TCP_PROTOCOL_VERSION 54462 #define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 @@ -73,4 +73,4 @@ #define DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES 54461 -#define DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES 54461 +#define DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES 54462 From 3a918ae66a984451e0db0f56ffa6232b897ad62f Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 7 Mar 2023 02:33:46 +0100 Subject: [PATCH 017/515] revert protocol changes, found better way --- src/Client/ClientBase.cpp | 25 ++++++++++++------------- src/Client/Connection.cpp | 5 ----- src/Client/MultiplexedConnections.cpp | 2 -- src/Client/Suggest.cpp | 1 - src/Core/Protocol.h | 4 +--- src/Core/ProtocolDefines.h | 4 +--- src/Server/TCPHandler.cpp | 10 ---------- src/Server/TCPHandler.h | 1 - 8 files changed, 14 insertions(+), 38 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 7ca6bbed6ba..09c510f01f3 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1020,10 +1020,6 @@ bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_) onProfileEvents(packet.block); return true; - case Protocol::Server::TimezoneUpdate: - DateLUT::setDefaultTimezone(packet.server_timezone); - return true; - default: throw Exception( ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from server {}", packet.type, connection->getDescription()); @@ -1188,10 +1184,6 @@ bool ClientBase::receiveSampleBlock(Block & out, ColumnsDescription & columns_de columns_description = ColumnsDescription::parse(packet.multistring_message[1]); return receiveSampleBlock(out, columns_description, parsed_query); - case Protocol::Server::TimezoneUpdate: - DateLUT::setDefaultTimezone(packet.server_timezone); - break; - default: throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER, "Unexpected packet from server (expected Data, Exception or Log, got {})", @@ -1500,7 +1492,7 @@ void ClientBase::receiveLogsAndProfileEvents(ASTPtr parsed_query) { auto packet_type = connection->checkPacket(0); - while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents)) + while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents )) { receiveAndProcessPacket(parsed_query, false); packet_type = connection->checkPacket(0); @@ -1537,10 +1529,6 @@ bool ClientBase::receiveEndOfQuery() onProfileEvents(packet.block); break; - case Protocol::Server::TimezoneUpdate: - DateLUT::setDefaultTimezone(packet.server_timezone); - break; - default: throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER, "Unexpected packet from server (expected Exception, EndOfStream, Log, Progress or ProfileEvents. Got {})", @@ -1611,6 +1599,8 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin progress_indication.resetProgress(); profile_events.watch.restart(); + const std::string old_timezone = DateLUT::instance().getTimeZone(); + { /// Temporarily apply query settings to context. std::optional old_settings; @@ -1659,6 +1649,9 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin bool is_async_insert = global_context->getSettingsRef().async_insert && insert && insert->hasInlinedData(); + if (!global_context->getSettingsRef().timezone.toString().empty()) + DateLUT::setDefaultTimezone(global_context->getSettingsRef().timezone); + /// INSERT query for which data transfer is needed (not an INSERT SELECT or input()) is processed separately. if (insert && (!insert->select || input_function) && !insert->watch && !is_async_insert) { @@ -1693,6 +1686,10 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin query_parameters.insert_or_assign(name, value); global_context->addQueryParameters(set_query->query_parameters); + + if (!global_context->getSettingsRef().timezone.toString().empty()) + DateLUT::setDefaultTimezone(global_context->getSettingsRef().timezone); + } if (const auto * use_query = parsed_query->as()) { @@ -1703,6 +1700,8 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin connection->setDefaultDatabase(new_database); } } + else + DateLUT::setDefaultTimezone(old_timezone); /// Always print last block (if it was not printed already) if (profile_events.last_block) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 87e9e20e8f7..eea007a8608 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -972,11 +972,6 @@ Packet Connection::receivePacket() res.block = receiveProfileEvents(); return res; - case Protocol::Server::TimezoneUpdate: - readStringBinary(server_timezone, *in); - res.server_timezone = server_timezone; - return res; - default: /// In unknown state, disconnect - to not leave unsynchronised connection. disconnect(); diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index 668833b2a84..cc260353339 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -258,7 +258,6 @@ Packet MultiplexedConnections::drain() switch (packet.type) { - case Protocol::Server::TimezoneUpdate: case Protocol::Server::MergeTreeAllRangesAnnounecement: case Protocol::Server::MergeTreeReadTaskRequest: case Protocol::Server::ReadTaskRequest: @@ -340,7 +339,6 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac switch (packet.type) { - case Protocol::Server::TimezoneUpdate: case Protocol::Server::MergeTreeAllRangesAnnounecement: case Protocol::Server::MergeTreeReadTaskRequest: case Protocol::Server::ReadTaskRequest: diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 4a29bead540..7027f35d21a 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -158,7 +158,6 @@ void Suggest::fetch(IServerConnection & connection, const ConnectionTimeouts & t fillWordsFromBlock(packet.block); continue; - case Protocol::Server::TimezoneUpdate: case Protocol::Server::Progress: case Protocol::Server::ProfileInfo: case Protocol::Server::Totals: diff --git a/src/Core/Protocol.h b/src/Core/Protocol.h index 97a2831ffe8..86c0a851c60 100644 --- a/src/Core/Protocol.h +++ b/src/Core/Protocol.h @@ -83,8 +83,7 @@ namespace Protocol ProfileEvents = 14, /// Packet with profile events from server. MergeTreeAllRangesAnnounecement = 15, MergeTreeReadTaskRequest = 16, /// Request from a MergeTree replica to a coordinator - TimezoneUpdate = 17, /// Receive server's (session-wide) default timezone - MAX = TimezoneUpdate, + MAX = MergeTreeReadTaskRequest, }; @@ -112,7 +111,6 @@ namespace Protocol "ProfileEvents", "MergeTreeAllRangesAnnounecement", "MergeTreeReadTaskRequest", - "TimezoneUpdate", }; return packet <= MAX ? data[packet] diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 5483489d5c4..3bbfb95f020 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -54,7 +54,7 @@ /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -#define DBMS_TCP_PROTOCOL_VERSION 54462 +#define DBMS_TCP_PROTOCOL_VERSION 54461 #define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 @@ -72,5 +72,3 @@ #define DBMS_MIN_PROTOCOL_VERSION_WITH_SERVER_QUERY_TIME_IN_PROGRESS 54460 #define DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES 54461 - -#define DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES 54462 diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 9bb11f34916..617b084a149 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -484,9 +484,6 @@ void TCPHandler::runImpl() { std::lock_guard lock(task_callback_mutex); sendLogs(); - if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES - && client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) - sendTimezone(); sendEndOfStream(); } @@ -1039,13 +1036,6 @@ void TCPHandler::sendInsertProfileEvents() sendProfileEvents(); } -void TCPHandler::sendTimezone() -{ - writeVarUInt(Protocol::Server::TimezoneUpdate, *out); - writeStringBinary(DateLUT::instance().getTimeZone(), *out); - out->next(); -} - bool TCPHandler::receiveProxyHeader() { diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index b19f908bc27..f06b0b060b3 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -262,7 +262,6 @@ private: void sendProfileEvents(); void sendSelectProfileEvents(); void sendInsertProfileEvents(); - void sendTimezone(); /// Creates state.block_in/block_out for blocks read/write, depending on whether compression is enabled. void initBlockInput(); From c859478db3a3964c49457e49bab62bdf975bed7f Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 7 Mar 2023 02:36:02 +0100 Subject: [PATCH 018/515] upd --- src/Client/IServerConnection.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index 52382ff9d45..cd4db8f5258 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -38,8 +38,6 @@ struct Packet ParallelReadRequest request; ParallelReadResponse response; - std::string server_timezone; - Packet() : type(Protocol::Server::Hello) {} }; From 5e7a861e688dea04ecfe9c54d30c642f65a28569 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 7 Mar 2023 02:45:47 +0100 Subject: [PATCH 019/515] fix --- src/Server/TCPHandler.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 617b084a149..a307b472a64 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -446,7 +446,6 @@ void TCPHandler::runImpl() sendSelectProfileEvents(); sendLogs(); - return false; }; @@ -1036,7 +1035,6 @@ void TCPHandler::sendInsertProfileEvents() sendProfileEvents(); } - bool TCPHandler::receiveProxyHeader() { if (in->eof()) From a7a3c9d1a675743e776fde32c96ccd9bbfc94e46 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 7 Mar 2023 02:52:42 +0100 Subject: [PATCH 020/515] fix style --- src/Client/ClientBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 09c510f01f3..25442c89f99 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1492,7 +1492,7 @@ void ClientBase::receiveLogsAndProfileEvents(ASTPtr parsed_query) { auto packet_type = connection->checkPacket(0); - while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents )) + while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents)) { receiveAndProcessPacket(parsed_query, false); packet_type = connection->checkPacket(0); From e92501d5dd7a9f3a77ad38f8750432a2286e9f0b Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 7 Mar 2023 13:02:02 +0100 Subject: [PATCH 021/515] update tests + exception --- src/Client/ClientBase.cpp | 28 +++++++++++++++++-- ...rence => 02674_timezone_setting.reference} | 0 ...setting.sql => 02674_timezone_setting.sql} | 4 +-- 3 files changed, 27 insertions(+), 5 deletions(-) rename tests/queries/0_stateless/{02668_timezone_setting.reference => 02674_timezone_setting.reference} (100%) rename tests/queries/0_stateless/{02668_timezone_setting.sql => 02674_timezone_setting.sql} (73%) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 25442c89f99..13f28806066 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -107,6 +107,7 @@ namespace ErrorCodes extern const int UNRECOGNIZED_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_PARSE_DATETIME; } } @@ -1599,6 +1600,9 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin progress_indication.resetProgress(); profile_events.watch.restart(); + /// A query may contain timezone setting. To handle this, old client-wide tz is saved here. + /// If timezone was set for a query, after its execution client tz will be back to old one. + /// If it was a settings query, new setting will be applied to client. const std::string old_timezone = DateLUT::instance().getTimeZone(); { @@ -1649,8 +1653,18 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin bool is_async_insert = global_context->getSettingsRef().async_insert && insert && insert->hasInlinedData(); - if (!global_context->getSettingsRef().timezone.toString().empty()) - DateLUT::setDefaultTimezone(global_context->getSettingsRef().timezone); + /// pre-load timezone from (query) settings -- new timezone may also be specified in query. + try + { + if (!global_context->getSettingsRef().timezone.toString().empty()) + DateLUT::setDefaultTimezone(global_context->getSettingsRef().timezone); + } + catch (Poco::Exception &) + { + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, + "Invalid time zone {} in client settings. Use `SET timezone = \'New/TZ\'` to set a proper timezone.", + global_context->getSettingsRef().timezone.toString()); + } /// INSERT query for which data transfer is needed (not an INSERT SELECT or input()) is processed separately. if (insert && (!insert->select || input_function) && !insert->watch && !is_async_insert) @@ -1687,9 +1701,17 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin global_context->addQueryParameters(set_query->query_parameters); + try + { if (!global_context->getSettingsRef().timezone.toString().empty()) DateLUT::setDefaultTimezone(global_context->getSettingsRef().timezone); - + } + catch (Poco::Exception &) + { + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, + "Invalid time zone {} in client settings. Use `SET timezone = \'New/TZ\'` to set a proper timezone.", + global_context->getSettingsRef().timezone.toString()); + } } if (const auto * use_query = parsed_query->as()) { diff --git a/tests/queries/0_stateless/02668_timezone_setting.reference b/tests/queries/0_stateless/02674_timezone_setting.reference similarity index 100% rename from tests/queries/0_stateless/02668_timezone_setting.reference rename to tests/queries/0_stateless/02674_timezone_setting.reference diff --git a/tests/queries/0_stateless/02668_timezone_setting.sql b/tests/queries/0_stateless/02674_timezone_setting.sql similarity index 73% rename from tests/queries/0_stateless/02668_timezone_setting.sql rename to tests/queries/0_stateless/02674_timezone_setting.sql index f331ab58307..51820fc2dca 100644 --- a/tests/queries/0_stateless/02668_timezone_setting.sql +++ b/tests/queries/0_stateless/02674_timezone_setting.sql @@ -5,5 +5,5 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric SET timezone = 'Europe/Zurich'; SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); -SET timezone = 'Абырвалг'; -select now(); -- { serverError POCO_EXCEPTION } \ No newline at end of file +SET timezone = 'Абырвалг'; -- { clientError CANNOT_PARSE_DATETIME } +select now(); -- { clientError CANNOT_PARSE_DATETIME } \ No newline at end of file From 1fd6e3f23b41dac6fde5b238e1a5da11a976b5ae Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 7 Mar 2023 16:02:30 +0100 Subject: [PATCH 022/515] Revert "fix style" This reverts commit a7a3c9d1a675743e776fde32c96ccd9bbfc94e46. --- src/Client/ClientBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 13f28806066..cfef1a5d3fe 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1493,7 +1493,7 @@ void ClientBase::receiveLogsAndProfileEvents(ASTPtr parsed_query) { auto packet_type = connection->checkPacket(0); - while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents)) + while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents )) { receiveAndProcessPacket(parsed_query, false); packet_type = connection->checkPacket(0); From 1ce697d8c06ce7f44e078f9b8809dcaa3e3ba8f8 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 7 Mar 2023 16:05:23 +0100 Subject: [PATCH 023/515] Revert "revert protocol changes, found better way" This reverts commit 3a918ae66a984451e0db0f56ffa6232b897ad62f. --- src/Client/ClientBase.cpp | 47 +++++-------------- src/Client/Connection.cpp | 5 ++ src/Client/IServerConnection.h | 2 + src/Client/MultiplexedConnections.cpp | 2 + src/Client/Suggest.cpp | 1 + src/Core/Protocol.h | 4 +- src/Core/ProtocolDefines.h | 4 +- src/Server/TCPHandler.cpp | 12 +++++ src/Server/TCPHandler.h | 1 + ...rence => 02668_timezone_setting.reference} | 0 ...setting.sql => 02668_timezone_setting.sql} | 4 +- 11 files changed, 44 insertions(+), 38 deletions(-) rename tests/queries/0_stateless/{02674_timezone_setting.reference => 02668_timezone_setting.reference} (100%) rename tests/queries/0_stateless/{02674_timezone_setting.sql => 02668_timezone_setting.sql} (73%) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index cfef1a5d3fe..7ca6bbed6ba 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -107,7 +107,6 @@ namespace ErrorCodes extern const int UNRECOGNIZED_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int CANNOT_OPEN_FILE; - extern const int CANNOT_PARSE_DATETIME; } } @@ -1021,6 +1020,10 @@ bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_) onProfileEvents(packet.block); return true; + case Protocol::Server::TimezoneUpdate: + DateLUT::setDefaultTimezone(packet.server_timezone); + return true; + default: throw Exception( ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from server {}", packet.type, connection->getDescription()); @@ -1185,6 +1188,10 @@ bool ClientBase::receiveSampleBlock(Block & out, ColumnsDescription & columns_de columns_description = ColumnsDescription::parse(packet.multistring_message[1]); return receiveSampleBlock(out, columns_description, parsed_query); + case Protocol::Server::TimezoneUpdate: + DateLUT::setDefaultTimezone(packet.server_timezone); + break; + default: throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER, "Unexpected packet from server (expected Data, Exception or Log, got {})", @@ -1493,7 +1500,7 @@ void ClientBase::receiveLogsAndProfileEvents(ASTPtr parsed_query) { auto packet_type = connection->checkPacket(0); - while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents )) + while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents)) { receiveAndProcessPacket(parsed_query, false); packet_type = connection->checkPacket(0); @@ -1530,6 +1537,10 @@ bool ClientBase::receiveEndOfQuery() onProfileEvents(packet.block); break; + case Protocol::Server::TimezoneUpdate: + DateLUT::setDefaultTimezone(packet.server_timezone); + break; + default: throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER, "Unexpected packet from server (expected Exception, EndOfStream, Log, Progress or ProfileEvents. Got {})", @@ -1600,11 +1611,6 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin progress_indication.resetProgress(); profile_events.watch.restart(); - /// A query may contain timezone setting. To handle this, old client-wide tz is saved here. - /// If timezone was set for a query, after its execution client tz will be back to old one. - /// If it was a settings query, new setting will be applied to client. - const std::string old_timezone = DateLUT::instance().getTimeZone(); - { /// Temporarily apply query settings to context. std::optional old_settings; @@ -1653,19 +1659,6 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin bool is_async_insert = global_context->getSettingsRef().async_insert && insert && insert->hasInlinedData(); - /// pre-load timezone from (query) settings -- new timezone may also be specified in query. - try - { - if (!global_context->getSettingsRef().timezone.toString().empty()) - DateLUT::setDefaultTimezone(global_context->getSettingsRef().timezone); - } - catch (Poco::Exception &) - { - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, - "Invalid time zone {} in client settings. Use `SET timezone = \'New/TZ\'` to set a proper timezone.", - global_context->getSettingsRef().timezone.toString()); - } - /// INSERT query for which data transfer is needed (not an INSERT SELECT or input()) is processed separately. if (insert && (!insert->select || input_function) && !insert->watch && !is_async_insert) { @@ -1700,18 +1693,6 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin query_parameters.insert_or_assign(name, value); global_context->addQueryParameters(set_query->query_parameters); - - try - { - if (!global_context->getSettingsRef().timezone.toString().empty()) - DateLUT::setDefaultTimezone(global_context->getSettingsRef().timezone); - } - catch (Poco::Exception &) - { - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, - "Invalid time zone {} in client settings. Use `SET timezone = \'New/TZ\'` to set a proper timezone.", - global_context->getSettingsRef().timezone.toString()); - } } if (const auto * use_query = parsed_query->as()) { @@ -1722,8 +1703,6 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin connection->setDefaultDatabase(new_database); } } - else - DateLUT::setDefaultTimezone(old_timezone); /// Always print last block (if it was not printed already) if (profile_events.last_block) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index eea007a8608..87e9e20e8f7 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -972,6 +972,11 @@ Packet Connection::receivePacket() res.block = receiveProfileEvents(); return res; + case Protocol::Server::TimezoneUpdate: + readStringBinary(server_timezone, *in); + res.server_timezone = server_timezone; + return res; + default: /// In unknown state, disconnect - to not leave unsynchronised connection. disconnect(); diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index cd4db8f5258..52382ff9d45 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -38,6 +38,8 @@ struct Packet ParallelReadRequest request; ParallelReadResponse response; + std::string server_timezone; + Packet() : type(Protocol::Server::Hello) {} }; diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index cc260353339..668833b2a84 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -258,6 +258,7 @@ Packet MultiplexedConnections::drain() switch (packet.type) { + case Protocol::Server::TimezoneUpdate: case Protocol::Server::MergeTreeAllRangesAnnounecement: case Protocol::Server::MergeTreeReadTaskRequest: case Protocol::Server::ReadTaskRequest: @@ -339,6 +340,7 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac switch (packet.type) { + case Protocol::Server::TimezoneUpdate: case Protocol::Server::MergeTreeAllRangesAnnounecement: case Protocol::Server::MergeTreeReadTaskRequest: case Protocol::Server::ReadTaskRequest: diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 7027f35d21a..4a29bead540 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -158,6 +158,7 @@ void Suggest::fetch(IServerConnection & connection, const ConnectionTimeouts & t fillWordsFromBlock(packet.block); continue; + case Protocol::Server::TimezoneUpdate: case Protocol::Server::Progress: case Protocol::Server::ProfileInfo: case Protocol::Server::Totals: diff --git a/src/Core/Protocol.h b/src/Core/Protocol.h index 86c0a851c60..97a2831ffe8 100644 --- a/src/Core/Protocol.h +++ b/src/Core/Protocol.h @@ -83,7 +83,8 @@ namespace Protocol ProfileEvents = 14, /// Packet with profile events from server. MergeTreeAllRangesAnnounecement = 15, MergeTreeReadTaskRequest = 16, /// Request from a MergeTree replica to a coordinator - MAX = MergeTreeReadTaskRequest, + TimezoneUpdate = 17, /// Receive server's (session-wide) default timezone + MAX = TimezoneUpdate, }; @@ -111,6 +112,7 @@ namespace Protocol "ProfileEvents", "MergeTreeAllRangesAnnounecement", "MergeTreeReadTaskRequest", + "TimezoneUpdate", }; return packet <= MAX ? data[packet] diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 3bbfb95f020..5483489d5c4 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -54,7 +54,7 @@ /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -#define DBMS_TCP_PROTOCOL_VERSION 54461 +#define DBMS_TCP_PROTOCOL_VERSION 54462 #define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 @@ -72,3 +72,5 @@ #define DBMS_MIN_PROTOCOL_VERSION_WITH_SERVER_QUERY_TIME_IN_PROGRESS 54460 #define DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES 54461 + +#define DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES 54462 diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a307b472a64..9bb11f34916 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -446,6 +446,7 @@ void TCPHandler::runImpl() sendSelectProfileEvents(); sendLogs(); + return false; }; @@ -483,6 +484,9 @@ void TCPHandler::runImpl() { std::lock_guard lock(task_callback_mutex); sendLogs(); + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES + && client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) + sendTimezone(); sendEndOfStream(); } @@ -1035,6 +1039,14 @@ void TCPHandler::sendInsertProfileEvents() sendProfileEvents(); } +void TCPHandler::sendTimezone() +{ + writeVarUInt(Protocol::Server::TimezoneUpdate, *out); + writeStringBinary(DateLUT::instance().getTimeZone(), *out); + out->next(); +} + + bool TCPHandler::receiveProxyHeader() { if (in->eof()) diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index f06b0b060b3..b19f908bc27 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -262,6 +262,7 @@ private: void sendProfileEvents(); void sendSelectProfileEvents(); void sendInsertProfileEvents(); + void sendTimezone(); /// Creates state.block_in/block_out for blocks read/write, depending on whether compression is enabled. void initBlockInput(); diff --git a/tests/queries/0_stateless/02674_timezone_setting.reference b/tests/queries/0_stateless/02668_timezone_setting.reference similarity index 100% rename from tests/queries/0_stateless/02674_timezone_setting.reference rename to tests/queries/0_stateless/02668_timezone_setting.reference diff --git a/tests/queries/0_stateless/02674_timezone_setting.sql b/tests/queries/0_stateless/02668_timezone_setting.sql similarity index 73% rename from tests/queries/0_stateless/02674_timezone_setting.sql rename to tests/queries/0_stateless/02668_timezone_setting.sql index 51820fc2dca..f331ab58307 100644 --- a/tests/queries/0_stateless/02674_timezone_setting.sql +++ b/tests/queries/0_stateless/02668_timezone_setting.sql @@ -5,5 +5,5 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric SET timezone = 'Europe/Zurich'; SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); -SET timezone = 'Абырвалг'; -- { clientError CANNOT_PARSE_DATETIME } -select now(); -- { clientError CANNOT_PARSE_DATETIME } \ No newline at end of file +SET timezone = 'Абырвалг'; +select now(); -- { serverError POCO_EXCEPTION } \ No newline at end of file From bbb31cf8913527eb21823216821ca536c2779563 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 15 Mar 2023 18:37:23 +0100 Subject: [PATCH 024/515] added validation on setting modification --- src/Client/ClientBase.cpp | 17 ++++++++++++----- src/Client/ClientBase.h | 1 + src/Core/Settings.h | 2 +- src/Core/SettingsFields.cpp | 11 +++++++++++ src/Core/SettingsFields.h | 38 +++++++++++++++++++++++++++++++++++++ src/Server/TCPHandler.cpp | 17 +++++++++++++---- 6 files changed, 76 insertions(+), 10 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 7ca6bbed6ba..5b086d675ba 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1021,7 +1021,7 @@ bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_) return true; case Protocol::Server::TimezoneUpdate: - DateLUT::setDefaultTimezone(packet.server_timezone); + onTimezoneUpdate(packet.server_timezone); return true; default: @@ -1046,6 +1046,11 @@ void ClientBase::onProgress(const Progress & value) progress_indication.writeProgress(*tty_buf); } +void ClientBase::onTimezoneUpdate(const String & tz) +{ + DateLUT::setDefaultTimezone(tz); +} + void ClientBase::onEndOfStream() { @@ -1189,12 +1194,12 @@ bool ClientBase::receiveSampleBlock(Block & out, ColumnsDescription & columns_de return receiveSampleBlock(out, columns_description, parsed_query); case Protocol::Server::TimezoneUpdate: - DateLUT::setDefaultTimezone(packet.server_timezone); + onTimezoneUpdate(packet.server_timezone); break; default: throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER, - "Unexpected packet from server (expected Data, Exception or Log, got {})", + "Unexpected packet from server (expected Data, Exception, Log or TimezoneUpdate, got {})", String(Protocol::Server::toString(packet.type))); } } @@ -1500,7 +1505,9 @@ void ClientBase::receiveLogsAndProfileEvents(ASTPtr parsed_query) { auto packet_type = connection->checkPacket(0); - while (packet_type && (*packet_type == Protocol::Server::Log || *packet_type == Protocol::Server::ProfileEvents)) + while (packet_type && (*packet_type == Protocol::Server::Log || + *packet_type == Protocol::Server::ProfileEvents || + *packet_type == Protocol::Server::TimezoneUpdate)) { receiveAndProcessPacket(parsed_query, false); packet_type = connection->checkPacket(0); @@ -1538,7 +1545,7 @@ bool ClientBase::receiveEndOfQuery() break; case Protocol::Server::TimezoneUpdate: - DateLUT::setDefaultTimezone(packet.server_timezone); + onTimezoneUpdate(packet.server_timezone); break; default: diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 52e15a1a075..18d9a30cac0 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -139,6 +139,7 @@ private: void cancelQuery(); void onProgress(const Progress & value); + void onTimezoneUpdate(const String & tz); void onData(Block & block, ASTPtr parsed_query); void onLogData(Block & block); void onTotals(Block & block, ASTPtr parsed_query); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e508818a26a..ced59219a5b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -712,7 +712,7 @@ class IColumn; M(Float, insert_keeper_fault_injection_probability, 0.0f, "Approximate probability of failure for a keeper request during insert. Valid value is in interval [0.0f, 1.0f]", 0) \ M(UInt64, insert_keeper_fault_injection_seed, 0, "0 - random seed, otherwise the setting value", 0) \ M(Bool, force_aggregation_in_order, false, "Force use of aggregation in order on remote nodes during distributed aggregation. PLEASE, NEVER CHANGE THIS SETTING VALUE MANUALLY!", IMPORTANT) \ - M(String, timezone, "", "Use specified timezone for interpreting Date and DateTime instead of server's timezone.", 0) \ + M(Timezone, timezone, "", "Use specified timezone for interpreting Date and DateTime instead of server's timezone.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 4164bf1e27e..44369c7c8a0 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -445,6 +445,17 @@ String SettingFieldEnumHelpers::readBinary(ReadBuffer & in) return str; } +void SettingFieldTimezone::writeBinary(WriteBuffer & out) const +{ + writeStringBinary(value, out); +} + +void SettingFieldTimezone::readBinary(ReadBuffer & in) +{ + String str; + readStringBinary(str, in); + *this = std::move(str); +} String SettingFieldCustom::toString() const { diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index c6fe46c9f6b..b580122d3db 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -539,6 +540,43 @@ void SettingFieldMultiEnum::readBinary(ReadBuffer & in) return std::initializer_list> __VA_ARGS__ .size();\ } +/* Setting field for specifying user-defined timezone. It is basically a string, but it needs validation. + */ +struct SettingFieldTimezone +{ + String value; + bool changed = false; + + explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(std::string(str)); value = str; } + explicit SettingFieldTimezone(const String & str) { validateTimezone(str); value = str; } + explicit SettingFieldTimezone(String && str) { validateTimezone(std::string(str)); value = std::move(str); } + explicit SettingFieldTimezone(const char * str) { validateTimezone(str); value = str; } + explicit SettingFieldTimezone(const Field & f) { const String & str = f.safeGet(); validateTimezone(str); value = str; } + + SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(std::string(str)); value = str; changed = true; return *this; } + SettingFieldTimezone & operator =(const String & str) { *this = std::string_view{str}; return *this; } + SettingFieldTimezone & operator =(String && str) { validateTimezone(str); value = std::move(str); changed = true; return *this; } + SettingFieldTimezone & operator =(const char * str) { *this = std::string_view{str}; return *this; } + SettingFieldTimezone & operator =(const Field & f) { *this = f.safeGet(); return *this; } + + operator const String &() const { return value; } /// NOLINT + explicit operator Field() const { return value; } + + const String & toString() const { return value; } + void parseFromString(const String & str) { *this = str; } + + void writeBinary(WriteBuffer & out) const; + void readBinary(ReadBuffer & in); + +private: + cctz::time_zone validated_tz; + void validateTimezone(const std::string & str) + { + if (str != "" && !cctz::load_time_zone(str, &validated_tz)) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); + } +}; + /// Can keep a value of any type. Used for user-defined settings. struct SettingFieldCustom { diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 9bb11f34916..f43982c5133 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -442,6 +442,9 @@ void TCPHandler::runImpl() if (isQueryCancelled()) return true; + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES + && client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) + sendTimezone(); sendProgress(); sendSelectProfileEvents(); sendLogs(); @@ -483,10 +486,10 @@ void TCPHandler::runImpl() { std::lock_guard lock(task_callback_mutex); - sendLogs(); if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES && client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) sendTimezone(); + sendLogs(); sendEndOfStream(); } @@ -1041,9 +1044,15 @@ void TCPHandler::sendInsertProfileEvents() void TCPHandler::sendTimezone() { - writeVarUInt(Protocol::Server::TimezoneUpdate, *out); - writeStringBinary(DateLUT::instance().getTimeZone(), *out); - out->next(); +// const String & tz = CurrentThread::get().getQueryContext()->getSettingsRef().timezone.toString(); + const String & tz = query_context->getSettingsRef().timezone.toString(); + if (!tz.empty()) + { + LOG_DEBUG(log, "Sent timezone: {}", tz); + writeVarUInt(Protocol::Server::TimezoneUpdate, *out); + writeStringBinary(tz, *out); + out->next(); + } } From e36addb96a7eaaba8f9a90383d3e77020a1a61e8 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 11 Apr 2023 13:03:03 +0200 Subject: [PATCH 025/515] Hackish way of setting up timezone on the client Warning: lots of debug logging --- programs/client/Client.cpp | 12 ++++- src/Client/ClientBase.cpp | 47 +++++++++++++++++++ src/Functions/timezoneOf.cpp | 13 +++++ src/Interpreters/Context.cpp | 9 ++++ src/Server/TCPHandler.cpp | 21 ++++++--- .../0_stateless/02668_timezone_setting.sql | 8 ++-- 6 files changed, 99 insertions(+), 11 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 660b8d7c00a..2aa75e60294 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -4,8 +4,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -307,7 +309,7 @@ int Client::main(const std::vector & /*args*/) try { UseSSL use_ssl; - MainThreadStatus::getInstance(); + auto & thread_status = MainThreadStatus::getInstance(); setupSignalHandler(); std::cout << std::fixed << std::setprecision(3); @@ -320,6 +322,14 @@ try processConfig(); initTtyBuffer(toProgressOption(config().getString("progress", "default"))); + { + // All that just to set DB::CurrentThread::get().getGlobalContext() + // which is required for client timezone (pushed as from server) to work. + auto thread_group = std::make_shared(); + thread_group->global_context = global_context; + thread_status.attachQuery(thread_group, false); + } + /// Includes delayed_interactive. if (is_interactive) { diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index ca32b9b97d7..e3e0364523a 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -65,6 +65,7 @@ #include #include +#include #include #include #include @@ -73,11 +74,44 @@ #include "config_version.h" #include "config.h" +#include + namespace fs = std::filesystem; using namespace std::literals; +namespace +{ +using namespace DB; +using ContetGetterFunc = std::function const; +const void* getContextPtrOrNull(ContetGetterFunc contextFunc) +{ + try + { + return contextFunc().get(); + } + catch(...) + { + } + return nullptr; +} + +void LogContextes(const std::string_view scope, const ContextPtr global_context) +{ + const auto * context = global_context.get(); + std::cerr << scope << " contextes" + << "\n\tglobal: " << reinterpret_cast(context) + << "\n\tsession: " << getContextPtrOrNull([&]() { return context ? context->getSessionContext() : nullptr; }) + << "\n\tquery: " << getContextPtrOrNull([&]() { return context ? context->getQueryContext() : nullptr; }) + << "\n\tcurrent T query: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getQueryContext(); }) + << "\n\tcurrent T global: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getGlobalContext(); }) +// << "\n\tbuffer: " << getContextPtrOrNull(context, &Context::getBufferContext) + << std::endl; +} + +} + namespace CurrentMetrics { extern const Metric MemoryTracking; @@ -438,7 +472,12 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) /// output_format, do not output it. /// Also do not output too much data if we're fuzzing. if (block.rows() == 0 || (query_fuzzer_runs != 0 && processed_rows >= 100)) + { + LogContextes("ClientBase::onData header", global_context); return; + } + + LogContextes("ClientBase::onData DATA block", global_context); /// If results are written INTO OUTFILE, we can avoid clearing progress to avoid flicker. if (need_render_progress && tty_buf && (!select_into_file || select_into_file_and_stdout)) @@ -1048,7 +1087,15 @@ void ClientBase::onProgress(const Progress & value) void ClientBase::onTimezoneUpdate(const String & tz) { + std::cerr << "ClientBase::onTimezoneUpdate received new TZ from server: " << tz << std::endl; DateLUT::setDefaultTimezone(tz); + + Settings settings; + settings.timezone = tz; + global_context->applySettingsChanges(settings.changes()); +// DB::CurrentThread::get().getQueryContext()->applySettingsChanges(settings.changes()); + + LogContextes("ClientBase::onTimezoneUpdate", global_context); } diff --git a/src/Functions/timezoneOf.cpp b/src/Functions/timezoneOf.cpp index 6454b1cd735..ce419b7b4cd 100644 --- a/src/Functions/timezoneOf.cpp +++ b/src/Functions/timezoneOf.cpp @@ -5,7 +5,11 @@ #include #include #include +#include "Poco/Logger.h" +#include +#include +#include namespace DB { @@ -52,6 +56,15 @@ public: { DataTypePtr type_no_nullable = removeNullable(arguments[0].type); + { + const auto query_context = DB::CurrentThread::get().getQueryContext(); + + LOG_DEBUG(&Poco::Logger::get("Function timezoneOf"), "query context: {}, timezone: {} ({})", + reinterpret_cast(query_context.get()), + query_context->getSettingsRef().timezone.toString(), + (query_context->getSettingsRef().timezone.changed ? "changed" : "UNCHANGED")); + } + return DataTypeString().createColumnConst(input_rows_count, dynamic_cast(*type_no_nullable).getTimeZone().getTimeZone()); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index cf1d5203bf7..e27889702c5 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -118,6 +118,8 @@ #include #endif +#include + namespace fs = std::filesystem; namespace ProfileEvents @@ -1543,6 +1545,13 @@ void Context::applySettingChange(const SettingChange & change) void Context::applySettingsChanges(const SettingsChanges & changes) { auto lock = getLock(); + LOG_DEBUG(shared->log, "Context::applySettingsChanges {} applying settings changes: {}", reinterpret_cast(this), + fmt::join(std::ranges::transform_view(changes, + [](const SettingChange & change) + { + return change.name + ": " + change.value.dump(); + }), ", ")); + for (const SettingChange & change : changes) applySettingChange(change); applySettingsQuirks(settings); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index ef4bf81a5c1..4d5402d65d5 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -345,6 +345,7 @@ void TCPHandler::runImpl() /// Send block to the client - input storage structure. state.input_header = metadata_snapshot->getSampleBlock(); sendData(state.input_header); + sendTimezone(); }); query_context->setInputBlocksReaderCallback([this] (ContextPtr context) -> Block @@ -452,9 +453,7 @@ void TCPHandler::runImpl() if (isQueryCancelled()) return true; - if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES - && client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) - sendTimezone(); +// sendTimezone(); sendProgress(); sendSelectProfileEvents(); sendLogs(); @@ -496,9 +495,7 @@ void TCPHandler::runImpl() { std::lock_guard lock(task_callback_mutex); - if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES - && client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) - sendTimezone(); +// sendTimezone(); sendLogs(); sendEndOfStream(); } @@ -764,7 +761,7 @@ void TCPHandler::processInsertQuery() /// Send block to the client - table structure. sendData(executor.getHeader()); - + sendTimezone(); sendLogs(); while (readDataNext()) @@ -809,6 +806,7 @@ void TCPHandler::processOrdinaryQueryWithProcessors() { std::lock_guard lock(task_callback_mutex); sendData(header); + sendTimezone(); } } @@ -1061,7 +1059,16 @@ void TCPHandler::sendInsertProfileEvents() void TCPHandler::sendTimezone() { +// if (client_tcp_protocol_version <= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES +// || client_tcp_protocol_version <= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) +// return; + // const String & tz = CurrentThread::get().getQueryContext()->getSettingsRef().timezone.toString(); + LOG_DEBUG(log, "TCPHandler::sendTimezone() query context: {}, timezone: {} ({})", + reinterpret_cast(query_context.get()), + query_context->getSettingsRef().timezone.toString(), + (query_context->getSettingsRef().timezone.changed ? "changed" : "UNCHANGED")); + const String & tz = query_context->getSettingsRef().timezone.toString(); if (!tz.empty()) { diff --git a/tests/queries/0_stateless/02668_timezone_setting.sql b/tests/queries/0_stateless/02668_timezone_setting.sql index f331ab58307..d85efaa8a39 100644 --- a/tests/queries/0_stateless/02668_timezone_setting.sql +++ b/tests/queries/0_stateless/02668_timezone_setting.sql @@ -1,9 +1,11 @@ +SET timezone = 'Абырвалг'; -- { serverError BAD_ARGUMENTS} + SET timezone = 'Asia/Novosibirsk'; SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich'); SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'Europe/Zurich'; -SET timezone = 'Europe/Zurich'; +SET timezone = 'Asia/Manila'; SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); -SET timezone = 'Абырвалг'; -select now(); -- { serverError POCO_EXCEPTION } \ No newline at end of file +SELECT timezone(), serverTimeZone(), timezoneOf(now()) SETTINGS timezone = 'Europe/Zurich'; +SELECT timezone(), serverTimeZone(), timezoneOf(now()) SETTINGS timezone = 'Pacific/Pitcairn'; From 5d18343fb8ac1b0cae8085a660b8c995b9e33ea2 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 00:15:07 +0200 Subject: [PATCH 026/515] fixed delay --- src/Client/ClientBase.cpp | 36 +++++++++++++++++------------------- src/Server/TCPHandler.cpp | 12 ++++++------ 2 files changed, 23 insertions(+), 25 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index e3e0364523a..8da4ac200d9 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -65,7 +65,7 @@ #include #include -#include +//#include #include #include #include @@ -74,7 +74,7 @@ #include "config_version.h" #include "config.h" -#include +//#include namespace fs = std::filesystem; @@ -97,18 +97,18 @@ const void* getContextPtrOrNull(ContetGetterFunc contextFunc) return nullptr; } -void LogContextes(const std::string_view scope, const ContextPtr global_context) -{ - const auto * context = global_context.get(); - std::cerr << scope << " contextes" - << "\n\tglobal: " << reinterpret_cast(context) - << "\n\tsession: " << getContextPtrOrNull([&]() { return context ? context->getSessionContext() : nullptr; }) - << "\n\tquery: " << getContextPtrOrNull([&]() { return context ? context->getQueryContext() : nullptr; }) - << "\n\tcurrent T query: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getQueryContext(); }) - << "\n\tcurrent T global: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getGlobalContext(); }) -// << "\n\tbuffer: " << getContextPtrOrNull(context, &Context::getBufferContext) - << std::endl; -} +//void LogContextes(const std::string_view scope, const ContextPtr global_context) +//{ +// const auto * context = global_context.get(); +// std::cerr << scope << " contextes" +// << "\n\tglobal: " << reinterpret_cast(context) +// << "\n\tsession: " << getContextPtrOrNull([&]() { return context ? context->getSessionContext() : nullptr; }) +// << "\n\tquery: " << getContextPtrOrNull([&]() { return context ? context->getQueryContext() : nullptr; }) +// << "\n\tcurrent T query: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getQueryContext(); }) +// << "\n\tcurrent T global: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getGlobalContext(); }) +//// << "\n\tbuffer: " << getContextPtrOrNull(context, &Context::getBufferContext) +// << std::endl; +//} } @@ -473,11 +473,11 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) /// Also do not output too much data if we're fuzzing. if (block.rows() == 0 || (query_fuzzer_runs != 0 && processed_rows >= 100)) { - LogContextes("ClientBase::onData header", global_context); +// LogContextes("ClientBase::onData header", global_context); return; } - LogContextes("ClientBase::onData DATA block", global_context); +// LogContextes("ClientBase::onData DATA block", global_context); /// If results are written INTO OUTFILE, we can avoid clearing progress to avoid flicker. if (need_render_progress && tty_buf && (!select_into_file || select_into_file_and_stdout)) @@ -1088,14 +1088,12 @@ void ClientBase::onProgress(const Progress & value) void ClientBase::onTimezoneUpdate(const String & tz) { std::cerr << "ClientBase::onTimezoneUpdate received new TZ from server: " << tz << std::endl; - DateLUT::setDefaultTimezone(tz); Settings settings; settings.timezone = tz; global_context->applySettingsChanges(settings.changes()); -// DB::CurrentThread::get().getQueryContext()->applySettingsChanges(settings.changes()); - LogContextes("ClientBase::onTimezoneUpdate", global_context); +// LogContextes("ClientBase::onTimezoneUpdate", global_context); } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 4d5402d65d5..6ff7acf025a 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -453,7 +453,7 @@ void TCPHandler::runImpl() if (isQueryCancelled()) return true; -// sendTimezone(); + sendTimezone(); sendProgress(); sendSelectProfileEvents(); sendLogs(); @@ -495,7 +495,7 @@ void TCPHandler::runImpl() { std::lock_guard lock(task_callback_mutex); -// sendTimezone(); + sendTimezone(); sendLogs(); sendEndOfStream(); } @@ -1069,14 +1069,14 @@ void TCPHandler::sendTimezone() query_context->getSettingsRef().timezone.toString(), (query_context->getSettingsRef().timezone.changed ? "changed" : "UNCHANGED")); - const String & tz = query_context->getSettingsRef().timezone.toString(); - if (!tz.empty()) - { + const String & tz = CurrentThread::get().getQueryContext()->getSettingsRef().timezone.toString(); +// if (!tz.empty()) +// { LOG_DEBUG(log, "Sent timezone: {}", tz); writeVarUInt(Protocol::Server::TimezoneUpdate, *out); writeStringBinary(tz, *out); out->next(); - } +// } } From a9499eed794731a3fed2305e4d5f0e3607815816 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 12:47:05 +0200 Subject: [PATCH 027/515] moved getting server TZ DateLUT to separate place, upd tests and fix --- programs/copier/ClusterCopierApp.cpp | 2 +- programs/keeper/Keeper.cpp | 4 +-- programs/obfuscator/Obfuscator.cpp | 2 +- programs/server/Server.cpp | 4 +-- src/Client/ClientBase.cpp | 26 +++++++++---------- src/Common/DateLUT.h | 7 +++++ src/Daemon/BaseDaemon.cpp | 2 +- src/Functions/serverConstants.cpp | 2 +- src/Loggers/OwnPatternFormatter.h | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- .../MergeTree/MergeFromLogEntryTask.cpp | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 6 ++--- .../MergeTree/MergeTreeMutationEntry.cpp | 2 +- src/Storages/MergeTree/MergeTreePartInfo.cpp | 4 +-- src/Storages/MergeTree/MergeTreePartition.cpp | 4 +-- .../MergeTree/ReplicatedMergeTreeLogEntry.cpp | 2 +- .../ReplicatedMergeTreeMutationEntry.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 2 +- .../02668_timezone_setting.reference | 3 --- .../0_stateless/02668_timezone_setting.sql | 11 -------- .../02681_timezone_setting.reference | 5 ++++ .../0_stateless/02681_timezone_setting.sql | 11 ++++++++ 23 files changed, 59 insertions(+), 50 deletions(-) delete mode 100644 tests/queries/0_stateless/02668_timezone_setting.reference delete mode 100644 tests/queries/0_stateless/02668_timezone_setting.sql create mode 100644 tests/queries/0_stateless/02681_timezone_setting.reference create mode 100644 tests/queries/0_stateless/02681_timezone_setting.sql diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index b2994b90e23..92657f81c2a 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -43,7 +43,7 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self) time_t timestamp = Poco::Timestamp().epochTime(); auto curr_pid = Poco::Process::id(); - process_id = std::to_string(DateLUT::instance().toNumYYYYMMDDhhmmss(timestamp)) + "_" + std::to_string(curr_pid); + process_id = std::to_string(DateLUT::serverTimezoneInstance().toNumYYYYMMDDhhmmss(timestamp)) + "_" + std::to_string(curr_pid); host_id = escapeForFileName(getFQDNOrHostName()) + '#' + process_id; process_path = fs::weakly_canonical(fs::path(base_dir) / ("clickhouse-copier_" + process_id)); fs::create_directories(process_path); diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index ed3297ed7cb..58a87057363 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -399,8 +399,8 @@ try /// Initialize DateLUT early, to not interfere with running time of first query. LOG_DEBUG(log, "Initializing DateLUT."); - DateLUT::instance(); - LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone()); + DateLUT::serverTimezoneInstance(); + LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::serverTimezoneInstance().getTimeZone()); /// Don't want to use DNS cache DNSResolver::instance().setDisableCacheFlag(); diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 274ad29a174..9b7f2c424d3 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -492,7 +492,7 @@ private: const DateLUTImpl & date_lut; public: - explicit DateTimeModel(UInt64 seed_) : seed(seed_), date_lut(DateLUT::instance()) {} + explicit DateTimeModel(UInt64 seed_) : seed(seed_), date_lut(DateLUT::serverTimezoneInstance()) {} void train(const IColumn &) override {} void finalize() override {} diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 711dfb3820a..23113686aa1 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1041,8 +1041,8 @@ try /// Initialize DateLUT early, to not interfere with running time of first query. LOG_DEBUG(log, "Initializing DateLUT."); - DateLUT::instance(); - LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone()); + DateLUT::serverTimezoneInstance(); + LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::serverTimezoneInstance().getTimeZone()); /// Storage with temporary data for processing of heavy queries. if (!server_settings.tmp_policy.value.empty()) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 8da4ac200d9..7a91a382787 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -84,18 +84,18 @@ using namespace std::literals; namespace { using namespace DB; -using ContetGetterFunc = std::function const; -const void* getContextPtrOrNull(ContetGetterFunc contextFunc) -{ - try - { - return contextFunc().get(); - } - catch(...) - { - } - return nullptr; -} +//using ContetGetterFunc = std::function const; +//const void* getContextPtrOrNull(ContetGetterFunc contextFunc) +//{ +// try +// { +// return contextFunc().get(); +// } +// catch(...) +// { +// } +// return nullptr; +//} //void LogContextes(const std::string_view scope, const ContextPtr global_context) //{ @@ -1087,7 +1087,7 @@ void ClientBase::onProgress(const Progress & value) void ClientBase::onTimezoneUpdate(const String & tz) { - std::cerr << "ClientBase::onTimezoneUpdate received new TZ from server: " << tz << std::endl; +// std::cerr << "ClientBase::onTimezoneUpdate received new TZ from server: " << tz << std::endl; Settings settings; settings.timezone = tz; diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index f17fe772dbc..810810edb6c 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -17,6 +17,13 @@ class DateLUT : private boost::noncopyable { public: + /// Return singleton DateLUTImpl instance for server's (native) time zone. + static ALWAYS_INLINE const DateLUTImpl & serverTimezoneInstance() + { + const auto & date_lut = getInstance(); + return *date_lut.default_impl.load(std::memory_order_acquire); + } + /// Return singleton DateLUTImpl instance for timezone set by `timezone` setting for current session is used. /// If it is not set, server's timezone (the one which server has) is being used. static ALWAYS_INLINE const DateLUTImpl & instance() diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 18c4c0d97a0..c6b5be3ea87 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -1003,7 +1003,7 @@ void BaseDaemon::shouldSetupWatchdog(char * argv0_) void BaseDaemon::setupWatchdog() { /// Initialize in advance to avoid double initialization in forked processes. - DateLUT::instance(); + DateLUT::serverTimezoneInstance(); std::string original_process_name; if (argv0) diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index 57a6279bd7a..5d54815818d 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -75,7 +75,7 @@ namespace public: static constexpr auto name = "serverTimezone"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionServerTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance("").getTimeZone()}, context->isDistributed()) {} + explicit FunctionServerTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::serverTimezoneInstance().getTimeZone()}, context->isDistributed()) {} }; diff --git a/src/Loggers/OwnPatternFormatter.h b/src/Loggers/OwnPatternFormatter.h index 07d0409b0ae..8b0d11bcec1 100644 --- a/src/Loggers/OwnPatternFormatter.h +++ b/src/Loggers/OwnPatternFormatter.h @@ -31,6 +31,6 @@ public: virtual void formatExtended(const DB::ExtendedLogMessage & msg_ext, std::string & text) const; private: - const DateLUTImpl & server_timezone = DateLUT::instance(""); + const DateLUTImpl & server_timezone = DateLUT::serverTimezoneInstance(); bool color; }; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 85420cabb8d..f50a7169d39 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1030,7 +1030,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() DayNum max_date; MergeTreePartInfo::parseMinMaxDatesFromPartName(name, min_date, max_date); - const auto & date_lut = DateLUT::instance(); + const auto & date_lut = DateLUT::serverTimezoneInstance(); partition = MergeTreePartition(date_lut.toNumYYYYMM(min_date)); minmax_idx = std::make_shared(min_date, max_date); } diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index e017c9681e8..28e30b5f64f 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -57,7 +57,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() { LOG_INFO(log, "Will try to fetch part {} until '{}' because this part assigned to recompression merge. " "Source replica {} will try to merge this part first", entry.new_part_name, - DateLUT::instance().timeToString(entry.create_time + storage_settings_ptr->try_fetch_recompressed_part_timeout.totalSeconds()), entry.source_replica); + DateLUT::serverTimezoneInstance().timeToString(entry.create_time + storage_settings_ptr->try_fetch_recompressed_part_timeout.totalSeconds()), entry.source_replica); /// Waiting other replica to recompress part. No need to check it. return PrepareResult{ .prepared_successfully = false, diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 37cfe4d065e..48a1cb97c89 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -115,7 +115,7 @@ void updateTTL( if (const ColumnUInt16 * column_date = typeid_cast(ttl_column.get())) { - const auto & date_lut = DateLUT::instance(); + const auto & date_lut = DateLUT::serverTimezoneInstance(); for (const auto & val : column_date->getData()) ttl_info.update(date_lut.fromDayNum(DayNum(val))); } @@ -128,7 +128,7 @@ void updateTTL( { if (typeid_cast(&column_const->getDataColumn())) { - const auto & date_lut = DateLUT::instance(); + const auto & date_lut = DateLUT::serverTimezoneInstance(); ttl_info.update(date_lut.fromDayNum(DayNum(column_const->getValue()))); } else if (typeid_cast(&column_const->getDataColumn())) @@ -369,7 +369,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( DayNum min_date(minmax_idx->hyperrectangle[data.minmax_idx_date_column_pos].left.get()); DayNum max_date(minmax_idx->hyperrectangle[data.minmax_idx_date_column_pos].right.get()); - const auto & date_lut = DateLUT::instance(); + const auto & date_lut = DateLUT::serverTimezoneInstance(); auto min_month = date_lut.toNumYYYYMM(min_date); auto max_month = date_lut.toNumYYYYMM(max_date); diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 2e30a3f3986..2c0359b0f3f 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -127,7 +127,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat LocalDateTime create_time_dt; *buf >> "create time: " >> create_time_dt >> "\n"; - create_time = DateLUT::instance().makeDateTime( + create_time = DateLUT::serverTimezoneInstance().makeDateTime( create_time_dt.year(), create_time_dt.month(), create_time_dt.day(), create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second()); diff --git a/src/Storages/MergeTree/MergeTreePartInfo.cpp b/src/Storages/MergeTree/MergeTreePartInfo.cpp index 84432a293d7..e1b52d8a7b7 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.cpp +++ b/src/Storages/MergeTree/MergeTreePartInfo.cpp @@ -148,7 +148,7 @@ void MergeTreePartInfo::parseMinMaxDatesFromPartName(const String & part_name, D throw Exception(ErrorCodes::BAD_DATA_PART_NAME, "Unexpected part name: {}", part_name); } - const auto & date_lut = DateLUT::instance(); + const auto & date_lut = DateLUT::serverTimezoneInstance(); min_date = date_lut.YYYYMMDDToDayNum(min_yyyymmdd); max_date = date_lut.YYYYMMDDToDayNum(max_yyyymmdd); @@ -219,7 +219,7 @@ String MergeTreePartInfo::getPartNameV1() const String MergeTreePartInfo::getPartNameV0(DayNum left_date, DayNum right_date) const { - const auto & date_lut = DateLUT::instance(); + const auto & date_lut = DateLUT::serverTimezoneInstance(); /// Directory name for the part has form: `YYYYMMDD_YYYYMMDD_N_N_L`. diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 3b28012e7d6..b0fc34ac2f7 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -239,7 +239,7 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const result += '-'; if (typeid_cast(partition_key_sample.getByPosition(i).type.get())) - result += toString(DateLUT::instance().toNumYYYYMMDD(DayNum(value[i].safeGet()))); + result += toString(DateLUT::serverTimezoneInstance().toNumYYYYMMDD(DayNum(value[i].safeGet()))); else if (typeid_cast(partition_key_sample.getByPosition(i).type.get())) result += toString(value[i].get().toUnderType()); else @@ -320,7 +320,7 @@ std::optional MergeTreePartition::tryParseValueFromID(const String & partit throw Exception( ErrorCodes::INVALID_PARTITION_VALUE, "Cannot parse partition_id: got unexpected Date: {}", date_yyyymmdd); - UInt32 date = DateLUT::instance().YYYYMMDDToDayNum(date_yyyymmdd); + UInt32 date = DateLUT::serverTimezoneInstance().YYYYMMDDToDayNum(date_yyyymmdd); res.emplace_back(date); break; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index 79b0beb0933..ac956433eab 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -199,7 +199,7 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in, MergeTreeDataFor { LocalDateTime create_time_dt; in >> "create_time: " >> create_time_dt >> "\n"; - create_time = DateLUT::instance().makeDateTime( + create_time = DateLUT::serverTimezoneInstance().makeDateTime( create_time_dt.year(), create_time_dt.month(), create_time_dt.day(), create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second()); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp index 1efb3f6826b..17f3637e722 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp @@ -38,7 +38,7 @@ void ReplicatedMergeTreeMutationEntry::readText(ReadBuffer & in) LocalDateTime create_time_dt; in >> "create time: " >> create_time_dt >> "\n"; - create_time = DateLUT::instance().makeDateTime( + create_time = DateLUT::serverTimezoneInstance().makeDateTime( create_time_dt.year(), create_time_dt.month(), create_time_dt.day(), create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second()); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index fe4a144deaa..356663496a6 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5318,7 +5318,7 @@ String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { /// The date range is all month long. - const auto & lut = DateLUT::instance(); + const auto & lut = DateLUT::serverTimezoneInstance(); time_t start_time = lut.YYYYMMDDToDate(parse(part_info.partition_id + "01")); DayNum left_date = DayNum(lut.toDayNum(start_time).toUnderType()); DayNum right_date = DayNum(static_cast(left_date) + lut.daysInMonth(start_time) - 1); diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 3471e4ea6bf..8546fdd3c9f 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1335,7 +1335,7 @@ ASTPtr StorageWindowView::innerQueryParser(const ASTSelectQuery & query) time_zone = &DateLUT::instance(window_view_timezone); } else - time_zone = &DateLUT::instance(); + time_zone = &DateLUT::serverTimezoneInstance(); return result; } diff --git a/tests/queries/0_stateless/02668_timezone_setting.reference b/tests/queries/0_stateless/02668_timezone_setting.reference deleted file mode 100644 index 8ed8024f652..00000000000 --- a/tests/queries/0_stateless/02668_timezone_setting.reference +++ /dev/null @@ -1,3 +0,0 @@ -1999-12-12 18:23:23.123 -1999-12-12 23:23:23.123 -1999-12-13 04:23:23.123 diff --git a/tests/queries/0_stateless/02668_timezone_setting.sql b/tests/queries/0_stateless/02668_timezone_setting.sql deleted file mode 100644 index d85efaa8a39..00000000000 --- a/tests/queries/0_stateless/02668_timezone_setting.sql +++ /dev/null @@ -1,11 +0,0 @@ -SET timezone = 'Абырвалг'; -- { serverError BAD_ARGUMENTS} - -SET timezone = 'Asia/Novosibirsk'; -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich'); -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'Europe/Zurich'; - -SET timezone = 'Asia/Manila'; -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); - -SELECT timezone(), serverTimeZone(), timezoneOf(now()) SETTINGS timezone = 'Europe/Zurich'; -SELECT timezone(), serverTimeZone(), timezoneOf(now()) SETTINGS timezone = 'Pacific/Pitcairn'; diff --git a/tests/queries/0_stateless/02681_timezone_setting.reference b/tests/queries/0_stateless/02681_timezone_setting.reference new file mode 100644 index 00000000000..8850d77ab03 --- /dev/null +++ b/tests/queries/0_stateless/02681_timezone_setting.reference @@ -0,0 +1,5 @@ +2022-12-12 17:23:23.123 +2022-12-12 23:23:23.123 +2022-12-12 22:23:23.123 +Europe/Zurich Europe/Zurich +Pacific/Pitcairn Pacific/Pitcairn diff --git a/tests/queries/0_stateless/02681_timezone_setting.sql b/tests/queries/0_stateless/02681_timezone_setting.sql new file mode 100644 index 00000000000..73afb4c029b --- /dev/null +++ b/tests/queries/0_stateless/02681_timezone_setting.sql @@ -0,0 +1,11 @@ +SET timezone = 'Абырвалг'; -- { serverError BAD_ARGUMENTS} + +SET timezone = 'Asia/Novosibirsk'; +SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich'); +SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'Europe/Zurich'; + +SET timezone = 'Asia/Manila'; +SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); + +SELECT timezone(), timezoneOf(now()) SETTINGS timezone = 'Europe/Zurich' FORMAT TSV; +SELECT timezone(), timezoneOf(now()) SETTINGS timezone = 'Pacific/Pitcairn' FORMAT TSV; From daef5d818a5cdc6e358efaa49e56d90ee530e6bf Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 15:31:58 +0200 Subject: [PATCH 028/515] fix according to updates in ThreadStatus.h --- programs/client/Client.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index aa563198c82..528c504e555 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -325,9 +325,8 @@ try { // All that just to set DB::CurrentThread::get().getGlobalContext() // which is required for client timezone (pushed as from server) to work. - auto thread_group = std::make_shared(); - thread_group->global_context = global_context; - thread_status.attachQuery(thread_group, false); + auto thread_group = std::make_shared(); + thread_status.attachToGroup(thread_group, false); } /// Includes delayed_interactive. From 3f8956f854253a5b17c6fa4163372f7e0f6cf664 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 17:45:11 +0200 Subject: [PATCH 029/515] remove additional logging --- src/Client/ClientBase.cpp | 30 ------------------------------ src/Functions/timezoneOf.cpp | 14 -------------- src/Interpreters/Context.cpp | 8 -------- src/Server/TCPHandler.cpp | 24 +++++++----------------- 4 files changed, 7 insertions(+), 69 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index d722d39e8f6..f4253ab90f6 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -66,7 +66,6 @@ #include #include -//#include #include #include #include @@ -75,9 +74,6 @@ #include "config_version.h" #include "config.h" -//#include - - namespace fs = std::filesystem; using namespace std::literals; @@ -85,32 +81,6 @@ using namespace std::literals; namespace { using namespace DB; -//using ContetGetterFunc = std::function const; -//const void* getContextPtrOrNull(ContetGetterFunc contextFunc) -//{ -// try -// { -// return contextFunc().get(); -// } -// catch(...) -// { -// } -// return nullptr; -//} - -//void LogContextes(const std::string_view scope, const ContextPtr global_context) -//{ -// const auto * context = global_context.get(); -// std::cerr << scope << " contextes" -// << "\n\tglobal: " << reinterpret_cast(context) -// << "\n\tsession: " << getContextPtrOrNull([&]() { return context ? context->getSessionContext() : nullptr; }) -// << "\n\tquery: " << getContextPtrOrNull([&]() { return context ? context->getQueryContext() : nullptr; }) -// << "\n\tcurrent T query: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getQueryContext(); }) -// << "\n\tcurrent T global: " << getContextPtrOrNull([&]() { return DB::CurrentThread::get().getGlobalContext(); }) -//// << "\n\tbuffer: " << getContextPtrOrNull(context, &Context::getBufferContext) -// << std::endl; -//} - } namespace CurrentMetrics diff --git a/src/Functions/timezoneOf.cpp b/src/Functions/timezoneOf.cpp index ce419b7b4cd..7a5957a5dbc 100644 --- a/src/Functions/timezoneOf.cpp +++ b/src/Functions/timezoneOf.cpp @@ -5,11 +5,6 @@ #include #include #include -#include "Poco/Logger.h" - -#include -#include -#include namespace DB { @@ -56,15 +51,6 @@ public: { DataTypePtr type_no_nullable = removeNullable(arguments[0].type); - { - const auto query_context = DB::CurrentThread::get().getQueryContext(); - - LOG_DEBUG(&Poco::Logger::get("Function timezoneOf"), "query context: {}, timezone: {} ({})", - reinterpret_cast(query_context.get()), - query_context->getSettingsRef().timezone.toString(), - (query_context->getSettingsRef().timezone.changed ? "changed" : "UNCHANGED")); - } - return DataTypeString().createColumnConst(input_rows_count, dynamic_cast(*type_no_nullable).getTimeZone().getTimeZone()); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 9e0b1dfd032..e888902ae29 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -119,8 +119,6 @@ #include #endif -#include - namespace fs = std::filesystem; namespace ProfileEvents @@ -1683,12 +1681,6 @@ void Context::applySettingChange(const SettingChange & change) void Context::applySettingsChanges(const SettingsChanges & changes) { auto lock = getLock(); - LOG_DEBUG(shared->log, "Context::applySettingsChanges {} applying settings changes: {}", reinterpret_cast(this), - fmt::join(std::ranges::transform_view(changes, - [](const SettingChange & change) - { - return change.name + ": " + change.value.dump(); - }), ", ")); for (const SettingChange & change : changes) applySettingChange(change); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a48097a649f..9a1b64eaf89 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -767,7 +767,6 @@ void TCPHandler::processInsertQuery() /// Send block to the client - table structure. sendData(executor.getHeader()); - sendTimezone(); sendLogs(); while (readDataNext()) @@ -1070,24 +1069,15 @@ void TCPHandler::sendInsertProfileEvents() void TCPHandler::sendTimezone() { -// if (client_tcp_protocol_version <= DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES -// || client_tcp_protocol_version <= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) -// return; + if (client_tcp_protocol_version < DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES) + return; -// const String & tz = CurrentThread::get().getQueryContext()->getSettingsRef().timezone.toString(); - LOG_DEBUG(log, "TCPHandler::sendTimezone() query context: {}, timezone: {} ({})", - reinterpret_cast(query_context.get()), - query_context->getSettingsRef().timezone.toString(), - (query_context->getSettingsRef().timezone.changed ? "changed" : "UNCHANGED")); + const String & tz = query_context->getSettingsRef().timezone.toString(); - const String & tz = CurrentThread::get().getQueryContext()->getSettingsRef().timezone.toString(); -// if (!tz.empty()) -// { - LOG_DEBUG(log, "Sent timezone: {}", tz); - writeVarUInt(Protocol::Server::TimezoneUpdate, *out); - writeStringBinary(tz, *out); - out->next(); -// } + LOG_DEBUG(log, "TCPHandler::sendTimezone(): {}", tz); + writeVarUInt(Protocol::Server::TimezoneUpdate, *out); + writeStringBinary(tz, *out); + out->next(); } From 73675cd8d29ffd3c5e1b1a57a023ee1ac946ef8c Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 19:17:48 +0200 Subject: [PATCH 030/515] tryfix fasttest --- src/Server/TCPHandler.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 9a1b64eaf89..152c7aba56e 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -458,7 +458,6 @@ void TCPHandler::runImpl() if (getQueryCancellationStatus() == CancellationStatus::FULLY_CANCELLED) return true; - sendTimezone(); sendProgress(); sendSelectProfileEvents(); sendLogs(); @@ -811,7 +810,6 @@ void TCPHandler::processOrdinaryQueryWithProcessors() { std::lock_guard lock(task_callback_mutex); sendData(header); - sendTimezone(); } } From 981a73cd867c435c74adf06cece8ec279fb8fde8 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 21:20:12 +0200 Subject: [PATCH 031/515] upd remotequeryexecutor on receive timezone --- src/QueryPipeline/RemoteQueryExecutor.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index b7490a2ad9c..23c1412dc76 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -449,6 +449,9 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::processPacket(Packet packet throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push into profile queue"); break; + case Protocol::Server::TimezoneUpdate: + break; + default: got_unknown_packet_from_replica = true; throw Exception( @@ -546,6 +549,9 @@ void RemoteQueryExecutor::finish(std::unique_ptr * read_context) if (!profile_queue->emplace(std::move(packet.block))) throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push into profile queue"); break; + + case Protocol::Server::TimezoneUpdate: + break; default: got_unknown_packet_from_replica = true; From 0c616ac0a287ce017c18561013b87e576ac8e74b Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 21:31:13 +0200 Subject: [PATCH 032/515] fix style --- src/QueryPipeline/RemoteQueryExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 23c1412dc76..56b5357c522 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -549,7 +549,7 @@ void RemoteQueryExecutor::finish(std::unique_ptr * read_context) if (!profile_queue->emplace(std::move(packet.block))) throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push into profile queue"); break; - + case Protocol::Server::TimezoneUpdate: break; From 1ec32d374d8872aedd2f13bfdbbb263d98feed17 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 22:27:54 +0200 Subject: [PATCH 033/515] update Timezone packet processing --- src/Client/HedgedConnections.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 13e4fe75b3d..d11954f3838 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -418,6 +418,7 @@ Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & repli } replica_with_last_received_packet = replica_location; break; + case Protocol::Server::TimezoneUpdate: case Protocol::Server::PartUUIDs: case Protocol::Server::ProfileInfo: case Protocol::Server::Totals: From 16292eb5a18d1f410421217461a7b3e44b39dbec Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 12 Apr 2023 22:54:51 +0200 Subject: [PATCH 034/515] update timezone packet handling in remote inserter --- src/QueryPipeline/RemoteInserter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/QueryPipeline/RemoteInserter.cpp b/src/QueryPipeline/RemoteInserter.cpp index b8a878b56c3..134c169e35f 100644 --- a/src/QueryPipeline/RemoteInserter.cpp +++ b/src/QueryPipeline/RemoteInserter.cpp @@ -130,7 +130,7 @@ void RemoteInserter::onFinish() break; else if (Protocol::Server::Exception == packet.type) packet.exception->rethrow(); - else if (Protocol::Server::Log == packet.type) + else if (Protocol::Server::Log == packet.type || Protocol::Server::TimezoneUpdate == packet.type) { // Do nothing } From bac5fbc3d2cf1b9606d66543244036797221a4b3 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 13 Apr 2023 13:26:09 +0200 Subject: [PATCH 035/515] fix error on connection drop after 1st query --- src/Client/ClientBase.cpp | 4 ---- src/Client/Connection.cpp | 4 ++++ src/Server/TCPHandler.cpp | 4 +++- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index f4253ab90f6..2e82144e64d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1074,13 +1074,9 @@ void ClientBase::onProgress(const Progress & value) void ClientBase::onTimezoneUpdate(const String & tz) { -// std::cerr << "ClientBase::onTimezoneUpdate received new TZ from server: " << tz << std::endl; - Settings settings; settings.timezone = tz; global_context->applySettingsChanges(settings.changes()); - -// LogContextes("ClientBase::onTimezoneUpdate", global_context); } diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index a0025eafd64..08549265848 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -335,6 +335,10 @@ void Connection::receiveHello() nonce.emplace(read_nonce); } } + else if (packet_type == Protocol::Server::TimezoneUpdate) + { + // skip this packet at hello, will receive and process it later + } else if (packet_type == Protocol::Server::Exception) receiveException()->rethrow(); else diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 152c7aba56e..a875507d227 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -183,8 +183,11 @@ void TCPHandler::runImpl() /// User will be authenticated here. It will also set settings from user profile into connection_context. try { + LOG_DEBUG(log, "Before receiveHello"); receiveHello(); + LOG_DEBUG(log, "Before sendHello"); sendHello(); + LOG_DEBUG(log, "Before receiveAddendum"); if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) receiveAddendum(); @@ -499,7 +502,6 @@ void TCPHandler::runImpl() { std::lock_guard lock(task_callback_mutex); - sendTimezone(); sendLogs(); sendEndOfStream(); } From d5ea52e4b9b4005356f79e3eaadd4d6458fa116e Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 14 Apr 2023 01:28:59 +0200 Subject: [PATCH 036/515] optimize --- src/Common/DateLUT.h | 11 +++++------ src/Core/SettingsFields.h | 6 +++--- src/DataTypes/TimezoneMixin.h | 2 +- 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index 810810edb6c..59b280240ea 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -28,11 +28,11 @@ public: /// If it is not set, server's timezone (the one which server has) is being used. static ALWAYS_INLINE const DateLUTImpl & instance() { - std::string effective_time_zone; const auto & date_lut = getInstance(); if (DB::CurrentThread::isInitialized()) { + std::string effective_time_zone; const auto query_context = DB::CurrentThread::get().getQueryContext(); if (query_context) @@ -43,6 +43,8 @@ public: return date_lut.getImplementation(effective_time_zone); } + /// Timezone is passed in query_context, but on CH-Client we have no query context, + /// and each time we modify client's global context const auto global_context = DB::CurrentThread::get().getGlobalContext(); if (global_context) { @@ -56,15 +58,12 @@ public: return *date_lut.default_impl.load(std::memory_order_acquire); } - /// Return singleton DateLUTImpl instance for a given time zone. If timezone is an empty string, - /// server's timezone is used. The `timezone` setting is not considered here. static ALWAYS_INLINE const DateLUTImpl & instance(const std::string & time_zone) { - const auto & date_lut = getInstance(); - if (time_zone.empty()) - return *date_lut.default_impl.load(std::memory_order_acquire); + return instance(); + const auto & date_lut = getInstance(); return date_lut.getImplementation(time_zone); } diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 8e9ffe03008..0ee3ddd4862 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -576,10 +576,10 @@ struct SettingFieldTimezone private: cctz::time_zone validated_tz; - void validateTimezone(const std::string & str) + void validateTimezone(const std::string & tz_str) { - if (str != "" && !cctz::load_time_zone(str, &validated_tz)) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); + if (!tz_str.empty() && !cctz::load_time_zone(tz_str, &validated_tz)) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", tz_str); } }; diff --git a/src/DataTypes/TimezoneMixin.h b/src/DataTypes/TimezoneMixin.h index 5b7870c7b9a..03ecde5dd0a 100644 --- a/src/DataTypes/TimezoneMixin.h +++ b/src/DataTypes/TimezoneMixin.h @@ -15,7 +15,7 @@ public: explicit TimezoneMixin(const String & time_zone_name = "") : has_explicit_time_zone(!time_zone_name.empty()) - , time_zone(time_zone_name.empty() ? DateLUT::instance() : DateLUT::instance(time_zone_name)) + , time_zone(DateLUT::instance(time_zone_name)) , utc_time_zone(DateLUT::instance("UTC")) { } From 22be85d9764d6ebe3511313c9dadcbdf070c53ad Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 18 Apr 2023 02:42:30 +0200 Subject: [PATCH 037/515] renamed setting --- docs/en/operations/settings/settings.md | 8 +++---- .../functions/date-time-functions.md | 2 +- docs/ru/operations/settings/settings.md | 8 +++---- .../functions/date-time-functions.md | 2 +- src/Client/ClientBase.cpp | 5 +--- src/Common/DateLUT.cpp | 2 +- src/Core/Settings.h | 2 +- src/Core/SettingsFields.cpp | 15 ++++++++++++ src/Core/SettingsFields.h | 23 +++++++++++-------- src/Interpreters/Context.cpp | 1 - src/Server/TCPHandler.cpp | 6 +---- .../0_stateless/02681_timezone_setting.sql | 12 +++++----- 12 files changed, 48 insertions(+), 38 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 7caf3d4333f..dd81b07b9c0 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4008,9 +4008,9 @@ Default value: `0`. Use this setting only for backward compatibility if your use cases depend on old syntax. ::: -## timezone {#timezone} +## session_timezone {#session_timezone} -If specified, sets a implicit timezone (instead of server-default). All DateTime/DateTime64 values (and/or functions results) that have no explicit timezone specified are treated as having this timezone instead of default. +If specified, sets an implicit timezone (instead of server-default). All DateTime/DateTime64 values (and/or functions results) that have no explicit timezone specified are treated as having this timezone instead of default. Examples: ```clickhouse @@ -4020,13 +4020,13 @@ Europe/Berlin Europe/Berlin ``` ```clickhouse -SELECT timeZone(), serverTimezone() SETTINGS timezone = 'Asia/Novosibirsk' FORMAT TSV +SELECT timeZone(), serverTimezone() SETTINGS session_timezone = 'Asia/Novosibirsk' FORMAT TSV Asia/Novosibirsk Europe/Berlin ``` ```clickhouse -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'America/Denver' FORMAT TSV +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'America/Denver' FORMAT TSV 1999-12-13 07:23:23.123 ``` diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index f96041996d4..c1b8d201745 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -26,7 +26,7 @@ SELECT ## timeZone -Returns the default timezone of the server for current session. This can be modified using `SET timezone = 'New/Value'` +Returns the default timezone of the server for current session. This can be modified using `SET session_timezone = 'New/Value'` If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. **Syntax** diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 1687e37dba2..fd4d1e11df7 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4075,9 +4075,9 @@ SELECT sum(number) FROM numbers(10000000000) SETTINGS partial_result_on_first_ca Значение по умолчанию: `false` -## timezone {#timezone} +## session_timezone {#session_timezone} -Задаёт значение часового пояса (timezone) по умолчанию для текущей сессии вместо часового пояса сервера. То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. +Задаёт значение часового пояса (session_timezone) по умолчанию для текущей сессии вместо часового пояса сервера. То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. Примеры: ```clickhouse @@ -4087,13 +4087,13 @@ Europe/Berlin Europe/Berlin ``` ```clickhouse -SELECT timeZone(), serverTimezone() SETTINGS timezone = 'Asia/Novosibirsk' FORMAT TSV +SELECT timeZone(), serverTimezone() SETTINGS session_timezone = 'Asia/Novosibirsk' FORMAT TSV Asia/Novosibirsk Europe/Berlin ``` ```clickhouse -SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'America/Denver' FORMAT TSV +SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'America/Denver' FORMAT TSV 1999-12-13 07:23:23.123 ``` diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 132b54c1040..3e378c08308 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -26,7 +26,7 @@ SELECT ## timeZone {#timezone} -Возвращает часовой пояс сервера, считающийся умолчанием для текущей сессии. +Возвращает часовой пояс сервера, считающийся умолчанием для текущей сессии. Можно изменить значение с помощью `SET session_timezone = 'New/Timezone''` Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями, актуальными для каждого шарда. Иначе возвращается константа. **Синтаксис** diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 2e82144e64d..5126777fa1e 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -451,12 +451,9 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) /// Also do not output too much data if we're fuzzing. if (block.rows() == 0 || (query_fuzzer_runs != 0 && processed_rows >= 100)) { -// LogContextes("ClientBase::onData header", global_context); return; } -// LogContextes("ClientBase::onData DATA block", global_context); - /// If results are written INTO OUTFILE, we can avoid clearing progress to avoid flicker. if (need_render_progress && tty_buf && (!select_into_file || select_into_file_and_stdout)) progress_indication.clearProgressOutput(*tty_buf); @@ -1075,7 +1072,7 @@ void ClientBase::onProgress(const Progress & value) void ClientBase::onTimezoneUpdate(const String & tz) { Settings settings; - settings.timezone = tz; + settings.session_timezone = tz; global_context->applySettingsChanges(settings.changes()); } diff --git a/src/Common/DateLUT.cpp b/src/Common/DateLUT.cpp index 16bd69a20f8..f7e7df016cb 100644 --- a/src/Common/DateLUT.cpp +++ b/src/Common/DateLUT.cpp @@ -167,5 +167,5 @@ DateLUT & DateLUT::getInstance() std::string DateLUT::extractTimezoneFromContext(const DB::ContextPtr query_context) { - return query_context->getSettingsRef().timezone.value; + return query_context->getSettingsRef().session_timezone.value; } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b3e9f3fb220..837958aa5b0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -734,7 +734,7 @@ class IColumn; M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(Bool, function_json_value_return_type_allow_nullable, false, "Allow function to return nullable type.", 0) \ M(Bool, function_json_value_return_type_allow_complex, false, "Allow function to return complex type, such as: struct, array, map.", 0) \ - M(Timezone, timezone, "", "Use specified timezone for interpreting Date and DateTime instead of server's timezone.", 0) \ + M(Timezone, session_timezone, "", "Use specified timezone for interpreting Date and DateTime instead of server's timezone in current session.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index f4169aa0c64..65720056c8a 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -13,6 +13,7 @@ #include +extern const char * auto_time_zones[]; namespace DB { @@ -26,6 +27,14 @@ namespace ErrorCodes namespace { + bool checkIsExitingTimeZone(const std::string_view timezone) + { + for (auto * it = auto_time_zones; *it; ++it) + if (timezone == *it) + return true; + return false; + } + template T stringToNumber(const String & str) { @@ -463,6 +472,12 @@ void SettingFieldTimezone::readBinary(ReadBuffer & in) *this = std::move(str); } +void SettingFieldTimezone::validateTimezone(std::string_view str) +{ + if (str != "" && !checkIsExitingTimeZone(str)) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); +} + String SettingFieldCustom::toString() const { return value.dump(); diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 0ee3ddd4862..e3b18a606a1 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -553,13 +552,16 @@ struct SettingFieldTimezone String value; bool changed = false; - explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(std::string(str)); value = str; } + explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(str); value = str; } +// explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(std::string(str)); value = str; } explicit SettingFieldTimezone(const String & str) { validateTimezone(str); value = str; } - explicit SettingFieldTimezone(String && str) { validateTimezone(std::string(str)); value = std::move(str); } +// explicit SettingFieldTimezone(String && str) { validateTimezone(std::string(str)); value = std::move(str); } + explicit SettingFieldTimezone(String && str) { validateTimezone(str); value = std::move(str); } explicit SettingFieldTimezone(const char * str) { validateTimezone(str); value = str; } explicit SettingFieldTimezone(const Field & f) { const String & str = f.safeGet(); validateTimezone(str); value = str; } - SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(std::string(str)); value = str; changed = true; return *this; } +// SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(std::string(str)); value = str; changed = true; return *this; } + SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(str); value = str; changed = true; return *this; } SettingFieldTimezone & operator =(const String & str) { *this = std::string_view{str}; return *this; } SettingFieldTimezone & operator =(String && str) { validateTimezone(str); value = std::move(str); changed = true; return *this; } SettingFieldTimezone & operator =(const char * str) { *this = std::string_view{str}; return *this; } @@ -575,12 +577,13 @@ struct SettingFieldTimezone void readBinary(ReadBuffer & in); private: - cctz::time_zone validated_tz; - void validateTimezone(const std::string & tz_str) - { - if (!tz_str.empty() && !cctz::load_time_zone(tz_str, &validated_tz)) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", tz_str); - } + static void validateTimezone(std::string_view str); +// cctz::time_zone validated_tz; +// void validateTimezone(const std::string & tz_str) +// { +// if (!tz_str.empty() && !cctz::load_time_zone(tz_str, &validated_tz)) +// throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", tz_str); +// } }; /// Can keep a value of any type. Used for user-defined settings. diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 82ecd87faa0..400eb570131 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1680,7 +1680,6 @@ void Context::applySettingChange(const SettingChange & change) void Context::applySettingsChanges(const SettingsChanges & changes) { auto lock = getLock(); - for (const SettingChange & change : changes) applySettingChange(change); applySettingsQuirks(settings); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a875507d227..e44609529ba 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -183,11 +183,8 @@ void TCPHandler::runImpl() /// User will be authenticated here. It will also set settings from user profile into connection_context. try { - LOG_DEBUG(log, "Before receiveHello"); receiveHello(); - LOG_DEBUG(log, "Before sendHello"); sendHello(); - LOG_DEBUG(log, "Before receiveAddendum"); if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) receiveAddendum(); @@ -465,7 +462,6 @@ void TCPHandler::runImpl() sendSelectProfileEvents(); sendLogs(); - return false; }; @@ -1072,7 +1068,7 @@ void TCPHandler::sendTimezone() if (client_tcp_protocol_version < DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES) return; - const String & tz = query_context->getSettingsRef().timezone.toString(); + const String & tz = query_context->getSettingsRef().session_timezone.toString(); LOG_DEBUG(log, "TCPHandler::sendTimezone(): {}", tz); writeVarUInt(Protocol::Server::TimezoneUpdate, *out); diff --git a/tests/queries/0_stateless/02681_timezone_setting.sql b/tests/queries/0_stateless/02681_timezone_setting.sql index 73afb4c029b..f66e8d2b646 100644 --- a/tests/queries/0_stateless/02681_timezone_setting.sql +++ b/tests/queries/0_stateless/02681_timezone_setting.sql @@ -1,11 +1,11 @@ -SET timezone = 'Абырвалг'; -- { serverError BAD_ARGUMENTS} +SET session_timezone = 'Абырвалг'; -- { serverError BAD_ARGUMENTS} -SET timezone = 'Asia/Novosibirsk'; +SET session_timezone = 'Asia/Novosibirsk'; SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich'); -SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS timezone = 'Europe/Zurich'; +SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'Europe/Zurich'; -SET timezone = 'Asia/Manila'; +SET session_timezone = 'Asia/Manila'; SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); -SELECT timezone(), timezoneOf(now()) SETTINGS timezone = 'Europe/Zurich' FORMAT TSV; -SELECT timezone(), timezoneOf(now()) SETTINGS timezone = 'Pacific/Pitcairn' FORMAT TSV; +SELECT timezone(), timezoneOf(now()) SETTINGS session_timezone = 'Europe/Zurich' FORMAT TSV; +SELECT timezone(), timezoneOf(now()) SETTINGS session_timezone = 'Pacific/Pitcairn' FORMAT TSV; From 0550b0640ce0020d4e4f0015447631c4b742ab13 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 18 Apr 2023 03:35:29 +0200 Subject: [PATCH 038/515] fix linking issue --- src/Core/SettingsFields.cpp | 15 --------------- src/Core/SettingsFields.h | 23 ++++++++++------------- 2 files changed, 10 insertions(+), 28 deletions(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 65720056c8a..f4169aa0c64 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -13,7 +13,6 @@ #include -extern const char * auto_time_zones[]; namespace DB { @@ -27,14 +26,6 @@ namespace ErrorCodes namespace { - bool checkIsExitingTimeZone(const std::string_view timezone) - { - for (auto * it = auto_time_zones; *it; ++it) - if (timezone == *it) - return true; - return false; - } - template T stringToNumber(const String & str) { @@ -472,12 +463,6 @@ void SettingFieldTimezone::readBinary(ReadBuffer & in) *this = std::move(str); } -void SettingFieldTimezone::validateTimezone(std::string_view str) -{ - if (str != "" && !checkIsExitingTimeZone(str)) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); -} - String SettingFieldCustom::toString() const { return value.dump(); diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index e3b18a606a1..0ee3ddd4862 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -552,16 +553,13 @@ struct SettingFieldTimezone String value; bool changed = false; - explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(str); value = str; } -// explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(std::string(str)); value = str; } + explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(std::string(str)); value = str; } explicit SettingFieldTimezone(const String & str) { validateTimezone(str); value = str; } -// explicit SettingFieldTimezone(String && str) { validateTimezone(std::string(str)); value = std::move(str); } - explicit SettingFieldTimezone(String && str) { validateTimezone(str); value = std::move(str); } + explicit SettingFieldTimezone(String && str) { validateTimezone(std::string(str)); value = std::move(str); } explicit SettingFieldTimezone(const char * str) { validateTimezone(str); value = str; } explicit SettingFieldTimezone(const Field & f) { const String & str = f.safeGet(); validateTimezone(str); value = str; } -// SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(std::string(str)); value = str; changed = true; return *this; } - SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(str); value = str; changed = true; return *this; } + SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(std::string(str)); value = str; changed = true; return *this; } SettingFieldTimezone & operator =(const String & str) { *this = std::string_view{str}; return *this; } SettingFieldTimezone & operator =(String && str) { validateTimezone(str); value = std::move(str); changed = true; return *this; } SettingFieldTimezone & operator =(const char * str) { *this = std::string_view{str}; return *this; } @@ -577,13 +575,12 @@ struct SettingFieldTimezone void readBinary(ReadBuffer & in); private: - static void validateTimezone(std::string_view str); -// cctz::time_zone validated_tz; -// void validateTimezone(const std::string & tz_str) -// { -// if (!tz_str.empty() && !cctz::load_time_zone(tz_str, &validated_tz)) -// throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", tz_str); -// } + cctz::time_zone validated_tz; + void validateTimezone(const std::string & tz_str) + { + if (!tz_str.empty() && !cctz::load_time_zone(tz_str, &validated_tz)) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", tz_str); + } }; /// Can keep a value of any type. Used for user-defined settings. From 1e8c0a2db9671f0862975499f16b923a49c3a2ec Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 19 Apr 2023 00:06:15 +0200 Subject: [PATCH 039/515] Lighter timezone validation Reused external variable from src/Storages/System/StorageSystemTimeZones.generated.cpp Required changes to CMakeLists of some standalone modules to link properly --- programs/library-bridge/CMakeLists.txt | 8 +++++++- programs/odbc-bridge/CMakeLists.txt | 6 +++++- src/Core/SettingsFields.cpp | 15 +++++++++++++++ src/Core/SettingsFields.h | 24 ++++++++++++++---------- utils/check-marks/CMakeLists.txt | 6 +++++- utils/keeper-data-dumper/CMakeLists.txt | 8 +++++++- utils/wal-dump/CMakeLists.txt | 6 +++++- 7 files changed, 58 insertions(+), 15 deletions(-) diff --git a/programs/library-bridge/CMakeLists.txt b/programs/library-bridge/CMakeLists.txt index 1cacc391ca5..97af7c3b22e 100644 --- a/programs/library-bridge/CMakeLists.txt +++ b/programs/library-bridge/CMakeLists.txt @@ -13,11 +13,17 @@ set (CLICKHOUSE_LIBRARY_BRIDGE_SOURCES library-bridge.cpp ) +set(CLICKHOUSE_LIBRARY_BRIDGE_EXTERNAL_SOURCES + ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp +) + if (OS_LINUX) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") endif () -clickhouse_add_executable(clickhouse-library-bridge ${CLICKHOUSE_LIBRARY_BRIDGE_SOURCES}) +clickhouse_add_executable(clickhouse-library-bridge + ${CLICKHOUSE_LIBRARY_BRIDGE_SOURCES} + ${CLICKHOUSE_LIBRARY_BRIDGE_EXTERNAL_SOURCES}) target_link_libraries(clickhouse-library-bridge PRIVATE daemon diff --git a/programs/odbc-bridge/CMakeLists.txt b/programs/odbc-bridge/CMakeLists.txt index 118610e4dcd..bf1b42df026 100644 --- a/programs/odbc-bridge/CMakeLists.txt +++ b/programs/odbc-bridge/CMakeLists.txt @@ -15,13 +15,17 @@ set (CLICKHOUSE_ODBC_BRIDGE_SOURCES validateODBCConnectionString.cpp ) +set(ODBC_BRIDGE_EXTERNAL_SOURCES + ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp +) + if (OS_LINUX) # clickhouse-odbc-bridge is always a separate binary. # Reason: it must not export symbols from SSL, mariadb-client, etc. to not break ABI compatibility with ODBC drivers. set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") endif () -clickhouse_add_executable(clickhouse-odbc-bridge ${CLICKHOUSE_ODBC_BRIDGE_SOURCES}) +clickhouse_add_executable(clickhouse-odbc-bridge ${CLICKHOUSE_ODBC_BRIDGE_SOURCES} ${ODBC_BRIDGE_EXTERNAL_SOURCES}) target_link_libraries(clickhouse-odbc-bridge PRIVATE daemon diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index f4169aa0c64..65720056c8a 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -13,6 +13,7 @@ #include +extern const char * auto_time_zones[]; namespace DB { @@ -26,6 +27,14 @@ namespace ErrorCodes namespace { + bool checkIsExitingTimeZone(const std::string_view timezone) + { + for (auto * it = auto_time_zones; *it; ++it) + if (timezone == *it) + return true; + return false; + } + template T stringToNumber(const String & str) { @@ -463,6 +472,12 @@ void SettingFieldTimezone::readBinary(ReadBuffer & in) *this = std::move(str); } +void SettingFieldTimezone::validateTimezone(std::string_view str) +{ + if (str != "" && !checkIsExitingTimeZone(str)) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); +} + String SettingFieldCustom::toString() const { return value.dump(); diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 0ee3ddd4862..e78fef9f455 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -2,7 +2,7 @@ #include #include -#include +//#include #include #include #include @@ -553,13 +553,16 @@ struct SettingFieldTimezone String value; bool changed = false; - explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(std::string(str)); value = str; } +// explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(std::string(str)); value = str; } + explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(str); value = str; } explicit SettingFieldTimezone(const String & str) { validateTimezone(str); value = str; } - explicit SettingFieldTimezone(String && str) { validateTimezone(std::string(str)); value = std::move(str); } +// explicit SettingFieldTimezone(String && str) { validateTimezone(std::string(str)); value = std::move(str); } + explicit SettingFieldTimezone(String && str) { validateTimezone(str); value = std::move(str); } explicit SettingFieldTimezone(const char * str) { validateTimezone(str); value = str; } explicit SettingFieldTimezone(const Field & f) { const String & str = f.safeGet(); validateTimezone(str); value = str; } - SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(std::string(str)); value = str; changed = true; return *this; } +// SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(std::string(str)); value = str; changed = true; return *this; } + SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(str); value = str; changed = true; return *this; } SettingFieldTimezone & operator =(const String & str) { *this = std::string_view{str}; return *this; } SettingFieldTimezone & operator =(String && str) { validateTimezone(str); value = std::move(str); changed = true; return *this; } SettingFieldTimezone & operator =(const char * str) { *this = std::string_view{str}; return *this; } @@ -575,12 +578,13 @@ struct SettingFieldTimezone void readBinary(ReadBuffer & in); private: - cctz::time_zone validated_tz; - void validateTimezone(const std::string & tz_str) - { - if (!tz_str.empty() && !cctz::load_time_zone(tz_str, &validated_tz)) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", tz_str); - } +// cctz::time_zone validated_tz; +// void validateTimezone(const std::string & str) +// { +// if (!str.empty() && !cctz::load_time_zone(str, &validated_tz)) +// throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); +// } + static void validateTimezone(std::string_view str); }; /// Can keep a value of any type. Used for user-defined settings. diff --git a/utils/check-marks/CMakeLists.txt b/utils/check-marks/CMakeLists.txt index 05546a2989b..456fb3d7112 100644 --- a/utils/check-marks/CMakeLists.txt +++ b/utils/check-marks/CMakeLists.txt @@ -1,2 +1,6 @@ -clickhouse_add_executable (check-marks main.cpp) +set(CHECK_MARKS_EXTERNAL_SOURCES + ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp +) + +clickhouse_add_executable (check-marks ${CHECK_MARKS_EXTERNAL_SOURCES} main.cpp) target_link_libraries(check-marks PRIVATE dbms boost::program_options) diff --git a/utils/keeper-data-dumper/CMakeLists.txt b/utils/keeper-data-dumper/CMakeLists.txt index 1f55e50e68e..a6858a29e8b 100644 --- a/utils/keeper-data-dumper/CMakeLists.txt +++ b/utils/keeper-data-dumper/CMakeLists.txt @@ -1,2 +1,8 @@ -clickhouse_add_executable(keeper-data-dumper main.cpp) +set(KEEPER_DATA_DUMPER_EXTERNAL_SOURCES + ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp +) + +clickhouse_add_executable(keeper-data-dumper + ${KEEPER_DATA_DUMPER_EXTERNAL_SOURCES} + main.cpp) target_link_libraries(keeper-data-dumper PRIVATE dbms) diff --git a/utils/wal-dump/CMakeLists.txt b/utils/wal-dump/CMakeLists.txt index 3d59e95b4ca..754799a6faf 100644 --- a/utils/wal-dump/CMakeLists.txt +++ b/utils/wal-dump/CMakeLists.txt @@ -1,2 +1,6 @@ -clickhouse_add_executable (wal-dump main.cpp) +set(WAL_DUMP_EXTERNAL_SOURCES + ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp +) + +clickhouse_add_executable (wal-dump ${WAL_DUMP_EXTERNAL_SOURCES} main.cpp) target_link_libraries(wal-dump PRIVATE dbms boost::program_options) From 24be7203d931b57a35241fec1abe31a9099ba096 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 19 Apr 2023 00:39:08 +0200 Subject: [PATCH 040/515] add errorcode reference --- src/Core/SettingsFields.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 65720056c8a..e952688a968 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -22,6 +22,7 @@ namespace ErrorCodes extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH; extern const int CANNOT_PARSE_BOOL; extern const int CANNOT_PARSE_NUMBER; + extern const int BAD_ARGUMENTS; } From 542c09cb518988cf54261edbab691c459efa9a88 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 19 Apr 2023 01:35:49 +0200 Subject: [PATCH 041/515] fix keeper standalone linking --- programs/keeper/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 761335fb707..ff2de3f581c 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -107,6 +107,8 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/GraphiteWriter.cpp ${CMAKE_CURRENT_BINARY_DIR}/../../src/Daemon/GitHash.generated.cpp + ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp + Keeper.cpp clickhouse-keeper.cpp ) From ce7dc8b123502aee8af1578d87133f8283c66a5b Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 19 Apr 2023 13:42:07 +0200 Subject: [PATCH 042/515] tidy --- src/Core/SettingsFields.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index e952688a968..6af38586ed8 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -475,7 +475,7 @@ void SettingFieldTimezone::readBinary(ReadBuffer & in) void SettingFieldTimezone::validateTimezone(std::string_view str) { - if (str != "" && !checkIsExitingTimeZone(str)) + if (!str.empty() && !checkIsExitingTimeZone(str)) throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); } From daae5025e8586156a016687672100a8ec0db6016 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 19 Apr 2023 14:45:51 +0200 Subject: [PATCH 043/515] small updates due to review --- docs/en/operations/settings/settings.md | 1 + docs/en/sql-reference/functions/date-time-functions.md | 2 +- docs/ru/operations/settings/settings.md | 1 + programs/client/Client.cpp | 2 +- src/Client/ClientBase.cpp | 5 ----- 5 files changed, 4 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index dd81b07b9c0..2010f763c84 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4011,6 +4011,7 @@ Use this setting only for backward compatibility if your use cases depend on old ## session_timezone {#session_timezone} If specified, sets an implicit timezone (instead of server-default). All DateTime/DateTime64 values (and/or functions results) that have no explicit timezone specified are treated as having this timezone instead of default. +Setting this to `''` (empty string) effectively resets implicit timezone to server timezone. Examples: ```clickhouse diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index c1b8d201745..265ce676ef7 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -51,7 +51,7 @@ If it is executed in the context of a distributed table, then it generates a nor **Syntax** ``` sql -timeZone() +serverTimeZone() ``` Alias: `ServerTimezone`, `servertimezone`. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index fd4d1e11df7..8180f5435b8 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4078,6 +4078,7 @@ SELECT sum(number) FROM numbers(10000000000) SETTINGS partial_result_on_first_ca ## session_timezone {#session_timezone} Задаёт значение часового пояса (session_timezone) по умолчанию для текущей сессии вместо часового пояса сервера. То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. +При значении настройки `''` (пустая строка), будет совпадать с часовым поясом сервера. Примеры: ```clickhouse diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 528c504e555..b760efc21d1 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -324,7 +324,7 @@ try { // All that just to set DB::CurrentThread::get().getGlobalContext() - // which is required for client timezone (pushed as from server) to work. + // which is required for client timezone (pushed from server) to work. auto thread_group = std::make_shared(); thread_status.attachToGroup(thread_group, false); } diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 5126777fa1e..6df86db886b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -78,11 +78,6 @@ namespace fs = std::filesystem; using namespace std::literals; -namespace -{ -using namespace DB; -} - namespace CurrentMetrics { extern const Metric MemoryTracking; From 61f55930ceee99ce23cdab794ce77945f9a6ee1c Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Wed, 19 Apr 2023 15:46:17 +0300 Subject: [PATCH 044/515] Update docs/en/operations/settings/settings.md Co-authored-by: Vasily Nemkov --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2010f763c84..366e7de8d28 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4034,7 +4034,7 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric Possible values: -- Any valid timezone in `Region/Place` notation, e.g. `Europe/Berlin` +- Any timezone name from `system.time_zones`, e.g. `Europe/Berlin`, `UTC` or `Zulu` Default value: `''`. From b281ceacbb4cc8f66a56b9bc12d5ab521098ce08 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 19 Apr 2023 14:47:57 +0200 Subject: [PATCH 045/515] Update docs/ru/operations/settings/settings.md --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 8180f5435b8..f0f497b6254 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4101,6 +4101,6 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric Возможные значения: -- Строка вида `Регион/Город`, например `Europe/Zurich` +- Любая зона из `system.time_zones`, например `Europe/Berlin`, `UTC` или `Zulu` Значение по умолчанию: `''`. From b06d7355d597abdf6692b5c232fb12449d57aa5b Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Wed, 19 Apr 2023 17:39:40 +0300 Subject: [PATCH 046/515] Update src/Core/SettingsFields.h Co-authored-by: Vasily Nemkov --- src/Core/SettingsFields.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index e78fef9f455..8bd7370c980 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -2,7 +2,6 @@ #include #include -//#include #include #include #include From b81ce64fa23cf2d05edd488eeb1adbf981784a54 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Wed, 19 Apr 2023 17:39:52 +0300 Subject: [PATCH 047/515] Update src/Client/ClientBase.cpp Co-authored-by: Vasily Nemkov --- src/Client/ClientBase.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 6df86db886b..bd83246871b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1531,9 +1531,9 @@ void ClientBase::receiveLogsAndProfileEvents(ASTPtr parsed_query) { auto packet_type = connection->checkPacket(0); - while (packet_type && (*packet_type == Protocol::Server::Log || - *packet_type == Protocol::Server::ProfileEvents || - *packet_type == Protocol::Server::TimezoneUpdate)) + while (packet_type && (*packet_type == Protocol::Server::Log + || *packet_type == Protocol::Server::ProfileEvents + || *packet_type == Protocol::Server::TimezoneUpdate)) { receiveAndProcessPacket(parsed_query, false); packet_type = connection->checkPacket(0); From bf55f43e1933fdcbbc2ec85e5b0823c6a7e3eb5e Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 20 Apr 2023 02:53:42 +0200 Subject: [PATCH 048/515] update cmakelists --- src/CMakeLists.txt | 5 +++++ src/Core/SettingsFields.cpp | 4 ++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 76e5ef83e41..5ac3f6e1654 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -191,6 +191,11 @@ list (APPEND dbms_headers Dictionaries/DictionaryStructure.h Dictionaries/getDictionaryConfigurationFromAST.h) +# Required for validation of Timezone in session_timezone setting. +# This way we don't need to create timezone via cctz each time, but check against pregenerated char** +list (APPEND dbms_sources + Storages/System/StorageSystemTimeZones.generated.cpp) + if (NOT ENABLE_SSL) list (REMOVE_ITEM clickhouse_common_io_sources Common/OpenSSLHelpers.cpp) list (REMOVE_ITEM clickhouse_common_io_headers Common/OpenSSLHelpers.h) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 6af38586ed8..c0556519563 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -28,7 +28,7 @@ namespace ErrorCodes namespace { - bool checkIsExitingTimeZone(const std::string_view timezone) + bool checkIsExistingTimeZone(const std::string_view timezone) { for (auto * it = auto_time_zones; *it; ++it) if (timezone == *it) @@ -475,7 +475,7 @@ void SettingFieldTimezone::readBinary(ReadBuffer & in) void SettingFieldTimezone::validateTimezone(std::string_view str) { - if (!str.empty() && !checkIsExitingTimeZone(str)) + if (!str.empty() && !checkIsExistingTimeZone(str)) throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); } From f4af76ab8baee97c06cf1e53346da6107c7ccbbd Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 20 Apr 2023 11:58:51 +0200 Subject: [PATCH 049/515] cleanup cmakelists --- programs/library-bridge/CMakeLists.txt | 7 +------ programs/odbc-bridge/CMakeLists.txt | 6 +----- utils/check-marks/CMakeLists.txt | 6 +----- utils/keeper-data-dumper/CMakeLists.txt | 8 +------- utils/wal-dump/CMakeLists.txt | 6 +----- 5 files changed, 5 insertions(+), 28 deletions(-) diff --git a/programs/library-bridge/CMakeLists.txt b/programs/library-bridge/CMakeLists.txt index 97af7c3b22e..79497d5fb2e 100644 --- a/programs/library-bridge/CMakeLists.txt +++ b/programs/library-bridge/CMakeLists.txt @@ -13,17 +13,12 @@ set (CLICKHOUSE_LIBRARY_BRIDGE_SOURCES library-bridge.cpp ) -set(CLICKHOUSE_LIBRARY_BRIDGE_EXTERNAL_SOURCES - ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp -) - if (OS_LINUX) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") endif () clickhouse_add_executable(clickhouse-library-bridge - ${CLICKHOUSE_LIBRARY_BRIDGE_SOURCES} - ${CLICKHOUSE_LIBRARY_BRIDGE_EXTERNAL_SOURCES}) + ${CLICKHOUSE_LIBRARY_BRIDGE_SOURCES}) target_link_libraries(clickhouse-library-bridge PRIVATE daemon diff --git a/programs/odbc-bridge/CMakeLists.txt b/programs/odbc-bridge/CMakeLists.txt index bf1b42df026..118610e4dcd 100644 --- a/programs/odbc-bridge/CMakeLists.txt +++ b/programs/odbc-bridge/CMakeLists.txt @@ -15,17 +15,13 @@ set (CLICKHOUSE_ODBC_BRIDGE_SOURCES validateODBCConnectionString.cpp ) -set(ODBC_BRIDGE_EXTERNAL_SOURCES - ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp -) - if (OS_LINUX) # clickhouse-odbc-bridge is always a separate binary. # Reason: it must not export symbols from SSL, mariadb-client, etc. to not break ABI compatibility with ODBC drivers. set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") endif () -clickhouse_add_executable(clickhouse-odbc-bridge ${CLICKHOUSE_ODBC_BRIDGE_SOURCES} ${ODBC_BRIDGE_EXTERNAL_SOURCES}) +clickhouse_add_executable(clickhouse-odbc-bridge ${CLICKHOUSE_ODBC_BRIDGE_SOURCES}) target_link_libraries(clickhouse-odbc-bridge PRIVATE daemon diff --git a/utils/check-marks/CMakeLists.txt b/utils/check-marks/CMakeLists.txt index 456fb3d7112..05546a2989b 100644 --- a/utils/check-marks/CMakeLists.txt +++ b/utils/check-marks/CMakeLists.txt @@ -1,6 +1,2 @@ -set(CHECK_MARKS_EXTERNAL_SOURCES - ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp -) - -clickhouse_add_executable (check-marks ${CHECK_MARKS_EXTERNAL_SOURCES} main.cpp) +clickhouse_add_executable (check-marks main.cpp) target_link_libraries(check-marks PRIVATE dbms boost::program_options) diff --git a/utils/keeper-data-dumper/CMakeLists.txt b/utils/keeper-data-dumper/CMakeLists.txt index a6858a29e8b..1f55e50e68e 100644 --- a/utils/keeper-data-dumper/CMakeLists.txt +++ b/utils/keeper-data-dumper/CMakeLists.txt @@ -1,8 +1,2 @@ -set(KEEPER_DATA_DUMPER_EXTERNAL_SOURCES - ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp -) - -clickhouse_add_executable(keeper-data-dumper - ${KEEPER_DATA_DUMPER_EXTERNAL_SOURCES} - main.cpp) +clickhouse_add_executable(keeper-data-dumper main.cpp) target_link_libraries(keeper-data-dumper PRIVATE dbms) diff --git a/utils/wal-dump/CMakeLists.txt b/utils/wal-dump/CMakeLists.txt index 754799a6faf..3d59e95b4ca 100644 --- a/utils/wal-dump/CMakeLists.txt +++ b/utils/wal-dump/CMakeLists.txt @@ -1,6 +1,2 @@ -set(WAL_DUMP_EXTERNAL_SOURCES - ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp -) - -clickhouse_add_executable (wal-dump ${WAL_DUMP_EXTERNAL_SOURCES} main.cpp) +clickhouse_add_executable (wal-dump main.cpp) target_link_libraries(wal-dump PRIVATE dbms boost::program_options) From e37745811cd6000348655c7c42cdc25436a3090e Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 20 Apr 2023 12:04:12 +0200 Subject: [PATCH 050/515] style --- programs/library-bridge/CMakeLists.txt | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/programs/library-bridge/CMakeLists.txt b/programs/library-bridge/CMakeLists.txt index 79497d5fb2e..1cacc391ca5 100644 --- a/programs/library-bridge/CMakeLists.txt +++ b/programs/library-bridge/CMakeLists.txt @@ -17,8 +17,7 @@ if (OS_LINUX) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") endif () -clickhouse_add_executable(clickhouse-library-bridge - ${CLICKHOUSE_LIBRARY_BRIDGE_SOURCES}) +clickhouse_add_executable(clickhouse-library-bridge ${CLICKHOUSE_LIBRARY_BRIDGE_SOURCES}) target_link_libraries(clickhouse-library-bridge PRIVATE daemon From 65d23af611f28fbe0f2baf1dac6cd1aa317676b5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 24 Apr 2023 19:35:49 +0000 Subject: [PATCH 051/515] Refactor PreparedSets [1] --- src/Interpreters/PreparedSets.cpp | 101 +++++++++++-------- src/Interpreters/PreparedSets.h | 157 ++++++++++++++++++++++++++---- 2 files changed, 200 insertions(+), 58 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 7b0efddae87..11af41cae8f 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -5,6 +5,8 @@ #include #include #include +#include "Common/logger_useful.h" +#include "Processors/QueryPlan/CreatingSetsStep.h" namespace DB { @@ -66,30 +68,30 @@ String PreparedSetKey::toString() const return buf.str(); } -SubqueryForSet & PreparedSets::createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, - SizeLimits set_size_limit, bool transform_null_in) -{ - SubqueryForSet & subquery = subqueries[subquery_id]; +// SubqueryForSet & PreparedSets::createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, +// SizeLimits set_size_limit, bool transform_null_in) +// { +// SubqueryForSet & subquery = subqueries[subquery_id]; - /// If you already created a Set with the same subquery / table for another ast - /// In that case several PreparedSetKey would share same subquery and set - /// Not sure if it's really possible case (maybe for distributed query when set was filled by external table?) - if (subquery.set.isValid()) - sets[key] = subquery.set; - else - { - subquery.set_in_progress = std::make_shared(set_size_limit, false, transform_null_in); - sets[key] = FutureSet(subquery.promise_to_fill_set.get_future()); - } +// /// If you already created a Set with the same subquery / table for another ast +// /// In that case several PreparedSetKey would share same subquery and set +// /// Not sure if it's really possible case (maybe for distributed query when set was filled by external table?) +// if (subquery.set.isValid()) +// sets[key] = subquery.set; +// else +// { +// subquery.set_in_progress = std::make_shared(set_size_limit, false, transform_null_in); +// sets[key] = FutureSet(subquery.promise_to_fill_set.get_future()); +// } - if (!subquery.set_in_progress) - { - subquery.key = key.toString(); - subquery.set_in_progress = std::make_shared(set_size_limit, false, transform_null_in); - } +// if (!subquery.set_in_progress) +// { +// subquery.key = key.toString(); +// subquery.set_in_progress = std::make_shared(set_size_limit, false, transform_null_in); +// } - return subquery; -} +// return subquery; +// } /// If the subquery is not associated with any set, create default-constructed SubqueryForSet. /// It's aimed to fill external table passed to SubqueryForSet::createSource. @@ -154,26 +156,6 @@ QueryPlanPtr SubqueryForSet::detachSource() } -FutureSet::FutureSet(SetPtr set) -{ - std::promise promise; - promise.set_value(set); - *this = FutureSet(promise.get_future()); -} - - -bool FutureSet::isReady() const -{ - return future_set.valid() && - future_set.wait_for(std::chrono::seconds(0)) == std::future_status::ready; -} - -bool FutureSet::isCreated() const -{ - return isReady() && get() != nullptr && get()->isCreated(); -} - - std::variant, SharedSet> PreparedSetsCache::findOrPromiseToBuild(const String & key) { std::lock_guard lock(cache_mutex); @@ -194,4 +176,41 @@ std::variant, SharedSet> PreparedSetsCache::findOrPromiseTo return promise_to_fill_set; } +std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & context, bool create_ordered_set) +{ + if (set) + return nullptr; + + auto set_cache = context->getPreparedSetsCache(); + if (set_cache) + { + auto from_cache = set_cache->findOrPromiseToBuild(subquery.key); + if (from_cache.index() == 0) + { + subquery.promise_to_fill_set = std::move(std::get<0>(from_cache)); + } + else + { + LOG_TRACE(&Poco::Logger::get("FutureSetFromSubquery"), "Waiting for set, key: {}", subquery.key); + set = std::get<1>(from_cache).get(); + return nullptr; + } + } + + subquery.set = set = std::make_shared(size_limits, create_ordered_set, transform_null_in); + + auto plan = subquery.detachSource(); + + const Settings & settings = context->getSettingsRef(); + auto creating_set = std::make_unique( + plan->getCurrentDataStream(), + subquery_id, + std::move(subquery), + SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), + context); + creating_set->setStepDescription("Create set for subquery"); + plan->addStep(std::move(creating_set)); + return plan; +} + }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 4a7d1c3de46..a4898fe5ec6 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -10,6 +10,14 @@ #include #include #include +#include "Core/Block.h" +#include "Interpreters/Context.h" +#include "Interpreters/Set.h" +#include "Processors/Executors/CompletedPipelineExecutor.h" +#include "Processors/QueryPlan/BuildQueryPipelineSettings.h" +#include "Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h" +#include "Processors/Sinks/NullSink.h" +#include namespace DB { @@ -25,30 +33,83 @@ class InterpreterSelectWithUnionQuery; /// At analysis stage the FutureSets are created but not necessarily filled. Then for non-constant sets there /// must be an explicit step to build them before they can be used. /// FutureSet objects can be stored in PreparedSets and are not intended to be used from multiple threads. -class FutureSet final +// class FutureSet final +// { +// public: +// FutureSet() = default; + +// /// Create FutureSet from an object that will be created in the future. +// explicit FutureSet(const std::shared_future & future_set_) : future_set(future_set_) {} + +// /// Create FutureSet from a ready set. +// explicit FutureSet(SetPtr readySet); + +// /// The set object will be ready in the future, as opposed to 'null' object when FutureSet is default constructed. +// bool isValid() const { return future_set.valid(); } + +// /// The the value of SetPtr is ready, but the set object might not have been filled yet. +// bool isReady() const; + +// /// The set object is ready and filled. +// bool isCreated() const; + +// SetPtr get() const { chassert(isReady()); return future_set.get(); } + +// private: +// std::shared_future future_set; +// }; + +class FutureSet { public: - FutureSet() = default; + virtual ~FutureSet() = default; - /// Create FutureSet from an object that will be created in the future. - explicit FutureSet(const std::shared_future & future_set_) : future_set(future_set_) {} + virtual bool isReady() const = 0; + virtual SetPtr get() const = 0; - /// Create FutureSet from a ready set. - explicit FutureSet(SetPtr readySet); + virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; + virtual std::unique_ptr build(const ContextPtr & context) = 0; +}; - /// The set object will be ready in the future, as opposed to 'null' object when FutureSet is default constructed. - bool isValid() const { return future_set.valid(); } +using FutureSetPtr = std::unique_ptr; - /// The the value of SetPtr is ready, but the set object might not have been filled yet. - bool isReady() const; +class FutureSetFromTuple final : public FutureSet +{ +public: + FutureSetFromTuple(Block block_, const SizeLimits & size_limits_, bool transform_null_in_); - /// The set object is ready and filled. - bool isCreated() const; + bool isReady() const override { return set != nullptr; } + SetPtr get() const override { return set; } - SetPtr get() const { chassert(isReady()); return future_set.get(); } + SetPtr buildOrderedSetInplace(const ContextPtr &) override + { + fill(true); + return set; + } + + std::unique_ptr build(const ContextPtr &) override + { + fill(false); + return nullptr; + } private: - std::shared_future future_set; + Block block; + SizeLimits size_limits; + bool transform_null_in; + + SetPtr set; + + void fill(bool create_ordered_set) + { + if (set) + return; + + set = std::make_shared(size_limits, create_ordered_set, transform_null_in); + set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); + set->insertFromBlock(block.getColumnsWithTypeAndName()); + set->finishInsert(); + } }; /// Information on how to build set for the [GLOBAL] IN section. @@ -66,11 +127,12 @@ public: /// Build this set from the result of the subquery. String key; - SetPtr set_in_progress; + SetPtr set; /// After set_in_progress is finished it will be put into promise_to_fill_set and thus all FutureSet's /// that are referencing this set will be filled. + std::promise promise_to_fill_set; - FutureSet set = FutureSet{promise_to_fill_set.get_future()}; + // FutureSet set = FutureSet{promise_to_fill_set.get_future()}; /// If set, put the result into the table. /// This is a temporary table for transferring to remote servers for distributed query processing. @@ -80,6 +142,67 @@ public: std::unique_ptr source; }; +class FutureSetFromSubquery : public FutureSet +{ +public: + FutureSetFromSubquery(SubqueryForSet subquery_, String subquery_id, SizeLimits set_size_limit_, bool transform_null_in_); + + bool isReady() const override { return set != nullptr; } + SetPtr get() const override { return set; } + + SetPtr buildOrderedSetInplace(const ContextPtr & context) override + { + auto plan = buildPlan(context, true); + + auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + pipeline.complete(std::make_shared(Block())); + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); + + return set; + } + + std::unique_ptr build(const ContextPtr & context) override + { + return buildPlan(context, false); + } + +private: + SetPtr set; + SubqueryForSet subquery; + String subquery_id; + SizeLimits size_limits; + bool transform_null_in; + + std::unique_ptr buildPlan(const ContextPtr & context, bool create_ordered_set); +}; + +// class FutureSetFromFuture : public FutureSet +// { +// public: +// FutureSetFromFuture(std::shared_future future_set_); + +// bool isReady() const override { return future_set.wait_for(std::chrono::seconds(0)) == std::future_status::ready; } +// SetPtr get() const override { return future_set.get(); } + +// SetPtr buildOrderedSetInplace(const ContextPtr &) override +// { +// fill(true); +// return set; +// } + +// std::unique_ptr build(const ContextPtr &) override +// { +// fill(false); +// return nullptr; +// } + +// private: +// std::shared_future future_set; +// } + struct PreparedSetKey { /// Prepared sets for tuple literals are indexed by the hash of the tree contents and by the desired @@ -132,7 +255,7 @@ private: std::unordered_map sets; /// This is the information required for building sets - SubqueriesForSets subqueries; + // SubqueriesForSets subqueries; }; using PreparedSetsPtr = std::shared_ptr; From 80a2f30a0cc651db608735391d5173452f7b41ff Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 25 Apr 2023 18:14:08 +0000 Subject: [PATCH 052/515] Refactor PreparedSets [2] --- src/Columns/ColumnSet.h | 4 +- src/Interpreters/ActionsVisitor.cpp | 56 +++--- src/Interpreters/ActionsVisitor.h | 7 +- src/Interpreters/ExpressionAnalyzer.cpp | 188 ++++++++++----------- src/Interpreters/ExpressionAnalyzer.h | 4 +- src/Interpreters/GlobalSubqueriesVisitor.h | 24 +-- src/Interpreters/PreparedSets.cpp | 110 +++++++++--- src/Interpreters/PreparedSets.h | 75 +++++--- src/Planner/PlannerContext.h | 20 +-- 9 files changed, 283 insertions(+), 205 deletions(-) diff --git a/src/Columns/ColumnSet.h b/src/Columns/ColumnSet.h index 3f5cf4ad280..ccd9aa19896 100644 --- a/src/Columns/ColumnSet.h +++ b/src/Columns/ColumnSet.h @@ -21,7 +21,7 @@ class ColumnSet final : public COWHelper private: friend class COWHelper; - ColumnSet(size_t s_, FutureSet data_) : data(std::move(data_)) { s = s_; } + ColumnSet(size_t s_, FutureSetPtr data_) : data(std::move(data_)) { s = s_; } ColumnSet(const ColumnSet &) = default; public: @@ -35,7 +35,7 @@ public: Field operator[](size_t) const override { return {}; } private: - FutureSet data; + FutureSetPtr data; }; } diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 3bb3ea67e29..00feecb44f5 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -422,9 +422,8 @@ Block createBlockForSet( } -SetPtr makeExplicitSet( - const ASTFunction * node, const ActionsDAG & actions, bool create_ordered_set, - ContextPtr context, const SizeLimits & size_limits, PreparedSets & prepared_sets) +FutureSetPtr makeExplicitSet( + const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets) { const IAST & args = *node->arguments; @@ -448,7 +447,7 @@ SetPtr makeExplicitSet( element_type = low_cardinality_type->getDictionaryType(); auto set_key = PreparedSetKey::forLiteral(*right_arg, set_element_types); - if (auto set = prepared_sets.get(set_key)) + if (auto set = prepared_sets.getFuture(set_key)) return set; /// Already prepared. Block block; @@ -458,14 +457,7 @@ SetPtr makeExplicitSet( else block = createBlockForSet(left_arg_type, right_arg, set_element_types, context); - SetPtr set - = std::make_shared(size_limits, create_ordered_set, context->getSettingsRef().transform_null_in); - set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); - set->insertFromBlock(block.getColumnsWithTypeAndName()); - set->finishInsert(); - - prepared_sets.set(set_key, set); - return set; + return prepared_sets.addFromTuple(set_key, block); } class ScopeStack::Index @@ -952,7 +944,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & return; } - FutureSet prepared_set; + FutureSetPtr prepared_set; if (checkFunctionIsInOrGlobalInOperator(node)) { /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). @@ -961,7 +953,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (!data.no_makeset && !(data.is_create_parameterized_view && !analyzeReceiveQueryParams(ast).empty())) prepared_set = makeSet(node, data, data.no_subqueries); - if (prepared_set.isValid()) + if (prepared_set) { /// Transform tuple or subquery into a set. } @@ -1174,14 +1166,14 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & num_arguments += columns.size() - 1; arg += columns.size() - 1; } - else if (checkFunctionIsInOrGlobalInOperator(node) && arg == 1 && prepared_set.isValid()) + else if (checkFunctionIsInOrGlobalInOperator(node) && arg == 1 && prepared_set) { ColumnWithTypeAndName column; column.type = std::make_shared(); /// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name, /// so that sets with the same literal representation do not fuse together (they can have different types). - const bool is_constant_set = prepared_set.isCreated(); + const bool is_constant_set = typeid_cast(prepared_set.get()) == nullptr; if (is_constant_set) column.name = data.getUniqueName("__set"); else @@ -1373,7 +1365,7 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */, data.addColumn(std::move(column)); } -FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_subqueries) +FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_subqueries) { if (!data.prepared_sets) return {}; @@ -1394,11 +1386,8 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no return {}; auto set_key = PreparedSetKey::forSubquery(*right_in_operand); - { - auto set = data.prepared_sets->getFuture(set_key); - if (set.isValid()) - return set; - } + if (auto set = data.prepared_sets->getFuture(set_key)) + return set; /// A special case is if the name of the table is specified on the right side of the IN statement, /// and the table has the type Set (a previously prepared set). @@ -1409,20 +1398,16 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no if (table) { - StorageSet * storage_set = dynamic_cast(table.get()); - if (storage_set) - { - SetPtr set = storage_set->getSet(); - data.prepared_sets->set(set_key, set); - return FutureSet(set); - } + if (StorageSet * storage_set = dynamic_cast(table.get())) + return data.prepared_sets->addFromStorage(set_key, storage_set->getSet()); } } /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. - String set_id = right_in_operand->getColumnName(); - bool transform_null_in = data.getContext()->getSettingsRef().transform_null_in; - SubqueryForSet & subquery_for_set = data.prepared_sets->createOrGetSubquery(set_id, set_key, data.set_size_limit, transform_null_in); + // String set_id = right_in_operand->getColumnName(); + //bool transform_null_in = data.getContext()->getSettingsRef().transform_null_in; + SubqueryForSet subquery_for_set; // = data.prepared_sets->createOrGetSubquery(set_id, set_key, data.set_size_limit, transform_null_in); + subquery_for_set.key = right_in_operand->getColumnName(); /** The following happens for GLOBAL INs or INs: * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, @@ -1432,13 +1417,13 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no * In case that we have HAVING with IN subquery, we have to force creating set for it. * Also it doesn't make sense if it is GLOBAL IN or ordinary IN. */ - if (data.create_source_for_in && !subquery_for_set.hasSource()) + if (data.create_source_for_in) { auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth, {}); subquery_for_set.createSource(*interpreter); } - return subquery_for_set.set; + return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set)); } else { @@ -1446,8 +1431,7 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no const auto & index = data.actions_stack.getLastActionsIndex(); if (data.prepared_sets && index.contains(left_in_operand->getColumnName())) /// An explicit enumeration of values in parentheses. - return FutureSet( - makeExplicitSet(&node, last_actions, false, data.getContext(), data.set_size_limit, *data.prepared_sets)); + return makeExplicitSet(&node, last_actions, data.getContext(), *data.prepared_sets); else return {}; } diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 260fd5ab2c0..71d57620196 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -25,9 +25,8 @@ class IFunctionOverloadResolver; using FunctionOverloadResolverPtr = std::shared_ptr; /// The case of an explicit enumeration of values. -SetPtr makeExplicitSet( - const ASTFunction * node, const ActionsDAG & actions, bool create_ordered_set, - ContextPtr context, const SizeLimits & limits, PreparedSets & prepared_sets); +FutureSetPtr makeExplicitSet( + const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets); /** For ActionsVisitor * A stack of ExpressionActions corresponding to nested lambda expressions. @@ -219,7 +218,7 @@ private: static void visit(const ASTLiteral & literal, const ASTPtr & ast, Data & data); static void visit(ASTExpressionList & expression_list, const ASTPtr & ast, Data & data); - static FutureSet makeSet(const ASTFunction & node, Data & data, bool no_subqueries); + static FutureSetPtr makeSet(const ASTFunction & node, Data & data, bool no_subqueries); static ASTs doUntuple(const ASTFunction * function, ActionsMatcher::Data & data); static std::optional getNameAndTypeFromAST(const ASTPtr & ast, Data & data); }; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index cc54e7620f6..7dece81734c 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -450,76 +450,76 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global, b } -void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options) -{ - if (!prepared_sets) - return; +// void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options) +// { +// if (!prepared_sets) +// return; - auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name); +// auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name); - if (prepared_sets->getFuture(set_key).isValid()) - return; /// Already prepared. +// if (prepared_sets->getFuture(set_key).isValid()) +// return; /// Already prepared. - if (auto set_ptr_from_storage_set = isPlainStorageSetInSubquery(subquery_or_table_name)) - { - prepared_sets->set(set_key, set_ptr_from_storage_set); - return; - } +// if (auto set_ptr_from_storage_set = isPlainStorageSetInSubquery(subquery_or_table_name)) +// { +// prepared_sets->set(set_key, set_ptr_from_storage_set); +// return; +// } - auto build_set = [&] () -> SetPtr - { - LOG_TRACE(getLogger(), "Building set, key: {}", set_key.toString()); +// auto build_set = [&] () -> SetPtr +// { +// LOG_TRACE(getLogger(), "Building set, key: {}", set_key.toString()); - auto interpreter_subquery = interpretSubquery(subquery_or_table_name, getContext(), {}, query_options); - auto io = interpreter_subquery->execute(); - PullingAsyncPipelineExecutor executor(io.pipeline); +// auto interpreter_subquery = interpretSubquery(subquery_or_table_name, getContext(), {}, query_options); +// auto io = interpreter_subquery->execute(); +// PullingAsyncPipelineExecutor executor(io.pipeline); - SetPtr set = std::make_shared(settings.size_limits_for_set_used_with_index, true, getContext()->getSettingsRef().transform_null_in); - set->setHeader(executor.getHeader().getColumnsWithTypeAndName()); +// SetPtr set = std::make_shared(settings.size_limits_for_set_used_with_index, true, getContext()->getSettingsRef().transform_null_in); +// set->setHeader(executor.getHeader().getColumnsWithTypeAndName()); - Block block; - while (executor.pull(block)) - { - if (block.rows() == 0) - continue; +// Block block; +// while (executor.pull(block)) +// { +// if (block.rows() == 0) +// continue; - /// If the limits have been exceeded, give up and let the default subquery processing actions take place. - if (!set->insertFromBlock(block.getColumnsWithTypeAndName())) - return nullptr; - } +// /// If the limits have been exceeded, give up and let the default subquery processing actions take place. +// if (!set->insertFromBlock(block.getColumnsWithTypeAndName())) +// return nullptr; +// } - set->finishInsert(); +// set->finishInsert(); - return set; - }; +// return set; +// }; - SetPtr set; +// SetPtr set; - auto set_cache = getContext()->getPreparedSetsCache(); - if (set_cache) - { - auto from_cache = set_cache->findOrPromiseToBuild(set_key.toString()); - if (from_cache.index() == 0) - { - set = build_set(); - std::get<0>(from_cache).set_value(set); - } - else - { - LOG_TRACE(getLogger(), "Waiting for set, key: {}", set_key.toString()); - set = std::get<1>(from_cache).get(); - } - } - else - { - set = build_set(); - } +// auto set_cache = getContext()->getPreparedSetsCache(); +// if (set_cache) +// { +// auto from_cache = set_cache->findOrPromiseToBuild(set_key.toString()); +// if (from_cache.index() == 0) +// { +// set = build_set(); +// std::get<0>(from_cache).set_value(set); +// } +// else +// { +// LOG_TRACE(getLogger(), "Waiting for set, key: {}", set_key.toString()); +// set = std::get<1>(from_cache).get(); +// } +// } +// else +// { +// set = build_set(); +// } - if (!set) - return; +// if (!set) +// return; - prepared_sets->set(set_key, std::move(set)); -} +// prepared_sets->set(set_key, std::move(set)); +// } SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name) { @@ -536,50 +536,50 @@ SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_o /// Performance optimization for IN() if storage supports it. -void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) -{ - if (!node || !storage() || !storage()->supportsIndexForIn()) - return; +// void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) +// { +// if (!node || !storage() || !storage()->supportsIndexForIn()) +// return; - for (auto & child : node->children) - { - /// Don't descend into subqueries. - if (child->as()) - continue; +// for (auto & child : node->children) +// { +// /// Don't descend into subqueries. +// if (child->as()) +// continue; - /// Don't descend into lambda functions - const auto * func = child->as(); - if (func && func->name == "lambda") - continue; +// /// Don't descend into lambda functions +// const auto * func = child->as(); +// if (func && func->name == "lambda") +// continue; - makeSetsForIndex(child); - } +// makeSetsForIndex(child); +// } - const auto * func = node->as(); - if (func && functionIsInOrGlobalInOperator(func->name)) - { - const IAST & args = *func->arguments; - const ASTPtr & left_in_operand = args.children.at(0); +// const auto * func = node->as(); +// if (func && functionIsInOrGlobalInOperator(func->name)) +// { +// const IAST & args = *func->arguments; +// const ASTPtr & left_in_operand = args.children.at(0); - if (storage()->mayBenefitFromIndexForIn(left_in_operand, getContext(), metadata_snapshot)) - { - const ASTPtr & arg = args.children.at(1); - if (arg->as() || arg->as()) - { - if (settings.use_index_for_in_with_subqueries) - tryMakeSetForIndexFromSubquery(arg, query_options); - } - else - { - auto temp_actions = std::make_shared(columns_after_join); - getRootActions(left_in_operand, true, temp_actions); +// if (storage()->mayBenefitFromIndexForIn(left_in_operand, getContext(), metadata_snapshot)) +// { +// const ASTPtr & arg = args.children.at(1); +// if (arg->as() || arg->as()) +// { +// if (settings.use_index_for_in_with_subqueries) +// tryMakeSetForIndexFromSubquery(arg, query_options); +// } +// else +// { +// auto temp_actions = std::make_shared(columns_after_join); +// getRootActions(left_in_operand, true, temp_actions); - if (prepared_sets && temp_actions->tryFindInOutputs(left_in_operand->getColumnName())) - makeExplicitSet(func, *temp_actions, true, getContext(), settings.size_limits_for_set, *prepared_sets); - } - } - } -} +// if (prepared_sets && temp_actions->tryFindInOutputs(left_in_operand->getColumnName())) +// makeExplicitSet(func, *temp_actions, true, getContext(), settings.size_limits_for_set, *prepared_sets); +// } +// } +// } +// } void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts) diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 1b6e8e24091..e4926b80625 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -144,7 +144,7 @@ public: /** Create Set from a subquery or a table expression in the query. The created set is suitable for using the index. * The set will not be created if its size hits the limit. */ - void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options = {}); + // void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options = {}); /** Checks if subquery is not a plain StorageSet. * Because while making set we will read data from StorageSet which is not allowed. @@ -364,7 +364,7 @@ public: ActionsDAGPtr appendProjectResult(ExpressionActionsChain & chain) const; /// Create Set-s that we make from IN section to use index on them. - void makeSetsForIndex(const ASTPtr & node); + // void makeSetsForIndex(const ASTPtr & node); private: StorageMetadataPtr metadata_snapshot; diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index b105cae31c6..9fb2c02bd58 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -167,19 +167,21 @@ public: { /// Do not materialize external tables if it's explain statement. } - else if (getContext()->getSettingsRef().use_index_for_in_with_subqueries) - { - auto external_table = external_storage_holder->getTable(); - auto table_out = external_table->write({}, external_table->getInMemoryMetadataPtr(), getContext()); - auto io = interpreter->execute(); - io.pipeline.complete(std::move(table_out)); - CompletedPipelineExecutor executor(io.pipeline); - executor.execute(); - } + // else if (getContext()->getSettingsRef().use_index_for_in_with_subqueries) + // { + // auto external_table = external_storage_holder->getTable(); + // auto table_out = external_table->write({}, external_table->getInMemoryMetadataPtr(), getContext()); + // auto io = interpreter->execute(); + // io.pipeline.complete(std::move(table_out)); + // CompletedPipelineExecutor executor(io.pipeline); + // executor.execute(); + // } else { - auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); - subquery_for_set.createSource(*interpreter, external_storage); + // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); + // subquery_for_set.createSource(*interpreter, external_storage); + auto key = subquery_or_table_name->getColumnName(); + prepared_sets->addStorageToSubquery(key, std::move(external_storage)); } /** NOTE If it was written IN tmp_table - the existing temporary (but not external) table, diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 11af41cae8f..300ef2aadb6 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -95,11 +95,51 @@ String PreparedSetKey::toString() const /// If the subquery is not associated with any set, create default-constructed SubqueryForSet. /// It's aimed to fill external table passed to SubqueryForSet::createSource. -SubqueryForSet & PreparedSets::getSubquery(const String & subquery_id) { return subqueries[subquery_id]; } +void PreparedSets::addStorageToSubquery(const String & subquery_id, StoragePtr storage) +{ + auto it = subqueries.find(subquery_id); + if (it == subqueries.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find subquery {}", subquery_id); -void PreparedSets::set(const PreparedSetKey & key, SetPtr set_) { sets[key] = FutureSet(set_); } + it->second->addStorage(std::move(storage)); +} -FutureSet PreparedSets::getFuture(const PreparedSetKey & key) const +FutureSetPtr PreparedSets::addFromStorage(const PreparedSetKey & key, SetPtr set_) +{ + auto from_storage = std::make_shared(std::move(set_)); + auto [it, inserted] = sets.emplace(key, std::move(from_storage)); + + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + + return it->second; +} + +FutureSetPtr PreparedSets::addFromTuple(const PreparedSetKey & key, Block block) +{ + auto from_tuple = std::make_shared(std::move(block)); + auto [it, inserted] = sets.emplace(key, std::move(from_tuple)); + + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + + return it->second; +} + +FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery) +{ + auto id = subquery.key; + auto from_subquery = std::make_shared(std::move(subquery)); + auto [it, inserted] = sets.emplace(key, from_subquery); + + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + + subqueries.emplace(id, std::move(from_subquery)); + return it->second; +} + +FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const { auto it = sets.find(key); if (it == sets.end()) @@ -107,24 +147,24 @@ FutureSet PreparedSets::getFuture(const PreparedSetKey & key) const return it->second; } -SetPtr PreparedSets::get(const PreparedSetKey & key) const -{ - auto it = sets.find(key); - if (it == sets.end() || !it->second.isReady()) - return nullptr; - return it->second.get(); -} +// SetPtr PreparedSets::get(const PreparedSetKey & key) const +// { +// auto it = sets.find(key); +// if (it == sets.end() || !it->second.isReady()) +// return nullptr; +// return it->second.get(); +// } -std::vector PreparedSets::getByTreeHash(IAST::Hash ast_hash) const -{ - std::vector res; - for (const auto & it : this->sets) - { - if (it.first.ast_hash == ast_hash) - res.push_back(it.second); - } - return res; -} +// std::vector PreparedSets::getByTreeHash(IAST::Hash ast_hash) const +// { +// std::vector res; +// for (const auto & it : this->sets) +// { +// if (it.first.ast_hash == ast_hash) +// res.push_back(it.second); +// } +// return res; +// } PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries() { @@ -197,14 +237,18 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c } } - subquery.set = set = std::make_shared(size_limits, create_ordered_set, transform_null_in); + + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); + + subquery.set = set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); auto plan = subquery.detachSource(); + auto description = subquery.key; - const Settings & settings = context->getSettingsRef(); auto creating_set = std::make_unique( plan->getCurrentDataStream(), - subquery_id, + description, std::move(subquery), SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), context); @@ -213,4 +257,24 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c return plan; } + +static SizeLimits getSizeLimitsForUnorderedSet(const Settings & settings) +{ + return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); +} + +static SizeLimits getSizeLimitsForOrderedSet(const Settings & settings) +{ + if (settings.use_index_for_in_with_subqueries_max_values && + settings.use_index_for_in_with_subqueries_max_values < settings.max_rows_in_set) + return getSizeLimitsForUnorderedSet(settings); + + return SizeLimits(settings.use_index_for_in_with_subqueries_max_values, settings.max_bytes_in_set, OverflowMode::BREAK); +} + +SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordered_set) +{ + return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); +} + }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index a4898fe5ec6..8ebabc32b0a 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -69,43 +69,46 @@ public: virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; virtual std::unique_ptr build(const ContextPtr & context) = 0; + + static SizeLimits getSizeLimitsForSet(const Settings & settings, bool ordered_set); }; -using FutureSetPtr = std::unique_ptr; +using FutureSetPtr = std::shared_ptr; class FutureSetFromTuple final : public FutureSet { public: - FutureSetFromTuple(Block block_, const SizeLimits & size_limits_, bool transform_null_in_); + FutureSetFromTuple(Block block_); bool isReady() const override { return set != nullptr; } SetPtr get() const override { return set; } - SetPtr buildOrderedSetInplace(const ContextPtr &) override + SetPtr buildOrderedSetInplace(const ContextPtr & context) override { - fill(true); + fill(context, true); return set; } - std::unique_ptr build(const ContextPtr &) override + std::unique_ptr build(const ContextPtr & context) override { - fill(false); + fill(context, false); return nullptr; } private: Block block; - SizeLimits size_limits; - bool transform_null_in; SetPtr set; - void fill(bool create_ordered_set) + void fill(const ContextPtr & context, bool create_ordered_set) { if (set) return; - set = std::make_shared(size_limits, create_ordered_set, transform_null_in); + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); + + set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); set->insertFromBlock(block.getColumnsWithTypeAndName()); set->finishInsert(); @@ -145,13 +148,16 @@ public: class FutureSetFromSubquery : public FutureSet { public: - FutureSetFromSubquery(SubqueryForSet subquery_, String subquery_id, SizeLimits set_size_limit_, bool transform_null_in_); + FutureSetFromSubquery(SubqueryForSet subquery_); bool isReady() const override { return set != nullptr; } SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override { + if (!context->getSettingsRef().use_index_for_in_with_subqueries) + return nullptr; + auto plan = buildPlan(context, true); auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); @@ -169,16 +175,34 @@ public: return buildPlan(context, false); } + void addStorage(StoragePtr storage) { subquery.table = std::move(storage); } + private: SetPtr set; SubqueryForSet subquery; - String subquery_id; - SizeLimits size_limits; - bool transform_null_in; std::unique_ptr buildPlan(const ContextPtr & context, bool create_ordered_set); }; +class FutureSetFromStorage : public FutureSet +{ +public: + FutureSetFromStorage(SetPtr set_); // : set(std::move(set_) {} + + bool isReady() const override { return set != nullptr; } + SetPtr get() const override { return set; } + + SetPtr buildOrderedSetInplace(const ContextPtr &) override + { + return set->hasExplicitSetElements() ? set : nullptr; + } + + std::unique_ptr build(const ContextPtr &) override { return nullptr; } + +private: + SetPtr set; +}; + // class FutureSetFromFuture : public FutureSet // { // public: @@ -230,15 +254,20 @@ struct PreparedSetKey class PreparedSets { public: - using SubqueriesForSets = std::unordered_map; + using SubqueriesForSets = std::unordered_map>; - SubqueryForSet & createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, - SizeLimits set_size_limit, bool transform_null_in); - SubqueryForSet & getSubquery(const String & subquery_id); + // SubqueryForSet & createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, + // SizeLimits set_size_limit, bool transform_null_in); - void set(const PreparedSetKey & key, SetPtr set_); - FutureSet getFuture(const PreparedSetKey & key) const; - SetPtr get(const PreparedSetKey & key) const; + FutureSetPtr addFromStorage(const PreparedSetKey & key, SetPtr set_); + FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block); + FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery); + + void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage); + + FutureSetPtr getFuture(const PreparedSetKey & key) const; + //SubqueryForSet & getSubquery(const String & subquery_id); + // SetPtr get(const PreparedSetKey & key) const; /// Get subqueries and clear them. /// We need to build a plan for subqueries just once. That's why we can clear them after accessing them. @@ -252,10 +281,10 @@ public: bool empty() const; private: - std::unordered_map sets; + std::unordered_map sets; /// This is the information required for building sets - // SubqueriesForSets subqueries; + SubqueriesForSets subqueries; }; using PreparedSetsPtr = std::shared_ptr; diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index ccc4ab43638..9ecfdb6117a 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -57,18 +57,18 @@ class PlannerSet { public: /// Construct planner set that is ready for execution - explicit PlannerSet(FutureSet set_) + explicit PlannerSet(FutureSetPtr set_) : set(std::move(set_)) {} /// Construct planner set with set and subquery node explicit PlannerSet(QueryTreeNodePtr subquery_node_) - : set(promise_to_build_set.get_future()) - , subquery_node(std::move(subquery_node_)) + //: set(promise_to_build_set.get_future()) + : subquery_node(std::move(subquery_node_)) {} /// Get a reference to a set that might be not built yet - const FutureSet & getSet() const + const FutureSetPtr & getSet() const { return set; } @@ -80,14 +80,14 @@ public: } /// This promise will be fulfilled when set is built and all FutureSet objects will become ready - std::promise extractPromiseToBuildSet() - { - return std::move(promise_to_build_set); - } + // std::promise extractPromiseToBuildSet() + // { + // return std::move(promise_to_build_set); + // } private: - std::promise promise_to_build_set; - FutureSet set; + //std::promise promise_to_build_set; + FutureSetPtr set; QueryTreeNodePtr subquery_node; }; From f598a39ea261161cefa363f21112ddb49c7300a5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 4 May 2023 17:54:08 +0000 Subject: [PATCH 053/515] Refactor PreparedSets [3] --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 18 ++- src/Analyzer/SetUtils.cpp | 24 ++-- src/Analyzer/SetUtils.h | 3 +- src/Columns/ColumnSet.h | 2 +- src/DataTypes/DataTypeSet.h | 2 +- src/Functions/in.cpp | 21 +++- src/Interpreters/ActionsVisitor.cpp | 4 +- src/Interpreters/ExpressionActions.cpp | 2 +- src/Interpreters/GlobalSubqueriesVisitor.h | 14 ++- src/Interpreters/InterpreterSelectQuery.cpp | 11 +- src/Interpreters/PreparedSets.cpp | 22 ++-- src/Interpreters/PreparedSets.h | 42 ++++--- src/Planner/CollectSets.cpp | 61 ++++++++-- src/Planner/CollectSets.h | 4 +- src/Planner/Planner.cpp | 90 ++++++++------- src/Planner/PlannerActionsVisitor.cpp | 35 ++++-- src/Planner/PlannerContext.cpp | 72 ++++++------ src/Planner/PlannerContext.h | 107 +++++++++--------- src/Planner/PlannerJoinTree.cpp | 21 ++-- src/Planner/Utils.cpp | 3 +- src/Planner/Utils.h | 1 + src/Processors/QueryPlan/CreatingSetsStep.cpp | 72 +++++++++--- src/Processors/QueryPlan/CreatingSetsStep.h | 19 +++- .../QueryPlan/Optimizations/Optimizations.h | 1 + .../Optimizations/addPlansForSets.cpp | 35 ++++++ .../QueryPlan/Optimizations/optimizeTree.cpp | 1 + src/Processors/QueryPlan/QueryPlan.cpp | 5 + src/Processors/QueryPlan/QueryPlan.h | 5 +- .../Transforms/CreatingSetsTransform.cpp | 24 ++-- src/Storages/KVStorageUtils.cpp | 13 ++- src/Storages/MergeTree/KeyCondition.cpp | 20 +++- .../MergeTreeIndexConditionBloomFilter.cpp | 8 +- .../MergeTree/MergeTreeIndexFullText.cpp | 9 +- .../MergeTree/MergeTreeIndexInverted.cpp | 9 +- src/Storages/MergeTree/RPNBuilder.cpp | 42 ++++--- src/Storages/MergeTree/RPNBuilder.h | 6 +- .../System/StorageSystemZooKeeper.cpp | 11 +- src/Storages/VirtualColumnUtils.cpp | 38 +++---- 38 files changed, 585 insertions(+), 292 deletions(-) create mode 100644 src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 8afb9078fae..78e8b1760cb 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5117,14 +5117,26 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi const auto & second_argument_constant_literal = second_argument_constant_node->getValue(); const auto & second_argument_constant_type = second_argument_constant_node->getResultType(); - auto set = makeSetForConstantValue(first_argument_constant_type, + const auto & settings = scope.context->getSettingsRef(); + + auto result_block = makeSetForConstantValue(first_argument_constant_type, second_argument_constant_literal, second_argument_constant_type, - scope.context->getSettingsRef()); + settings.transform_null_in); + + SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; + + auto set = std::make_shared(size_limits_for_set, true /*fill_set_elements*/, settings.transform_null_in); + + set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName()); + set->insertFromBlock(result_block.getColumnsWithTypeAndName()); + set->finishInsert(); + + auto future_set = std::make_shared(std::move(set)); /// Create constant set column for constant folding - auto column_set = ColumnSet::create(1, FutureSet(std::move(set))); + auto column_set = ColumnSet::create(1, std::move(future_set)); argument_columns[1].column = ColumnConst::create(std::move(column_set), 1); } diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index 0fb075e925b..2fb05449714 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -118,7 +118,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes & } -SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, const Settings & settings) +Block makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in) { DataTypes set_element_types = {expression_type}; const auto * lhs_tuple_type = typeid_cast(expression_type.get()); @@ -135,8 +135,8 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field size_t lhs_type_depth = getCompoundTypeDepth(*expression_type); size_t rhs_type_depth = getCompoundTypeDepth(*value_type); - SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; - bool tranform_null_in = settings.transform_null_in; + // SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; + // bool transform_null_in = settings.transform_null_in; Block result_block; @@ -145,7 +145,7 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field /// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc. Array array{value}; - result_block = createBlockFromCollection(array, set_element_types, tranform_null_in); + result_block = createBlockFromCollection(array, set_element_types, transform_null_in); } else if (lhs_type_depth + 1 == rhs_type_depth) { @@ -154,9 +154,9 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field WhichDataType rhs_which_type(value_type); if (rhs_which_type.isArray()) - result_block = createBlockFromCollection(value.get(), set_element_types, tranform_null_in); + result_block = createBlockFromCollection(value.get(), set_element_types, transform_null_in); else if (rhs_which_type.isTuple()) - result_block = createBlockFromCollection(value.get(), set_element_types, tranform_null_in); + result_block = createBlockFromCollection(value.get(), set_element_types, transform_null_in); else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Unsupported type at the right-side of IN. Expected Array or Tuple. Actual {}", @@ -170,13 +170,15 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field value_type->getName()); } - auto set = std::make_shared(size_limits_for_set, true /*fill_set_elements*/, tranform_null_in); + return result_block; - set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName()); - set->insertFromBlock(result_block.getColumnsWithTypeAndName()); - set->finishInsert(); + // auto set = std::make_shared(size_limits_for_set, true /*fill_set_elements*/, tranform_null_in); - return set; + // set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName()); + // set->insertFromBlock(result_block.getColumnsWithTypeAndName()); + // set->finishInsert(); + + // return set; } } diff --git a/src/Analyzer/SetUtils.h b/src/Analyzer/SetUtils.h index 7afc8e5259c..fdeaa4a3c48 100644 --- a/src/Analyzer/SetUtils.h +++ b/src/Analyzer/SetUtils.h @@ -21,10 +21,9 @@ using SetPtr = std::shared_ptr; * @param expression_type - type of first argument of function IN. * @param value - constant value of second argument of function IN. * @param value_type - type of second argument of function IN. - * @param settings - query settings. * * @return SetPtr for constant value. */ -SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, const Settings & settings); +Block makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in); } diff --git a/src/Columns/ColumnSet.h b/src/Columns/ColumnSet.h index ccd9aa19896..935a72e551a 100644 --- a/src/Columns/ColumnSet.h +++ b/src/Columns/ColumnSet.h @@ -29,7 +29,7 @@ public: TypeIndex getDataType() const override { return TypeIndex::Set; } MutableColumnPtr cloneDummy(size_t s_) const override { return ColumnSet::create(s_, data); } - ConstSetPtr getData() const { if (!data.isReady()) return nullptr; return data.get(); } + FutureSetPtr getData() const { return data; } // Used only for debugging, making it DUMPABLE Field operator[](size_t) const override { return {}; } diff --git a/src/DataTypes/DataTypeSet.h b/src/DataTypes/DataTypeSet.h index 7ddfeb9fe30..7ef0d931279 100644 --- a/src/DataTypes/DataTypeSet.h +++ b/src/DataTypes/DataTypeSet.h @@ -20,7 +20,7 @@ public: bool isParametric() const override { return true; } // Used for expressions analysis. - MutableColumnPtr createColumn() const override { return ColumnSet::create(0, FutureSet{}); } + MutableColumnPtr createColumn() const override { return ColumnSet::create(0, nullptr); } // Used only for debugging, making it DUMPABLE Field getDefault() const override { return Tuple(); } diff --git a/src/Functions/in.cpp b/src/Functions/in.cpp index 7a41ae2e3ea..eb623951bf3 100644 --- a/src/Functions/in.cpp +++ b/src/Functions/in.cpp @@ -55,9 +55,13 @@ public: /// It is needed to perform type analysis without creation of set. static constexpr auto name = FunctionInName::name; - static FunctionPtr create(ContextPtr) + FunctionIn(SizeLimits size_limits_, bool transform_null_in_) + : size_limits(std::move(size_limits_)), transform_null_in(transform_null_in_) {} + + static FunctionPtr create(ContextPtr context) { - return std::make_shared(); + const auto & settings = context->getSettingsRef(); + return std::make_shared(FutureSet::getSizeLimitsForSet(settings, false), settings.transform_null_in); } String getName() const override @@ -122,10 +126,15 @@ public: tuple = typeid_cast(materialized_tuple.get()); } - auto set = column_set->getData(); - if (!set) + auto future_set = column_set->getData(); + if (!future_set || !future_set->isFilled()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Not-ready Set passed as the second argument for function '{}'", getName()); + if (auto * for_tuple = typeid_cast(future_set.get())) + if (!for_tuple->isReady()) + for_tuple->buildForTuple(size_limits, transform_null_in); + + auto set = future_set->get(); auto set_types = set->getDataTypes(); if (tuple && set_types.size() != 1 && set_types.size() == tuple->tupleSize()) @@ -173,6 +182,10 @@ public: return res; } + +private: + SizeLimits size_limits; + bool transform_null_in; }; template diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 00feecb44f5..18e44ab9f2f 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -446,7 +446,7 @@ FutureSetPtr makeExplicitSet( if (const auto * low_cardinality_type = typeid_cast(element_type.get())) element_type = low_cardinality_type->getDictionaryType(); - auto set_key = PreparedSetKey::forLiteral(*right_arg, set_element_types); + auto set_key = PreparedSetKey::forLiteral(right_arg->getTreeHash(), set_element_types); if (auto set = prepared_sets.getFuture(set_key)) return set; /// Already prepared. @@ -1384,7 +1384,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool { if (no_subqueries) return {}; - auto set_key = PreparedSetKey::forSubquery(*right_in_operand); + auto set_key = PreparedSetKey::forSubquery(right_in_operand->getTreeHash()); if (auto set = data.prepared_sets->getFuture(set_key)) return set; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 36725f36804..989fa12cba0 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -941,7 +941,7 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con if (const auto * column_set = checkAndGetColumn(action.node->column.get())) { auto set = column_set->getData(); - if (set && set->isCreated() && set->getTotalRowCount() == 0) + if (set && set->isReady() && set->get()->getTotalRowCount() == 0) return true; } } diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 9fb2c02bd58..3a846bb4bc3 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -181,7 +181,19 @@ public: // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); // subquery_for_set.createSource(*interpreter, external_storage); auto key = subquery_or_table_name->getColumnName(); - prepared_sets->addStorageToSubquery(key, std::move(external_storage)); + auto set_key = PreparedSetKey::forSubquery(subquery_or_table_name->getTreeHash()); + + if (!prepared_sets->getFuture(set_key)) + { + SubqueryForSet subquery_for_set; + subquery_for_set.key = std::move(key); + subquery_for_set.table = std::move(external_storage); + subquery_for_set.createSource(*interpreter); + + prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set)); + } + else + prepared_sets->addStorageToSubquery(key, std::move(external_storage)); } /** NOTE If it was written IN tmp_table - the existing temporary (but not external) table, diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index afd34cb044e..2487381e2fb 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -905,8 +905,8 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (storage && !options.only_analyze) { - query_analyzer->makeSetsForIndex(select_query.where()); - query_analyzer->makeSetsForIndex(select_query.prewhere()); + // query_analyzer->makeSetsForIndex(select_query.where()); + // query_analyzer->makeSetsForIndex(select_query.prewhere()); query_info.prepared_sets = query_analyzer->getPreparedSets(); from_stage = storage->getQueryProcessingStage(context, options.to_stage, storage_snapshot, query_info); @@ -3088,7 +3088,12 @@ void InterpreterSelectQuery::executeExtremes(QueryPlan & query_plan) void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_plan) { - addCreatingSetsStep(query_plan, prepared_sets, context); + auto step = std::make_unique( + query_plan.getCurrentDataStream(), + prepared_sets->detachSubqueries(context), + context); + + query_plan.addStep(std::move(step)); } diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 300ef2aadb6..733eb1c24bb 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -11,7 +11,7 @@ namespace DB { -PreparedSetKey PreparedSetKey::forLiteral(const IAST & ast, DataTypes types_) +PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) { /// Remove LowCardinality types from type list because Set doesn't support LowCardinality keys now, /// just converts LowCardinality to ordinary types. @@ -19,15 +19,15 @@ PreparedSetKey PreparedSetKey::forLiteral(const IAST & ast, DataTypes types_) type = recursiveRemoveLowCardinality(type); PreparedSetKey key; - key.ast_hash = ast.getTreeHash(); + key.ast_hash = hash; key.types = std::move(types_); return key; } -PreparedSetKey PreparedSetKey::forSubquery(const IAST & ast) +PreparedSetKey PreparedSetKey::forSubquery(Hash hash) { PreparedSetKey key; - key.ast_hash = ast.getTreeHash(); + key.ast_hash = hash; return key; } @@ -155,9 +155,9 @@ FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const // return it->second.get(); // } -// std::vector PreparedSets::getByTreeHash(IAST::Hash ast_hash) const +// std::vector PreparedSets::getByTreeHash(IAST::Hash ast_hash) const // { -// std::vector res; +// std::vector res; // for (const auto & it : this->sets) // { // if (it.first.ast_hash == ast_hash) @@ -166,7 +166,7 @@ FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const // return res; // } -PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries() +PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries(const ContextPtr &) { auto res = std::move(subqueries); subqueries = SubqueriesForSets(); @@ -221,6 +221,8 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c if (set) return nullptr; + std::cerr << StackTrace().toString() << std::endl; + auto set_cache = context->getPreparedSetsCache(); if (set_cache) { @@ -277,4 +279,10 @@ SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordere return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); } +FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) {} + +FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquery(std::move(subquery_)) {} + +FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} + }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 8ebabc32b0a..8661d81a96a 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -65,6 +65,7 @@ public: virtual ~FutureSet() = default; virtual bool isReady() const = 0; + virtual bool isFilled() const = 0; virtual SetPtr get() const = 0; virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; @@ -81,34 +82,41 @@ public: FutureSetFromTuple(Block block_); bool isReady() const override { return set != nullptr; } + bool isFilled() const override { return true; } SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override { - fill(context, true); + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, true); + fill(size_limits, settings.transform_null_in, true); return set; } std::unique_ptr build(const ContextPtr & context) override { - fill(context, false); + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, false); + fill(size_limits, settings.transform_null_in, false); return nullptr; } + void buildForTuple(SizeLimits size_limits, bool transform_null_in) + { + fill(size_limits, transform_null_in, false); + } + private: Block block; SetPtr set; - void fill(const ContextPtr & context, bool create_ordered_set) + void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set) { if (set) return; - const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); - - set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); + set = std::make_shared(size_limits, create_ordered_set, transform_null_in); set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); set->insertFromBlock(block.getColumnsWithTypeAndName()); set->finishInsert(); @@ -151,6 +159,7 @@ public: FutureSetFromSubquery(SubqueryForSet subquery_); bool isReady() const override { return set != nullptr; } + bool isFilled() const override { return isReady(); } SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override @@ -190,6 +199,7 @@ public: FutureSetFromStorage(SetPtr set_); // : set(std::move(set_) {} bool isReady() const override { return set != nullptr; } + bool isFilled() const override { return isReady(); } SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr &) override @@ -229,23 +239,25 @@ private: struct PreparedSetKey { + using Hash = std::pair; + /// Prepared sets for tuple literals are indexed by the hash of the tree contents and by the desired /// data types of set elements (two different Sets can be required for two tuples with the same contents /// if left hand sides of the IN operators have different types). - static PreparedSetKey forLiteral(const IAST & ast, DataTypes types_); + static PreparedSetKey forLiteral(Hash hash, DataTypes types_); /// Prepared sets for subqueries are indexed only by the AST contents because the type of the resulting /// set is fully determined by the subquery. - static PreparedSetKey forSubquery(const IAST & ast); + static PreparedSetKey forSubquery(Hash hash); - IAST::Hash ast_hash; + Hash ast_hash; DataTypes types; /// Empty for subqueries. bool operator==(const PreparedSetKey & other) const; String toString() const; - struct Hash + struct Hashing { UInt64 operator()(const PreparedSetKey & key) const { return key.ast_hash.first; } }; @@ -272,16 +284,18 @@ public: /// Get subqueries and clear them. /// We need to build a plan for subqueries just once. That's why we can clear them after accessing them. /// SetPtr would still be available for consumers of PreparedSets. - SubqueriesForSets detachSubqueries(); + SubqueriesForSets detachSubqueries(const ContextPtr &); /// Returns all sets that match the given ast hash not checking types /// Used in KeyCondition and MergeTreeIndexConditionBloomFilter to make non exact match for types in PreparedSetKey - std::vector getByTreeHash(IAST::Hash ast_hash) const; + //std::vector getByTreeHash(IAST::Hash ast_hash) const; + + const std::unordered_map & getSets() const { return sets; } bool empty() const; private: - std::unordered_map sets; + std::unordered_map sets; /// This is the information required for building sets SubqueriesForSets subqueries; diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 02069aad292..6c970e0e91b 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -11,6 +11,8 @@ #include #include #include +#include +#include namespace DB { @@ -26,8 +28,9 @@ namespace class CollectSetsVisitor : public ConstInDepthQueryTreeVisitor { public: - explicit CollectSetsVisitor(PlannerContext & planner_context_) + explicit CollectSetsVisitor(PlannerContext & planner_context_, const SelectQueryOptions & select_query_options_) : planner_context(planner_context_) + , select_query_options(select_query_options_) {} void visitImpl(const QueryTreeNodePtr & node) @@ -42,10 +45,12 @@ public: const auto & settings = planner_context.getQueryContext()->getSettingsRef(); - String set_key = planner_context.createSetKey(in_second_argument); + // String set_key = planner_context.createSetKey(in_second_argument); - if (planner_context.hasSet(set_key)) - return; + // if (planner_context.hasSet(set_key)) + // return; + + auto & sets = planner_context.getPreparedSets(); /// Tables and table functions are replaced with subquery at Analysis stage, except special Set table. auto * second_argument_table = in_second_argument->as(); @@ -54,7 +59,9 @@ public: if (storage_set) { /// Handle storage_set as ready set. - planner_context.registerSet(set_key, PlannerSet(FutureSet(storage_set->getSet()))); + auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash()); + sets.addFromStorage(set_key, storage_set->getSet()); + //planner_context.registerSet(set_key, PlannerSet(FutureSet(storage_set->getSet()))); } else if (const auto * constant_node = in_second_argument->as()) { @@ -62,14 +69,47 @@ public: in_first_argument->getResultType(), constant_node->getValue(), constant_node->getResultType(), - settings); + settings.transform_null_in); - planner_context.registerSet(set_key, PlannerSet(FutureSet(std::move(set)))); + DataTypes set_element_types = {in_first_argument->getResultType()}; + const auto * left_tuple_type = typeid_cast(set_element_types.front().get()); + if (left_tuple_type && left_tuple_type->getElements().size() != 1) + set_element_types = left_tuple_type->getElements(); + + for (auto & element_type : set_element_types) + if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + element_type = low_cardinality_type->getDictionaryType(); + + auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); + + sets.addFromTuple(set_key, std::move(set)); + + //planner_context.registerSet(set_key, PlannerSet(FutureSet(std::move(set)))); } else if (in_second_argument_node_type == QueryTreeNodeType::QUERY || in_second_argument_node_type == QueryTreeNodeType::UNION) { - planner_context.registerSet(set_key, PlannerSet(in_second_argument)); + auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash()); + + auto subquery_options = select_query_options.subquery(); + Planner subquery_planner( + in_second_argument, + subquery_options, + planner_context.getGlobalPlannerContext()); + subquery_planner.buildQueryPlanIfNeeded(); + + // const auto & settings = planner_context.getQueryContext()->getSettingsRef(); + // SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; + // bool tranform_null_in = settings.transform_null_in; + // auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); + + SubqueryForSet subquery_for_set; + subquery_for_set.key = planner_context.createSetKey(in_second_argument); + subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); + + sets.addFromSubquery(set_key, std::move(subquery_for_set)); + + //planner_context.registerSet(set_key, PlannerSet(in_second_argument)); } else { @@ -87,13 +127,14 @@ public: private: PlannerContext & planner_context; + const SelectQueryOptions & select_query_options; }; } -void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context) +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context, const SelectQueryOptions & select_query_options) { - CollectSetsVisitor visitor(planner_context); + CollectSetsVisitor visitor(planner_context, select_query_options); visitor.visit(node); } diff --git a/src/Planner/CollectSets.h b/src/Planner/CollectSets.h index 94f792e877b..57e662a392e 100644 --- a/src/Planner/CollectSets.h +++ b/src/Planner/CollectSets.h @@ -7,9 +7,11 @@ namespace DB { +struct SelectQueryOptions; + /** Collect prepared sets and sets for subqueries that are necessary to execute IN function and its variations. * Collected sets are registered in planner context. */ -void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context); +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context, const SelectQueryOptions & select_query_options); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index d036c895fbb..38d0aa29d24 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -878,50 +878,50 @@ void addOffsetStep(QueryPlan & query_plan, const QueryAnalysisResult & query_ana query_plan.addStep(std::move(offsets_step)); } -void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, - const SelectQueryOptions & select_query_options, - const PlannerContextPtr & planner_context, - const std::vector & result_actions_to_execute) -{ - PreparedSets::SubqueriesForSets subqueries_for_sets; +// void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, +// const SelectQueryOptions & select_query_options, +// const PlannerContextPtr & planner_context, +// const std::vector & result_actions_to_execute) +// { +// PreparedSets::SubqueriesForSets subqueries_for_sets; - for (const auto & actions_to_execute : result_actions_to_execute) - { - for (const auto & node : actions_to_execute->getNodes()) - { - const auto & set_key = node.result_name; - auto * planner_set = planner_context->getSetOrNull(set_key); - if (!planner_set) - continue; +// for (const auto & actions_to_execute : result_actions_to_execute) +// { +// for (const auto & node : actions_to_execute->getNodes()) +// { +// const auto & set_key = node.result_name; +// auto * planner_set = planner_context->getSetOrNull(set_key); +// if (!planner_set) +// continue; - if (planner_set->getSet().isCreated() || !planner_set->getSubqueryNode()) - continue; +// if (planner_set->getSet().isCreated() || !planner_set->getSubqueryNode()) +// continue; - auto subquery_options = select_query_options.subquery(); - Planner subquery_planner( - planner_set->getSubqueryNode(), - subquery_options, - planner_context->getGlobalPlannerContext()); - subquery_planner.buildQueryPlanIfNeeded(); +// auto subquery_options = select_query_options.subquery(); +// Planner subquery_planner( +// planner_set->getSubqueryNode(), +// subquery_options, +// planner_context->getGlobalPlannerContext()); +// subquery_planner.buildQueryPlanIfNeeded(); - const auto & settings = planner_context->getQueryContext()->getSettingsRef(); - SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; - bool tranform_null_in = settings.transform_null_in; - auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); +// const auto & settings = planner_context->getQueryContext()->getSettingsRef(); +// SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; +// bool tranform_null_in = settings.transform_null_in; +// auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); - SubqueryForSet subquery_for_set; - subquery_for_set.key = set_key; - subquery_for_set.set_in_progress = set; - subquery_for_set.set = planner_set->getSet(); - subquery_for_set.promise_to_fill_set = planner_set->extractPromiseToBuildSet(); - subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); +// SubqueryForSet subquery_for_set; +// subquery_for_set.key = set_key; +// subquery_for_set.set_in_progress = set; +// subquery_for_set.set = planner_set->getSet(); +// subquery_for_set.promise_to_fill_set = planner_set->extractPromiseToBuildSet(); +// subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); - subqueries_for_sets.emplace(set_key, std::move(subquery_for_set)); - } - } +// subqueries_for_sets.emplace(set_key, std::move(subquery_for_set)); +// } +// } - addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); -} +// addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); +// } /// Support for `additional_result_filter` setting void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan, @@ -951,7 +951,7 @@ void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan, auto storage = std::make_shared(StorageID{"dummy", "dummy"}, fake_column_descriptions); auto fake_table_expression = std::make_shared(std::move(storage), query_context); - auto filter_info = buildFilterInfo(additional_result_filter_ast, fake_table_expression, planner_context, std::move(fake_name_set)); + auto filter_info = buildFilterInfo(additional_result_filter_ast, fake_table_expression, planner_context, select_query_options, std::move(fake_name_set)); if (!filter_info.actions || !query_plan.isInitialized()) return; @@ -1179,7 +1179,7 @@ void Planner::buildPlanForQueryNode() } checkStoragesSupportTransactions(planner_context); - collectSets(query_tree, *planner_context); + collectSets(query_tree, *planner_context, select_query_options); collectTableExpressionData(query_tree, planner_context); const auto & settings = query_context->getSettingsRef(); @@ -1467,7 +1467,17 @@ void Planner::buildPlanForQueryNode() } if (!select_query_options.only_analyze) - addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); + { + auto step = std::make_unique( + query_plan.getCurrentDataStream(), + planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()), + planner_context->getQueryContext()); + + query_plan.addStep(std::move(step)); + + //addCreatingSetsStep(query_plan, planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()), planner_context->getQueryContext()); + //addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); + } } SelectQueryInfo Planner::buildSelectQueryInfo() const diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index c64d82299ca..453b02a2f8f 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -16,6 +16,7 @@ #include #include +#include #include #include @@ -623,33 +624,51 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::makeSetForInFunction(const QueryTreeNodePtr & node) { const auto & function_node = node->as(); + auto in_first_argument = function_node.getArguments().getNodes().at(0); auto in_second_argument = function_node.getArguments().getNodes().at(1); - auto set_key = planner_context->createSetKey(in_second_argument); - const auto & planner_set = planner_context->getSetOrThrow(set_key); + //auto set_key = planner_context->createSetKey(in_second_argument); + + DataTypes set_element_types = {in_first_argument->getResultType()}; + const auto * left_tuple_type = typeid_cast(set_element_types.front().get()); + if (left_tuple_type && left_tuple_type->getElements().size() != 1) + set_element_types = left_tuple_type->getElements(); + + for (auto & element_type : set_element_types) + if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + element_type = low_cardinality_type->getDictionaryType(); + + auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); + + + auto set = planner_context->getPreparedSets().getFuture(set_key); + if (!set) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "No set is registered for key {}", + set_key.toString()); ColumnWithTypeAndName column; - column.name = set_key; + column.name = planner_context->createSetKey(in_second_argument); column.type = std::make_shared(); - bool set_is_created = planner_set.getSet().isCreated(); - auto column_set = ColumnSet::create(1, planner_set.getSet()); + bool set_is_created = set->isFilled(); + auto column_set = ColumnSet::create(1, std::move(set)); if (set_is_created) column.column = ColumnConst::create(std::move(column_set), 1); else column.column = std::move(column_set); - actions_stack[0].addConstantIfNecessary(set_key, column); + actions_stack[0].addConstantIfNecessary(column.name, column); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) { auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputConstantColumnIfNecessary(set_key, column); + actions_stack_node.addInputConstantColumnIfNecessary(column.name, column); } - return {set_key, 0}; + return {column.name, 0}; } PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitIndexHintFunction(const QueryTreeNodePtr & node) diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index 346cc6d2080..e8c7bb7ef48 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -126,49 +126,49 @@ PlannerContext::SetKey PlannerContext::createSetKey(const QueryTreeNodePtr & set return "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second); } -void PlannerContext::registerSet(const SetKey & key, PlannerSet planner_set) -{ - if (!planner_set.getSet().isValid()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Set must be initialized"); +// void PlannerContext::registerSet(const SetKey & key, PlannerSet planner_set) +// { +// if (!planner_set.getSet().isValid()) +// throw Exception(ErrorCodes::LOGICAL_ERROR, "Set must be initialized"); - const auto & subquery_node = planner_set.getSubqueryNode(); - if (subquery_node) - { - auto node_type = subquery_node->getNodeType(); +// const auto & subquery_node = planner_set.getSubqueryNode(); +// if (subquery_node) +// { +// auto node_type = subquery_node->getNodeType(); - if (node_type != QueryTreeNodeType::QUERY && - node_type != QueryTreeNodeType::UNION) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Invalid node for set table expression. Expected query or union. Actual {}", - subquery_node->formatASTForErrorMessage()); - } +// if (node_type != QueryTreeNodeType::QUERY && +// node_type != QueryTreeNodeType::UNION) +// throw Exception(ErrorCodes::LOGICAL_ERROR, +// "Invalid node for set table expression. Expected query or union. Actual {}", +// subquery_node->formatASTForErrorMessage()); +// } - set_key_to_set.emplace(key, std::move(planner_set)); -} +// set_key_to_set.emplace(key, std::move(planner_set)); +// } -bool PlannerContext::hasSet(const SetKey & key) const -{ - return set_key_to_set.contains(key); -} +// bool PlannerContext::hasSet(const SetKey & key) const +// { +// return set_key_to_set.contains(key); +// } -const PlannerSet & PlannerContext::getSetOrThrow(const SetKey & key) const -{ - auto it = set_key_to_set.find(key); - if (it == set_key_to_set.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "No set is registered for key {}", - key); +// const PlannerSet & PlannerContext::getSetOrThrow(const SetKey & key) const +// { +// auto it = set_key_to_set.find(key); +// if (it == set_key_to_set.end()) +// throw Exception(ErrorCodes::LOGICAL_ERROR, +// "No set is registered for key {}", +// key); - return it->second; -} +// return it->second; +// } -PlannerSet * PlannerContext::getSetOrNull(const SetKey & key) -{ - auto it = set_key_to_set.find(key); - if (it == set_key_to_set.end()) - return nullptr; +// PlannerSet * PlannerContext::getSetOrNull(const SetKey & key) +// { +// auto it = set_key_to_set.find(key); +// if (it == set_key_to_set.end()) +// return nullptr; - return &it->second; -} +// return &it->second; +// } } diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index 9ecfdb6117a..df3fad01824 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -55,43 +55,47 @@ using GlobalPlannerContextPtr = std::shared_ptr; */ class PlannerSet { -public: - /// Construct planner set that is ready for execution - explicit PlannerSet(FutureSetPtr set_) - : set(std::move(set_)) - {} - /// Construct planner set with set and subquery node - explicit PlannerSet(QueryTreeNodePtr subquery_node_) - //: set(promise_to_build_set.get_future()) - : subquery_node(std::move(subquery_node_)) - {} - - /// Get a reference to a set that might be not built yet - const FutureSetPtr & getSet() const - { - return set; - } - - /// Get subquery node - const QueryTreeNodePtr & getSubqueryNode() const - { - return subquery_node; - } - - /// This promise will be fulfilled when set is built and all FutureSet objects will become ready - // std::promise extractPromiseToBuildSet() - // { - // return std::move(promise_to_build_set); - // } - -private: - //std::promise promise_to_build_set; - FutureSetPtr set; - - QueryTreeNodePtr subquery_node; }; +// { +// public: +// /// Construct planner set that is ready for execution +// explicit PlannerSet(FutureSetPtr set_) +// : set(std::move(set_)) +// {} + +// /// Construct planner set with set and subquery node +// explicit PlannerSet(QueryTreeNodePtr subquery_node_) +// //: set(promise_to_build_set.get_future()) +// : subquery_node(std::move(subquery_node_)) +// {} + +// /// Get a reference to a set that might be not built yet +// const FutureSetPtr & getSet() const +// { +// return set; +// } + +// /// Get subquery node +// const QueryTreeNodePtr & getSubqueryNode() const +// { +// return subquery_node; +// } + +// /// This promise will be fulfilled when set is built and all FutureSet objects will become ready +// // std::promise extractPromiseToBuildSet() +// // { +// // return std::move(promise_to_build_set); +// // } + +// private: +// //std::promise promise_to_build_set; +// FutureSetPtr set; + +// QueryTreeNodePtr subquery_node; +// }; + class PlannerContext { public: @@ -179,28 +183,30 @@ public: using SetKey = std::string; - using SetKeyToSet = std::unordered_map; + // using SetKeyToSet = std::unordered_map; - /// Create set key for set source node + // /// Create set key for set source node static SetKey createSetKey(const QueryTreeNodePtr & set_source_node); - /// Register set for set key - void registerSet(const SetKey & key, PlannerSet planner_set); + // /// Register set for set key + // void registerSet(const SetKey & key, PlannerSet planner_set); - /// Returns true if set is registered for key, false otherwise - bool hasSet(const SetKey & key) const; + // /// Returns true if set is registered for key, false otherwise + // bool hasSet(const SetKey & key) const; - /// Get set for key, if no set is registered logical exception is thrown - const PlannerSet & getSetOrThrow(const SetKey & key) const; + // /// Get set for key, if no set is registered logical exception is thrown + // const PlannerSet & getSetOrThrow(const SetKey & key) const; - /// Get set for key, if no set is registered null is returned - PlannerSet * getSetOrNull(const SetKey & key); + // /// Get set for key, if no set is registered null is returned + // PlannerSet * getSetOrNull(const SetKey & key); - /// Get registered sets - const SetKeyToSet & getRegisteredSets() const - { - return set_key_to_set; - } + // /// Get registered sets + // const SetKeyToSet & getRegisteredSets() const + // { + // return set_key_to_set; + // } + + PreparedSets & getPreparedSets() { return prepared_sets; } private: /// Query context @@ -216,8 +222,7 @@ private: std::unordered_map table_expression_node_to_data; /// Set key to set - SetKeyToSet set_key_to_set; - + PreparedSets prepared_sets; }; using PlannerContextPtr = std::shared_ptr; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 93a9945b1ca..77f53a6b7ac 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -388,7 +388,8 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, SelectQueryInfo & table_expression_query_info, - PlannerContextPtr & planner_context) + PlannerContextPtr & planner_context, + const SelectQueryOptions & select_query_options) { auto storage_id = storage->getStorageID(); const auto & query_context = planner_context->getQueryContext(); @@ -397,12 +398,13 @@ FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, if (!row_policy_filter) return {}; - return buildFilterInfo(row_policy_filter->expression, table_expression_query_info.table_expression, planner_context); + return buildFilterInfo(row_policy_filter->expression, table_expression_query_info.table_expression, planner_context, select_query_options); } FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, SelectQueryInfo & table_expression_query_info, - PlannerContextPtr & planner_context) + PlannerContextPtr & planner_context, + const SelectQueryOptions & select_query_options) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); @@ -428,14 +430,15 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, *storage, query_context); - return buildFilterInfo(parallel_replicas_custom_filter_ast, table_expression_query_info.table_expression, planner_context); + return buildFilterInfo(parallel_replicas_custom_filter_ast, table_expression_query_info.table_expression, planner_context, select_query_options); } /// Apply filters from additional_table_filters setting FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, const String & table_expression_alias, SelectQueryInfo & table_expression_query_info, - PlannerContextPtr & planner_context) + PlannerContextPtr & planner_context, + const SelectQueryOptions & select_query_options) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); @@ -469,7 +472,7 @@ FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, return {}; table_expression_query_info.additional_filter_ast = additional_filter_ast; - return buildFilterInfo(additional_filter_ast, table_expression_query_info.table_expression, planner_context); + return buildFilterInfo(additional_filter_ast, table_expression_query_info.table_expression, planner_context, select_query_options); } JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, @@ -679,14 +682,14 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } }; - auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context); + auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, select_query_options); add_filter(row_policy_filter_info, "Row-level security filter"); if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY) { if (settings.parallel_replicas_count > 1) { - auto parallel_replicas_custom_key_filter_info = buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context); + auto parallel_replicas_custom_key_filter_info = buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context, select_query_options); add_filter(parallel_replicas_custom_key_filter_info, "Parallel replicas custom key filter"); } else @@ -701,7 +704,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } const auto & table_expression_alias = table_expression->getAlias(); - auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context); + auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context, select_query_options); add_filter(additional_filters_info, "additional filter"); from_stage = storage->getQueryProcessingStage(query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 5c61b2fc2c7..2b4febf58ea 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -426,6 +426,7 @@ SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, const QueryTreeNodePtr & table_expression, PlannerContextPtr & planner_context, + const SelectQueryOptions & select_query_options, NameSet table_expression_required_names_without_filter) { const auto & query_context = planner_context->getQueryContext(); @@ -443,7 +444,7 @@ FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, } collectSourceColumns(filter_query_tree, planner_context); - collectSets(filter_query_tree, *planner_context); + collectSets(filter_query_tree, *planner_context, select_query_options); auto filter_actions_dag = std::make_shared(); diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index d9412800e61..8071e201f88 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -82,6 +82,7 @@ SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, const QueryTreeNodePtr & table_expression, PlannerContextPtr & planner_context, + const SelectQueryOptions & select_query_options, NameSet table_expression_required_names_without_filter = {}); ASTPtr parseAdditionalResultFilter(const Settings & settings); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 9eec3e90494..d77c6627994 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -38,16 +38,16 @@ CreatingSetStep::CreatingSetStep( SizeLimits network_transfer_limits_, ContextPtr context_) : ITransformingStep(input_stream_, Block{}, getTraits()) - , WithContext(context_) , description(std::move(description_)) , subquery_for_set(std::move(subquery_for_set_)) , network_transfer_limits(std::move(network_transfer_limits_)) + , context(std::move(context_)) { } void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, getContext()); + pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, context); } void CreatingSetStep::updateOutputStream() @@ -60,7 +60,7 @@ void CreatingSetStep::describeActions(FormatSettings & settings) const String prefix(settings.offset, ' '); settings.out << prefix; - if (subquery_for_set.set_in_progress) + if (subquery_for_set.set) settings.out << "Set: "; settings.out << description << '\n'; @@ -68,7 +68,7 @@ void CreatingSetStep::describeActions(FormatSettings & settings) const void CreatingSetStep::describeActions(JSONBuilder::JSONMap & map) const { - if (subquery_for_set.set_in_progress) + if (subquery_for_set.set) map.add("Set", description); } @@ -130,22 +130,14 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets plans.emplace_back(std::make_unique(std::move(query_plan))); query_plan = QueryPlan(); - for (auto & [description, subquery_for_set] : subqueries_for_sets) + for (auto & [description, future_set] : subqueries_for_sets) { - if (!subquery_for_set.hasSource()) + if (future_set->isReady()) continue; - auto plan = subquery_for_set.detachSource(); - - const Settings & settings = context->getSettingsRef(); - auto creating_set = std::make_unique( - plan->getCurrentDataStream(), - description, - std::move(subquery_for_set), - SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), - context); - creating_set->setStepDescription("Create set for subquery"); - plan->addStep(std::move(creating_set)); + auto plan = future_set->build(context); + if (!plan) + continue; input_streams.emplace_back(plan->getCurrentDataStream()); plans.emplace_back(std::move(plan)); @@ -162,12 +154,56 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets query_plan.unitePlans(std::move(creating_sets), std::move(plans)); } +//void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets subqueries_for_sets, ContextPtr context) + +std::vector> DelayedCreatingSetsStep::makePlansForSets(DelayedCreatingSetsStep && step) +{ + // DataStreams input_streams; + // input_streams.emplace_back(query_plan.getCurrentDataStream()); + + std::vector> plans; + // plans.emplace_back(std::make_unique(std::move(query_plan))); + // query_plan = QueryPlan(); + + for (auto & [description, future_set] : step.subqueries_for_sets) + { + if (future_set->isReady()) + continue; + + auto plan = future_set->build(step.context); + if (!plan) + continue; + + plan->optimize(QueryPlanOptimizationSettings::fromContext(step.context)); + + //input_streams.emplace_back(plan->getCurrentDataStream()); + plans.emplace_back(std::move(plan)); + } + + return plans; +} + void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context) { if (!prepared_sets || prepared_sets->empty()) return; - addCreatingSetsStep(query_plan, prepared_sets->detachSubqueries(), context); + addCreatingSetsStep(query_plan, prepared_sets->detachSubqueries(context), context); +} + +DelayedCreatingSetsStep::DelayedCreatingSetsStep( + DataStream input_stream, PreparedSets::SubqueriesForSets subqueries_for_sets_, ContextPtr context_) + : subqueries_for_sets(std::move(subqueries_for_sets_)), context(std::move(context_)) +{ + input_streams = {input_stream}; + output_stream = std::move(input_stream); +} + +QueryPipelineBuilderPtr DelayedCreatingSetsStep::updatePipeline(QueryPipelineBuilders, const BuildQueryPipelineSettings &) +{ + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot build pipeline in DelayedCreatingSets. This step should be optimized out."); } } diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index b4777578a30..96ab26077fc 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -9,7 +9,7 @@ namespace DB { /// Creates sets for subqueries and JOIN. See CreatingSetsTransform. -class CreatingSetStep : public ITransformingStep, WithContext +class CreatingSetStep : public ITransformingStep { public: CreatingSetStep( @@ -32,6 +32,7 @@ private: String description; SubqueryForSet subquery_for_set; SizeLimits network_transfer_limits; + ContextPtr context; }; class CreatingSetsStep : public IQueryPlanStep @@ -46,6 +47,22 @@ public: void describePipeline(FormatSettings & settings) const override; }; +class DelayedCreatingSetsStep final : public IQueryPlanStep +{ +public: + DelayedCreatingSetsStep(DataStream input_stream, PreparedSets::SubqueriesForSets subqueries_for_sets_, ContextPtr context_); + + String getName() const override { return "DelayedCreatingSets"; } + + QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders, const BuildQueryPipelineSettings &) override; + + static std::vector> makePlansForSets(DelayedCreatingSetsStep && step); + +private: + PreparedSets::SubqueriesForSets subqueries_for_sets; + ContextPtr context; +}; + void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets subqueries_for_sets, ContextPtr context); void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context); diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index de1d43bed1b..2b934ec440b 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -110,6 +110,7 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes); bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes); +bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes); /// Enable memory bound merging of aggregation states for remote queries /// in case it was enabled for local plan diff --git a/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp b/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp new file mode 100644 index 00000000000..e9100ae9d02 --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp @@ -0,0 +1,35 @@ +#include +#include +#include +#include + +namespace DB::QueryPlanOptimizations +{ + +bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes) +{ + auto * delayed = typeid_cast(node.step.get()); + if (!delayed) + return false; + + auto plans = DelayedCreatingSetsStep::makePlansForSets(std::move(*delayed)); + node.children.reserve(1 + plans.size()); + + DataStreams input_streams; + input_streams.reserve(1 + plans.size()); + input_streams.push_back(node.children.front()->step->getOutputStream()); + + for (const auto & plan : plans) + { + input_streams.push_back(plan->getCurrentDataStream()); + node.children.push_back(plan->getRootNode()); + nodes.splice(nodes.end(), QueryPlan::detachNodes(std::move(*plan))); + } + + auto creating_sets = std::make_unique(std::move(input_streams)); + creating_sets->setStepDescription("Create sets before main query execution"); + node.step = std::move(creating_sets); + return true; +} + +} diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 37e3b2f67d8..c74487f7cc1 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -163,6 +163,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s optimizePrewhere(stack, nodes); optimizePrimaryKeyCondition(stack); enableMemoryBoundMerging(*frame.node, nodes); + addPlansForSets(*frame.node, nodes); stack.pop_back(); } diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 3fbe3d89845..cb732e58855 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -541,4 +541,9 @@ void QueryPlan::explainEstimate(MutableColumns & columns) } } +QueryPlan::Nodes QueryPlan::detachNodes(QueryPlan && plan) +{ + return std::move(plan.nodes); +} + } diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 19d87b101de..d89bdc534be 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -105,10 +105,11 @@ public: std::vector children = {}; }; - const Node * getRootNode() const { return root; } - using Nodes = std::list; + Node * getRootNode() const { return root; } + static Nodes detachNodes(QueryPlan && plan); + private: QueryPlanResourceHolder resources; Nodes nodes; diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index e3ae2d4fd4e..de10be599c8 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -76,7 +76,7 @@ void CreatingSetsTransform::startSubquery() } subquery.promise_to_fill_set.set_value(ready_set); - subquery.set_in_progress.reset(); + subquery.set.reset(); done_with_set = true; set_from_cache = true; } @@ -84,7 +84,7 @@ void CreatingSetsTransform::startSubquery() } } - if (subquery.set_in_progress) + if (subquery.set) LOG_TRACE(log, "Creating set, key: {}", subquery.key); if (subquery.table) LOG_TRACE(log, "Filling temporary table."); @@ -93,7 +93,7 @@ void CreatingSetsTransform::startSubquery() /// TODO: make via port table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), getContext())); - done_with_set = !subquery.set_in_progress; + done_with_set = !subquery.set; done_with_table = !subquery.table; if ((done_with_set && !set_from_cache) /*&& done_with_join*/ && done_with_table) @@ -116,8 +116,8 @@ void CreatingSetsTransform::finishSubquery() } else if (read_rows != 0) { - if (subquery.set_in_progress) - LOG_DEBUG(log, "Created Set with {} entries from {} rows in {} sec.", subquery.set_in_progress->getTotalRowCount(), read_rows, seconds); + if (subquery.set) + LOG_DEBUG(log, "Created Set with {} entries from {} rows in {} sec.", subquery.set->getTotalRowCount(), read_rows, seconds); if (subquery.table) LOG_DEBUG(log, "Created Table with {} rows in {} sec.", read_rows, seconds); } @@ -131,9 +131,9 @@ void CreatingSetsTransform::init() { is_initialized = true; - if (subquery.set_in_progress) + if (subquery.set) { - subquery.set_in_progress->setHeader(getInputPort().getHeader().getColumnsWithTypeAndName()); + subquery.set->setHeader(getInputPort().getHeader().getColumnsWithTypeAndName()); } watch.restart(); @@ -147,7 +147,7 @@ void CreatingSetsTransform::consume(Chunk chunk) if (!done_with_set) { - if (!subquery.set_in_progress->insertFromBlock(block.getColumnsWithTypeAndName())) + if (!subquery.set->insertFromBlock(block.getColumnsWithTypeAndName())) done_with_set = true; } @@ -170,12 +170,12 @@ void CreatingSetsTransform::consume(Chunk chunk) Chunk CreatingSetsTransform::generate() { - if (subquery.set_in_progress) + if (subquery.set) { - subquery.set_in_progress->finishInsert(); - subquery.promise_to_fill_set.set_value(subquery.set_in_progress); + subquery.set->finishInsert(); + subquery.promise_to_fill_set.set_value(subquery.set); if (promise_to_build) - promise_to_build->set_value(subquery.set_in_progress); + promise_to_build->set_value(subquery.set); } if (table_out.initialized()) diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index d5563108ad7..281236e631e 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -68,11 +68,18 @@ bool traverseASTFilter( PreparedSetKey set_key; if ((value->as() || value->as())) - set_key = PreparedSetKey::forSubquery(*value); + set_key = PreparedSetKey::forSubquery(value->getTreeHash()); else - set_key = PreparedSetKey::forLiteral(*value, {primary_key_type}); + set_key = PreparedSetKey::forLiteral(value->getTreeHash(), {primary_key_type}); - SetPtr set = prepared_sets->get(set_key); + FutureSetPtr future_set = prepared_sets->getFuture(set_key); + if (!future_set) + return false; + + if (!future_set->isReady()) + future_set->buildOrderedSetInplace(context); + + auto set = future_set->get(); if (!set) return false; diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 5d961425469..7a1c3b10c8a 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1204,14 +1204,32 @@ bool KeyCondition::tryPrepareSetIndex( const auto right_arg = func.getArgumentAt(1); - auto prepared_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types); + LOG_TRACE(&Poco::Logger::get("KK"), "Trying to get set for {}", right_arg.getColumnName()); + + auto future_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types); + if (!future_set) + return false; + + LOG_TRACE(&Poco::Logger::get("KK"), "Found set for {}", right_arg.getColumnName()); + + if (!future_set->isReady()) + { + LOG_TRACE(&Poco::Logger::get("KK"), "Building set inplace for {}", right_arg.getColumnName()); + future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); + } + + auto prepared_set = future_set->get(); if (!prepared_set) return false; + LOG_TRACE(&Poco::Logger::get("KK"), "Set if ready for {}", right_arg.getColumnName()); + /// The index can be prepared if the elements of the set were saved in advance. if (!prepared_set->hasExplicitSetElements()) return false; + LOG_TRACE(&Poco::Logger::get("KK"), "Has explicit elements for {}", right_arg.getColumnName()); + prepared_set->checkColumnsNumber(left_args_count); for (size_t i = 0; i < indexes_mapping.size(); ++i) prepared_set->checkTypesEqual(indexes_mapping[i].tuple_index, data_types[i]); diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 235d90bb974..5e186c25b83 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -310,7 +310,13 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNo if (functionIsInOrGlobalInOperator(function_name)) { - ConstSetPtr prepared_set = rhs_argument.tryGetPreparedSet(); + auto future_set = rhs_argument.tryGetPreparedSet(); + if (future_set && !future_set->isReady()) + future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); + + ConstSetPtr prepared_set; + if (future_set) + prepared_set = future_set->get(); if (prepared_set && prepared_set->hasExplicitSetElements()) { diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 06fddd51cb8..5e1d23df3c7 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -624,7 +624,14 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( if (key_tuple_mapping.empty()) return false; - auto prepared_set = right_argument.tryGetPreparedSet(data_types); + auto future_set = right_argument.tryGetPreparedSet(data_types); + if (future_set && !future_set->isReady()) + future_set->buildOrderedSetInplace(right_argument.getTreeContext().getQueryContext()); + + ConstSetPtr prepared_set; + if (future_set) + prepared_set = future_set->get(); + if (!prepared_set || !prepared_set->hasExplicitSetElements()) return false; diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index baa11368c8b..6ffba0ad029 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -655,7 +655,14 @@ bool MergeTreeConditionInverted::tryPrepareSetGinFilter( if (key_tuple_mapping.empty()) return false; - ConstSetPtr prepared_set = rhs.tryGetPreparedSet(); + auto future_set = rhs.tryGetPreparedSet(); + if (future_set && !future_set->isReady()) + future_set->buildOrderedSetInplace(rhs.getTreeContext().getQueryContext()); + + ConstSetPtr prepared_set; + if (future_set) + prepared_set = future_set->get(); + if (!prepared_set || !prepared_set->hasExplicitSetElements()) return false; diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index e49459d3d17..e8843ff1489 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -275,7 +275,7 @@ bool RPNBuilderTreeNode::tryGetConstant(Field & output_value, DataTypePtr & outp namespace { -ConstSetPtr tryGetSetFromDAGNode(const ActionsDAG::Node * dag_node) +FutureSetPtr tryGetSetFromDAGNode(const ActionsDAG::Node * dag_node) { if (!dag_node->column) return {}; @@ -285,28 +285,20 @@ ConstSetPtr tryGetSetFromDAGNode(const ActionsDAG::Node * dag_node) column = &column_const->getDataColumn(); if (const auto * column_set = typeid_cast(column)) - { - auto set = column_set->getData(); - - if (set && set->isCreated()) - return set; - } + return column_set->getData(); return {}; } } -ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const +FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const { const auto & prepared_sets = getTreeContext().getPreparedSets(); if (ast_node && prepared_sets) { - auto prepared_sets_with_same_hash = prepared_sets->getByTreeHash(ast_node->getTreeHash()); - for (auto & set : prepared_sets_with_same_hash) - if (set.isCreated()) - return set.get(); + return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); } else if (dag_node) { @@ -317,16 +309,16 @@ ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const return {}; } -ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) const +FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) const { const auto & prepared_sets = getTreeContext().getPreparedSets(); if (prepared_sets && ast_node) { if (ast_node->as() || ast_node->as()) - return prepared_sets->get(PreparedSetKey::forSubquery(*ast_node)); + return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); - return prepared_sets->get(PreparedSetKey::forLiteral(*ast_node, data_types)); + return prepared_sets->getFuture(PreparedSetKey::forLiteral(ast_node->getTreeHash(), data_types)); } else if (dag_node) { @@ -337,7 +329,7 @@ ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) return nullptr; } -ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet( +FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( const std::vector & indexes_mapping, const DataTypes & data_types) const { @@ -346,19 +338,25 @@ ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet( if (prepared_sets && ast_node) { if (ast_node->as() || ast_node->as()) - return prepared_sets->get(PreparedSetKey::forSubquery(*ast_node)); + return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); /// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information /// about types in left argument of the IN operator. Instead, we manually iterate through all the sets /// and find the one for the right arg based on the AST structure (getTreeHash), after that we check /// that the types it was prepared with are compatible with the types of the primary key. - auto types_match = [&indexes_mapping, &data_types](const SetPtr & candidate_set) + auto types_match = [&indexes_mapping, &data_types](const DataTypes & set_types) { assert(indexes_mapping.size() == data_types.size()); for (size_t i = 0; i < indexes_mapping.size(); ++i) { - if (!candidate_set->areTypesEqual(indexes_mapping[i].tuple_index, data_types[i])) + if (indexes_mapping[i].tuple_index >= set_types.size()) + return false; + + auto lhs = recursiveRemoveLowCardinality(data_types[i]); + auto rhs = recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index]); + + if (!lhs->equals(*rhs)) return false; } @@ -366,10 +364,10 @@ ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet( }; auto tree_hash = ast_node->getTreeHash(); - for (const auto & set : prepared_sets->getByTreeHash(tree_hash)) + for (const auto & [key, future_set] : prepared_sets->getSets()) { - if (set.isCreated() && types_match(set.get())) - return set.get(); + if (key.ast_hash == tree_hash && types_match(key.types)) + return future_set; } } else diff --git a/src/Storages/MergeTree/RPNBuilder.h b/src/Storages/MergeTree/RPNBuilder.h index 626eb288493..6f624d93cd6 100644 --- a/src/Storages/MergeTree/RPNBuilder.h +++ b/src/Storages/MergeTree/RPNBuilder.h @@ -109,13 +109,13 @@ public: bool tryGetConstant(Field & output_value, DataTypePtr & output_type) const; /// Try get prepared set from node - ConstSetPtr tryGetPreparedSet() const; + FutureSetPtr tryGetPreparedSet() const; /// Try get prepared set from node that match data types - ConstSetPtr tryGetPreparedSet(const DataTypes & data_types) const; + FutureSetPtr tryGetPreparedSet(const DataTypes & data_types) const; /// Try get prepared set from node that match indexes mapping and data types - ConstSetPtr tryGetPreparedSet( + FutureSetPtr tryGetPreparedSet( const std::vector & indexes_mapping, const DataTypes & data_types) const; diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 6ca74406b17..0f45f6825f6 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -313,8 +313,15 @@ static void extractPathImpl(const ActionsDAG::Node & node, Paths & res, ContextP if (!column_set) return; - auto set = column_set->getData(); - if (!set || !set->isCreated()) + auto future_set = column_set->getData(); + if (!future_set) + return; + + if (!future_set->isReady()) + future_set->buildOrderedSetInplace(context); + + auto set = future_set->get(); + if (!set) return; if (!set->hasExplicitSetElements()) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 4ff00facfdc..6e7b9ea5849 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -80,24 +80,24 @@ ASTPtr buildWhereExpression(const ASTs & functions) return makeASTFunction("and", functions); } -void buildSets(const ASTPtr & expression, ExpressionAnalyzer & analyzer) -{ - const auto * func = expression->as(); - if (func && functionIsInOrGlobalInOperator(func->name)) - { - const IAST & args = *func->arguments; - const ASTPtr & arg = args.children.at(1); - if (arg->as() || arg->as()) - { - analyzer.tryMakeSetForIndexFromSubquery(arg); - } - } - else - { - for (const auto & child : expression->children) - buildSets(child, analyzer); - } -} +// void buildSets(const ASTPtr & expression, ExpressionAnalyzer & analyzer) +// { +// const auto * func = expression->as(); +// if (func && functionIsInOrGlobalInOperator(func->name)) +// { +// const IAST & args = *func->arguments; +// const ASTPtr & arg = args.children.at(1); +// if (arg->as() || arg->as()) +// { +// analyzer.tryMakeSetForIndexFromSubquery(arg); +// } +// } +// else +// { +// for (const auto & child : expression->children) +// buildSets(child, analyzer); +// } +// } } @@ -199,7 +199,7 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex /// Let's analyze and calculate the prepared expression. auto syntax_result = TreeRewriter(context).analyze(expression_ast, block.getNamesAndTypesList()); ExpressionAnalyzer analyzer(expression_ast, syntax_result, context); - buildSets(expression_ast, analyzer); + //buildSets(expression_ast, analyzer); ExpressionActionsPtr actions = analyzer.getActions(false /* add alises */, true /* project result */, CompileExpressions::yes); Block block_with_filter = block; From a218f010e85c0bc995df17f5416bee4bcd8e6148 Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 5 May 2023 14:12:14 +0200 Subject: [PATCH 054/515] First portion of speedup Do not generate DateLUT for each serialized and deserialized date --- src/Common/LocalDate.h | 8 ++++---- .../Serializations/SerializationDate.cpp | 11 +++++++---- src/DataTypes/Serializations/SerializationDate.h | 5 ++++- src/Functions/FunctionsConversion.h | 12 ++++++------ src/IO/ReadHelpers.h | 16 ++++++++-------- src/IO/WriteHelpers.h | 8 ++++---- 6 files changed, 33 insertions(+), 27 deletions(-) diff --git a/src/Common/LocalDate.h b/src/Common/LocalDate.h index dc36f92bebf..4a383129ae4 100644 --- a/src/Common/LocalDate.h +++ b/src/Common/LocalDate.h @@ -61,17 +61,17 @@ public: init(time); } - LocalDate(DayNum day_num) /// NOLINT + LocalDate(DayNum day_num, const DateLUTImpl & time_zone = DateLUT::instance()) /// NOLINT { - const auto & values = DateLUT::instance().getValues(day_num); + const auto & values = time_zone.getValues(day_num); m_year = values.year; m_month = values.month; m_day = values.day_of_month; } - explicit LocalDate(ExtendedDayNum day_num) + explicit LocalDate(ExtendedDayNum day_num, const DateLUTImpl & time_zone = DateLUT::instance()) { - const auto & values = DateLUT::instance().getValues(day_num); + const auto & values = time_zone.getValues(day_num); m_year = values.year; m_month = values.month; m_day = values.day_of_month; diff --git a/src/DataTypes/Serializations/SerializationDate.cpp b/src/DataTypes/Serializations/SerializationDate.cpp index 678817017e0..bc2057d549e 100644 --- a/src/DataTypes/Serializations/SerializationDate.cpp +++ b/src/DataTypes/Serializations/SerializationDate.cpp @@ -13,7 +13,7 @@ namespace DB void SerializationDate::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeDateText(DayNum(assert_cast(column).getData()[row_num]), ostr); + writeDateText(DayNum(assert_cast(column).getData()[row_num]), ostr, time_zone); } void SerializationDate::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -26,7 +26,7 @@ void SerializationDate::deserializeWholeText(IColumn & column, ReadBuffer & istr void SerializationDate::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { DayNum x; - readDateText(x, istr); + readDateText(x, istr, time_zone); assert_cast(column).getData().push_back(x); } @@ -46,7 +46,7 @@ void SerializationDate::deserializeTextQuoted(IColumn & column, ReadBuffer & ist { DayNum x; assertChar('\'', istr); - readDateText(x, istr); + readDateText(x, istr, time_zone); assertChar('\'', istr); assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. } @@ -62,7 +62,7 @@ void SerializationDate::deserializeTextJSON(IColumn & column, ReadBuffer & istr, { DayNum x; assertChar('"', istr); - readDateText(x, istr); + readDateText(x, istr, time_zone); assertChar('"', istr); assert_cast(column).getData().push_back(x); } @@ -80,5 +80,8 @@ void SerializationDate::deserializeTextCSV(IColumn & column, ReadBuffer & istr, readCSV(value, istr); assert_cast(column).getData().push_back(value); } +SerializationDate::SerializationDate(const TimezoneMixin & time_zone_) : TimezoneMixin(time_zone_) +{ +} } diff --git a/src/DataTypes/Serializations/SerializationDate.h b/src/DataTypes/Serializations/SerializationDate.h index 099d7444c3d..c4e57470673 100644 --- a/src/DataTypes/Serializations/SerializationDate.h +++ b/src/DataTypes/Serializations/SerializationDate.h @@ -1,13 +1,16 @@ #pragma once #include +#include namespace DB { -class SerializationDate final : public SerializationNumber +class SerializationDate final : public SerializationNumber, public TimezoneMixin { public: + explicit SerializationDate(const TimezoneMixin & time_zone_ = TimezoneMixin()); + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 28002d34acc..645504df829 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -718,9 +718,9 @@ template <> struct FormatImpl { template - static ReturnType execute(const DataTypeDate::FieldType x, WriteBuffer & wb, const DataTypeDate *, const DateLUTImpl *) + static ReturnType execute(const DataTypeDate::FieldType x, WriteBuffer & wb, const DataTypeDate *, const DateLUTImpl * time_zone) { - writeDateText(DayNum(x), wb); + writeDateText(DayNum(x), wb, *time_zone); return ReturnType(true); } }; @@ -729,9 +729,9 @@ template <> struct FormatImpl { template - static ReturnType execute(const DataTypeDate32::FieldType x, WriteBuffer & wb, const DataTypeDate32 *, const DateLUTImpl *) + static ReturnType execute(const DataTypeDate32::FieldType x, WriteBuffer & wb, const DataTypeDate32 *, const DateLUTImpl * time_zone) { - writeDateText(ExtendedDayNum(x), wb); + writeDateText(ExtendedDayNum(x), wb, *time_zone); return ReturnType(true); } }; @@ -830,8 +830,8 @@ struct ConvertImpl(*col_with_type_and_name.type); const DateLUTImpl * time_zone = nullptr; - /// For argument of DateTime type, second argument with time zone could be specified. - if constexpr (std::is_same_v || std::is_same_v) + /// For argument of Date or DateTime type, second argument with time zone could be specified. + if constexpr (std::is_same_v || std::is_same_v || std::is_same_v) { auto non_null_args = createBlockWithNestedColumns(arguments); time_zone = &extractTimeZoneFromFunctionArguments(non_null_args, 1, 0); diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 0b0a0640cb1..0127809c832 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -723,7 +723,7 @@ inline void convertToDayNum(DayNum & date, ExtendedDayNum & from) } template -inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf) +inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) { static constexpr bool throw_exception = std::is_same_v; @@ -734,13 +734,13 @@ inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf) else if (!readDateTextImpl(local_date, buf)) return false; - ExtendedDayNum ret = DateLUT::instance().makeDayNum(local_date.year(), local_date.month(), local_date.day()); + ExtendedDayNum ret = date_lut.makeDayNum(local_date.year(), local_date.month(), local_date.day()); convertToDayNum(date,ret); return ReturnType(true); } template -inline ReturnType readDateTextImpl(ExtendedDayNum & date, ReadBuffer & buf) +inline ReturnType readDateTextImpl(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) { static constexpr bool throw_exception = std::is_same_v; @@ -752,7 +752,7 @@ inline ReturnType readDateTextImpl(ExtendedDayNum & date, ReadBuffer & buf) return false; /// When the parameter is out of rule or out of range, Date32 uses 1925-01-01 as the default value (-DateLUT::instance().getDayNumOffsetEpoch(), -16436) and Date uses 1970-01-01. - date = DateLUT::instance().makeDayNum(local_date.year(), local_date.month(), local_date.day(), -static_cast(DateLUT::instance().getDayNumOffsetEpoch())); + date = date_lut.makeDayNum(local_date.year(), local_date.month(), local_date.day(), -static_cast(date_lut.getDayNumOffsetEpoch())); return ReturnType(true); } @@ -762,14 +762,14 @@ inline void readDateText(LocalDate & date, ReadBuffer & buf) readDateTextImpl(date, buf); } -inline void readDateText(DayNum & date, ReadBuffer & buf) +inline void readDateText(DayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) { - readDateTextImpl(date, buf); + readDateTextImpl(date, buf, date_lut); } -inline void readDateText(ExtendedDayNum & date, ReadBuffer & buf) +inline void readDateText(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) { - readDateTextImpl(date, buf); + readDateTextImpl(date, buf, date_lut); } inline bool tryReadDateText(LocalDate & date, ReadBuffer & buf) diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index e08e451e0a7..9ee11d3cc9f 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -694,15 +694,15 @@ inline void writeDateText(const LocalDate & date, WriteBuffer & buf) } template -inline void writeDateText(DayNum date, WriteBuffer & buf) +inline void writeDateText(DayNum date, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { - writeDateText(LocalDate(date), buf); + writeDateText(LocalDate(date, time_zone), buf); } template -inline void writeDateText(ExtendedDayNum date, WriteBuffer & buf) +inline void writeDateText(ExtendedDayNum date, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { - writeDateText(LocalDate(date), buf); + writeDateText(LocalDate(date, time_zone), buf); } /// In the format YYYY-MM-DD HH:MM:SS From 2b08801ae9bc1ca456247282ebfe060a9df0bce4 Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 5 May 2023 15:50:19 +0200 Subject: [PATCH 055/515] add timezone param --- src/IO/WriteHelpers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 9ee11d3cc9f..8a7cd72f79a 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -883,7 +883,7 @@ inline void writeText(is_enum auto x, WriteBuffer & buf) { writeText(magic_enum: inline void writeText(std::string_view x, WriteBuffer & buf) { writeString(x.data(), x.size(), buf); } -inline void writeText(const DayNum & x, WriteBuffer & buf) { writeDateText(LocalDate(x), buf); } +inline void writeText(const DayNum & x, WriteBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { writeDateText(LocalDate(x, time_zone), buf); } inline void writeText(const LocalDate & x, WriteBuffer & buf) { writeDateText(x, buf); } inline void writeText(const LocalDateTime & x, WriteBuffer & buf) { writeDateTimeText(x, buf); } inline void writeText(const UUID & x, WriteBuffer & buf) { writeUUIDText(x, buf); } From 54ef6769f1131e49c87235c6948a39951adeae49 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 5 May 2023 18:30:08 +0000 Subject: [PATCH 056/515] Refactor PreparedSets [4] --- src/Interpreters/PreparedSets.cpp | 8 +- src/Interpreters/PreparedSets.h | 3 +- src/Planner/PlannerActionsVisitor.cpp | 20 +- .../optimizePrimaryKeyCondition.cpp | 2 + .../QueryPlan/ReadFromMergeTree.cpp | 173 ++++++++++++------ src/Processors/QueryPlan/ReadFromMergeTree.h | 11 +- .../QueryPlan/SourceStepWithFilter.h | 2 + .../Transforms/CreatingSetsTransform.cpp | 1 + src/Storages/MergeTree/KeyCondition.cpp | 10 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +- 10 files changed, 160 insertions(+), 74 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 733eb1c24bb..5d9a0f27496 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -221,7 +221,7 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c if (set) return nullptr; - std::cerr << StackTrace().toString() << std::endl; + // std::cerr << StackTrace().toString() << std::endl; auto set_cache = context->getPreparedSetsCache(); if (set_cache) @@ -248,6 +248,10 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c auto plan = subquery.detachSource(); auto description = subquery.key; + // WriteBufferFromOwnString buf; + // plan->explainPlan(buf, {.header=true}); + // std::cerr << buf.str() << std::endl; + auto creating_set = std::make_unique( plan->getCurrentDataStream(), description, @@ -279,7 +283,7 @@ SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordere return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); } -FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) {} +FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) { std::cerr << block.dumpStructure() << std::endl; } FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquery(std::move(subquery_)) {} diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 8661d81a96a..b4d01754ea8 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -16,6 +16,7 @@ #include "Processors/Executors/CompletedPipelineExecutor.h" #include "Processors/QueryPlan/BuildQueryPipelineSettings.h" #include "Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h" +#include "Processors/Sinks/EmptySink.h" #include "Processors/Sinks/NullSink.h" #include @@ -171,7 +172,7 @@ public: auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); - pipeline.complete(std::make_shared(Block())); + pipeline.complete(std::make_shared(Block())); CompletedPipelineExecutor executor(pipeline); executor.execute(); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 453b02a2f8f..e0844a6d2b1 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -629,14 +629,20 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma //auto set_key = planner_context->createSetKey(in_second_argument); - DataTypes set_element_types = {in_first_argument->getResultType()}; - const auto * left_tuple_type = typeid_cast(set_element_types.front().get()); - if (left_tuple_type && left_tuple_type->getElements().size() != 1) - set_element_types = left_tuple_type->getElements(); + DataTypes set_element_types; - for (auto & element_type : set_element_types) - if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - element_type = low_cardinality_type->getDictionaryType(); + auto in_second_argument_node_type = in_second_argument->getNodeType(); + if (!(in_second_argument_node_type == QueryTreeNodeType::QUERY || in_second_argument_node_type == QueryTreeNodeType::UNION)) + { + set_element_types = {in_first_argument->getResultType()}; + const auto * left_tuple_type = typeid_cast(set_element_types.front().get()); + if (left_tuple_type && left_tuple_type->getElements().size() != 1) + set_element_types = left_tuple_type->getElements(); + + for (auto & element_type : set_element_types) + if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + element_type = low_cardinality_type->getDictionaryType(); + } auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index e98386a6ee9..5ef786ff975 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -28,6 +28,8 @@ void optimizePrimaryKeyCondition(const Stack & stack) else break; } + + source_step_with_filter->onAddFilterFinish(); } } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 185ec9bace8..ad63b486c7c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -247,7 +247,7 @@ ReadFromMergeTree::ReadFromMergeTree( { /// build sort description for output stream SortDescription sort_description; - const Names & sorting_key_columns = storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(); + const Names & sorting_key_columns = metadata_for_reading->getSortingKeyColumns(); const Block & header = output_stream->header; const int sort_direction = getSortDirection(); for (const auto & column_name : sorting_key_columns) @@ -1118,7 +1118,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(Merge prewhere_info, filter_nodes, storage_snapshot->metadata, - storage_snapshot->getMetadataForQuery(), + metadata_for_reading, query_info, context, requested_num_streams, @@ -1126,7 +1126,90 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(Merge data, real_column_names, sample_factor_column_queried, - log); + log, + key_condition); +} + +static ActionsDAGPtr buildFilterDAG( + const ContextPtr & context, + const PrewhereInfoPtr & prewhere_info, + const ActionDAGNodes & added_filter_nodes, + const SelectQueryInfo & query_info) +{ + const auto & settings = context->getSettingsRef(); + ActionsDAG::NodeRawConstPtrs nodes; + + if (prewhere_info) + { + { + const auto & node = prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name); + nodes.push_back(&node); + } + + if (prewhere_info->row_level_filter) + { + const auto & node = prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name); + nodes.push_back(&node); + } + } + + for (const auto & node : added_filter_nodes.nodes) + nodes.push_back(node); + + std::unordered_map node_name_to_input_node_column; + + if (settings.allow_experimental_analyzer && query_info.planner_context) + { + const auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(query_info.table_expression); + for (const auto & [column_identifier, column_name] : table_expression_data.getColumnIdentifierToColumnName()) + { + const auto & column = table_expression_data.getColumnOrThrow(column_name); + node_name_to_input_node_column.emplace(column_identifier, ColumnWithTypeAndName(column.type, column_name)); + } + } + + return ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_node_column, context); +} + +static void buildKeyCondition( + std::optional & key_condition, + ActionsDAGPtr filter_actions_dag, + const ContextPtr & context, + const SelectQueryInfo & query_info, + const StorageMetadataPtr & metadata_snapshot) +{ + key_condition.reset(); + + // Build and check if primary key is used when necessary + const auto & primary_key = metadata_snapshot->getPrimaryKey(); + const Names & primary_key_column_names = primary_key.column_names; + + const auto & settings = context->getSettingsRef(); + if (settings.query_plan_optimize_primary_key) + { + NameSet array_join_name_set; + if (query_info.syntax_analyzer_result) + array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(); + + key_condition.emplace(filter_actions_dag, + context, + primary_key_column_names, + primary_key.expression, + array_join_name_set); + } + else + { + key_condition.emplace(query_info, context, primary_key_column_names, primary_key.expression); + } +} + +void ReadFromMergeTree::onAddFilterFinish() +{ + if (!filter_nodes.nodes.empty()) + { + auto filter_actions_dag = buildFilterDAG(context, prewhere_info, filter_nodes, query_info); + buildKeyCondition(key_condition, filter_actions_dag, context, query_info, metadata_for_reading); + } } MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( @@ -1142,44 +1225,14 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( const MergeTreeData & data, const Names & real_column_names, bool sample_factor_column_queried, - Poco::Logger * log) + Poco::Logger * log, + std::optional & key_condition) { const auto & settings = context->getSettingsRef(); if (settings.allow_experimental_analyzer || settings.query_plan_optimize_primary_key) { - ActionsDAG::NodeRawConstPtrs nodes; - - if (prewhere_info) - { - { - const auto & node = prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name); - nodes.push_back(&node); - } - - if (prewhere_info->row_level_filter) - { - const auto & node = prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name); - nodes.push_back(&node); - } - } - - for (const auto & node : added_filter_nodes.nodes) - nodes.push_back(node); - - std::unordered_map node_name_to_input_node_column; - - if (settings.allow_experimental_analyzer && query_info.planner_context) - { - const auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(query_info.table_expression); - for (const auto & [column_identifier, column_name] : table_expression_data.getColumnIdentifierToColumnName()) - { - const auto & column = table_expression_data.getColumnOrThrow(column_name); - node_name_to_input_node_column.emplace(column_identifier, ColumnWithTypeAndName(column.type, column_name)); - } - } - auto updated_query_info_with_filter_dag = query_info; - updated_query_info_with_filter_dag.filter_actions_dag = ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_node_column, context); + updated_query_info_with_filter_dag.filter_actions_dag = buildFilterDAG(context, prewhere_info, added_filter_nodes, query_info); return selectRangesToReadImpl( parts, @@ -1192,7 +1245,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( data, real_column_names, sample_factor_column_queried, - log); + log, + key_condition); } return selectRangesToReadImpl( @@ -1206,7 +1260,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( data, real_column_names, sample_factor_column_queried, - log); + log, + key_condition); } MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( @@ -1220,7 +1275,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( const MergeTreeData & data, const Names & real_column_names, bool sample_factor_column_queried, - Poco::Logger * log) + Poco::Logger * log, + std::optional & key_condition) { AnalysisResult result; const auto & settings = context->getSettingsRef(); @@ -1246,24 +1302,29 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( // Build and check if primary key is used when necessary const auto & primary_key = metadata_snapshot->getPrimaryKey(); const Names & primary_key_column_names = primary_key.column_names; - std::optional key_condition; - if (settings.query_plan_optimize_primary_key) - { - NameSet array_join_name_set; - if (query_info.syntax_analyzer_result) - array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(); + // if (!key_condition) + // { + // if (settings.query_plan_optimize_primary_key) + // { + // NameSet array_join_name_set; + // if (query_info.syntax_analyzer_result) + // array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(); - key_condition.emplace(query_info.filter_actions_dag, - context, - primary_key_column_names, - primary_key.expression, - array_join_name_set); - } - else - { - key_condition.emplace(query_info, context, primary_key_column_names, primary_key.expression); - } + // key_condition.emplace(query_info.filter_actions_dag, + // context, + // primary_key_column_names, + // primary_key.expression, + // array_join_name_set); + // } + // else + // { + // key_condition.emplace(query_info, context, primary_key_column_names, primary_key.expression); + // } + // } + + if (!key_condition) + buildKeyCondition(key_condition, query_info.filter_actions_dag, context, query_info, metadata_snapshot); if (settings.force_primary_key && key_condition->alwaysUnknownOrTrue()) { @@ -1395,7 +1456,7 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, /// update sort info for output stream SortDescription sort_description; - const Names & sorting_key_columns = storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(); + const Names & sorting_key_columns = metadata_for_reading->getSortingKeyColumns(); const Block & header = output_stream->header; const int sort_direction = getSortDirection(); for (const auto & column_name : sorting_key_columns) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 5e4ba117967..121970f2ca9 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -145,7 +145,8 @@ public: const MergeTreeData & data, const Names & real_column_names, bool sample_factor_column_queried, - Poco::Logger * log); + Poco::Logger * log, + std::optional & key_condition); MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(MergeTreeData::DataPartsVector parts) const; @@ -177,6 +178,8 @@ public: size_t getNumStreams() const { return requested_num_streams; } bool isParallelReadingEnabled() const { return read_task_callback != std::nullopt; } + void onAddFilterFinish() override; + private: static MergeTreeDataSelectAnalysisResultPtr selectRangesToReadImpl( MergeTreeData::DataPartsVector parts, @@ -189,7 +192,8 @@ private: const MergeTreeData & data, const Names & real_column_names, bool sample_factor_column_queried, - Poco::Logger * log); + Poco::Logger * log, + std::optional & key_condition); int getSortDirection() const { @@ -228,6 +232,9 @@ private: std::shared_ptr max_block_numbers_to_read; + /// Pre-computed value, needed to trigger sets creatin for PK + mutable std::optional key_condition; + Poco::Logger * log; UInt64 selected_parts = 0; UInt64 selected_rows = 0; diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.h b/src/Processors/QueryPlan/SourceStepWithFilter.h index a363451fff2..34b6e3c6a7b 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.h +++ b/src/Processors/QueryPlan/SourceStepWithFilter.h @@ -37,6 +37,8 @@ public: filter_dags.push_back(std::move(filter_dag)); } + virtual void onAddFilterFinish() {} + protected: std::vector filter_dags; ActionDAGNodes filter_nodes; diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index de10be599c8..6626d4b9795 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -133,6 +133,7 @@ void CreatingSetsTransform::init() if (subquery.set) { + //std::cerr << "=========== " << getInputPort().getHeader().dumpStructure() << std::endl; subquery.set->setHeader(getInputPort().getHeader().getColumnsWithTypeAndName()); } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 7a1c3b10c8a..efeb9e40dd4 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1204,17 +1204,17 @@ bool KeyCondition::tryPrepareSetIndex( const auto right_arg = func.getArgumentAt(1); - LOG_TRACE(&Poco::Logger::get("KK"), "Trying to get set for {}", right_arg.getColumnName()); + // LOG_TRACE(&Poco::Logger::get("KK"), "Trying to get set for {}", right_arg.getColumnName()); auto future_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types); if (!future_set) return false; - LOG_TRACE(&Poco::Logger::get("KK"), "Found set for {}", right_arg.getColumnName()); + // LOG_TRACE(&Poco::Logger::get("KK"), "Found set for {}", right_arg.getColumnName()); if (!future_set->isReady()) { - LOG_TRACE(&Poco::Logger::get("KK"), "Building set inplace for {}", right_arg.getColumnName()); + // LOG_TRACE(&Poco::Logger::get("KK"), "Building set inplace for {}", right_arg.getColumnName()); future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); } @@ -1222,13 +1222,13 @@ bool KeyCondition::tryPrepareSetIndex( if (!prepared_set) return false; - LOG_TRACE(&Poco::Logger::get("KK"), "Set if ready for {}", right_arg.getColumnName()); + // LOG_TRACE(&Poco::Logger::get("KK"), "Set if ready for {}", right_arg.getColumnName()); /// The index can be prepared if the elements of the set were saved in advance. if (!prepared_set->hasExplicitSetElements()) return false; - LOG_TRACE(&Poco::Logger::get("KK"), "Has explicit elements for {}", right_arg.getColumnName()); + // LOG_TRACE(&Poco::Logger::get("KK"), "Has explicit elements for {}", right_arg.getColumnName()); prepared_set->checkColumnsNumber(left_args_count); for (size_t i = 0; i < indexes_mapping.size(); ++i) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 22df8f298c4..aae8f843a3c 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1291,6 +1291,7 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); + std::optional key_condition; return ReadFromMergeTree::selectRangesToRead( std::move(parts), prewhere_info, @@ -1304,7 +1305,8 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar data, real_column_names, sample_factor_column_queried, - log); + log, + key_condition); } QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( From b2c36fc3e5a968cd3223261585cb00f89be2c783 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 5 May 2023 20:51:38 +0000 Subject: [PATCH 057/515] Fixing style. --- src/Interpreters/PreparedSets.cpp | 7 ++++++- src/Processors/QueryPlan/ReadFromMergeTree.h | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 5d9a0f27496..1d7d90432b0 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -11,6 +11,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) { /// Remove LowCardinality types from type list because Set doesn't support LowCardinality keys now, @@ -283,7 +288,7 @@ SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordere return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); } -FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) { std::cerr << block.dumpStructure() << std::endl; } +FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) {} FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquery(std::move(subquery_)) {} diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 121970f2ca9..f13f75bfebc 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -232,7 +232,7 @@ private: std::shared_ptr max_block_numbers_to_read; - /// Pre-computed value, needed to trigger sets creatin for PK + /// Pre-computed value, needed to trigger sets creating for PK mutable std::optional key_condition; Poco::Logger * log; From fc02e9efc9eb2dcb9b7209e41eafb1e50abced7d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 8 May 2023 17:23:46 +0000 Subject: [PATCH 058/515] update fasttest a bit --- tests/ci/fast_test_check.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 89066ade2cb..fe211d79810 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -214,8 +214,11 @@ def main(): # Refuse other checks to run if fast test failed if state != "success": - if FORCE_TESTS_LABEL in pr_info.labels and state != "error": - print(f"'{FORCE_TESTS_LABEL}' enabled, will report success") + if state == "error": + print("The status is 'error', report failure disregard the labels") + sys.exit(1) + elif FORCE_TESTS_LABEL in pr_info.labels: + print(f"'{FORCE_TESTS_LABEL}' enabled, reporting success") else: sys.exit(1) From 3fedd683ef97e61ebcc17b2f8b38feb297fbc26c Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 8 May 2023 22:28:31 +0200 Subject: [PATCH 059/515] speedup vol. II --- .../Serializations/SerializationDate.cpp | 2 +- src/Functions/FunctionsConversion.h | 16 ++++---- src/IO/ReadHelpers.h | 40 +++++++++++++++---- 3 files changed, 42 insertions(+), 16 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationDate.cpp b/src/DataTypes/Serializations/SerializationDate.cpp index bc2057d549e..8b4956f7826 100644 --- a/src/DataTypes/Serializations/SerializationDate.cpp +++ b/src/DataTypes/Serializations/SerializationDate.cpp @@ -77,7 +77,7 @@ void SerializationDate::serializeTextCSV(const IColumn & column, size_t row_num, void SerializationDate::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { DayNum value; - readCSV(value, istr); + readCSV(value, istr, time_zone); assert_cast(column).getData().push_back(value); } SerializationDate::SerializationDate(const TimezoneMixin & time_zone_) : TimezoneMixin(time_zone_) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 0f2d49f2557..6af5c44eb5e 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -981,18 +981,18 @@ void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTI } template <> -inline void parseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline void parseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { DayNum tmp(0); - readDateText(tmp, rb); + readDateText(tmp, rb, *time_zone); x = tmp; } template <> -inline void parseImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline void parseImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { ExtendedDayNum tmp(0); - readDateText(tmp, rb); + readDateText(tmp, rb, *time_zone); x = tmp; } @@ -1040,20 +1040,20 @@ bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateL } template <> -inline bool tryParseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline bool tryParseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { DayNum tmp(0); - if (!tryReadDateText(tmp, rb)) + if (!tryReadDateText(tmp, rb, *time_zone)) return false; x = tmp; return true; } template <> -inline bool tryParseImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline bool tryParseImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { ExtendedDayNum tmp(0); - if (!tryReadDateText(tmp, rb)) + if (!tryReadDateText(tmp, rb, *time_zone)) return false; x = tmp; return true; diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 16c28b89667..f9e21418a41 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -760,14 +760,14 @@ inline bool tryReadDateText(LocalDate & date, ReadBuffer & buf) return readDateTextImpl(date, buf); } -inline bool tryReadDateText(DayNum & date, ReadBuffer & buf) +inline bool tryReadDateText(DayNum & date, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { - return readDateTextImpl(date, buf); + return readDateTextImpl(date, buf, time_zone); } -inline bool tryReadDateText(ExtendedDayNum & date, ReadBuffer & buf) +inline bool tryReadDateText(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { - return readDateTextImpl(date, buf); + return readDateTextImpl(date, buf, time_zone); } template @@ -1160,7 +1160,7 @@ inline void readText(is_floating_point auto & x, ReadBuffer & buf) { readFloatTe inline void readText(String & x, ReadBuffer & buf) { readEscapedString(x, buf); } inline void readText(LocalDate & x, ReadBuffer & buf) { readDateText(x, buf); } -inline void readText(DayNum & x, ReadBuffer & buf) { readDateText(x, buf); } +inline void readText(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { readDateText(x, buf, time_zone); } inline void readText(LocalDateTime & x, ReadBuffer & buf) { readDateTimeText(x, buf); } inline void readText(UUID & x, ReadBuffer & buf) { readUUIDText(x, buf); } inline void readText(IPv4 & x, ReadBuffer & buf) { readIPv4Text(x, buf); } @@ -1172,6 +1172,10 @@ template requires is_arithmetic_v inline void readQuoted(T & x, ReadBuffer & buf) { readText(x, buf); } +template +requires is_arithmetic_v +inline void readQuoted(T & x, ReadBuffer & buf, const DateLUTImpl & time_zone) { readText(x, buf, time_zone); } + inline void readQuoted(String & x, ReadBuffer & buf) { readQuotedString(x, buf); } inline void readQuoted(LocalDate & x, ReadBuffer & buf) @@ -1214,6 +1218,10 @@ template requires is_arithmetic_v inline void readDoubleQuoted(T & x, ReadBuffer & buf) { readText(x, buf); } +template + requires is_arithmetic_v +inline void readDoubleQuoted(T & x, ReadBuffer & buf, const DateLUTImpl & time_zone) { readText(x, buf, time_zone); } + inline void readDoubleQuoted(String & x, ReadBuffer & buf) { readDoubleQuotedString(x, buf); } inline void readDoubleQuoted(LocalDate & x, ReadBuffer & buf) @@ -1230,7 +1238,7 @@ inline void readDoubleQuoted(LocalDateTime & x, ReadBuffer & buf) assertChar('"', buf); } -/// CSV, for numbers, dates: quotes are optional, no special escaping rules. +/// CSV for numbers: quotes are optional, no special escaping rules. template inline void readCSVSimple(T & x, ReadBuffer & buf) { @@ -1248,6 +1256,24 @@ inline void readCSVSimple(T & x, ReadBuffer & buf) assertChar(maybe_quote, buf); } +// standalone overload for dates: to avoid instantiating DateLUTs while parsing other types +template +inline void readCSVSimple(T & x, ReadBuffer & buf, const DateLUTImpl & time_zone) +{ + if (buf.eof()) [[unlikely]] + throwReadAfterEOF(); + + char maybe_quote = *buf.position(); + + if (maybe_quote == '\'' || maybe_quote == '\"') + ++buf.position(); + + readText(x, buf, time_zone); + + if (maybe_quote == '\'' || maybe_quote == '\"') + assertChar(maybe_quote, buf); +} + template requires is_arithmetic_v inline void readCSV(T & x, ReadBuffer & buf) @@ -1257,7 +1283,7 @@ inline void readCSV(T & x, ReadBuffer & buf) inline void readCSV(String & x, ReadBuffer & buf, const FormatSettings::CSV & settings) { readCSVString(x, buf, settings); } inline void readCSV(LocalDate & x, ReadBuffer & buf) { readCSVSimple(x, buf); } -inline void readCSV(DayNum & x, ReadBuffer & buf) { readCSVSimple(x, buf); } +inline void readCSV(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { readCSVSimple(x, buf, time_zone); } inline void readCSV(LocalDateTime & x, ReadBuffer & buf) { readCSVSimple(x, buf); } inline void readCSV(UUID & x, ReadBuffer & buf) { readCSVSimple(x, buf); } inline void readCSV(IPv4 & x, ReadBuffer & buf) { readCSVSimple(x, buf); } From 1751ccc7aca3830d21a06ec4f09bd28bf9254f79 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 9 May 2023 14:18:04 +0200 Subject: [PATCH 060/515] fix stateless --- src/Functions/FunctionsConversion.h | 15 ++++++++++++--- src/IO/ReadHelpers.h | 12 ++++++++---- 2 files changed, 20 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 6af5c44eb5e..e0e188f68c2 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -439,7 +439,7 @@ struct ToDate32Transform32Or64Signed static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { - static const Int32 daynum_min_offset = -static_cast(DateLUT::instance().getDayNumOffsetEpoch()); + static const Int32 daynum_min_offset = -static_cast(time_zone.getDayNumOffsetEpoch()); if (from < daynum_min_offset) return daynum_min_offset; return (from < DATE_LUT_MAX_EXTEND_DAY_NUM) @@ -830,8 +830,11 @@ struct ConvertImpl(*col_with_type_and_name.type); const DateLUTImpl * time_zone = nullptr; + + if constexpr (std::is_same_v) + time_zone = &DateLUT::instance(); /// For argument of Date or DateTime type, second argument with time zone could be specified. - if constexpr (std::is_same_v || std::is_same_v || std::is_same_v) + if constexpr (std::is_same_v || std::is_same_v) { auto non_null_args = createBlockWithNestedColumns(arguments); time_zone = &extractTimeZoneFromFunctionArguments(non_null_args, 1, 0); @@ -1193,7 +1196,7 @@ struct ConvertThroughParsing const DateLUTImpl * local_time_zone [[maybe_unused]] = nullptr; const DateLUTImpl * utc_time_zone [[maybe_unused]] = nullptr; - /// For conversion to DateTime type, second argument with time zone could be specified. + /// For conversion to Date or DateTime type, second argument with time zone could be specified. if constexpr (std::is_same_v || to_datetime64) { const auto result_type = removeNullable(res_type); @@ -1206,6 +1209,12 @@ struct ConvertThroughParsing if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort || parsing_mode == ConvertFromStringParsingMode::BestEffortUS) utc_time_zone = &DateLUT::instance("UTC"); } + else if constexpr (std::is_same_v || std::is_same_v) + { + // Timezone is more or less dummy when parsing Date/Date32 from string. + local_time_zone = &DateLUT::instance(); + utc_time_zone = &DateLUT::instance("UTC"); + } const IColumn * col_from = arguments[0].column.get(); const ColumnString * col_from_string = checkAndGetColumn(col_from); diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index f9e21418a41..ea565d11914 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -718,7 +718,7 @@ inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf, const DateLU return false; ExtendedDayNum ret = date_lut.makeDayNum(local_date.year(), local_date.month(), local_date.day()); - convertToDayNum(date,ret); + convertToDayNum(date, ret); return ReturnType(true); } @@ -1159,8 +1159,11 @@ inline bool tryReadText(IPv6 & x, ReadBuffer & buf) { return tryReadIPv6Text(x, inline void readText(is_floating_point auto & x, ReadBuffer & buf) { readFloatText(x, buf); } inline void readText(String & x, ReadBuffer & buf) { readEscapedString(x, buf); } + +inline void readText(DayNum & x, ReadBuffer & buf) { readDateText(x, buf); } +inline void readText(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone) { readDateText(x, buf, time_zone); } + inline void readText(LocalDate & x, ReadBuffer & buf) { readDateText(x, buf); } -inline void readText(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { readDateText(x, buf, time_zone); } inline void readText(LocalDateTime & x, ReadBuffer & buf) { readDateTimeText(x, buf); } inline void readText(UUID & x, ReadBuffer & buf) { readUUIDText(x, buf); } inline void readText(IPv4 & x, ReadBuffer & buf) { readIPv4Text(x, buf); } @@ -1219,7 +1222,7 @@ requires is_arithmetic_v inline void readDoubleQuoted(T & x, ReadBuffer & buf) { readText(x, buf); } template - requires is_arithmetic_v +requires is_arithmetic_v inline void readDoubleQuoted(T & x, ReadBuffer & buf, const DateLUTImpl & time_zone) { readText(x, buf, time_zone); } inline void readDoubleQuoted(String & x, ReadBuffer & buf) { readDoubleQuotedString(x, buf); } @@ -1283,7 +1286,8 @@ inline void readCSV(T & x, ReadBuffer & buf) inline void readCSV(String & x, ReadBuffer & buf, const FormatSettings::CSV & settings) { readCSVString(x, buf, settings); } inline void readCSV(LocalDate & x, ReadBuffer & buf) { readCSVSimple(x, buf); } -inline void readCSV(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { readCSVSimple(x, buf, time_zone); } +inline void readCSV(DayNum & x, ReadBuffer & buf) { readCSVSimple(x, buf); } +inline void readCSV(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone) { readCSVSimple(x, buf, time_zone); } inline void readCSV(LocalDateTime & x, ReadBuffer & buf) { readCSVSimple(x, buf); } inline void readCSV(UUID & x, ReadBuffer & buf) { readCSVSimple(x, buf); } inline void readCSV(IPv4 & x, ReadBuffer & buf) { readCSVSimple(x, buf); } From 297188ce583a94f9942f7fd141a85dbdcfdcd587 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 9 May 2023 22:37:25 +0200 Subject: [PATCH 061/515] fix Date32 --- src/DataTypes/Serializations/SerializationDate32.cpp | 11 +++++++---- src/DataTypes/Serializations/SerializationDate32.h | 5 ++++- src/Functions/FunctionsConversion.h | 7 +++++-- 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationDate32.cpp b/src/DataTypes/Serializations/SerializationDate32.cpp index ef92202f89d..8dcaee8d266 100644 --- a/src/DataTypes/Serializations/SerializationDate32.cpp +++ b/src/DataTypes/Serializations/SerializationDate32.cpp @@ -11,7 +11,7 @@ namespace DB void SerializationDate32::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeDateText(ExtendedDayNum(assert_cast(column).getData()[row_num]), ostr); + writeDateText(ExtendedDayNum(assert_cast(column).getData()[row_num]), ostr, time_zone); } void SerializationDate32::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const @@ -24,7 +24,7 @@ void SerializationDate32::deserializeWholeText(IColumn & column, ReadBuffer & is void SerializationDate32::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { ExtendedDayNum x; - readDateText(x, istr); + readDateText(x, istr, time_zone); assert_cast(column).getData().push_back(x); } @@ -44,7 +44,7 @@ void SerializationDate32::deserializeTextQuoted(IColumn & column, ReadBuffer & i { ExtendedDayNum x; assertChar('\'', istr); - readDateText(x, istr); + readDateText(x, istr, time_zone); assertChar('\'', istr); assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. } @@ -60,7 +60,7 @@ void SerializationDate32::deserializeTextJSON(IColumn & column, ReadBuffer & ist { ExtendedDayNum x; assertChar('"', istr); - readDateText(x, istr); + readDateText(x, istr, time_zone); assertChar('"', istr); assert_cast(column).getData().push_back(x); } @@ -78,4 +78,7 @@ void SerializationDate32::deserializeTextCSV(IColumn & column, ReadBuffer & istr readCSV(value, istr); assert_cast(column).getData().push_back(value.getExtenedDayNum()); } +SerializationDate32::SerializationDate32(const TimezoneMixin & time_zone_) : TimezoneMixin(time_zone_) +{ +} } diff --git a/src/DataTypes/Serializations/SerializationDate32.h b/src/DataTypes/Serializations/SerializationDate32.h index 484b4f4a958..e8e8f1a74d6 100644 --- a/src/DataTypes/Serializations/SerializationDate32.h +++ b/src/DataTypes/Serializations/SerializationDate32.h @@ -1,12 +1,15 @@ #pragma once #include +#include namespace DB { -class SerializationDate32 final : public SerializationNumber +class SerializationDate32 final : public SerializationNumber, public TimezoneMixin { public: + explicit SerializationDate32(const TimezoneMixin & time_zone_ = TimezoneMixin()); + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index e0e188f68c2..2f751e72222 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -731,7 +731,10 @@ struct FormatImpl template static ReturnType execute(const DataTypeDate32::FieldType x, WriteBuffer & wb, const DataTypeDate32 *, const DateLUTImpl * time_zone) { + std::cerr << "BEFORE: " << std::endl; + std::cerr << time_zone->getTimeZone() << std::endl; writeDateText(ExtendedDayNum(x), wb, *time_zone); + std::cerr << "AFTER" << std::endl; return ReturnType(true); } }; @@ -831,7 +834,7 @@ struct ConvertImpl) + if constexpr (std::is_same_v || std::is_same_v) time_zone = &DateLUT::instance(); /// For argument of Date or DateTime type, second argument with time zone could be specified. if constexpr (std::is_same_v || std::is_same_v) @@ -1765,7 +1768,7 @@ public: || std::is_same_v // toDate(value[, timezone : String]) || std::is_same_v // TODO: shall we allow timestamp argument for toDate? DateTime knows nothing about timezones and this argument is ignored below. - // toDate(value[, timezone : String]) + // toDate32(value[, timezone : String]) || std::is_same_v // toDateTime(value[, timezone: String]) || std::is_same_v From 8d0644e79301a9a0ccf67b66d44c43e4766d8aa7 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 9 May 2023 23:02:03 +0200 Subject: [PATCH 062/515] cleanup --- src/Functions/FunctionsConversion.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 2f751e72222..b10d9f4a31a 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -731,10 +731,7 @@ struct FormatImpl template static ReturnType execute(const DataTypeDate32::FieldType x, WriteBuffer & wb, const DataTypeDate32 *, const DateLUTImpl * time_zone) { - std::cerr << "BEFORE: " << std::endl; - std::cerr << time_zone->getTimeZone() << std::endl; writeDateText(ExtendedDayNum(x), wb, *time_zone); - std::cerr << "AFTER" << std::endl; return ReturnType(true); } }; From 07630ef43fd40f46dfba9adca487c3b69ca2ad3c Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 11 May 2023 01:10:34 +0200 Subject: [PATCH 063/515] upd --- src/Client/ClientBase.cpp | 4 +--- src/Client/Connection.cpp | 8 ++++---- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 8b5db85fc02..fad9494ba4b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1082,9 +1082,7 @@ void ClientBase::onProgress(const Progress & value) void ClientBase::onTimezoneUpdate(const String & tz) { - Settings settings; - settings.session_timezone = tz; - global_context->applySettingsChanges(settings.changes()); + global_context->setSetting("session_timezone", tz); } diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 062f05105aa..86585d805d9 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -355,10 +355,10 @@ void Connection::receiveHello() nonce.emplace(read_nonce); } } - else if (packet_type == Protocol::Server::TimezoneUpdate) - { - // skip this packet at hello, will receive and process it later - } +// else if (packet_type == Protocol::Server::TimezoneUpdate) +// { +// // skip this packet at hello, will receive and process it later +// } else if (packet_type == Protocol::Server::Exception) receiveException()->rethrow(); else From 58bdcc29315a712e1255c13d31669f4545de9edb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 10 May 2023 23:55:13 +0000 Subject: [PATCH 064/515] allow to cast IPv6 to IPv4 for address in proper mapping block --- src/Functions/FunctionsConversion.h | 30 ++++++++++++++++++++++++++--- 1 file changed, 27 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 9c4085f9745..9cdd09780e3 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -57,6 +57,7 @@ #include #include #include +#include #include @@ -217,13 +218,13 @@ struct ConvertImpl } else if constexpr ( (std::is_same_v != std::is_same_v) - && !(is_any_of || is_any_of) + && !(is_any_of || is_any_of) ) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Conversion from {} to {} is not supported", TypeName, TypeName); } - else if constexpr (std::is_same_v != std::is_same_v) + else if constexpr (std::is_same_v != std::is_same_v && !std::is_same_v) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Conversion between numeric types and IPv6 is not supported. " @@ -304,7 +305,30 @@ struct ConvertImpl } else { - if constexpr (std::is_same_v && std::is_same_v) + if constexpr (std::is_same_v && std::is_same_v) + { + const uint8_t ip4_cidr[] {0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xff, 0xff, 0x00, 0x00, 0x00, 0x00}; + const uint8_t * src = reinterpret_cast(&vec_from[i].toUnderType()); + if (!matchIPv6Subnet(src, ip4_cidr, 96)) + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 in column {} is not in IPv4 mapping block", named_from.column->getName()); + + uint8_t * dst = reinterpret_cast(&vec_to[i].toUnderType()); + if constexpr (std::endian::native == std::endian::little) + { + dst[0] = src[15]; + dst[1] = src[14]; + dst[2] = src[13]; + dst[3] = src[12]; + } + else + { + dst[3] = src[15]; + dst[2] = src[14]; + dst[1] = src[13]; + dst[0] = src[12]; + } + } + else if constexpr (std::is_same_v && std::is_same_v) vec_to[i] = static_cast(static_cast(vec_from[i])); else vec_to[i] = static_cast(vec_from[i]); From 108e256578574b26f8adeb3916b15238f0557ee9 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 11 May 2023 16:17:52 +0000 Subject: [PATCH 065/515] allow to cast IPv4 to IPv6 --- src/Functions/FunctionsConversion.h | 32 ++++++++++++++++++++++++----- 1 file changed, 27 insertions(+), 5 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 9cdd09780e3..5bf59f33cb5 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -218,13 +218,13 @@ struct ConvertImpl } else if constexpr ( (std::is_same_v != std::is_same_v) - && !(is_any_of || is_any_of) + && !(is_any_of || is_any_of) ) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Conversion from {} to {} is not supported", TypeName, TypeName); } - else if constexpr (std::is_same_v != std::is_same_v && !std::is_same_v) + else if constexpr (std::is_same_v != std::is_same_v && !(std::is_same_v || std::is_same_v)) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Conversion between numeric types and IPv6 is not supported. " @@ -322,10 +322,32 @@ struct ConvertImpl } else { - dst[3] = src[15]; - dst[2] = src[14]; - dst[1] = src[13]; dst[0] = src[12]; + dst[1] = src[13]; + dst[2] = src[14]; + dst[3] = src[15]; + } + } + else if constexpr (std::is_same_v && std::is_same_v) + { + const uint8_t * src = reinterpret_cast(&vec_from[i].toUnderType()); + uint8_t * dst = reinterpret_cast(&vec_to[i].toUnderType()); + std::memset(dst, '\0', IPV6_BINARY_LENGTH); + dst[10] = dst[11] = 0xff; + + if constexpr (std::endian::native == std::endian::little) + { + dst[12] = src[3]; + dst[13] = src[2]; + dst[14] = src[1]; + dst[15] = src[0]; + } + else + { + dst[12] = src[0]; + dst[13] = src[1]; + dst[14] = src[2]; + dst[15] = src[3]; } } else if constexpr (std::is_same_v && std::is_same_v) From 60b69601e9b1e3563eb43bf4ea1deee582e088fa Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 12 May 2023 00:27:11 +0200 Subject: [PATCH 066/515] update docs --- .../server-configuration-parameters/settings.md | 4 ++++ docs/en/operations/settings/settings.md | 16 +++++++++++----- .../functions/date-time-functions.md | 14 +++++++++++--- .../server-configuration-parameters/settings.md | 4 ++++ docs/ru/operations/settings/settings.md | 12 ++++++++---- .../functions/date-time-functions.md | 13 +++++++++++-- 6 files changed, 49 insertions(+), 14 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index e3ca04f5b9b..36ddf6faad0 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1612,6 +1612,10 @@ The time zone is necessary for conversions between String and DateTime formats w Asia/Istanbul ``` +**See also** + +- [session_timezone](../settings/settings.md#session_timezone) + ## tcp_port {#server_configuration_parameters-tcp_port} Port for communicating with clients over the TCP protocol. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index cc5f292f677..2a929acd5f2 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4036,23 +4036,25 @@ Use this setting only for backward compatibility if your use cases depend on old ## session_timezone {#session_timezone} -If specified, sets an implicit timezone (instead of server-default). All DateTime/DateTime64 values (and/or functions results) that have no explicit timezone specified are treated as having this timezone instead of default. -Setting this to `''` (empty string) effectively resets implicit timezone to server timezone. +If specified, sets an implicit timezone (instead of [server default](../server-configuration-parameters/settimgs.md#server_configuration_parameters-timezone). +All DateTime/DateTime64 values (and/or functions results) that have no explicit timezone specified are treated as having this timezone instead of default. +A value of `''` (empty string) configures the session timezone to the server default timezone. + Examples: -```clickhouse +```sql SELECT timeZone(), serverTimezone() FORMAT TSV Europe/Berlin Europe/Berlin ``` -```clickhouse +```sql SELECT timeZone(), serverTimezone() SETTINGS session_timezone = 'Asia/Novosibirsk' FORMAT TSV Asia/Novosibirsk Europe/Berlin ``` -```clickhouse +```sql SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'America/Denver' FORMAT TSV 1999-12-13 07:23:23.123 @@ -4064,6 +4066,10 @@ Possible values: Default value: `''`. +**See also** + +- [timezone](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone) + ## final {#final} Automatically applies [FINAL](../../sql-reference/statements/select/from.md#final-modifier) modifier to all tables in a query, to tables where [FINAL](../../sql-reference/statements/select/from.md#final-modifier) is applicable, including joined tables and tables in sub-queries, and diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 596a2c509cd..9207a135c67 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -139,7 +139,7 @@ makeDateTime32(year, month, day, hour, minute, second[, fraction[, precision[, t ## timeZone -Returns the default timezone of the server for current session. This can be modified using `SET session_timezone = 'New/Value'` +Returns the default timezone of the current session, i.e. the value of setting [session_timezone](../../operations/settings/settings.md#session_timezone). If the function is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard, otherwise it produces a constant value. **Syntax** @@ -156,9 +156,13 @@ Alias: `timezone`. Type: [String](../../sql-reference/data-types/string.md). +**See also** + +- [serverTimeZone](#serverTimeZone) + ## serverTimeZone -Returns the actual timezone in which the server runs in. +Returns the default timezone of the server, i.e. the value of setting [timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. **Syntax** @@ -175,6 +179,10 @@ Alias: `ServerTimezone`, `servertimezone`. Type: [String](../../sql-reference/data-types/string.md). +**See also** + +- [timeZone](#timeZone) + ## toTimeZone Converts a date or date with time to the specified time zone. Does not change the internal value (number of unix seconds) of the data, only the value's time zone attribute and the value's string representation changes. @@ -408,7 +416,7 @@ Result: ``` :::note -The return type of `toStartOf*`, `toLastDayOfMonth`, `toMonday`, `timeSlot` functions described below is determined by the configuration parameter [enable_extended_results_for_datetime_functions](../../operations/settings/settings.md#enable-extended-results-for-datetime-functions) which is `0` by default. +Thes return type of `toStartOf*`, `toLastDayOfMonth`, `toMonday`, `timeSlot` functions described below is determined by the configuration parameter [enable_extended_results_for_datetime_functions](../../operations/settings/settings.md#enable-extended-results-for-datetime-functions) which is `0` by default. Behavior for * `enable_extended_results_for_datetime_functions = 0`: Functions `toStartOfYear`, `toStartOfISOYear`, `toStartOfQuarter`, `toStartOfMonth`, `toStartOfWeek`, `toLastDayOfMonth`, `toMonday` return `Date` or `DateTime`. Functions `toStartOfDay`, `toStartOfHour`, `toStartOfFifteenMinutes`, `toStartOfTenMinutes`, `toStartOfFiveMinutes`, `toStartOfMinute`, `timeSlot` return `DateTime`. Though these functions can take values of the extended types `Date32` and `DateTime64` as an argument, passing them a time outside the normal range (year 1970 to 2149 for `Date` / 2106 for `DateTime`) will produce wrong results. diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 787153d4d19..33db6df0fdd 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1355,6 +1355,10 @@ Parameters: Europe/Moscow ``` +**См. также** + +- [session_timezone](../settings/settings.md#session_timezone) + ## tcp_port {#server_configuration_parameters-tcp_port} Порт для взаимодействия с клиентами по протоколу TCP. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 585a3995afe..56bfbf8a57f 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4077,23 +4077,23 @@ SELECT sum(number) FROM numbers(10000000000) SETTINGS partial_result_on_first_ca ## session_timezone {#session_timezone} -Задаёт значение часового пояса (session_timezone) по умолчанию для текущей сессии вместо часового пояса сервера. То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. +Задаёт значение часового пояса (session_timezone) по умолчанию для текущей сессии вместо [часового пояса сервера](../server-configuration-parameters/settimgs.md#server_configuration_parameters-timezone). То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. При значении настройки `''` (пустая строка), будет совпадать с часовым поясом сервера. Примеры: -```clickhouse +```sql SELECT timeZone(), serverTimezone() FORMAT TSV Europe/Berlin Europe/Berlin ``` -```clickhouse +```sql SELECT timeZone(), serverTimezone() SETTINGS session_timezone = 'Asia/Novosibirsk' FORMAT TSV Asia/Novosibirsk Europe/Berlin ``` -```clickhouse +```sql SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'America/Denver' FORMAT TSV 1999-12-13 07:23:23.123 @@ -4104,3 +4104,7 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric - Любая зона из `system.time_zones`, например `Europe/Berlin`, `UTC` или `Zulu` Значение по умолчанию: `''`. + +**Смотрите также** + +- [timezone](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone) \ No newline at end of file diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 3e378c08308..2d9f96c3199 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -26,7 +26,8 @@ SELECT ## timeZone {#timezone} -Возвращает часовой пояс сервера, считающийся умолчанием для текущей сессии. Можно изменить значение с помощью `SET session_timezone = 'New/Timezone''` +Возвращает часовой пояс сервера, считающийся умолчанием для текущей сессии: значение параметра [session_timezone](../../operations/settings/settings.md#session_timezone), если установлено. + Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями, актуальными для каждого шарда. Иначе возвращается константа. **Синтаксис** @@ -43,9 +44,13 @@ timeZone() Тип: [String](../../sql-reference/data-types/string.md). +**Смотрите также** + +- [serverTimeZone](#servertimezone) + ## serverTimeZone {#servertimezone} -Возвращает (истинный) часовой пояс сервера, в котором тот работает. +Возвращает часовой пояс сервера по умолчанию, в т.ч. установленный [timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями, актуальными для каждого шарда. Иначе возвращается константа. **Синтаксис** @@ -62,6 +67,10 @@ serverTimeZone() Тип: [String](../../sql-reference/data-types/string.md). +**Смотрите также** + +- [timeZone](#timezone) + ## toTimeZone {#totimezone} Переводит дату или дату с временем в указанный часовой пояс. Часовой пояс - это атрибут типов `Date` и `DateTime`. Внутреннее значение (количество секунд) поля таблицы или результирующего столбца не изменяется, изменяется тип поля и, соответственно, его текстовое отображение. From c3af36915f049794a9c44c55ebb6a6bc950eadc8 Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 12 May 2023 01:29:34 +0200 Subject: [PATCH 067/515] fixed docs 2 --- docs/en/operations/settings/settings.md | 2 +- docs/ru/operations/settings/settings.md | 2 +- src/DataTypes/Serializations/SerializationDate.cpp | 3 ++- src/DataTypes/Serializations/SerializationDate.h | 7 +++++-- src/DataTypes/Serializations/SerializationDate32.cpp | 3 ++- src/DataTypes/Serializations/SerializationDate32.h | 7 +++++-- src/IO/ReadHelpers.h | 3 +-- 7 files changed, 17 insertions(+), 10 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2a929acd5f2..e796ea83a6f 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4036,7 +4036,7 @@ Use this setting only for backward compatibility if your use cases depend on old ## session_timezone {#session_timezone} -If specified, sets an implicit timezone (instead of [server default](../server-configuration-parameters/settimgs.md#server_configuration_parameters-timezone). +If specified, sets an implicit timezone (instead of [server default](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone). All DateTime/DateTime64 values (and/or functions results) that have no explicit timezone specified are treated as having this timezone instead of default. A value of `''` (empty string) configures the session timezone to the server default timezone. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 56bfbf8a57f..98486847fd9 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4077,7 +4077,7 @@ SELECT sum(number) FROM numbers(10000000000) SETTINGS partial_result_on_first_ca ## session_timezone {#session_timezone} -Задаёт значение часового пояса (session_timezone) по умолчанию для текущей сессии вместо [часового пояса сервера](../server-configuration-parameters/settimgs.md#server_configuration_parameters-timezone). То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. +Задаёт значение часового пояса (session_timezone) по умолчанию для текущей сессии вместо [часового пояса сервера](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone). То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. При значении настройки `''` (пустая строка), будет совпадать с часовым поясом сервера. Примеры: diff --git a/src/DataTypes/Serializations/SerializationDate.cpp b/src/DataTypes/Serializations/SerializationDate.cpp index 8b4956f7826..1ed48fdd31d 100644 --- a/src/DataTypes/Serializations/SerializationDate.cpp +++ b/src/DataTypes/Serializations/SerializationDate.cpp @@ -80,7 +80,8 @@ void SerializationDate::deserializeTextCSV(IColumn & column, ReadBuffer & istr, readCSV(value, istr, time_zone); assert_cast(column).getData().push_back(value); } -SerializationDate::SerializationDate(const TimezoneMixin & time_zone_) : TimezoneMixin(time_zone_) + +SerializationDate::SerializationDate(const DateLUTImpl & time_zone_) : time_zone(time_zone_) { } diff --git a/src/DataTypes/Serializations/SerializationDate.h b/src/DataTypes/Serializations/SerializationDate.h index c4e57470673..4d6a6fa36ec 100644 --- a/src/DataTypes/Serializations/SerializationDate.h +++ b/src/DataTypes/Serializations/SerializationDate.h @@ -6,10 +6,10 @@ namespace DB { -class SerializationDate final : public SerializationNumber, public TimezoneMixin +class SerializationDate final : public SerializationNumber { public: - explicit SerializationDate(const TimezoneMixin & time_zone_ = TimezoneMixin()); + explicit SerializationDate(const DateLUTImpl & time_zone_ = DateLUT::instance()); void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; @@ -21,6 +21,9 @@ public: void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + +protected: + const DateLUTImpl & time_zone; }; } diff --git a/src/DataTypes/Serializations/SerializationDate32.cpp b/src/DataTypes/Serializations/SerializationDate32.cpp index 8dcaee8d266..851710de839 100644 --- a/src/DataTypes/Serializations/SerializationDate32.cpp +++ b/src/DataTypes/Serializations/SerializationDate32.cpp @@ -78,7 +78,8 @@ void SerializationDate32::deserializeTextCSV(IColumn & column, ReadBuffer & istr readCSV(value, istr); assert_cast(column).getData().push_back(value.getExtenedDayNum()); } -SerializationDate32::SerializationDate32(const TimezoneMixin & time_zone_) : TimezoneMixin(time_zone_) + +SerializationDate32::SerializationDate32(const DateLUTImpl & time_zone_) : time_zone(time_zone_) { } } diff --git a/src/DataTypes/Serializations/SerializationDate32.h b/src/DataTypes/Serializations/SerializationDate32.h index e8e8f1a74d6..6b6e5442240 100644 --- a/src/DataTypes/Serializations/SerializationDate32.h +++ b/src/DataTypes/Serializations/SerializationDate32.h @@ -5,10 +5,10 @@ namespace DB { -class SerializationDate32 final : public SerializationNumber, public TimezoneMixin +class SerializationDate32 final : public SerializationNumber { public: - explicit SerializationDate32(const TimezoneMixin & time_zone_ = TimezoneMixin()); + explicit SerializationDate32(const DateLUTImpl & time_zone_ = DateLUT::instance()); void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; @@ -20,5 +20,8 @@ public: void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + +protected: + const DateLUTImpl & time_zone; }; } diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index ea565d11914..3bd9275322e 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1160,8 +1160,7 @@ inline void readText(is_floating_point auto & x, ReadBuffer & buf) { readFloatTe inline void readText(String & x, ReadBuffer & buf) { readEscapedString(x, buf); } -inline void readText(DayNum & x, ReadBuffer & buf) { readDateText(x, buf); } -inline void readText(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone) { readDateText(x, buf, time_zone); } +inline void readText(DayNum & x, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) { readDateText(x, buf, time_zone); } inline void readText(LocalDate & x, ReadBuffer & buf) { readDateText(x, buf); } inline void readText(LocalDateTime & x, ReadBuffer & buf) { readDateTimeText(x, buf); } From 24067ea977b6e4484f68efba8858ba8d0ad1cd6b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 12 May 2023 15:54:50 +0000 Subject: [PATCH 068/515] allow conversion for toIPv4OrDefault --- src/Functions/FunctionsCodingIP.h | 81 +++++++++++++++++++++++++++++ src/Functions/FunctionsConversion.h | 20 +++++++ 2 files changed, 101 insertions(+) diff --git a/src/Functions/FunctionsCodingIP.h b/src/Functions/FunctionsCodingIP.h index d02cc81f608..bd53fa7e043 100644 --- a/src/Functions/FunctionsCodingIP.h +++ b/src/Functions/FunctionsCodingIP.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -16,6 +17,7 @@ namespace ErrorCodes extern const int CANNOT_PARSE_IPV4; extern const int CANNOT_PARSE_IPV6; extern const int ILLEGAL_COLUMN; + extern const int CANNOT_CONVERT_TYPE; } enum class IPStringToNumExceptionMode : uint8_t @@ -296,4 +298,83 @@ ColumnPtr convertToIPv4(ColumnPtr column, const PaddedPODArray * null_map return col_res; } +template +ColumnPtr convertIPv6ToIPv4(ColumnPtr column, const PaddedPODArray * null_map = nullptr) +{ + const ColumnIPv6 * column_ipv6 = checkAndGetColumn(column.get()); + + if (!column_ipv6) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column type {}. Expected IPv6.", column->getName()); + + size_t column_size = column_ipv6->size(); + + ColumnUInt8::MutablePtr col_null_map_to; + ColumnUInt8::Container * vec_null_map_to = nullptr; + + if constexpr (exception_mode == IPStringToNumExceptionMode::Null) + { + col_null_map_to = ColumnUInt8::create(column_size, false); + vec_null_map_to = &col_null_map_to->getData(); + } + + const uint8_t ip4_cidr[] {0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xff, 0xff, 0x00, 0x00, 0x00, 0x00}; + + auto col_res = ToColumn::create(); + auto & vec_res = col_res->getData(); + vec_res.resize(column_size); + const auto & vec_src = column_ipv6->getData(); + + for (size_t i = 0; i < vec_res.size(); ++i) + { + const uint8_t * src = reinterpret_cast(&vec_src[i]); + uint8_t * dst = reinterpret_cast(&vec_res[i]); + + if (null_map && (*null_map)[i]) + { + std::memset(dst, '\0', IPV4_BINARY_LENGTH); + if constexpr (exception_mode == IPStringToNumExceptionMode::Null) + (*vec_null_map_to)[i] = true; + continue; + } + + if (!matchIPv6Subnet(src, ip4_cidr, 96)) + { + if constexpr (exception_mode == IPStringToNumExceptionMode::Throw) + { + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 in column {} is not in IPv4 mapping block", column->getName()); + } + else if constexpr (exception_mode == IPStringToNumExceptionMode::Default) + { + std::memset(dst, '\0', IPV4_BINARY_LENGTH); + } + else if constexpr (exception_mode == IPStringToNumExceptionMode::Null) + { + (*vec_null_map_to)[i] = true; + std::memset(dst, '\0', IPV4_BINARY_LENGTH); + } + continue; + } + + if constexpr (std::endian::native == std::endian::little) + { + dst[0] = src[15]; + dst[1] = src[14]; + dst[2] = src[13]; + dst[3] = src[12]; + } + else + { + dst[0] = src[12]; + dst[1] = src[13]; + dst[2] = src[14]; + dst[3] = src[15]; + } + } + + if constexpr (exception_mode == IPStringToNumExceptionMode::Null) + return ColumnNullable::create(std::move(col_res), std::move(col_null_map_to)); + + return col_res; +} + } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 5bf59f33cb5..4d4efc84df1 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -4035,6 +4035,26 @@ private: return true; } } + else if constexpr (WhichDataType(FromDataType::type_id).isIPv6() && WhichDataType(ToDataType::type_id).isIPv4()) + { + ret = [cast_ipv4_ipv6_default_on_conversion_error_value, requested_result_is_nullable]( + ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t) + -> ColumnPtr + { + if (!WhichDataType(result_type).isIPv4()) + throw Exception( + ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv4", result_type->getName()); + + const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr; + if (cast_ipv4_ipv6_default_on_conversion_error_value || requested_result_is_nullable) + return convertIPv6ToIPv4(arguments[0].column, null_map); + else + return convertIPv6ToIPv4(arguments[0].column, null_map); + }; + + return true; + } + if constexpr (WhichDataType(ToDataType::type_id).isStringOrFixedString()) { if (from_type->getCustomSerialization()) From fc857aa2dbf297d2681af16ddfbafb47739db854 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 15 May 2023 03:06:03 +0000 Subject: [PATCH 069/515] tests added --- .../queries/0_stateless/02234_cast_to_ip_address.reference | 4 ++++ tests/queries/0_stateless/02234_cast_to_ip_address.sql | 7 +++++++ 2 files changed, 11 insertions(+) diff --git a/tests/queries/0_stateless/02234_cast_to_ip_address.reference b/tests/queries/0_stateless/02234_cast_to_ip_address.reference index 9023b36a9bf..fa9c6bd0f94 100644 --- a/tests/queries/0_stateless/02234_cast_to_ip_address.reference +++ b/tests/queries/0_stateless/02234_cast_to_ip_address.reference @@ -11,6 +11,10 @@ IPv4 functions 127.0.0.1 127.0.0.1 -- +1.2.3.4 +1.2.3.4 +0.0.0.0 +-- 127.0.0.1 -- 0 diff --git a/tests/queries/0_stateless/02234_cast_to_ip_address.sql b/tests/queries/0_stateless/02234_cast_to_ip_address.sql index 6c65fe86cc9..28f1afff57f 100644 --- a/tests/queries/0_stateless/02234_cast_to_ip_address.sql +++ b/tests/queries/0_stateless/02234_cast_to_ip_address.sql @@ -20,6 +20,13 @@ SELECT toIPv4OrNull('127.0.0.1'); SELECT '--'; +SELECT toIPv4(toIPv6('::ffff:1.2.3.4')); +SELECT toIPv4(toIPv6('::afff:1.2.3.4')); --{serverError CANNOT_CONVERT_TYPE} +SELECT toIPv4OrDefault(toIPv6('::ffff:1.2.3.4')); +SELECT toIPv4OrDefault(toIPv6('::afff:1.2.3.4')); + +SELECT '--'; + SELECT cast('test' , 'IPv4'); --{serverError CANNOT_PARSE_IPV4} SELECT cast('127.0.0.1' , 'IPv4'); From bf6afd27a655bb159583af2020113e101ca82e00 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 17 May 2023 13:32:51 +0000 Subject: [PATCH 070/515] build with disks --- programs/keeper-converter/KeeperConverter.cpp | 4 +- programs/keeper/CMakeLists.txt | 58 +++- programs/keeper/Keeper.cpp | 13 + programs/server/config.d/users.xml | 26 ++ src/Coordination/Context/Context.cpp | 259 ++++++++++++++++++ src/Coordination/Context/Context.h | 111 ++++++++ src/Coordination/Context/Settings.cpp | 24 ++ src/Coordination/Context/ThreadStatusExt.cpp | 14 + src/Coordination/CoordinationSettings.cpp | 60 ---- src/Coordination/CoordinationSettings.h | 9 - src/Coordination/KeeperContext.cpp | 104 +++++++ src/Coordination/KeeperContext.h | 37 ++- src/Coordination/KeeperDispatcher.cpp | 6 +- src/Coordination/KeeperServer.cpp | 33 ++- src/Coordination/KeeperSnapshotManager.cpp | 14 +- src/Coordination/KeeperStateMachine.cpp | 6 +- src/Coordination/KeeperStorage.cpp | 46 ++-- src/Coordination/ZooKeeperDataReader.cpp | 4 +- src/Coordination/pathUtils.cpp | 4 +- src/Coordination/pathUtils.h | 4 +- src/Coordination/tests/gtest_coordination.cpp | 2 +- src/Core/SettingsFields.cpp | 29 ++ src/Core/SettingsFields.h | 16 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 + src/Disks/IO/getThreadPoolReader.cpp | 24 -- .../ObjectStorages/DiskObjectStorage.cpp | 19 -- src/Disks/ObjectStorages/DiskObjectStorage.h | 14 +- .../ObjectStorages/DiskObjectStorageCache.cpp | 28 ++ src/Disks/registerDisks.cpp | 17 ++ src/Interpreters/Context.h | 8 + 30 files changed, 807 insertions(+), 188 deletions(-) create mode 100644 programs/server/config.d/users.xml create mode 100644 src/Coordination/Context/Context.cpp create mode 100644 src/Coordination/Context/Context.h create mode 100644 src/Coordination/Context/Settings.cpp create mode 100644 src/Coordination/Context/ThreadStatusExt.cpp create mode 100644 src/Coordination/KeeperContext.cpp create mode 100644 src/Disks/ObjectStorages/DiskObjectStorageCache.cpp diff --git a/programs/keeper-converter/KeeperConverter.cpp b/programs/keeper-converter/KeeperConverter.cpp index c81e61685fd..58d090ca8b9 100644 --- a/programs/keeper-converter/KeeperConverter.cpp +++ b/programs/keeper-converter/KeeperConverter.cpp @@ -40,8 +40,8 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv) try { - auto keeper_context = std::make_shared(); - keeper_context->digest_enabled = true; + auto keeper_context = std::make_shared(true); + keeper_context->setDigestEnabled(true); DB::KeeperStorage storage(/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false); diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index e5d56023f7b..c0c0a6dd1b0 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -46,6 +46,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManager.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManagerS3.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateMachine.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperContext.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateManager.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStorage.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperAsynchronousMetrics.cpp @@ -58,10 +59,14 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/SettingsFields.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/BaseSettings.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/ServerSettings.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/Field.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/SettingsEnums.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/ServerUUID.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/UUID.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/BackgroundSchedulePool.cpp + + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/IO/ReadBuffer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/KeeperTCPHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/TCPServer.cpp @@ -92,6 +97,10 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/ICompressionCodec.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/LZ4_decompress_faster.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/CurrentThread.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/NamedCollections/NamedCollections.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/NamedCollections/NamedCollectionConfiguration.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/ZooKeeper/IKeeper.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/ZooKeeper/TestKeeper.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -102,11 +111,56 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/ZooKeeper/ZooKeeperLock.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/ZooKeeper/ZooKeeperNodeCache.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/registerDisks.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IDisk.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/DiskFactory.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/DiskSelector.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/DiskLocal.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/DiskLocalCheckThread.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/LocalDirectorySyncGuard.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/TemporaryFileOnDisk.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/loadLocalDiskConfig.cpp + + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/IObjectStorage.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/MetadataFromDiskTransactionState.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorage.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp + + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/registerDiskS3.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/S3Capabilities.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/diskSettings.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/ProxyListConfiguration.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/ProxyResolverConfiguration.cpp + + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/createReadBufferFromFileBase.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ReadBufferFromRemoteFSGather.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/IOUringReader.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteBufferFromTemporaryFile.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/getThreadPoolReader.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolRemoteFSReader.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolReader.cpp + + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Storages/StorageS3Settings.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/BaseDaemon.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/SentryWriter.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/GraphiteWriter.cpp ${CMAKE_CURRENT_BINARY_DIR}/../../src/Daemon/GitHash.generated.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Context/Context.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Context/Settings.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Context/ThreadStatusExt.cpp + Keeper.cpp clickhouse-keeper.cpp ) @@ -117,10 +171,6 @@ if (BUILD_STANDALONE_KEEPER) target_compile_definitions (clickhouse-keeper PRIVATE -DCLICKHOUSE_PROGRAM_STANDALONE_BUILD) target_compile_definitions (clickhouse-keeper PUBLIC -DWITHOUT_TEXT_LOG) - target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_SOURCE_DIR}/../../src") # uses includes from src directory - target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/../../src/Core/include") # uses some includes from core - target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/../../src") # uses some includes from common - target_link_libraries(clickhouse-keeper PRIVATE ch_contrib::abseil_swiss_tables diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 3d1773260f5..996c4678450 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -21,6 +21,8 @@ #include #include +#include + #include #include @@ -40,6 +42,8 @@ #include #include +#include + int mainEntryClickHouseKeeper(int argc, char ** argv) { @@ -408,6 +412,15 @@ try std::mutex servers_lock; auto servers = std::make_shared>(); + auto shared_context = Context::createShared(); + auto global_context = Context::createGlobal(shared_context.get()); + + global_context->makeGlobalContext(); + global_context->setPath(path); + global_context->setRemoteHostFilter(config()); + + registerDisks(/*global_skip_access_check=*/false); + tiny_context = std::make_shared(); /// This object will periodically calculate some metrics. KeeperAsynchronousMetrics async_metrics( diff --git a/programs/server/config.d/users.xml b/programs/server/config.d/users.xml new file mode 100644 index 00000000000..d8a62b45baa --- /dev/null +++ b/programs/server/config.d/users.xml @@ -0,0 +1,26 @@ + + + + 10000000000 + 0 + 2 + + + 5000000000 + 20000000000 + + + + + + + + + + + + + default + + + diff --git a/src/Coordination/Context/Context.cpp b/src/Coordination/Context/Context.cpp new file mode 100644 index 00000000000..aeb4e405938 --- /dev/null +++ b/src/Coordination/Context/Context.cpp @@ -0,0 +1,259 @@ +#include + +#include +#include +#include + +#include + +#include + +#include +#include + +namespace ProfileEvents +{ + extern const Event ContextLock; +} + +namespace CurrentMetrics +{ + extern const Metric ContextLockWait; + extern const Metric BackgroundSchedulePoolTask; + extern const Metric BackgroundSchedulePoolSize; + extern const Metric IOWriterThreads; + extern const Metric IOWriterThreadsActive; +} + +namespace DB +{ + + +struct ContextSharedPart : boost::noncopyable +{ + ContextSharedPart() + : macros(std::make_unique()) + {} + + /// For access of most of shared objects. Recursive mutex. + mutable std::recursive_mutex mutex; + + ServerSettings server_settings; + + String path; /// Path to the data directory, with a slash at the end. + ConfigurationPtr config; /// Global configuration settings. + MultiVersion macros; /// Substitutions extracted from config. + mutable std::unique_ptr schedule_pool; /// A thread pool that can run different jobs in background + RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml + /// + mutable std::unique_ptr asynchronous_remote_fs_reader; + mutable std::unique_ptr asynchronous_local_fs_reader; + mutable std::unique_ptr synchronous_local_fs_reader; + + mutable std::unique_ptr threadpool_writer; + + mutable ThrottlerPtr remote_read_throttler; /// A server-wide throttler for remote IO reads + mutable ThrottlerPtr remote_write_throttler; /// A server-wide throttler for remote IO writes + + mutable ThrottlerPtr local_read_throttler; /// A server-wide throttler for local IO reads + mutable ThrottlerPtr local_write_throttler; /// A server-wide throttler for local IO writes + +}; + +Context::Context() = default; +Context::~Context() = default; +Context::Context(const Context &) = default; +Context & Context::operator=(const Context &) = default; + +SharedContextHolder::SharedContextHolder(SharedContextHolder &&) noexcept = default; +SharedContextHolder & SharedContextHolder::operator=(SharedContextHolder &&) noexcept = default; +SharedContextHolder::SharedContextHolder() = default; +SharedContextHolder::~SharedContextHolder() = default; +SharedContextHolder::SharedContextHolder(std::unique_ptr shared_context) + : shared(std::move(shared_context)) {} + +void SharedContextHolder::reset() { shared.reset(); } + +void Context::makeGlobalContext() +{ + initGlobal(); + global_context = shared_from_this(); +} + +ContextMutablePtr Context::createGlobal(ContextSharedPart * shared) +{ + auto res = std::shared_ptr(new Context); + res->shared = shared; + return res; +} + +void Context::initGlobal() +{ + assert(!global_context_instance); + global_context_instance = shared_from_this(); +} + +SharedContextHolder Context::createShared() +{ + return SharedContextHolder(std::make_unique()); +} + +ContextMutablePtr Context::getGlobalContext() const +{ + auto ptr = global_context.lock(); + if (!ptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no global context or global context has expired"); + return ptr; +} + +std::unique_lock Context::getLock() const +{ + ProfileEvents::increment(ProfileEvents::ContextLock); + CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; + return std::unique_lock(shared->mutex); +} + +String Context::getPath() const +{ + auto lock = getLock(); + return shared->path; +} + +void Context::setPath(const String & path) +{ + auto lock = getLock(); + shared->path = path; +} + +MultiVersion::Version Context::getMacros() const +{ + return shared->macros.get(); +} + +BackgroundSchedulePool & Context::getSchedulePool() const +{ + auto lock = getLock(); + if (!shared->schedule_pool) + { + shared->schedule_pool = std::make_unique( + shared->server_settings.background_schedule_pool_size, + CurrentMetrics::BackgroundSchedulePoolTask, + CurrentMetrics::BackgroundSchedulePoolSize, + "BgSchPool"); + } + + return *shared->schedule_pool; +} + +void Context::setRemoteHostFilter(const Poco::Util::AbstractConfiguration & config) +{ + shared->remote_host_filter.setValuesFromConfig(config); +} + +const RemoteHostFilter & Context::getRemoteHostFilter() const +{ + return shared->remote_host_filter; +} + +IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) const +{ + auto lock = getLock(); + + switch (type) + { + case FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER: + { + if (!shared->asynchronous_remote_fs_reader) + shared->asynchronous_remote_fs_reader = createThreadPoolReader(type, getConfigRef()); + return *shared->asynchronous_remote_fs_reader; + } + case FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER: + { + if (!shared->asynchronous_local_fs_reader) + shared->asynchronous_local_fs_reader = createThreadPoolReader(type, getConfigRef()); + + return *shared->asynchronous_local_fs_reader; + } + case FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER: + { + if (!shared->synchronous_local_fs_reader) + shared->synchronous_local_fs_reader = createThreadPoolReader(type, getConfigRef()); + + return *shared->synchronous_local_fs_reader; + } + } +} + +std::shared_ptr Context::getFilesystemCacheLog() const +{ + return nullptr; +} + +std::shared_ptr Context::getFilesystemReadPrefetchesLog() const +{ + return nullptr; +} + +void Context::setConfig(const ConfigurationPtr & config) +{ + auto lock = getLock(); + shared->config = config; +} + +const Poco::Util::AbstractConfiguration & Context::getConfigRef() const +{ + auto lock = getLock(); + return shared->config ? *shared->config : Poco::Util::Application::instance().config(); +} + +std::shared_ptr Context::getAsyncReadCounters() const +{ + auto lock = getLock(); + if (!async_read_counters) + async_read_counters = std::make_shared(); + return async_read_counters; +} + +ThreadPool & Context::getThreadPoolWriter() const +{ + const auto & config = getConfigRef(); + + auto lock = getLock(); + + if (!shared->threadpool_writer) + { + auto pool_size = config.getUInt(".threadpool_writer_pool_size", 100); + auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000); + + shared->threadpool_writer = std::make_unique( + CurrentMetrics::IOWriterThreads, CurrentMetrics::IOWriterThreadsActive, pool_size, pool_size, queue_size); + } + + return *shared->threadpool_writer; +} + +ThrottlerPtr Context::getRemoteReadThrottler() const +{ + return nullptr; +} + +ThrottlerPtr Context::getRemoteWriteThrottler() const +{ + return nullptr; +} + +ThrottlerPtr Context::getLocalReadThrottler() const +{ + return nullptr; +} + +ThrottlerPtr Context::getLocalWriteThrottler() const +{ + return nullptr; +} + +ReadSettings Context::getReadSettings() const +{ + return ReadSettings{}; +} + +} diff --git a/src/Coordination/Context/Context.h b/src/Coordination/Context/Context.h new file mode 100644 index 00000000000..683209d942e --- /dev/null +++ b/src/Coordination/Context/Context.h @@ -0,0 +1,111 @@ +#pragma once + +#include + +#include +#include + +#include + +#include +#include + +#include + +#include + +#include + +namespace DB +{ + +struct ContextSharedPart; +class Macros; +class FilesystemCacheLog; +class FilesystemReadPrefetchesLog; + +/// A small class which owns ContextShared. +/// We don't use something like unique_ptr directly to allow ContextShared type to be incomplete. +struct SharedContextHolder +{ + ~SharedContextHolder(); + SharedContextHolder(); + explicit SharedContextHolder(std::unique_ptr shared_context); + SharedContextHolder(SharedContextHolder &&) noexcept; + + SharedContextHolder & operator=(SharedContextHolder &&) noexcept; + + ContextSharedPart * get() const { return shared.get(); } + void reset(); +private: + std::unique_ptr shared; +}; + + +class Context : public std::enable_shared_from_this +{ +private: + /// Use copy constructor or createGlobal() instead + Context(); + Context(const Context &); + Context & operator=(const Context &); + + std::unique_lock getLock() const; + + ContextWeakMutablePtr global_context; + inline static ContextPtr global_context_instance; + ContextSharedPart * shared; + + /// Query metrics for reading data asynchronously with IAsynchronousReader. + mutable std::shared_ptr async_read_counters; + + Settings settings; /// Setting for query execution. +public: + /// Create initial Context with ContextShared and etc. + static ContextMutablePtr createGlobal(ContextSharedPart * shared); + static SharedContextHolder createShared(); + + ContextMutablePtr getGlobalContext() const; + static ContextPtr getGlobalContextInstance() { return global_context_instance; } + + void makeGlobalContext(); + void initGlobal(); + + ~Context(); + + using ConfigurationPtr = Poco::AutoPtr; + + /// Global application configuration settings. + void setConfig(const ConfigurationPtr & config); + const Poco::Util::AbstractConfiguration & getConfigRef() const; + + const Settings & getSettingsRef() const { return settings; } + + String getPath() const; + void setPath(const String & path); + + MultiVersion::Version getMacros() const; + + BackgroundSchedulePool & getSchedulePool() const; + + /// Storage of allowed hosts from config.xml + void setRemoteHostFilter(const Poco::Util::AbstractConfiguration & config); + const RemoteHostFilter & getRemoteHostFilter() const; + + std::shared_ptr getFilesystemCacheLog() const; + std::shared_ptr getFilesystemReadPrefetchesLog() const; + + IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type) const; + std::shared_ptr getAsyncReadCounters() const; + ThreadPool & getThreadPoolWriter() const; + + ThrottlerPtr getRemoteReadThrottler() const; + ThrottlerPtr getRemoteWriteThrottler() const; + + ThrottlerPtr getLocalReadThrottler() const; + ThrottlerPtr getLocalWriteThrottler() const; + + ReadSettings getReadSettings() const; +}; + +} diff --git a/src/Coordination/Context/Settings.cpp b/src/Coordination/Context/Settings.cpp new file mode 100644 index 00000000000..12a7a42ffac --- /dev/null +++ b/src/Coordination/Context/Settings.cpp @@ -0,0 +1,24 @@ +#include + +namespace DB +{ + +IMPLEMENT_SETTINGS_TRAITS(SettingsTraits, LIST_OF_SETTINGS) + +std::vector Settings::getAllRegisteredNames() const +{ + std::vector all_settings; + for (const auto & setting_field : all()) + { + all_settings.push_back(setting_field.getName()); + } + return all_settings; +} + +void Settings::set(std::string_view name, const Field & value) +{ + BaseSettings::set(name, value); +} + + +} diff --git a/src/Coordination/Context/ThreadStatusExt.cpp b/src/Coordination/Context/ThreadStatusExt.cpp new file mode 100644 index 00000000000..97f7287be8c --- /dev/null +++ b/src/Coordination/Context/ThreadStatusExt.cpp @@ -0,0 +1,14 @@ +#include + +namespace DB +{ + +void CurrentThread::detachFromGroupIfNotDetached() +{ +} + +void CurrentThread::attachToGroup(const ThreadGroupPtr &) +{ +} + +} diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 7a66134f43f..833ff7be7ae 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -85,14 +85,6 @@ void KeeperConfigurationAndSettings::dump(WriteBufferFromOwnString & buf) const writeText(four_letter_word_allow_list, buf); buf.write('\n'); - writeText("log_storage_path=", buf); - writeText(log_storage_path, buf); - buf.write('\n'); - - writeText("snapshot_storage_path=", buf); - writeText(snapshot_storage_path, buf); - buf.write('\n'); - /// coordination_settings writeText("max_requests_batch_size=", buf); @@ -188,61 +180,9 @@ KeeperConfigurationAndSettings::loadFromConfig(const Poco::Util::AbstractConfigu DEFAULT_FOUR_LETTER_WORD_CMD)); - ret->log_storage_path = getLogsPathFromConfig(config, standalone_keeper_); - ret->snapshot_storage_path = getSnapshotsPathFromConfig(config, standalone_keeper_); - - ret->state_file_path = getStateFilePathFromConfig(config, standalone_keeper_); - ret->coordination_settings->loadFromConfig("keeper_server.coordination_settings", config); return ret; } -String KeeperConfigurationAndSettings::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_) -{ - /// the most specialized path - if (config.has("keeper_server.log_storage_path")) - return config.getString("keeper_server.log_storage_path"); - - if (config.has("keeper_server.storage_path")) - return std::filesystem::path{config.getString("keeper_server.storage_path")} / "logs"; - - if (standalone_keeper_) - return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "logs"; - else - return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/logs"; -} - -String KeeperConfigurationAndSettings::getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_) -{ - /// the most specialized path - if (config.has("keeper_server.snapshot_storage_path")) - return config.getString("keeper_server.snapshot_storage_path"); - - if (config.has("keeper_server.storage_path")) - return std::filesystem::path{config.getString("keeper_server.storage_path")} / "snapshots"; - - if (standalone_keeper_) - return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "snapshots"; - else - return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/snapshots"; -} - -String KeeperConfigurationAndSettings::getStateFilePathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_) -{ - if (config.has("keeper_server.storage_path")) - return std::filesystem::path{config.getString("keeper_server.storage_path")} / "state"; - - if (config.has("keeper_server.snapshot_storage_path")) - return std::filesystem::path(config.getString("keeper_server.snapshot_storage_path")).parent_path() / "state"; - - if (config.has("keeper_server.log_storage_path")) - return std::filesystem::path(config.getString("keeper_server.log_storage_path")).parent_path() / "state"; - - if (standalone_keeper_) - return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "state"; - else - return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/state"; -} - } diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 90f66ccc09b..16da8404b44 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -81,17 +81,8 @@ struct KeeperConfigurationAndSettings bool standalone_keeper; CoordinationSettingsPtr coordination_settings; - String log_storage_path; - String snapshot_storage_path; - String state_file_path; - void dump(WriteBufferFromOwnString & buf) const; static std::shared_ptr loadFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_); - -private: - static String getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_); - static String getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_); - static String getStateFilePathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_); }; using KeeperConfigurationAndSettingsPtr = std::shared_ptr; diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp new file mode 100644 index 00000000000..9e504f5aa07 --- /dev/null +++ b/src/Coordination/KeeperContext.cpp @@ -0,0 +1,104 @@ +#include + +#include +#include +#include + +namespace DB +{ + + +KeeperContext::KeeperContext(bool standalone_keeper_) + : disk_selector(std::make_shared()) + , standalone_keeper(standalone_keeper_) +{} + +void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) +{ + digest_enabled = config.getBool("keeper_server.digest_enabled", false); + ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); + + disk_selector->initialize(config, "storage_configuration.disks", Context::getGlobalContextInstance()); + + log_storage_path = getLogsPathFromConfig(config); + snapshot_storage_path = getSnapshotsPathFromConfig(config); + + state_file_path = getStateFilePathFromConfig(config); +} + +KeeperContext::Phase KeeperContext::getServerState() const +{ + return server_state; +} + +void KeeperContext::setServerState(KeeperContext::Phase server_state_) +{ + server_state = server_state_; +} + +bool KeeperContext::ignoreSystemPathOnStartup() const +{ + return ignore_system_path_on_startup; +} + +bool KeeperContext::digestEnabled() const +{ + return digest_enabled; +} + +void KeeperContext::setDigestEnabled(bool digest_enabled_) +{ + digest_enabled = digest_enabled_; +} + +KeeperContext::Storage KeeperContext::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const +{ + /// the most specialized path + if (config.has("keeper_server.log_storage_path")) + return std::make_shared("LogDisk", config.getString("keeper_server.log_storage_path"), 0); + + if (config.has("keeper_server.log_storage_disk")) + return config.getString("keeper_server.log_storage_disk"); + + if (config.has("keeper_server.storage_path")) + return std::make_shared("LogDisk", std::filesystem::path{config.getString("keeper_server.storage_path")} / "logs", 0); + + if (standalone_keeper) + return std::make_shared("LogDisk", std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "logs", 0); + else + return std::make_shared("LogDisk", std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/logs", 0); +} + +std::string KeeperContext::getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config) +{ + /// the most specialized path + if (config.has("keeper_server.snapshot_storage_path")) + return config.getString("keeper_server.snapshot_storage_path"); + + if (config.has("keeper_server.storage_path")) + return std::filesystem::path{config.getString("keeper_server.storage_path")} / "snapshots"; + + if (standalone_keeper) + return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "snapshots"; + else + return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/snapshots"; +} + +std::string KeeperContext::getStateFilePathFromConfig(const Poco::Util::AbstractConfiguration & config) +{ + if (config.has("keeper_server.storage_path")) + return std::filesystem::path{config.getString("keeper_server.storage_path")} / "state"; + + if (config.has("keeper_server.snapshot_storage_path")) + return std::filesystem::path(config.getString("keeper_server.snapshot_storage_path")).parent_path() / "state"; + + if (config.has("keeper_server.log_storage_path")) + return std::filesystem::path(config.getString("keeper_server.log_storage_path")).parent_path() / "state"; + + if (standalone_keeper) + return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "state"; + else + return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/state"; +} + +} diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 64fa8cea6ec..2a215f9d58f 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -1,10 +1,20 @@ #pragma once +#include + +#include + +#include +#include + namespace DB { -struct KeeperContext +class KeeperContext { +public: + explicit KeeperContext(bool standalone_keeper_); + enum class Phase : uint8_t { INIT, @@ -12,10 +22,35 @@ struct KeeperContext SHUTDOWN }; + void initialize(const Poco::Util::AbstractConfiguration & config); + + Phase getServerState() const; + void setServerState(Phase server_state_); + + bool ignoreSystemPathOnStartup() const; + + bool digestEnabled() const; + void setDigestEnabled(bool digest_enabled_); +private: + /// local disk defined using path or disk name + using Storage = std::variant; + + Storage getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const; + std::string getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config); + std::string getStateFilePathFromConfig(const Poco::Util::AbstractConfiguration & config); + Phase server_state{Phase::INIT}; bool ignore_system_path_on_startup{false}; bool digest_enabled{true}; + + std::shared_ptr disk_selector; + + Storage log_storage_path; + Storage snapshot_storage_path; + Storage state_file_path; + + bool standalone_keeper; }; using KeeperContextPtr = std::shared_ptr; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index d64134f3024..6632e58782f 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -789,12 +789,14 @@ static uint64_t getDirSize(const fs::path & dir) uint64_t KeeperDispatcher::getLogDirSize() const { - return getDirSize(configuration_and_settings->log_storage_path); + //return getDirSize(configuration_and_settings->log_storage_path); + return 0; } uint64_t KeeperDispatcher::getSnapDirSize() const { - return getDirSize(configuration_and_settings->snapshot_storage_path); + //return getDirSize(configuration_and_settings->snapshot_storage_path); + return 0; } Keeper4LWInfo KeeperDispatcher::getKeeper4LWInfo() const diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 9ca792eecdd..d63593436f4 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -114,18 +114,17 @@ KeeperServer::KeeperServer( , coordination_settings(configuration_and_settings_->coordination_settings) , log(&Poco::Logger::get("KeeperServer")) , is_recovering(config.getBool("keeper_server.force_recovery", false)) - , keeper_context{std::make_shared()} + , keeper_context{std::make_shared(true)} , create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true)) { if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); - keeper_context->digest_enabled = config.getBool("keeper_server.digest_enabled", false); - keeper_context->ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); + keeper_context->initialize(config); - if (!fs::exists(configuration_and_settings_->snapshot_storage_path)) - fs::create_directories(configuration_and_settings_->snapshot_storage_path); - auto snapshots_disk = std::make_shared("Keeper-snapshots", configuration_and_settings_->snapshot_storage_path, 0); + //if (!fs::exists(keeper_context->snapshot_storage_path)) + // fs::create_directories(keeper_context->snapshot_storage_path); + auto snapshots_disk = std::make_shared("Keeper-snapshots", "", 0); state_machine = nuraft::cs_new( responses_queue_, @@ -137,23 +136,23 @@ KeeperServer::KeeperServer( commit_callback, checkAndGetSuperdigest(configuration_and_settings_->super_digest)); - auto state_path = fs::path(configuration_and_settings_->state_file_path).parent_path().generic_string(); - auto state_file_name = fs::path(configuration_and_settings_->state_file_path).filename().generic_string(); + //auto state_path = fs::path(keeper_context->state_file_path).parent_path().generic_string(); + //auto state_file_name = fs::path(configuration_and_settings_->state_file_path).filename().generic_string(); - if (!fs::exists(state_path)) - fs::create_directories(state_path); - auto state_disk = std::make_shared("Keeper-state", state_path, 0); + //if (!fs::exists(state_path)) + // fs::create_directories(state_path); + auto state_disk = std::make_shared("Keeper-state", "", 0); - if (!fs::exists(configuration_and_settings_->log_storage_path)) - fs::create_directories(configuration_and_settings_->log_storage_path); - auto logs_disk = std::make_shared("Keeper-logs", configuration_and_settings_->log_storage_path, 0); + //if (!fs::exists(configuration_and_settings_->log_storage_path)) + // fs::create_directories(configuration_and_settings_->log_storage_path); + auto logs_disk = std::make_shared("Keeper-logs", "", 0); state_manager = nuraft::cs_new( server_id, "keeper_server", logs_disk, state_disk, - state_file_name, + "state", config, coordination_settings); } @@ -431,7 +430,7 @@ void KeeperServer::startup(const Poco::Util::AbstractConfiguration & config, boo launchRaftServer(config, enable_ipv6); - keeper_context->server_state = KeeperContext::Phase::RUNNING; + keeper_context->setServerState(KeeperContext::Phase::RUNNING); } void KeeperServer::shutdownRaftServer() @@ -446,7 +445,7 @@ void KeeperServer::shutdownRaftServer() raft_instance->shutdown(); - keeper_context->server_state = KeeperContext::Phase::SHUTDOWN; + keeper_context->setServerState(KeeperContext::Phase::SHUTDOWN); if (create_snapshot_on_exit) raft_instance->create_snapshot(); diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index b7fca5c1eab..e1c0c034cff 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -9,11 +9,11 @@ #include #include #include -#include #include #include #include #include +#include #include #include #include @@ -157,7 +157,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr if (snapshot.version >= SnapshotVersion::V5) { writeBinary(snapshot.zxid, out); - if (keeper_context->digest_enabled) + if (keeper_context->digestEnabled()) { writeBinary(static_cast(KeeperStorage::CURRENT_DIGEST_VERSION), out); writeBinary(snapshot.nodes_digest, out); @@ -268,7 +268,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial deserialization_result.snapshot_meta = deserializeSnapshotMetadata(in); KeeperStorage & storage = *deserialization_result.storage; - bool recalculate_digest = keeper_context->digest_enabled; + bool recalculate_digest = keeper_context->digestEnabled(); if (version >= SnapshotVersion::V5) { readBinary(storage.zxid, in); @@ -350,7 +350,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial const std::string error_msg = fmt::format("Cannot read node on path {} from a snapshot because it is used as a system node", path); if (match_result == IS_CHILD) { - if (keeper_context->ignore_system_path_on_startup || keeper_context->server_state != KeeperContext::Phase::INIT) + if (keeper_context->ignoreSystemPathOnStartup() || keeper_context->getServerState() != KeeperContext::Phase::INIT) { LOG_ERROR(&Poco::Logger::get("KeeperSnapshotManager"), "{}. Ignoring it", error_msg); continue; @@ -366,7 +366,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial { if (!is_node_empty(node)) { - if (keeper_context->ignore_system_path_on_startup || keeper_context->server_state != KeeperContext::Phase::INIT) + if (keeper_context->ignoreSystemPathOnStartup() || keeper_context->getServerState() != KeeperContext::Phase::INIT) { LOG_ERROR(&Poco::Logger::get("KeeperSnapshotManager"), "{}. Ignoring it", error_msg); node = KeeperStorage::Node{}; @@ -395,9 +395,9 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial { if (itr.key != "/") { - auto parent_path = parentPath(itr.key); + auto parent_path = parentNodePath(itr.key); storage.container.updateValue( - parent_path, [version, path = itr.key](KeeperStorage::Node & value) { value.addChild(getBaseName(path), /*update_size*/ version < SnapshotVersion::V4); }); + parent_path, [version, path = itr.key](KeeperStorage::Node & value) { value.addChild(getBaseNodeName(path), /*update_size*/ version < SnapshotVersion::V4); }); } } diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 53657d9b0b9..a7c845e5017 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -223,7 +223,7 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req throw; } - if (keeper_context->digest_enabled && request_for_session.digest) + if (keeper_context->digestEnabled() && request_for_session.digest) assertDigest(*request_for_session.digest, storage->getNodesDigest(false), *request_for_session.request, false); return true; @@ -271,7 +271,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", response_for_session.session_id); } - if (keeper_context->digest_enabled && request_for_session.digest) + if (keeper_context->digestEnabled() && request_for_session.digest) assertDigest(*request_for_session.digest, storage->getNodesDigest(true), *request_for_session.request, true); } @@ -429,7 +429,7 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res }; - if (keeper_context->server_state == KeeperContext::Phase::SHUTDOWN) + if (keeper_context->getServerState() == KeeperContext::Phase::SHUTDOWN) { LOG_INFO(log, "Creating a snapshot during shutdown because 'create_snapshot_on_exit' is enabled."); auto snapshot_path = snapshot_task.create_snapshot(std::move(snapshot_task.snapshot)); diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 7a1a5e42632..64e785693e4 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -128,7 +128,7 @@ KeeperStorage::ResponsesForSessions processWatchesImpl( watches.erase(watch_it); } - auto parent_path = parentPath(path); + auto parent_path = parentNodePath(path); Strings paths_to_check_for_list_watches; if (event_type == Coordination::Event::CREATED) @@ -276,7 +276,7 @@ void KeeperStorage::initializeSystemNodes() [](auto & node) { ++node.stat.numChildren; - node.addChild(getBaseName(keeper_system_path)); + node.addChild(getBaseNodeName(keeper_system_path)); } ); addDigest(updated_root_it->value, "/"); @@ -290,9 +290,9 @@ void KeeperStorage::initializeSystemNodes() child_system_node.setData(data); auto [map_key, _] = container.insert(std::string{path}, child_system_node); /// Take child path from key owned by map. - auto child_path = getBaseName(map_key->getKey()); + auto child_path = getBaseNodeName(map_key->getKey()); container.updateValue( - parentPath(StringRef(path)), + parentNodePath(StringRef(path)), [child_path](auto & parent) { // don't update stats so digest is okay @@ -705,7 +705,7 @@ bool KeeperStorage::createNode( bool is_sequental, Coordination::ACLs node_acls) { - auto parent_path = parentPath(path); + auto parent_path = parentNodePath(path); auto node_it = container.find(parent_path); if (node_it == container.end()) @@ -728,7 +728,7 @@ bool KeeperStorage::createNode( created_node.is_sequental = is_sequental; auto [map_key, _] = container.insert(path, created_node); /// Take child path from key owned by map. - auto child_path = getBaseName(map_key->getKey()); + auto child_path = getBaseNodeName(map_key->getKey()); container.updateValue( parent_path, [child_path](KeeperStorage::Node & parent) @@ -758,8 +758,8 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t version) acl_map.removeUsage(prev_node.acl_id); container.updateValue( - parentPath(path), - [child_basename = getBaseName(node_it->key)](KeeperStorage::Node & parent) + parentNodePath(path), + [child_basename = getBaseNodeName(node_it->key)](KeeperStorage::Node & parent) { parent.removeChild(child_basename); chassert(parent.stat.numChildren == static_cast(parent.getChildren().size())); @@ -843,7 +843,7 @@ Coordination::ACLs getNodeACLs(KeeperStorage & storage, StringRef path, bool is_ void handleSystemNodeModification(const KeeperContext & keeper_context, std::string_view error_msg) { - if (keeper_context.server_state == KeeperContext::Phase::INIT && !keeper_context.ignore_system_path_on_startup) + if (keeper_context.getServerState() == KeeperContext::Phase::INIT && !keeper_context.ignoreSystemPathOnStartup()) throw Exception( ErrorCodes::LOGICAL_ERROR, "{}. Ignoring it can lead to data loss. " @@ -906,7 +906,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override { auto path = zk_request->getPath(); - return storage.checkACL(parentPath(path), Coordination::ACL::Create, session_id, is_local); + return storage.checkACL(parentNodePath(path), Coordination::ACL::Create, session_id, is_local); } std::vector @@ -917,7 +917,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr std::vector new_deltas; - auto parent_path = parentPath(request.path); + auto parent_path = parentNodePath(request.path); auto parent_node = storage.uncommitted_state.getNode(parent_path); if (parent_node == nullptr) return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; @@ -948,7 +948,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr if (storage.uncommitted_state.getNode(path_created)) return {KeeperStorage::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; - if (getBaseName(path_created).size == 0) + if (getBaseNodeName(path_created).size == 0) return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; Coordination::ACLs node_acls; @@ -1098,7 +1098,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr { bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override { - return storage.checkACL(parentPath(zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local); + return storage.checkACL(parentNodePath(zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local); } using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; @@ -1120,7 +1120,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr const auto update_parent_pzxid = [&]() { - auto parent_path = parentPath(request.path); + auto parent_path = parentNodePath(request.path); if (!storage.uncommitted_state.getNode(parent_path)) return; @@ -1155,7 +1155,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr update_parent_pzxid(); new_deltas.emplace_back( - std::string{parentPath(request.path)}, + std::string{parentNodePath(request.path)}, zxid, KeeperStorage::UpdateNodeDelta{[](KeeperStorage::Node & parent) { @@ -1298,7 +1298,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce request.version}); new_deltas.emplace_back( - parentPath(request.path).toString(), + parentNodePath(request.path).toString(), zxid, KeeperStorage::UpdateNodeDelta { @@ -1458,7 +1458,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override { auto path = zk_request->getPath(); - return storage.checkACL(check_not_exists ? parentPath(path) : path, Coordination::ACL::Read, session_id, is_local); + return storage.checkACL(check_not_exists ? parentNodePath(path) : path, Coordination::ACL::Read, session_id, is_local); } std::vector @@ -2001,7 +2001,7 @@ KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFactory() UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vector & new_deltas) const { - if (!keeper_context->digest_enabled) + if (!keeper_context->digestEnabled()) return current_digest; std::unordered_map> updated_nodes; @@ -2099,7 +2099,7 @@ void KeeperStorage::preprocessRequest( TransactionInfo transaction{.zxid = new_last_zxid}; uint64_t new_digest = getNodesDigest(false).value; SCOPE_EXIT({ - if (keeper_context->digest_enabled) + if (keeper_context->digestEnabled()) // if the version of digest we got from the leader is the same as the one this instances has, we can simply copy the value // and just check the digest on the commit // a mistake can happen while applying the changes to the uncommitted_state so for now let's just recalculate the digest here also @@ -2122,7 +2122,7 @@ void KeeperStorage::preprocessRequest( { new_deltas.emplace_back ( - parentPath(ephemeral_path).toString(), + parentNodePath(ephemeral_path).toString(), new_last_zxid, UpdateNodeDelta { @@ -2315,7 +2315,7 @@ void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed) const { - if (!keeper_context->digest_enabled) + if (!keeper_context->digestEnabled()) return {.version = DigestVersion::NO_DIGEST}; if (committed || uncommitted_transactions.empty()) @@ -2326,13 +2326,13 @@ KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed) const void KeeperStorage::removeDigest(const Node & node, const std::string_view path) { - if (keeper_context->digest_enabled) + if (keeper_context->digestEnabled()) nodes_digest -= node.getDigest(path); } void KeeperStorage::addDigest(const Node & node, const std::string_view path) { - if (keeper_context->digest_enabled) + if (keeper_context->digestEnabled()) { node.invalidateDigestCache(); nodes_digest += node.getDigest(path); diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index bdc462f3ea0..94fc07bcc4a 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -139,8 +139,8 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::L { if (itr.key != "/") { - auto parent_path = parentPath(itr.key); - storage.container.updateValue(parent_path, [my_path = itr.key] (KeeperStorage::Node & value) { value.addChild(getBaseName(my_path)); ++value.stat.numChildren; }); + auto parent_path = parentNodePath(itr.key); + storage.container.updateValue(parent_path, [my_path = itr.key] (KeeperStorage::Node & value) { value.addChild(getBaseNodeName(my_path)); ++value.stat.numChildren; }); } } diff --git a/src/Coordination/pathUtils.cpp b/src/Coordination/pathUtils.cpp index 1e1da339d2e..afa42b4a639 100644 --- a/src/Coordination/pathUtils.cpp +++ b/src/Coordination/pathUtils.cpp @@ -21,7 +21,7 @@ static size_t findLastSlash(StringRef path) return std::string::npos; } -StringRef parentPath(StringRef path) +StringRef parentNodePath(StringRef path) { auto rslash_pos = findLastSlash(path); if (rslash_pos > 0) @@ -29,7 +29,7 @@ StringRef parentPath(StringRef path) return "/"; } -StringRef getBaseName(StringRef path) +StringRef getBaseNodeName(StringRef path) { size_t basename_start = findLastSlash(path); return StringRef{path.data + basename_start + 1, path.size - basename_start - 1}; diff --git a/src/Coordination/pathUtils.h b/src/Coordination/pathUtils.h index 69ed2d8b177..b2b79b14110 100644 --- a/src/Coordination/pathUtils.h +++ b/src/Coordination/pathUtils.h @@ -6,8 +6,8 @@ namespace DB { -StringRef parentPath(StringRef path); +StringRef parentNodePath(StringRef path); -StringRef getBaseName(StringRef path); +StringRef getBaseNodeName(StringRef path); } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 62217fb2dd3..de5f2da262b 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -2366,7 +2366,7 @@ TEST_P(CoordinationTest, TestSystemNodeModify) int64_t zxid{0}; // On INIT we abort when a system path is modified - keeper_context->server_state = KeeperContext::Phase::RUNNING; + keeper_context->setServerState(KeeperContext::Phase::RUNNING); KeeperStorage storage{500, "", keeper_context}; const auto assert_create = [&](const std::string_view path, const auto expected_code) { diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 8cd4efb68c6..c1f9fa00f2a 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -379,6 +379,35 @@ void SettingFieldMap::readBinary(ReadBuffer & in) *this = map; } +#else + +SettingFieldMap::SettingFieldMap(const Field &) : value(Map()) {} +String SettingFieldMap::toString() const +{ + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Setting of type Map not supported"); +} + + +SettingFieldMap & SettingFieldMap::operator =(const Field &) +{ + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Setting of type Map not supported"); +} + +void SettingFieldMap::parseFromString(const String &) +{ + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Setting of type Map not supported"); +} + +void SettingFieldMap::writeBinary(WriteBuffer &) const +{ + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Setting of type Map not supported"); +} + +void SettingFieldMap::readBinary(ReadBuffer &) +{ + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Setting of type Map not supported"); +} + #endif namespace diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 2cd55e6b4c5..8cbce71b094 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -245,6 +245,12 @@ struct SettingFieldString void readBinary(ReadBuffer & in); }; +#ifdef CLICKHOUSE_PROGRAM_STANDALONE_BUILD +#define NORETURN [[noreturn]] +#else +#define NORETURN +#endif + struct SettingFieldMap { public: @@ -261,13 +267,15 @@ public: operator const Map &() const { return value; } /// NOLINT explicit operator Field() const { return value; } - String toString() const; - void parseFromString(const String & str); + NORETURN String toString() const; + NORETURN void parseFromString(const String & str); - void writeBinary(WriteBuffer & out) const; - void readBinary(ReadBuffer & in); + NORETURN void writeBinary(WriteBuffer & out) const; + NORETURN void readBinary(ReadBuffer & in); }; +#undef NORETURN + struct SettingFieldChar { public: diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 68b5a9c9d96..6c3556580a2 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -50,6 +50,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c size_t current_read_until_position = read_until_position ? read_until_position : object.bytes_size; auto current_read_buffer_creator = [=, this]() { return read_buffer_creator(object_path, current_read_until_position); }; +#ifndef CLICKHOUSE_PROGRAM_STANDALONE_BUILD if (with_cache) { auto cache_key = settings.remote_fs_cache->createKeyForPath(object_path); @@ -66,6 +67,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c read_until_position ? std::optional(read_until_position) : std::nullopt, cache_log); } +#endif return current_read_buffer_creator(); } diff --git a/src/Disks/IO/getThreadPoolReader.cpp b/src/Disks/IO/getThreadPoolReader.cpp index deb8f66106c..7dbff9ffe76 100644 --- a/src/Disks/IO/getThreadPoolReader.cpp +++ b/src/Disks/IO/getThreadPoolReader.cpp @@ -7,9 +7,7 @@ #include #include -#ifndef CLICKHOUSE_PROGRAM_STANDALONE_BUILD #include -#endif namespace DB { @@ -21,32 +19,10 @@ namespace ErrorCodes IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type) { -#ifdef CLICKHOUSE_PROGRAM_STANDALONE_BUILD - const auto & config = Poco::Util::Application::instance().config(); - switch (type) - { - case FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER: - { - static auto asynchronous_remote_fs_reader = createThreadPoolReader(type, config); - return *asynchronous_remote_fs_reader; - } - case FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER: - { - static auto asynchronous_local_fs_reader = createThreadPoolReader(type, config); - return *asynchronous_local_fs_reader; - } - case FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER: - { - static auto synchronous_local_fs_reader = createThreadPoolReader(type, config); - return *synchronous_local_fs_reader; - } - } -#else auto context = Context::getGlobalContextInstance(); if (!context) throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context not initialized"); return context->getThreadPoolReader(type); -#endif } std::unique_ptr createThreadPoolReader( diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index f832ba5b7b6..2a9cb86f91c 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include @@ -530,24 +529,6 @@ DiskObjectStoragePtr DiskObjectStorage::createDiskObjectStorage() threadpool_size); } -void DiskObjectStorage::wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name) -{ - object_storage = std::make_shared(object_storage, cache, cache_settings, layer_name); -} - -NameSet DiskObjectStorage::getCacheLayersNames() const -{ - NameSet cache_layers; - auto current_object_storage = object_storage; - while (current_object_storage->supportsCache()) - { - auto * cached_object_storage = assert_cast(current_object_storage.get()); - cache_layers.insert(cached_object_storage->getCacheConfigName()); - current_object_storage = cached_object_storage->getWrappedObjectStorage(); - } - return cache_layers; -} - std::unique_ptr DiskObjectStorage::readFile( const String & path, const ReadSettings & settings, diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 4372bc75950..2d942eda1d2 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -184,20 +184,22 @@ public: /// MergeTree table on this disk. bool isWriteOnce() const override; - /// Add a cache layer. - /// Example: DiskObjectStorage(S3ObjectStorage) -> DiskObjectStorage(CachedObjectStorage(S3ObjectStorage)) - /// There can be any number of cache layers: - /// DiskObjectStorage(CachedObjectStorage(...CacheObjectStorage(S3ObjectStorage)...)) - void wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name); - /// Get structure of object storage this disk works with. Examples: /// DiskObjectStorage(S3ObjectStorage) /// DiskObjectStorage(CachedObjectStorage(S3ObjectStorage)) /// DiskObjectStorage(CachedObjectStorage(CachedObjectStorage(S3ObjectStorage))) String getStructure() const { return fmt::format("DiskObjectStorage-{}({})", getName(), object_storage->getName()); } +#ifndef CLICKHOUSE_PROGRAM_STANDALONE_BUILD + /// Add a cache layer. + /// Example: DiskObjectStorage(S3ObjectStorage) -> DiskObjectStorage(CachedObjectStorage(S3ObjectStorage)) + /// There can be any number of cache layers: + /// DiskObjectStorage(CachedObjectStorage(...CacheObjectStorage(S3ObjectStorage)...)) + void wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name); + /// Get names of all cache layers. Name is how cache is defined in configuration file. NameSet getCacheLayersNames() const override; +#endif static std::shared_ptr getAsyncExecutor(const std::string & log_name, size_t size); diff --git a/src/Disks/ObjectStorages/DiskObjectStorageCache.cpp b/src/Disks/ObjectStorages/DiskObjectStorageCache.cpp new file mode 100644 index 00000000000..9e5012dec54 --- /dev/null +++ b/src/Disks/ObjectStorages/DiskObjectStorageCache.cpp @@ -0,0 +1,28 @@ +#include + +#include + +#include + +namespace DB +{ + +void DiskObjectStorage::wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name) +{ + object_storage = std::make_shared(object_storage, cache, cache_settings, layer_name); +} + +NameSet DiskObjectStorage::getCacheLayersNames() const +{ + NameSet cache_layers; + auto current_object_storage = object_storage; + while (current_object_storage->supportsCache()) + { + auto * cached_object_storage = assert_cast(current_object_storage.get()); + cache_layers.insert(cached_object_storage->getCacheConfigName()); + current_object_storage = cached_object_storage->getWrappedObjectStorage(); + } + return cache_layers; +} + +} diff --git a/src/Disks/registerDisks.cpp b/src/Disks/registerDisks.cpp index 48d5a19fb61..676744a8e79 100644 --- a/src/Disks/registerDisks.cpp +++ b/src/Disks/registerDisks.cpp @@ -32,6 +32,8 @@ void registerDiskCache(DiskFactory & factory, bool global_skip_access_check); void registerDiskLocalObjectStorage(DiskFactory & factory, bool global_skip_access_check); +#ifndef CLICKHOUSE_PROGRAM_STANDALONE_BUILD + void registerDisks(bool global_skip_access_check) { auto & factory = DiskFactory::instance(); @@ -61,4 +63,19 @@ void registerDisks(bool global_skip_access_check) registerDiskLocalObjectStorage(factory, global_skip_access_check); } +#else + +void registerDisks(bool global_skip_access_check) +{ + auto & factory = DiskFactory::instance(); + + registerDiskLocal(factory, global_skip_access_check); + +#if USE_AWS_S3 + registerDiskS3(factory, global_skip_access_check); +#endif +} + +#endif + } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 87843a458e8..23f3281330a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1,5 +1,7 @@ #pragma once +#ifndef CLICKHOUSE_PROGRAM_STANDALONE_BUILD + #include #include #include @@ -1226,3 +1228,9 @@ struct HTTPContext : public IHTTPContext }; } + +#else + +#include + +#endif From f459ac5517bcac49c7e6d583fca827504d2b0aa1 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sat, 20 May 2023 01:38:35 +0200 Subject: [PATCH 071/515] resolve ambiguity by adding a section to docs --- docs/en/operations/settings/settings.md | 20 ++++++++++++++++++++ docs/ru/operations/settings/settings.md | 20 ++++++++++++++++++++ 2 files changed, 40 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index cf9209e182f..9cca1ee5ec3 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4077,6 +4077,26 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric 1999-12-13 07:23:23.123 ``` +:::warning +The way this setting affects parsing of Date or DateTime types may seem non-obvious, see example and explanation below: +::: + +```sql +CREATE TABLE test_tz (`d` DateTime('UTC')) ENGINE = Memory AS SELECT toDateTime('2000-01-01 00:00:00', 'UTC'); + +SELECT *, timezone() FROM test_tz WHERE d = toDateTime('2000-01-01 00:00:00') SETTINGS session_timezone = 'Asia/Novosibirsk' +0 rows in set. + +SELECT *, timezone() FROM test_tz WHERE d = '2000-01-01 00:00:00' SETTINGS session_timezone = 'Asia/Novosibirsk' +┌───────────────────d─┬─timezone()───────┐ +│ 2000-01-01 00:00:00 │ Asia/Novosibirsk │ +└─────────────────────┴──────────────────┘ +``` + +This happens due to different parsing pipelines: + - `toDateTime('2000-01-01 00:00:00')` creates a new DateTime in a usual way, and thus `session_timezone` setting from query context is applied. + - `2000-01-01 00:00:00` is parsed to a DateTime inheriting type of `d` column, including DateTime's time zone, and `session_timezone` has no impact on this value. + Possible values: - Any timezone name from `system.time_zones`, e.g. `Europe/Berlin`, `UTC` or `Zulu` diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 98486847fd9..607082054cc 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4105,6 +4105,26 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric Значение по умолчанию: `''`. +:::warning +То, как этот параметр влияет на парсинг значений типа Date или DateTime, может показаться неочевидным. Пример и пояснение см. ниже: +::: + +```sql +CREATE TABLE test_tz (`d` DateTime('UTC')) ENGINE = Memory AS SELECT toDateTime('2000-01-01 00:00:00', 'UTC'); + +SELECT *, timezone() FROM test_tz WHERE d = toDateTime('2000-01-01 00:00:00') SETTINGS session_timezone = 'Asia/Novosibirsk' +0 rows in set. + +SELECT *, timezone() FROM test_tz WHERE d = '2000-01-01 00:00:00' SETTINGS session_timezone = 'Asia/Novosibirsk' +┌───────────────────d─┬─timezone()───────┐ +│ 2000-01-01 00:00:00 │ Asia/Novosibirsk │ +└─────────────────────┴──────────────────┘ +``` + +Это происходит из-за различного происхождения значения, используемого для сравнения: +- `toDateTime('2000-01-01 00:00:00')` создаёт значение типа `DateTime` как и в любом другом случае, в том числе применяет параметр `session_timezone` из контекста запроса, +- `2000-01-01 00:00:00` парсится в `DateTime` того же типа, что и колонка `d` (в том числе с той же `timezone`), и параметр `session_timezone` в данном случае не учитывается. + **Смотрите также** - [timezone](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone) \ No newline at end of file From af5793b26e0ee8fc02201f8d0439b5a15a019e7f Mon Sep 17 00:00:00 2001 From: zvonand Date: Sat, 20 May 2023 16:38:45 +0200 Subject: [PATCH 072/515] fix incode docs --- src/Functions/serverConstants.cpp | 28 +++++++++++++++------------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index 1460fc16265..9cd43be50c4 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -160,31 +160,33 @@ REGISTER_FUNCTION(TcpPort) REGISTER_FUNCTION(Timezone) { - factory.registerFunction({ - R"( + factory.registerFunction( + FunctionDocumentation{ + .description=R"( Returns the default timezone for current session. Used as default timezone for parsing DateTime|DateTime64 without explicitly specified timezone. Can be changed with SET timezone = 'New/Tz' [example:timezone] -)", - Documentation::Examples{{"timezone", "SELECT timezone();"}}, - Documentation::Categories{"Constant", "Miscellaneous"} - }); - factory.registerAlias("timeZone", "timezone"); + )", + .examples{{"timezone", "SELECT timezone();", ""}}, + .categories{"Constant", "Miscellaneous"} +}); +factory.registerAlias("timeZone", "timezone"); } REGISTER_FUNCTION(ServerTimezone) { - factory.registerFunction({ - R"( + factory.registerFunction( + FunctionDocumentation{ + .description=R"( Returns the timezone name in which server operates. [example:serverTimezone] -)", - Documentation::Examples{{"serverTimezone", "SELECT serverTimezone();"}}, - Documentation::Categories{"Constant", "Miscellaneous"} - }); + )", + .examples{{"serverTimezone", "SELECT serverTimezone();", ""}}, + .categories{"Constant", "Miscellaneous"} +}); factory.registerAlias("serverTimeZone", "serverTimezone"); factory.registerAlias("servertimezone", "serverTimezone"); } From 8c816a5c4a97044b1d3d902145ef4eefecd7beb8 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 22 May 2023 01:00:40 +0200 Subject: [PATCH 073/515] update --- src/Common/DateLUT.h | 37 ++++++++++++++++--------------- src/Core/Settings.h | 2 +- src/Functions/serverConstants.cpp | 4 +--- src/IO/ReadHelpers.h | 4 ++-- 4 files changed, 23 insertions(+), 24 deletions(-) diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index 59b280240ea..23698331afe 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -17,30 +17,24 @@ class DateLUT : private boost::noncopyable { public: - /// Return singleton DateLUTImpl instance for server's (native) time zone. - static ALWAYS_INLINE const DateLUTImpl & serverTimezoneInstance() - { - const auto & date_lut = getInstance(); - return *date_lut.default_impl.load(std::memory_order_acquire); - } - - /// Return singleton DateLUTImpl instance for timezone set by `timezone` setting for current session is used. - /// If it is not set, server's timezone (the one which server has) is being used. + /// Return singleton DateLUTImpl instance for session timezone. + /// The session timezone is configured by a session setting. + /// If not set (empty string), it is the server timezone. static ALWAYS_INLINE const DateLUTImpl & instance() { const auto & date_lut = getInstance(); if (DB::CurrentThread::isInitialized()) { - std::string effective_time_zone; - const auto query_context = DB::CurrentThread::get().getQueryContext(); + std::string context_timezone; + const DB::ContextPtr query_context = DB::CurrentThread::get().getQueryContext(); if (query_context) { - effective_time_zone = extractTimezoneFromContext(query_context); + context_timezone = extractTimezoneFromContext(query_context); - if (!effective_time_zone.empty()) - return date_lut.getImplementation(effective_time_zone); + if (!context_timezone.empty()) + return date_lut.getImplementation(context_timezone); } /// Timezone is passed in query_context, but on CH-Client we have no query context, @@ -48,10 +42,10 @@ public: const auto global_context = DB::CurrentThread::get().getGlobalContext(); if (global_context) { - effective_time_zone = extractTimezoneFromContext(global_context); + context_timezone = extractTimezoneFromContext(global_context); - if (!effective_time_zone.empty()) - return date_lut.getImplementation(effective_time_zone); + if (!context_timezone.empty()) + return date_lut.getImplementation(context_timezone); } } @@ -67,6 +61,13 @@ public: return date_lut.getImplementation(time_zone); } + // Return singleton DateLUTImpl for the server time zone. + static ALWAYS_INLINE const DateLUTImpl & serverTimezoneInstance() + { + const auto & date_lut = getInstance(); + return *date_lut.default_impl.load(std::memory_order_acquire); + } + static void setDefaultTimezone(const std::string & time_zone) { auto & date_lut = getInstance(); @@ -80,7 +81,7 @@ protected: private: static DateLUT & getInstance(); - static std::string extractTimezoneFromContext(const DB::ContextPtr query_context); + static std::string extractTimezoneFromContext(DB::ContextPtr query_context); const DateLUTImpl & getImplementation(const std::string & time_zone) const; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 90063f8efd4..2d766e8e18f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -747,7 +747,7 @@ class IColumn; M(Bool, allow_experimental_undrop_table_query, false, "Allow to use undrop query to restore dropped table in a limited time", 0) \ M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ - M(Timezone, session_timezone, "", "Use specified timezone for interpreting Date and DateTime instead of server's timezone in current session.", 0) \ + M(Timezone, session_timezone, "", "The default timezone for the current session. The server default timezone, if empty.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index 9cd43be50c4..d3e1e6e10fe 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -60,7 +60,7 @@ namespace }; - /// Returns default timezone for current session. + /// Returns timezone for current session. class FunctionTimezone : public FunctionConstantBase { public: @@ -187,8 +187,6 @@ Returns the timezone name in which server operates. .examples{{"serverTimezone", "SELECT serverTimezone();", ""}}, .categories{"Constant", "Miscellaneous"} }); - factory.registerAlias("serverTimeZone", "serverTimezone"); - factory.registerAlias("servertimezone", "serverTimezone"); } REGISTER_FUNCTION(Uptime) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 3bd9275322e..cbe18e11c9a 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -706,7 +706,7 @@ inline void convertToDayNum(DayNum & date, ExtendedDayNum & from) } template -inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) +inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut) { static constexpr bool throw_exception = std::is_same_v; @@ -723,7 +723,7 @@ inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf, const DateLU } template -inline ReturnType readDateTextImpl(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) +inline ReturnType readDateTextImpl(ExtendedDayNum & date, ReadBuffer & buf, const DateLUTImpl & date_lut) { static constexpr bool throw_exception = std::is_same_v; From bde2cf96b135235908b71cc0bf071a175321dc4d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 22 May 2023 12:24:16 +0000 Subject: [PATCH 074/515] Better --- programs/keeper/CMakeLists.txt | 7 +- programs/keeper/Keeper.cpp | 39 +++++--- programs/keeper/Keeper.h | 7 -- src/Coordination/Changelog.cpp | 55 +++++++---- src/Coordination/Changelog.h | 10 +- .../KeeperAsynchronousMetrics.cpp | 6 +- src/Coordination/KeeperAsynchronousMetrics.h | 6 +- src/Coordination/KeeperContext.cpp | 91 +++++++++++++++---- src/Coordination/KeeperContext.h | 16 +++- src/Coordination/KeeperLogStore.cpp | 9 +- src/Coordination/KeeperLogStore.h | 6 +- src/Coordination/KeeperServer.cpp | 21 +---- src/Coordination/KeeperSnapshotManager.cpp | 38 +++----- src/Coordination/KeeperSnapshotManager.h | 15 +-- src/Coordination/KeeperStateMachine.cpp | 2 - src/Coordination/KeeperStateMachine.h | 1 - src/Coordination/KeeperStateManager.cpp | 32 ++++--- src/Coordination/KeeperStateManager.h | 15 +-- .../{Context => Standalone}/Context.cpp | 58 ++++++++++++ .../{Context => Standalone}/Context.h | 9 ++ .../{Context => Standalone}/Settings.cpp | 0 .../ThreadStatusExt.cpp | 0 src/Coordination/TinyContext.cpp | 87 ------------------ src/Coordination/TinyContext.h | 36 -------- src/Interpreters/Context.h | 2 +- 25 files changed, 282 insertions(+), 286 deletions(-) rename src/Coordination/{Context => Standalone}/Context.cpp (80%) rename src/Coordination/{Context => Standalone}/Context.h (88%) rename src/Coordination/{Context => Standalone}/Settings.cpp (100%) rename src/Coordination/{Context => Standalone}/ThreadStatusExt.cpp (100%) delete mode 100644 src/Coordination/TinyContext.cpp delete mode 100644 src/Coordination/TinyContext.h diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index c0c0a6dd1b0..1b5b9e6a7b2 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -50,7 +50,6 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateManager.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStorage.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperAsynchronousMetrics.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/TinyContext.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/pathUtils.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SessionExpiryQueue.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SummingStateMachine.cpp @@ -157,9 +156,9 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/GraphiteWriter.cpp ${CMAKE_CURRENT_BINARY_DIR}/../../src/Daemon/GitHash.generated.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Context/Context.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Context/Settings.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Context/ThreadStatusExt.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Standalone/Context.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Standalone/Settings.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Standalone/ThreadStatusExt.cpp Keeper.cpp clickhouse-keeper.cpp diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 996c4678450..d5cf61daa6e 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -44,6 +44,8 @@ #include +#include + int mainEntryClickHouseKeeper(int argc, char ** argv) { @@ -280,9 +282,12 @@ void Keeper::defineOptions(Poco::Util::OptionSet & options) BaseDaemon::defineOptions(options); } -struct Keeper::KeeperHTTPContext : public IHTTPContext +namespace { - explicit KeeperHTTPContext(TinyContextPtr context_) + +struct KeeperHTTPContext : public IHTTPContext +{ + explicit KeeperHTTPContext(ContextPtr context_) : context(std::move(context_)) {} @@ -326,12 +331,14 @@ struct Keeper::KeeperHTTPContext : public IHTTPContext return {context->getConfigRef().getInt64("keeper_server.http_send_timeout", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0}; } - TinyContextPtr context; + ContextPtr context; }; -HTTPContextPtr Keeper::httpContext() +HTTPContextPtr httpContext() { - return std::make_shared(tiny_context); + return std::make_shared(Context::getGlobalContextInstance()); +} + } int Keeper::main(const std::vector & /*args*/) @@ -419,12 +426,14 @@ try global_context->setPath(path); global_context->setRemoteHostFilter(config()); + if (config().has("macros")) + global_context->setMacros(std::make_unique(config(), "macros", log)); + registerDisks(/*global_skip_access_check=*/false); - tiny_context = std::make_shared(); /// This object will periodically calculate some metrics. KeeperAsynchronousMetrics async_metrics( - tiny_context, + global_context, config().getUInt("asynchronous_metrics_update_period_s", 1), [&]() -> std::vector { @@ -449,12 +458,12 @@ try } /// Initialize keeper RAFT. Do nothing if no keeper_server in config. - tiny_context->initializeKeeperDispatcher(/* start_async = */ true); - FourLetterCommandFactory::registerCommands(*tiny_context->getKeeperDispatcher()); + global_context->initializeKeeperDispatcher(/* start_async = */ true); + FourLetterCommandFactory::registerCommands(*global_context->getKeeperDispatcher()); - auto config_getter = [this] () -> const Poco::Util::AbstractConfiguration & + auto config_getter = [&] () -> const Poco::Util::AbstractConfiguration & { - return tiny_context->getConfigRef(); + return global_context->getConfigRef(); }; auto tcp_receive_timeout = config().getInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC); @@ -476,7 +485,7 @@ try "Keeper (tcp): " + address.toString(), std::make_unique( new KeeperTCPHandlerFactory( - config_getter, tiny_context->getKeeperDispatcher(), + config_getter, global_context->getKeeperDispatcher(), tcp_receive_timeout, tcp_send_timeout, false), server_pool, socket)); }); @@ -494,7 +503,7 @@ try "Keeper with secure protocol (tcp_secure): " + address.toString(), std::make_unique( new KeeperTCPHandlerFactory( - config_getter, tiny_context->getKeeperDispatcher(), + config_getter, global_context->getKeeperDispatcher(), tcp_receive_timeout, tcp_send_timeout, true), server_pool, socket)); #else UNUSED(port); @@ -546,7 +555,7 @@ try [&](ConfigurationPtr config, bool /* initial_loading */) { if (config->has("keeper_server")) - tiny_context->updateKeeperConfiguration(*config); + global_context->updateKeeperConfiguration(*config); }, /* already_loaded = */ false); /// Reload it right now (initial loading) @@ -577,7 +586,7 @@ try else LOG_INFO(log, "Closed connections to Keeper."); - tiny_context->shutdownKeeperDispatcher(); + global_context->shutdownKeeperDispatcher(); /// Wait server pool to avoid use-after-free of destroyed context in the handlers server_pool.joinAll(); diff --git a/programs/keeper/Keeper.h b/programs/keeper/Keeper.h index 8a7724acb85..f889ffa595b 100644 --- a/programs/keeper/Keeper.h +++ b/programs/keeper/Keeper.h @@ -1,9 +1,7 @@ #pragma once #include -#include #include -#include namespace Poco { @@ -68,11 +66,6 @@ protected: std::string getDefaultConfigFileName() const override; private: - TinyContextPtr tiny_context; - - struct KeeperHTTPContext; - HTTPContextPtr httpContext(); - Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const; using CreateServerFunc = std::function; diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 1fda760cab0..852a21c1c45 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -85,17 +86,19 @@ class ChangelogWriter public: ChangelogWriter( std::map & existing_changelogs_, - DiskPtr disk_, + KeeperContextPtr keeper_context_, LogFileSettings log_file_settings_) : existing_changelogs(existing_changelogs_) , log_file_settings(log_file_settings_) - , disk(disk_) + , keeper_context(std::move(keeper_context_)) , log(&Poco::Logger::get("Changelog")) { } void setFile(ChangelogFileDescriptionPtr file_description, WriteMode mode) { + auto disk = getDisk(); + try { if (mode == WriteMode::Append && file_description->expectedEntriesCountInLog() != log_file_settings.rotate_interval) @@ -146,7 +149,7 @@ public: /// There is bug when compressed_buffer has value, file_buf's ownership transfer to compressed_buffer bool isFileSet() const { - return compressed_buffer.get() != nullptr || file_buf.get() != nullptr; + return compressed_buffer != nullptr || file_buf != nullptr; } bool appendRecord(ChangelogRecord && record) @@ -276,7 +279,7 @@ private: const auto * file_buffer = tryGetFileBuffer(); - if (log_file_settings.max_size != 0) + if (log_file_settings.max_size != 0 && isLocalDisk()) { int res = -1; do @@ -354,11 +357,12 @@ private: { initial_file_size = 0; prealloc_done = true; - LOG_WARNING(log, "Could not preallocate space on disk {} using fallocate", disk->getName()); + LOG_WARNING(log, "Could not preallocate space on disk {} using fallocate", getDisk()->getName()); return; } #ifdef OS_LINUX + if (isLocalDisk()) { int res = -1; do @@ -383,6 +387,16 @@ private: prealloc_done = true; } + DiskPtr getDisk() const + { + return keeper_context->getLogDisk(); + } + + bool isLocalDisk() const + { + return dynamic_cast(getDisk().get()) != nullptr; + } + std::map & existing_changelogs; ChangelogFileDescriptionPtr current_file_description{nullptr}; @@ -396,7 +410,7 @@ private: LogFileSettings log_file_settings; - DiskPtr disk; + KeeperContextPtr keeper_context; Poco::Logger * const log; }; @@ -533,18 +547,20 @@ private: }; Changelog::Changelog( - DiskPtr disk_, Poco::Logger * log_, - LogFileSettings log_file_settings) - : disk(disk_) - , changelogs_detached_dir("detached") + LogFileSettings log_file_settings, + KeeperContextPtr keeper_context_) + : changelogs_detached_dir("detached") , rotate_interval(log_file_settings.rotate_interval) , log(log_) , write_operations(std::numeric_limits::max()) , append_completion_queue(std::numeric_limits::max()) + , keeper_context(std::move(keeper_context_)) { /// Load all files in changelog directory + auto disk = getDisk(); + for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) { if (it->name() == changelogs_detached_dir) @@ -564,7 +580,7 @@ Changelog::Changelog( append_completion_thread = ThreadFromGlobalPool([this] { appendCompletionThread(); }); current_writer = std::make_unique( - existing_changelogs, disk, log_file_settings); + existing_changelogs, keeper_context, log_file_settings); } void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uint64_t logs_to_keep) @@ -636,7 +652,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin break; } - ChangelogReader reader(disk, changelog_description.path); + ChangelogReader reader(getDisk(), changelog_description.path); last_log_read_result = reader.readChangelog(logs, start_to_read_from, log); last_log_read_result->log_start_index = changelog_description.from_log_index; @@ -703,7 +719,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin if (last_log_read_result->last_read_index == 0 || last_log_read_result->error) /// If it's broken log then remove it { LOG_INFO(log, "Removing chagelog {} because it's empty or read finished with error", description->path); - disk->removeFile(description->path); + getDisk()->removeFile(description->path); existing_changelogs.erase(last_log_read_result->log_start_index); std::erase_if(logs, [last_log_read_result](const auto & item) { return item.first >= last_log_read_result->log_start_index; }); } @@ -748,8 +764,15 @@ std::string getCurrentTimestampFolder() } +DiskPtr Changelog::getDisk() const +{ + return keeper_context->getLogDisk(); +} + void Changelog::removeExistingLogs(ChangelogIter begin, ChangelogIter end) { + auto disk = getDisk(); + const auto timestamp_folder = (fs::path(changelogs_detached_dir) / getCurrentTimestampFolder()).generic_string(); for (auto itr = begin; itr != end;) @@ -920,7 +943,7 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry) auto to_remove_itr = existing_changelogs.upper_bound(index); for (auto itr = to_remove_itr; itr != existing_changelogs.end();) { - disk->removeFile(itr->second->path); + getDisk()->removeFile(itr->second->path); itr = existing_changelogs.erase(itr); } } @@ -974,7 +997,7 @@ void Changelog::compact(uint64_t up_to_log_index) { try { - disk->removeFile(itr->second->path); + getDisk()->removeFile(itr->second->path); LOG_INFO(log, "Removed changelog {} because of compaction.", itr->second->path); } catch (Exception & e) @@ -1179,7 +1202,7 @@ void Changelog::cleanLogThread() { try { - disk->removeFile(path); + getDisk()->removeFile(path); LOG_INFO(log, "Removed changelog {} because of compaction.", path); } catch (Exception & e) diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 7204c80a356..6f0c4e45605 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB { @@ -87,9 +88,9 @@ class Changelog { public: Changelog( - DiskPtr disk_, Poco::Logger * log_, - LogFileSettings log_file_settings); + LogFileSettings log_file_settings, + KeeperContextPtr keeper_context_); Changelog(Changelog &&) = delete; @@ -152,6 +153,8 @@ private: /// Pack log_entry into changelog record static ChangelogRecord buildRecord(uint64_t index, const LogEntryPtr & log_entry); + DiskPtr getDisk() const; + /// Currently existing changelogs std::map existing_changelogs; @@ -169,7 +172,6 @@ private: /// Clean useless log files in a background thread void cleanLogThread(); - DiskPtr disk; const String changelogs_detached_dir; const uint64_t rotate_interval; Poco::Logger * log; @@ -223,6 +225,8 @@ private: nuraft::wptr raft_server; + KeeperContextPtr keeper_context; + bool initialized = false; }; diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index 2d523a26dcc..1427130b184 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -108,8 +108,8 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM } KeeperAsynchronousMetrics::KeeperAsynchronousMetrics( - TinyContextPtr tiny_context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) - : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), tiny_context(std::move(tiny_context_)) + ContextPtr context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) + : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), context(std::move(context_)) { } @@ -117,7 +117,7 @@ void KeeperAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values { #if USE_NURAFT { - auto keeper_dispatcher = tiny_context->tryGetKeeperDispatcher(); + auto keeper_dispatcher = context->tryGetKeeperDispatcher(); if (keeper_dispatcher) updateKeeperInformation(*keeper_dispatcher, new_values); } diff --git a/src/Coordination/KeeperAsynchronousMetrics.h b/src/Coordination/KeeperAsynchronousMetrics.h index 8fa27336bc5..14092c11c15 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.h +++ b/src/Coordination/KeeperAsynchronousMetrics.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB @@ -13,10 +13,10 @@ class KeeperAsynchronousMetrics : public AsynchronousMetrics { public: KeeperAsynchronousMetrics( - TinyContextPtr tiny_context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_); + ContextPtr context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_); private: - TinyContextPtr tiny_context; + ContextPtr context; void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) override; }; diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 9e504f5aa07..ff3f96f199b 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -20,10 +20,10 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) disk_selector->initialize(config, "storage_configuration.disks", Context::getGlobalContextInstance()); - log_storage_path = getLogsPathFromConfig(config); - snapshot_storage_path = getSnapshotsPathFromConfig(config); + log_storage = getLogsPathFromConfig(config); + snapshot_storage = getSnapshotsPathFromConfig(config); - state_file_path = getStateFilePathFromConfig(config); + state_file_storage = getStatePathFromConfig(config); } KeeperContext::Phase KeeperContext::getServerState() const @@ -51,54 +51,109 @@ void KeeperContext::setDigestEnabled(bool digest_enabled_) digest_enabled = digest_enabled_; } +DiskPtr KeeperContext::getDisk(const Storage & storage) const +{ + if (const auto * storage_disk = std::get_if(&storage)) + return *storage_disk; + + const auto & disk_name = std::get(storage); + + return disk_selector->get(disk_name); +} + +DiskPtr KeeperContext::getLogDisk() const +{ + return getDisk(log_storage); +} + +DiskPtr KeeperContext::getSnapshotsDisk() const +{ + return getDisk(snapshot_storage); +} + +DiskPtr KeeperContext::getStateFileDisk() const +{ + return getDisk(state_file_storage); +} + KeeperContext::Storage KeeperContext::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const { + const auto create_local_disk = [](const auto & path) + { + if (!fs::exists(path)) + fs::create_directories(path); + + return std::make_shared("LogDisk", path, 0); + }; + /// the most specialized path if (config.has("keeper_server.log_storage_path")) - return std::make_shared("LogDisk", config.getString("keeper_server.log_storage_path"), 0); + return create_local_disk(config.getString("keeper_server.log_storage_path")); if (config.has("keeper_server.log_storage_disk")) return config.getString("keeper_server.log_storage_disk"); if (config.has("keeper_server.storage_path")) - return std::make_shared("LogDisk", std::filesystem::path{config.getString("keeper_server.storage_path")} / "logs", 0); + return create_local_disk(std::filesystem::path{config.getString("keeper_server.storage_path")} / "logs"); if (standalone_keeper) - return std::make_shared("LogDisk", std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "logs", 0); + return create_local_disk(std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "logs"); else - return std::make_shared("LogDisk", std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/logs", 0); + return create_local_disk(std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/logs"); } -std::string KeeperContext::getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config) +KeeperContext::Storage KeeperContext::getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const { + const auto create_local_disk = [](const auto & path) + { + if (!fs::exists(path)) + fs::create_directories(path); + + return std::make_shared("SnapshotDisk", path, 0); + }; + /// the most specialized path if (config.has("keeper_server.snapshot_storage_path")) - return config.getString("keeper_server.snapshot_storage_path"); + return create_local_disk(config.getString("keeper_server.snapshot_storage_path")); + + if (config.has("keeper_server.snapshot_storage_disk")) + return config.getString("keeper_server.snapshot_storage_disk"); if (config.has("keeper_server.storage_path")) - return std::filesystem::path{config.getString("keeper_server.storage_path")} / "snapshots"; + return create_local_disk(std::filesystem::path{config.getString("keeper_server.storage_path")} / "snapshots"); if (standalone_keeper) - return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "snapshots"; + return create_local_disk(std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "snapshots"); else - return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/snapshots"; + return create_local_disk(std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/snapshots"); } -std::string KeeperContext::getStateFilePathFromConfig(const Poco::Util::AbstractConfiguration & config) +KeeperContext::Storage KeeperContext::getStatePathFromConfig(const Poco::Util::AbstractConfiguration & config) const { + const auto create_local_disk = [](const auto & path) + { + if (!fs::exists(path)) + fs::create_directories(path); + + return std::make_shared("SnapshotDisk", path, 0); + }; + + if (config.has("keeper_server.state_storage_disk")) + return config.getString("keeper_server.state_storage_disk"); + if (config.has("keeper_server.storage_path")) - return std::filesystem::path{config.getString("keeper_server.storage_path")} / "state"; + return create_local_disk(std::filesystem::path{config.getString("keeper_server.storage_path")}); if (config.has("keeper_server.snapshot_storage_path")) - return std::filesystem::path(config.getString("keeper_server.snapshot_storage_path")).parent_path() / "state"; + return create_local_disk(std::filesystem::path(config.getString("keeper_server.snapshot_storage_path")).parent_path()); if (config.has("keeper_server.log_storage_path")) - return std::filesystem::path(config.getString("keeper_server.log_storage_path")).parent_path() / "state"; + return create_local_disk(std::filesystem::path(config.getString("keeper_server.log_storage_path")).parent_path()); if (standalone_keeper) - return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "state"; + return create_local_disk(std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)}); else - return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/state"; + return create_local_disk(std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination"); } } diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 2a215f9d58f..1fc01f12bba 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -31,13 +31,19 @@ public: bool digestEnabled() const; void setDigestEnabled(bool digest_enabled_); + + DiskPtr getLogDisk() const; + DiskPtr getSnapshotsDisk() const; + DiskPtr getStateFileDisk() const; private: /// local disk defined using path or disk name using Storage = std::variant; Storage getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const; - std::string getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config); - std::string getStateFilePathFromConfig(const Poco::Util::AbstractConfiguration & config); + Storage getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const; + Storage getStatePathFromConfig(const Poco::Util::AbstractConfiguration & config) const; + + DiskPtr getDisk(const Storage & storage) const; Phase server_state{Phase::INIT}; @@ -46,9 +52,9 @@ private: std::shared_ptr disk_selector; - Storage log_storage_path; - Storage snapshot_storage_path; - Storage state_file_path; + Storage log_storage; + Storage snapshot_storage; + Storage state_file_storage; bool standalone_keeper; }; diff --git a/src/Coordination/KeeperLogStore.cpp b/src/Coordination/KeeperLogStore.cpp index 7c4f76e0180..a9153475dbc 100644 --- a/src/Coordination/KeeperLogStore.cpp +++ b/src/Coordination/KeeperLogStore.cpp @@ -6,9 +6,9 @@ namespace DB { -KeeperLogStore::KeeperLogStore(DiskPtr disk_, LogFileSettings log_file_settings) +KeeperLogStore::KeeperLogStore(LogFileSettings log_file_settings, KeeperContextPtr keeper_context) : log(&Poco::Logger::get("KeeperLogStore")) - , changelog(disk_, log, log_file_settings) + , changelog(log, log_file_settings, keeper_context) { if (log_file_settings.force_sync) LOG_INFO(log, "force_sync enabled"); @@ -16,11 +16,6 @@ KeeperLogStore::KeeperLogStore(DiskPtr disk_, LogFileSettings log_file_settings) LOG_INFO(log, "force_sync disabled"); } -KeeperLogStore::KeeperLogStore(const std::string & changelogs_path, LogFileSettings log_file_settings) - : KeeperLogStore(std::make_shared("Keeper-logs", changelogs_path, 0), log_file_settings) -{ -} - uint64_t KeeperLogStore::start_index() const { std::lock_guard lock(changelog_lock); diff --git a/src/Coordination/KeeperLogStore.h b/src/Coordination/KeeperLogStore.h index 2902de129e7..6e71d8c55cf 100644 --- a/src/Coordination/KeeperLogStore.h +++ b/src/Coordination/KeeperLogStore.h @@ -4,6 +4,7 @@ #include #include #include +#include #include namespace DB @@ -13,10 +14,7 @@ namespace DB class KeeperLogStore : public nuraft::log_store { public: - KeeperLogStore(DiskPtr disk_, LogFileSettings log_file_settings); - - /// For gtest - KeeperLogStore(const std::string & changelogs_path, LogFileSettings log_file_settings); + KeeperLogStore(LogFileSettings log_file_settings, KeeperContextPtr keeper_context); /// Read log storage from filesystem starting from last_commited_log_index void init(uint64_t last_commited_log_index, uint64_t logs_to_keep); diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index d63593436f4..4cacf566df6 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -122,39 +122,22 @@ KeeperServer::KeeperServer( keeper_context->initialize(config); - //if (!fs::exists(keeper_context->snapshot_storage_path)) - // fs::create_directories(keeper_context->snapshot_storage_path); - auto snapshots_disk = std::make_shared("Keeper-snapshots", "", 0); - state_machine = nuraft::cs_new( responses_queue_, snapshots_queue_, - snapshots_disk, coordination_settings, keeper_context, config.getBool("keeper_server.upload_snapshot_on_exit", true) ? &snapshot_manager_s3 : nullptr, commit_callback, checkAndGetSuperdigest(configuration_and_settings_->super_digest)); - //auto state_path = fs::path(keeper_context->state_file_path).parent_path().generic_string(); - //auto state_file_name = fs::path(configuration_and_settings_->state_file_path).filename().generic_string(); - - //if (!fs::exists(state_path)) - // fs::create_directories(state_path); - auto state_disk = std::make_shared("Keeper-state", "", 0); - - //if (!fs::exists(configuration_and_settings_->log_storage_path)) - // fs::create_directories(configuration_and_settings_->log_storage_path); - auto logs_disk = std::make_shared("Keeper-logs", "", 0); - state_manager = nuraft::cs_new( server_id, "keeper_server", - logs_disk, - state_disk, "state", config, - coordination_settings); + coordination_settings, + keeper_context); } /** diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index e1c0c034cff..d47ea475c42 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -508,36 +508,18 @@ KeeperStorageSnapshot::~KeeperStorageSnapshot() } KeeperSnapshotManager::KeeperSnapshotManager( - const std::string & snapshots_path_, size_t snapshots_to_keep_, const KeeperContextPtr & keeper_context_, bool compress_snapshots_zstd_, const std::string & superdigest_, size_t storage_tick_time_) - : KeeperSnapshotManager( - std::make_shared("Keeper-snapshots", snapshots_path_, 0), - snapshots_to_keep_, - keeper_context_, - compress_snapshots_zstd_, - superdigest_, - storage_tick_time_) -{ -} - -KeeperSnapshotManager::KeeperSnapshotManager( - DiskPtr disk_, - size_t snapshots_to_keep_, - const KeeperContextPtr & keeper_context_, - bool compress_snapshots_zstd_, - const std::string & superdigest_, - size_t storage_tick_time_) - : disk(disk_) - , snapshots_to_keep(snapshots_to_keep_) + : snapshots_to_keep(snapshots_to_keep_) , compress_snapshots_zstd(compress_snapshots_zstd_) , superdigest(superdigest_) , storage_tick_time(storage_tick_time_) , keeper_context(keeper_context_) { + auto disk = getDisk(); for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) { const auto & name = it->name(); @@ -566,6 +548,7 @@ std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; + auto disk = getDisk(); auto plain_buf = disk->writeFile(tmp_snapshot_file_name); copyData(reader, *plain_buf); plain_buf->sync(); @@ -589,7 +572,7 @@ nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBuff } catch (const DB::Exception &) { - disk->removeFile(latest_itr->second); + getDisk()->removeFile(latest_itr->second); existing_snapshots.erase(latest_itr->first); tryLogCurrentException(__PRETTY_FUNCTION__); } @@ -602,7 +585,7 @@ nuraft::ptr KeeperSnapshotManager::deserializeSnapshotBufferFrom { const std::string & snapshot_path = existing_snapshots.at(up_to_log_idx); WriteBufferFromNuraftBuffer writer; - auto reader = disk->readFile(snapshot_path); + auto reader = getDisk()->readFile(snapshot_path); copyData(*reader, writer); return writer.getBuffer(); } @@ -664,6 +647,11 @@ SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot() return deserializeSnapshotFromBuffer(buffer); } +DiskPtr KeeperSnapshotManager::getDisk() const +{ + return keeper_context->getSnapshotsDisk(); +} + void KeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded() { while (existing_snapshots.size() > snapshots_to_keep) @@ -675,7 +663,7 @@ void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) auto itr = existing_snapshots.find(log_idx); if (itr == existing_snapshots.end()) throw Exception(ErrorCodes::UNKNOWN_SNAPSHOT, "Unknown snapshot with log index {}", log_idx); - disk->removeFile(itr->second); + getDisk()->removeFile(itr->second); existing_snapshots.erase(itr); } @@ -685,7 +673,7 @@ std::pair KeeperSnapshotManager::serializeSnapshot auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; - auto writer = disk->writeFile(tmp_snapshot_file_name); + auto writer = getDisk()->writeFile(tmp_snapshot_file_name); std::unique_ptr compressed_writer; if (compress_snapshots_zstd) compressed_writer = wrapWriteBufferWithCompressionMethod(std::move(writer), CompressionMethod::Zstd, 3); @@ -700,7 +688,7 @@ std::pair KeeperSnapshotManager::serializeSnapshot try { - disk->moveFile(tmp_snapshot_file_name, snapshot_file_name); + getDisk()->moveFile(tmp_snapshot_file_name, snapshot_file_name); } catch (fs::filesystem_error & e) { diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 9babad9ed98..7b1129018d8 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -99,16 +99,6 @@ class KeeperSnapshotManager { public: KeeperSnapshotManager( - DiskPtr disk_, - size_t snapshots_to_keep_, - const KeeperContextPtr & keeper_context_, - bool compress_snapshots_zstd_ = true, - const std::string & superdigest_ = "", - size_t storage_tick_time_ = 500); - - /// For gtest - KeeperSnapshotManager( - const std::string & snapshots_path_, size_t snapshots_to_keep_, const KeeperContextPtr & keeper_context_, bool compress_snapshots_zstd_ = true, @@ -157,7 +147,7 @@ public: try { - if (disk->exists(path)) + if (getDisk()->exists(path)) return path; } catch (...) @@ -170,11 +160,12 @@ public: private: void removeOutdatedSnapshotsIfNeeded(); + DiskPtr getDisk() const; + /// Checks first 4 buffer bytes to became sure that snapshot compressed with /// ZSTD codec. static bool isZstdCompressed(nuraft::ptr buffer); - DiskPtr disk; /// How many snapshots to keep before remove const size_t snapshots_to_keep; /// All existing snapshots in our path (log_index -> path) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index a7c845e5017..69c15db51da 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -43,7 +43,6 @@ namespace KeeperStateMachine::KeeperStateMachine( ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, - DiskPtr disk_, const CoordinationSettingsPtr & coordination_settings_, const KeeperContextPtr & keeper_context_, KeeperSnapshotManagerS3 * snapshot_manager_s3_, @@ -52,7 +51,6 @@ KeeperStateMachine::KeeperStateMachine( : commit_callback(commit_callback_) , coordination_settings(coordination_settings_) , snapshot_manager( - disk_, coordination_settings->snapshots_to_keep, keeper_context_, coordination_settings->compress_snapshots_with_zstd_format, diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 1fe4db4de68..afe11150f36 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -26,7 +26,6 @@ public: KeeperStateMachine( ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, - DiskPtr disk_, const CoordinationSettingsPtr & coordination_settings_, const KeeperContextPtr & keeper_context_, KeeperSnapshotManagerS3 * snapshot_manager_s3_, diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index bcd5ce89e49..352e67e7c43 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -212,12 +212,14 @@ KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfigur return result; } -KeeperStateManager::KeeperStateManager( - int server_id_, const std::string & host, int port, const std::string & logs_path, const std::string & state_file_path) +KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, int port, KeeperContextPtr keeper_context_) : my_server_id(server_id_) , secure(false) - , log_store(nuraft::cs_new(std::make_shared("Keeper-logs", logs_path, 0), LogFileSettings{.force_sync =false, .compress_logs = false, .rotate_interval = 5000})) - , server_state_file_name(fs::path(state_file_path).filename().generic_string()) + , log_store(nuraft::cs_new( + LogFileSettings{.force_sync = false, .compress_logs = false, .rotate_interval = 5000}, + keeper_context_)) + , server_state_file_name("state") + , keeper_context(keeper_context_) , logger(&Poco::Logger::get("KeeperStateManager")) { auto peer_config = nuraft::cs_new(my_server_id, host + ":" + std::to_string(port)); @@ -230,17 +232,15 @@ KeeperStateManager::KeeperStateManager( KeeperStateManager::KeeperStateManager( int my_server_id_, const std::string & config_prefix_, - DiskPtr log_disk_, - DiskPtr state_disk_, const std::string & server_state_file_name_, const Poco::Util::AbstractConfiguration & config, - const CoordinationSettingsPtr & coordination_settings) + const CoordinationSettingsPtr & coordination_settings, + KeeperContextPtr keeper_context_) : my_server_id(my_server_id_) , secure(config.getBool(config_prefix_ + ".raft_configuration.secure", false)) , config_prefix(config_prefix_) , configuration_wrapper(parseServersConfiguration(config, false)) , log_store(nuraft::cs_new( - log_disk_, LogFileSettings { .force_sync = coordination_settings->force_sync, @@ -248,9 +248,10 @@ KeeperStateManager::KeeperStateManager( .rotate_interval = coordination_settings->rotate_log_storage_interval, .max_size = coordination_settings->max_log_file_size, .overallocate_size = coordination_settings->log_file_overallocate_size - })) - , disk(state_disk_) + }, + keeper_context_)) , server_state_file_name(server_state_file_name_) + , keeper_context(keeper_context_) , logger(&Poco::Logger::get("KeeperStateManager")) { } @@ -299,6 +300,11 @@ const String & KeeperStateManager::getOldServerStatePath() return old_path; } +DiskPtr KeeperStateManager::getStateFileDisk() const +{ + return keeper_context->getStateFileDisk(); +} + namespace { enum ServerStateVersion : uint8_t @@ -314,6 +320,8 @@ void KeeperStateManager::save_state(const nuraft::srv_state & state) { const auto & old_path = getOldServerStatePath(); + auto disk = getStateFileDisk(); + if (disk->exists(server_state_file_name)) disk->moveFile(server_state_file_name, old_path); @@ -338,7 +346,9 @@ nuraft::ptr KeeperStateManager::read_state() { const auto & old_path = getOldServerStatePath(); - const auto try_read_file = [this](const auto & path) -> nuraft::ptr + auto disk = getStateFileDisk(); + + const auto try_read_file = [&](const auto & path) -> nuraft::ptr { try { diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index d8369100d1c..f24f0c2b1e5 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -39,19 +39,17 @@ public: KeeperStateManager( int server_id_, const std::string & config_prefix_, - DiskPtr logs_disk_, - DiskPtr state_disk_, - const std::string & state_file_path, + const std::string & server_state_file_name_, const Poco::Util::AbstractConfiguration & config, - const CoordinationSettingsPtr & coordination_settings); + const CoordinationSettingsPtr & coordination_settings, + KeeperContextPtr keeper_context_); /// Constructor for tests KeeperStateManager( int server_id_, const std::string & host, int port, - const std::string & logs_path, - const std::string & state_file_path); + KeeperContextPtr keeper_context_); void loadLogStore(uint64_t last_commited_index, uint64_t logs_to_keep); @@ -114,6 +112,8 @@ public: private: const String & getOldServerStatePath(); + DiskPtr getStateFileDisk() const; + /// Wrapper struct for Keeper cluster config. We parse this /// info from XML files. struct KeeperConfigurationWrapper @@ -137,9 +137,10 @@ private: nuraft::ptr log_store; - DiskPtr disk; const String server_state_file_name; + KeeperContextPtr keeper_context; + Poco::Logger * logger; public: diff --git a/src/Coordination/Context/Context.cpp b/src/Coordination/Standalone/Context.cpp similarity index 80% rename from src/Coordination/Context/Context.cpp rename to src/Coordination/Standalone/Context.cpp index aeb4e405938..667fabf4d66 100644 --- a/src/Coordination/Context/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -38,6 +38,9 @@ struct ContextSharedPart : boost::noncopyable /// For access of most of shared objects. Recursive mutex. mutable std::recursive_mutex mutex; + mutable std::mutex keeper_dispatcher_mutex; + mutable std::shared_ptr keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); + ServerSettings server_settings; String path; /// Path to the data directory, with a slash at the end. @@ -129,6 +132,11 @@ MultiVersion::Version Context::getMacros() const return shared->macros.get(); } +void Context::setMacros(std::unique_ptr && macros) +{ + shared->macros.set(std::move(macros)); +} + BackgroundSchedulePool & Context::getSchedulePool() const { auto lock = getLock(); @@ -256,4 +264,54 @@ ReadSettings Context::getReadSettings() const return ReadSettings{}; } +void Context::initializeKeeperDispatcher([[maybe_unused]] bool start_async) const +{ + const auto & config_ref = getConfigRef(); + + std::lock_guard lock(shared->keeper_dispatcher_mutex); + + if (shared->keeper_dispatcher) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize Keeper multiple times"); + + if (config_ref.has("keeper_server")) + { + shared->keeper_dispatcher = std::make_shared(); + shared->keeper_dispatcher->initialize(config_ref, true, start_async, getMacros()); + } +} + +std::shared_ptr Context::getKeeperDispatcher() const +{ + std::lock_guard lock(shared->keeper_dispatcher_mutex); + if (!shared->keeper_dispatcher) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Keeper must be initialized before requests"); + + return shared->keeper_dispatcher; +} + +std::shared_ptr Context::tryGetKeeperDispatcher() const +{ + std::lock_guard lock(shared->keeper_dispatcher_mutex); + return shared->keeper_dispatcher; +} + +void Context::shutdownKeeperDispatcher() const +{ + std::lock_guard lock(shared->keeper_dispatcher_mutex); + if (shared->keeper_dispatcher) + { + shared->keeper_dispatcher->shutdown(); + shared->keeper_dispatcher.reset(); + } +} + +void Context::updateKeeperConfiguration([[maybe_unused]] const Poco::Util::AbstractConfiguration & config_) +{ + std::lock_guard lock(shared->keeper_dispatcher_mutex); + if (!shared->keeper_dispatcher) + return; + + shared->keeper_dispatcher->updateConfiguration(getConfigRef(), getMacros()); +} + } diff --git a/src/Coordination/Context/Context.h b/src/Coordination/Standalone/Context.h similarity index 88% rename from src/Coordination/Context/Context.h rename to src/Coordination/Standalone/Context.h index 683209d942e..c2eee981aaa 100644 --- a/src/Coordination/Context/Context.h +++ b/src/Coordination/Standalone/Context.h @@ -2,6 +2,8 @@ #include +#include + #include #include @@ -85,6 +87,7 @@ public: void setPath(const String & path); MultiVersion::Version getMacros() const; + void setMacros(std::unique_ptr && macros); BackgroundSchedulePool & getSchedulePool() const; @@ -106,6 +109,12 @@ public: ThrottlerPtr getLocalWriteThrottler() const; ReadSettings getReadSettings() const; + + std::shared_ptr getKeeperDispatcher() const; + std::shared_ptr tryGetKeeperDispatcher() const; + void initializeKeeperDispatcher(bool start_async) const; + void shutdownKeeperDispatcher() const; + void updateKeeperConfiguration(const Poco::Util::AbstractConfiguration & config); }; } diff --git a/src/Coordination/Context/Settings.cpp b/src/Coordination/Standalone/Settings.cpp similarity index 100% rename from src/Coordination/Context/Settings.cpp rename to src/Coordination/Standalone/Settings.cpp diff --git a/src/Coordination/Context/ThreadStatusExt.cpp b/src/Coordination/Standalone/ThreadStatusExt.cpp similarity index 100% rename from src/Coordination/Context/ThreadStatusExt.cpp rename to src/Coordination/Standalone/ThreadStatusExt.cpp diff --git a/src/Coordination/TinyContext.cpp b/src/Coordination/TinyContext.cpp deleted file mode 100644 index 47b0a48dcda..00000000000 --- a/src/Coordination/TinyContext.cpp +++ /dev/null @@ -1,87 +0,0 @@ -#include - -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -void TinyContext::setConfig(const ConfigurationPtr & config_) -{ - std::lock_guard lock(keeper_dispatcher_mutex); - config = config_; -} - -const Poco::Util::AbstractConfiguration & TinyContext::getConfigRef() const -{ - std::lock_guard lock(keeper_dispatcher_mutex); - return config ? *config : Poco::Util::Application::instance().config(); -} - - -void TinyContext::initializeKeeperDispatcher([[maybe_unused]] bool start_async) const -{ - const auto & config_ref = getConfigRef(); - - std::lock_guard lock(keeper_dispatcher_mutex); - - if (keeper_dispatcher) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize Keeper multiple times"); - - if (config_ref.has("keeper_server")) - { - keeper_dispatcher = std::make_shared(); - - MultiVersion::Version macros; - - if (config_ref.has("macros")) - macros = std::make_unique(config_ref, "macros", &Poco::Logger::get("TinyContext")); - keeper_dispatcher->initialize(config_ref, true, start_async, macros); - } -} - -std::shared_ptr TinyContext::getKeeperDispatcher() const -{ - std::lock_guard lock(keeper_dispatcher_mutex); - if (!keeper_dispatcher) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Keeper must be initialized before requests"); - - return keeper_dispatcher; -} - -std::shared_ptr TinyContext::tryGetKeeperDispatcher() const -{ - std::lock_guard lock(keeper_dispatcher_mutex); - return keeper_dispatcher; -} - -void TinyContext::shutdownKeeperDispatcher() const -{ - std::lock_guard lock(keeper_dispatcher_mutex); - if (keeper_dispatcher) - { - keeper_dispatcher->shutdown(); - keeper_dispatcher.reset(); - } -} - -void TinyContext::updateKeeperConfiguration([[maybe_unused]] const Poco::Util::AbstractConfiguration & config_) -{ - std::lock_guard lock(keeper_dispatcher_mutex); - if (!keeper_dispatcher) - return; - - MultiVersion::Version macros; - - if (config_.has("macros")) - macros = std::make_unique(config_, "macros", &Poco::Logger::get("TinyContext")); - - keeper_dispatcher->updateConfiguration(config_, macros); -} - -} diff --git a/src/Coordination/TinyContext.h b/src/Coordination/TinyContext.h deleted file mode 100644 index b966d445004..00000000000 --- a/src/Coordination/TinyContext.h +++ /dev/null @@ -1,36 +0,0 @@ -#pragma once -#include -#include - -#include -#include - -namespace DB -{ - -class KeeperDispatcher; - -class TinyContext : public std::enable_shared_from_this -{ -public: - std::shared_ptr getKeeperDispatcher() const; - std::shared_ptr tryGetKeeperDispatcher() const; - void initializeKeeperDispatcher(bool start_async) const; - void shutdownKeeperDispatcher() const; - void updateKeeperConfiguration(const Poco::Util::AbstractConfiguration & config); - - using ConfigurationPtr = Poco::AutoPtr; - - void setConfig(const ConfigurationPtr & config); - const Poco::Util::AbstractConfiguration & getConfigRef() const; - -private: - mutable std::mutex keeper_dispatcher_mutex; - mutable std::shared_ptr keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); - - ConfigurationPtr config TSA_GUARDED_BY(keeper_dispatcher_mutex); -}; - -using TinyContextPtr = std::shared_ptr; - -} diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b2aee8d9ec0..0ec39f18757 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1237,6 +1237,6 @@ struct HTTPContext : public IHTTPContext #else -#include +#include #endif From 6de52e9fced5c71c24a2f40a2b13b9c3d5656b14 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 23 May 2023 18:36:02 +0000 Subject: [PATCH 075/515] Fixing some tests. --- src/Interpreters/InterpreterSelectQuery.cpp | 15 ++- src/Interpreters/PreparedSets.cpp | 36 ++++- src/Interpreters/PreparedSets.h | 32 +---- src/Planner/Planner.cpp | 15 ++- src/Processors/QueryPlan/CreatingSetsStep.h | 3 + .../Optimizations/filterPushDown.cpp | 13 ++ .../QueryPlan/ReadFromMergeTree.cpp | 107 +++++++++------ src/Processors/QueryPlan/ReadFromMergeTree.h | 44 +++++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 127 ++++++------------ .../MergeTree/MergeTreeDataSelectExecutor.h | 2 +- .../MergeTreeIndexConditionBloomFilter.cpp | 10 ++ .../MergeTree/MergeTreeIndexInverted.cpp | 5 + src/Storages/MergeTree/RPNBuilder.cpp | 20 ++- src/Storages/SelectQueryInfo.h | 2 + 14 files changed, 259 insertions(+), 172 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index bd96ba693fe..8d305c07ce9 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -3088,12 +3088,17 @@ void InterpreterSelectQuery::executeExtremes(QueryPlan & query_plan) void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_plan) { - auto step = std::make_unique( - query_plan.getCurrentDataStream(), - prepared_sets->detachSubqueries(context), - context); + auto subqueries = prepared_sets->detachSubqueries(context); - query_plan.addStep(std::move(step)); + if (!subqueries.empty()) + { + auto step = std::make_unique( + query_plan.getCurrentDataStream(), + std::move(subqueries), + context); + + query_plan.addStep(std::move(step)); + } } diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 1d7d90432b0..cd6b2a81ba0 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -226,7 +226,7 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c if (set) return nullptr; - // std::cerr << StackTrace().toString() << std::endl; + //std::cerr << StackTrace().toString() << std::endl; auto set_cache = context->getPreparedSetsCache(); if (set_cache) @@ -294,4 +294,38 @@ FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquer FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} +SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) +{ + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, true); + fill(size_limits, settings.transform_null_in, true); + return set; +} + +std::unique_ptr FutureSetFromTuple::build(const ContextPtr & context) +{ + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, false); + fill(size_limits, settings.transform_null_in, false); + return nullptr; +} + +void FutureSetFromTuple::buildForTuple(SizeLimits size_limits, bool transform_null_in) +{ + fill(size_limits, transform_null_in, false); +} + +void FutureSetFromTuple::fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set) +{ + //std::cerr << StackTrace().toString() << std::endl; + + if (set) + return; + + set = std::make_shared(size_limits, create_ordered_set, transform_null_in); + set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); + set->insertFromBlock(block.getColumnsWithTypeAndName()); + set->finishInsert(); +} + }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index b4d01754ea8..ef7aba38f24 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -86,42 +86,18 @@ public: bool isFilled() const override { return true; } SetPtr get() const override { return set; } - SetPtr buildOrderedSetInplace(const ContextPtr & context) override - { - const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, true); - fill(size_limits, settings.transform_null_in, true); - return set; - } + SetPtr buildOrderedSetInplace(const ContextPtr & context) override; - std::unique_ptr build(const ContextPtr & context) override - { - const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, false); - fill(size_limits, settings.transform_null_in, false); - return nullptr; - } + std::unique_ptr build(const ContextPtr & context) override; - void buildForTuple(SizeLimits size_limits, bool transform_null_in) - { - fill(size_limits, transform_null_in, false); - } + void buildForTuple(SizeLimits size_limits, bool transform_null_in); private: Block block; SetPtr set; - void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set) - { - if (set) - return; - - set = std::make_shared(size_limits, create_ordered_set, transform_null_in); - set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); - set->insertFromBlock(block.getColumnsWithTypeAndName()); - set->finishInsert(); - } + void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set); }; /// Information on how to build set for the [GLOBAL] IN section. diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 38d0aa29d24..b1780212e51 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1468,12 +1468,17 @@ void Planner::buildPlanForQueryNode() if (!select_query_options.only_analyze) { - auto step = std::make_unique( - query_plan.getCurrentDataStream(), - planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()), - planner_context->getQueryContext()); + auto subqueries = planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()); - query_plan.addStep(std::move(step)); + if (!subqueries.empty()) + { + auto step = std::make_unique( + query_plan.getCurrentDataStream(), + std::move(subqueries), + planner_context->getQueryContext()); + + query_plan.addStep(std::move(step)); + } //addCreatingSetsStep(query_plan, planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()), planner_context->getQueryContext()); //addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index 96ab26077fc..244bb27ba78 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -58,6 +58,9 @@ public: static std::vector> makePlansForSets(DelayedCreatingSetsStep && step); + ContextPtr getContext() const { return context; } + PreparedSets::SubqueriesForSets detachSubqueries() { return std::move(subqueries_for_sets); } + private: PreparedSets::SubqueriesForSets subqueries_for_sets; ContextPtr context; diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 37bc894339f..63ba3d5b56c 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -268,6 +268,19 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes return 2; } + if (auto * delayed = typeid_cast(child.get())) + { + /// CreatingSets does not change header. + /// We can push down filter and update header. + /// Filter - DelayedCreatingSets - Something + child = std::make_unique(filter->getOutputStream(), delayed->detachSubqueries(), delayed->getContext()); + std::swap(parent, child); + std::swap(parent_node->children, child_node->children); + std::swap(parent_node->children.front(), child_node->children.front()); + /// DelayedCreatingSets - Filter - Something + return 2; + } + if (auto * totals_having = typeid_cast(child.get())) { /// If totals step has HAVING expression, skip it for now. diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index ef013a36069..a096538a298 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1127,7 +1127,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(Merge real_column_names, sample_factor_column_queried, log, - key_condition); + indexes); } static ActionsDAGPtr buildFilterDAG( @@ -1171,14 +1171,14 @@ static ActionsDAGPtr buildFilterDAG( return ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_node_column, context); } -static void buildKeyCondition( - std::optional & key_condition, +static void buildIndexes( + std::optional & indexes, ActionsDAGPtr filter_actions_dag, const ContextPtr & context, const SelectQueryInfo & query_info, const StorageMetadataPtr & metadata_snapshot) { - key_condition.reset(); + indexes.reset(); // Build and check if primary key is used when necessary const auto & primary_key = metadata_snapshot->getPrimaryKey(); @@ -1191,16 +1191,58 @@ static void buildKeyCondition( if (query_info.syntax_analyzer_result) array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(); - key_condition.emplace(filter_actions_dag, + indexes.emplace(ReadFromMergeTree::Indexes{{ + filter_actions_dag, context, primary_key_column_names, primary_key.expression, - array_join_name_set); + array_join_name_set}, {}, false}); } else { - key_condition.emplace(query_info, context, primary_key_column_names, primary_key.expression); + indexes.emplace(ReadFromMergeTree::Indexes{{ + query_info, + context, + primary_key_column_names, + primary_key.expression}, {}, false}); } + + indexes->use_skip_indexes = settings.use_skip_indexes; + bool final = query_info.isFinal(); + + if (final && !settings.use_skip_indexes_if_final) + indexes->use_skip_indexes = false; + + if (!indexes->use_skip_indexes) + return; + + UsefulSkipIndexes skip_indexes; + using Key = std::pair; + std::map merged; + + for (const auto & index : metadata_snapshot->getSecondaryIndices()) + { + auto index_helper = MergeTreeIndexFactory::instance().get(index); + if (index_helper->isMergeable()) + { + auto [it, inserted] = merged.emplace(Key{index_helper->index.type, index_helper->getGranularity()}, skip_indexes.merged_indices.size()); + if (inserted) + { + skip_indexes.merged_indices.emplace_back(); + skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); + } + + skip_indexes.merged_indices[it->second].addIndex(index_helper); + } + else + { + auto condition = index_helper->createIndexCondition(query_info, context); + if (!condition->alwaysUnknownOrTrue()) + skip_indexes.useful_indices.emplace_back(index_helper, condition); + } + } + + indexes->skip_indexes = std::move(skip_indexes); } void ReadFromMergeTree::onAddFilterFinish() @@ -1208,7 +1250,7 @@ void ReadFromMergeTree::onAddFilterFinish() if (!filter_nodes.nodes.empty()) { auto filter_actions_dag = buildFilterDAG(context, prewhere_info, filter_nodes, query_info); - buildKeyCondition(key_condition, filter_actions_dag, context, query_info, metadata_for_reading); + buildIndexes(indexes, filter_actions_dag, context, query_info, metadata_for_reading); } } @@ -1226,7 +1268,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( const Names & real_column_names, bool sample_factor_column_queried, Poco::Logger * log, - std::optional & key_condition) + std::optional & indexes) { const auto & settings = context->getSettingsRef(); if (settings.allow_experimental_analyzer || settings.query_plan_optimize_primary_key) @@ -1246,7 +1288,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( real_column_names, sample_factor_column_queried, log, - key_condition); + indexes); } return selectRangesToReadImpl( @@ -1261,7 +1303,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( real_column_names, sample_factor_column_queried, log, - key_condition); + indexes); } MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( @@ -1276,7 +1318,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( const Names & real_column_names, bool sample_factor_column_queried, Poco::Logger * log, - std::optional & key_condition) + std::optional & indexes) { AnalysisResult result; const auto & settings = context->getSettingsRef(); @@ -1323,10 +1365,10 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( // } // } - if (!key_condition) - buildKeyCondition(key_condition, query_info.filter_actions_dag, context, query_info, metadata_snapshot); + if (!indexes) + buildIndexes(indexes, query_info.filter_actions_dag, context, query_info, metadata_snapshot); - if (settings.force_primary_key && key_condition->alwaysUnknownOrTrue()) + if (settings.force_primary_key && indexes->key_condition.alwaysUnknownOrTrue()) { return std::make_shared(MergeTreeDataSelectAnalysisResult{ .result = std::make_exception_ptr(Exception( @@ -1334,9 +1376,9 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( "Primary key ({}) is not used and setting 'force_primary_key' is set", fmt::join(primary_key_column_names, ", ")))}); } - LOG_DEBUG(log, "Key condition: {}", key_condition->toString()); + LOG_DEBUG(log, "Key condition: {}", indexes->key_condition.toString()); - if (key_condition->alwaysFalse()) + if (indexes->key_condition.alwaysFalse()) return std::make_shared(MergeTreeDataSelectAnalysisResult{.result = std::move(result)}); size_t total_marks_pk = 0; @@ -1358,7 +1400,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( query_info, metadata_snapshot->getColumns().getAllPhysical(), parts, - *key_condition, + indexes->key_condition, data, metadata_snapshot, context, @@ -1374,23 +1416,17 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( auto reader_settings = getMergeTreeReaderSettings(context, query_info); - bool use_skip_indexes = settings.use_skip_indexes; - bool final = isFinal(query_info); - - if (final && !settings.use_skip_indexes_if_final) - use_skip_indexes = false; - result.parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( std::move(parts), metadata_snapshot, - query_info, context, - *key_condition, + indexes->key_condition, + indexes->skip_indexes, reader_settings, log, num_streams, result.index_stats, - use_skip_indexes); + indexes->use_skip_indexes); } catch (...) { @@ -1438,7 +1474,7 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, /// Disable read-in-order optimization for reverse order with final. /// Otherwise, it can lead to incorrect final behavior because the implementation may rely on the reading in direct order). - if (direction != 1 && isFinal(query_info)) + if (direction != 1 && query_info.isFinal()) return false; auto order_info = std::make_shared(SortDescription{}, prefix_size, direction, limit); @@ -1561,11 +1597,7 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::getAnalysisResult() const bool ReadFromMergeTree::isQueryWithFinal() const { - const auto & select = query_info.query->as(); - if (query_info.table_expression_modifiers) - return query_info.table_expression_modifiers->hasFinal(); - else - return select.final(); + return query_info.isFinal(); } bool ReadFromMergeTree::isQueryWithSampling() const @@ -2009,15 +2041,6 @@ void ReadFromMergeTree::describeIndexes(JSONBuilder::JSONMap & map) const } } -bool ReadFromMergeTree::isFinal(const SelectQueryInfo & query_info) -{ - if (query_info.table_expression_modifiers) - return query_info.table_expression_modifiers->hasFinal(); - - const auto & select = query_info.query->as(); - return select.final(); -} - bool MergeTreeDataSelectAnalysisResult::error() const { return std::holds_alternative(result); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index f13f75bfebc..6610b463726 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -24,6 +24,35 @@ struct MergeTreeDataSelectSamplingData ActionsDAGPtr filter_expression; }; +struct UsefulSkipIndexes +{ + struct DataSkippingIndexAndCondition + { + MergeTreeIndexPtr index; + MergeTreeIndexConditionPtr condition; + + DataSkippingIndexAndCondition(MergeTreeIndexPtr index_, MergeTreeIndexConditionPtr condition_) + : index(index_), condition(condition_) + { + } + }; + + struct MergedDataSkippingIndexAndCondition + { + std::vector indices; + MergeTreeIndexMergedConditionPtr condition; + + void addIndex(const MergeTreeIndexPtr & index) + { + indices.push_back(index); + condition->addIndex(indices.back()); + } + }; + + std::vector useful_indices; + std::vector merged_indices; +}; + struct MergeTreeDataSelectAnalysisResult; using MergeTreeDataSelectAnalysisResultPtr = std::shared_ptr; @@ -132,6 +161,13 @@ public: UInt64 getSelectedRows() const { return selected_rows; } UInt64 getSelectedMarks() const { return selected_marks; } + struct Indexes + { + KeyCondition key_condition; + UsefulSkipIndexes skip_indexes; + bool use_skip_indexes; + }; + static MergeTreeDataSelectAnalysisResultPtr selectRangesToRead( MergeTreeData::DataPartsVector parts, const PrewhereInfoPtr & prewhere_info, @@ -146,7 +182,7 @@ public: const Names & real_column_names, bool sample_factor_column_queried, Poco::Logger * log, - std::optional & key_condition); + std::optional & indexes); MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(MergeTreeData::DataPartsVector parts) const; @@ -159,8 +195,6 @@ public: bool requestReadingInOrder(size_t prefix_size, int direction, size_t limit); void updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value); - - static bool isFinal(const SelectQueryInfo & query_info); bool isQueryWithFinal() const; bool isQueryWithSampling() const; @@ -193,7 +227,7 @@ private: const Names & real_column_names, bool sample_factor_column_queried, Poco::Logger * log, - std::optional & key_condition); + std::optional & indexes); int getSortDirection() const { @@ -233,7 +267,7 @@ private: std::shared_ptr max_block_numbers_to_read; /// Pre-computed value, needed to trigger sets creating for PK - mutable std::optional key_condition; + mutable std::optional indexes; Poco::Logger * log; UInt64 selected_parts = 0; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 55dfc56d8ac..f99e15c0fc1 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -887,9 +887,9 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( MergeTreeData::DataPartsVector && parts, StorageMetadataPtr metadata_snapshot, - const SelectQueryInfo & query_info, const ContextPtr & context, const KeyCondition & key_condition, + const UsefulSkipIndexes & skip_indexes, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, size_t num_streams, @@ -900,66 +900,6 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd parts_with_ranges.resize(parts.size()); const Settings & settings = context->getSettingsRef(); - /// Let's start analyzing all useful indices - - struct IndexStat - { - std::atomic total_granules{0}; - std::atomic granules_dropped{0}; - std::atomic total_parts{0}; - std::atomic parts_dropped{0}; - }; - - struct DataSkippingIndexAndCondition - { - MergeTreeIndexPtr index; - MergeTreeIndexConditionPtr condition; - IndexStat stat; - - DataSkippingIndexAndCondition(MergeTreeIndexPtr index_, MergeTreeIndexConditionPtr condition_) - : index(index_), condition(condition_) - { - } - }; - - struct MergedDataSkippingIndexAndCondition - { - std::vector indices; - MergeTreeIndexMergedConditionPtr condition; - IndexStat stat; - - void addIndex(const MergeTreeIndexPtr & index) - { - indices.push_back(index); - condition->addIndex(indices.back()); - } - }; - - std::list useful_indices; - std::map, MergedDataSkippingIndexAndCondition> merged_indices; - - if (use_skip_indexes) - { - for (const auto & index : metadata_snapshot->getSecondaryIndices()) - { - auto index_helper = MergeTreeIndexFactory::instance().get(index); - if (index_helper->isMergeable()) - { - auto [it, inserted] = merged_indices.try_emplace({index_helper->index.type, index_helper->getGranularity()}); - if (inserted) - it->second.condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); - - it->second.addIndex(index_helper); - } - else - { - auto condition = index_helper->createIndexCondition(query_info, context); - if (!condition->alwaysUnknownOrTrue()) - useful_indices.emplace_back(index_helper, condition); - } - } - } - if (use_skip_indexes && settings.force_data_skipping_indices.changed) { const auto & indices = settings.force_data_skipping_indices.toString(); @@ -977,7 +917,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "No indices parsed from force_data_skipping_indices ('{}')", indices); std::unordered_set useful_indices_names; - for (const auto & useful_index : useful_indices) + for (const auto & useful_index : skip_indexes.useful_indices) useful_indices_names.insert(useful_index.index->index.name); for (const auto & index_name : forced_indices) @@ -992,6 +932,17 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd } } + struct IndexStat + { + std::atomic total_granules{0}; + std::atomic granules_dropped{0}; + std::atomic total_parts{0}; + std::atomic parts_dropped{0}; + }; + + std::vector useful_indices_stat(skip_indexes.useful_indices.size()); + std::vector merged_indices_stat(skip_indexes.merged_indices.size()); + std::atomic sum_marks_pk = 0; std::atomic sum_parts_pk = 0; @@ -1018,12 +969,14 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd if (!ranges.ranges.empty()) sum_parts_pk.fetch_add(1, std::memory_order_relaxed); - for (auto & index_and_condition : useful_indices) + for (size_t idx = 0; idx < skip_indexes.useful_indices.size(); ++idx) { if (ranges.ranges.empty()) break; - index_and_condition.stat.total_parts.fetch_add(1, std::memory_order_relaxed); + auto & index_and_condition = skip_indexes.useful_indices[idx]; + auto & stat = useful_indices_stat[idx]; + stat.total_parts.fetch_add(1, std::memory_order_relaxed); size_t total_granules = 0; size_t granules_dropped = 0; @@ -1040,19 +993,21 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd uncompressed_cache.get(), log); - index_and_condition.stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed); - index_and_condition.stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); + stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed); + stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); if (ranges.ranges.empty()) - index_and_condition.stat.parts_dropped.fetch_add(1, std::memory_order_relaxed); + stat.parts_dropped.fetch_add(1, std::memory_order_relaxed); } - for (auto & [_, indices_and_condition] : merged_indices) + for (size_t idx = 0; idx < skip_indexes.merged_indices.size(); ++idx) { if (ranges.ranges.empty()) break; - indices_and_condition.stat.total_parts.fetch_add(1, std::memory_order_relaxed); + auto & indices_and_condition = skip_indexes.merged_indices[idx]; + auto & stat = merged_indices_stat[idx]; + stat.total_parts.fetch_add(1, std::memory_order_relaxed); size_t total_granules = 0; size_t granules_dropped = 0; @@ -1063,11 +1018,11 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd total_granules, granules_dropped, mark_cache.get(), uncompressed_cache.get(), log); - indices_and_condition.stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed); - indices_and_condition.stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); + stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed); + stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); if (ranges.ranges.empty()) - indices_and_condition.stat.parts_dropped.fetch_add(1, std::memory_order_relaxed); + stat.parts_dropped.fetch_add(1, std::memory_order_relaxed); } if (!ranges.ranges.empty()) @@ -1134,15 +1089,17 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd .num_granules_after = sum_marks_pk.load(std::memory_order_relaxed)}); } - for (const auto & index_and_condition : useful_indices) + for (size_t idx = 0; idx < skip_indexes.useful_indices.size(); ++idx) { + const auto & index_and_condition = skip_indexes.useful_indices[idx]; + const auto & stat = useful_indices_stat[idx]; const auto & index_name = index_and_condition.index->index.name; LOG_DEBUG( log, "Index {} has dropped {}/{} granules.", backQuote(index_name), - index_and_condition.stat.granules_dropped, - index_and_condition.stat.total_granules); + stat.granules_dropped, + stat.total_granules); std::string description = index_and_condition.index->index.type + " GRANULARITY " + std::to_string(index_and_condition.index->index.granularity); @@ -1151,25 +1108,27 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd .type = ReadFromMergeTree::IndexType::Skip, .name = index_name, .description = std::move(description), - .num_parts_after = index_and_condition.stat.total_parts - index_and_condition.stat.parts_dropped, - .num_granules_after = index_and_condition.stat.total_granules - index_and_condition.stat.granules_dropped}); + .num_parts_after = stat.total_parts - stat.parts_dropped, + .num_granules_after = stat.total_granules - stat.granules_dropped}); } - for (const auto & [type_with_granularity, index_and_condition] : merged_indices) + for (size_t idx = 0; idx < skip_indexes.merged_indices.size(); ++idx) { + const auto & index_and_condition = skip_indexes.merged_indices[idx]; + const auto & stat = merged_indices_stat[idx]; const auto & index_name = "Merged"; LOG_DEBUG(log, "Index {} has dropped {}/{} granules.", backQuote(index_name), - index_and_condition.stat.granules_dropped, index_and_condition.stat.total_granules); + stat.granules_dropped, stat.total_granules); - std::string description = "MERGED GRANULARITY " + std::to_string(type_with_granularity.second); + std::string description = "MERGED GRANULARITY " + std::to_string(index_and_condition.indices.at(0)->index.granularity); index_stats.emplace_back(ReadFromMergeTree::IndexStat{ .type = ReadFromMergeTree::IndexType::Skip, .name = index_name, .description = std::move(description), - .num_parts_after = index_and_condition.stat.total_parts - index_and_condition.stat.parts_dropped, - .num_granules_after = index_and_condition.stat.total_granules - index_and_condition.stat.granules_dropped}); + .num_parts_after = stat.total_parts - stat.parts_dropped, + .num_granules_after = stat.total_granules - stat.granules_dropped}); } return parts_with_ranges; @@ -1291,7 +1250,7 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); - std::optional key_condition; + std::optional indexes; return ReadFromMergeTree::selectRangesToRead( std::move(parts), prewhere_info, @@ -1306,7 +1265,7 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar real_column_names, sample_factor_column_queried, log, - key_condition); + indexes); } QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index a337574bb64..8c8ce59bebe 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -189,9 +189,9 @@ public: static RangesInDataParts filterPartsByPrimaryKeyAndSkipIndexes( MergeTreeData::DataPartsVector && parts, StorageMetadataPtr metadata_snapshot, - const SelectQueryInfo & query_info, const ContextPtr & context, const KeyCondition & key_condition, + const UsefulSkipIndexes & skip_indexes, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, size_t num_streams, diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 5e186c25b83..2bd9db12b93 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -310,14 +310,24 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNo if (functionIsInOrGlobalInOperator(function_name)) { + //std::cerr << StackTrace().toString() << std::endl; + auto future_set = rhs_argument.tryGetPreparedSet(); + + //std::cerr << "==== Finding set for MergeTreeBF " << bool(future_set) << std::endl; + if (future_set && !future_set->isReady()) + { + //std::cerr << "==== not ready, building " << std::endl; future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); + } ConstSetPtr prepared_set; if (future_set) prepared_set = future_set->get(); + //std::cerr << "==== Prep set for MergeTreeBF " << bool(prepared_set) << ' ' << (prepared_set ? prepared_set->hasExplicitSetElements() : false) << std::endl; + if (prepared_set && prepared_set->hasExplicitSetElements()) { const auto prepared_info = getPreparedSetInfo(prepared_set); diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index 6ffba0ad029..a64f81807ae 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -655,7 +655,12 @@ bool MergeTreeConditionInverted::tryPrepareSetGinFilter( if (key_tuple_mapping.empty()) return false; + //std::cerr << "==== Finding set for MergeTreeConditionInverted\n"; + auto future_set = rhs.tryGetPreparedSet(); + + //std::cerr << "==== Set for MergeTreeConditionInverted" << bool(future_set) << std::endl; + if (future_set && !future_set->isReady()) future_set->buildOrderedSetInplace(rhs.getTreeContext().getQueryContext()); diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index e8843ff1489..cc7ec45be6a 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -298,11 +298,29 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const if (ast_node && prepared_sets) { - return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); + auto hash = ast_node->getTreeHash(); + auto key = PreparedSetKey::forSubquery(hash); + + // std::cerr << ".........Getting from AST \n" << ast_node->dumpTree() << std::endl + // << key.toString() << std::endl; + + for (const auto & [k, v] : prepared_sets->getSets()) + { + // std::cerr << "........... " << k.toString() << std::endl; + if (k.ast_hash == hash) + return v; + } + + //return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); } else if (dag_node) { + + // std::cerr << "...........Getting from DAG\n"; const auto * node_without_alias = getNodeWithoutAlias(dag_node); + // std::cerr << ".......... node_without_alias : " << node_without_alias->result_name + // << ' ' << node_without_alias->result_type->getName() + // << ' ' << (node_without_alias->column ? node_without_alias->column->getName() : "") << std::endl; return tryGetSetFromDAGNode(node_without_alias); } diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index fb895d04b8f..a8eb00adc87 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -264,5 +264,7 @@ struct SelectQueryInfo { return input_order_info ? input_order_info : (projection ? projection->input_order_info : nullptr); } + + bool isFinal() const; }; } From 9db7e8ed62415aac040ee5d220d6dabc749ccc7b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 23 May 2023 20:47:35 +0000 Subject: [PATCH 076/515] Fixing build. --- src/Storages/SelectQueryInfo.cpp | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) create mode 100644 src/Storages/SelectQueryInfo.cpp diff --git a/src/Storages/SelectQueryInfo.cpp b/src/Storages/SelectQueryInfo.cpp new file mode 100644 index 00000000000..665da7fee70 --- /dev/null +++ b/src/Storages/SelectQueryInfo.cpp @@ -0,0 +1,16 @@ +#include +#include + +namespace DB +{ + +bool SelectQueryInfo::isFinal() const +{ + if (table_expression_modifiers) + return table_expression_modifiers->hasFinal(); + + const auto & select = query->as(); + return select.final(); +} + +} From 161afea266bc8f3a13dacfb99f7333ab98f2ac46 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 May 2023 07:36:39 +0000 Subject: [PATCH 077/515] Add support for changelog --- programs/keeper/Keeper.cpp | 2 - src/Coordination/Changelog.cpp | 324 +++++++++++------- src/Coordination/Changelog.h | 4 +- src/Coordination/KeeperContext.cpp | 12 +- src/Coordination/KeeperContext.h | 2 + src/Coordination/KeeperSnapshotManager.cpp | 47 ++- src/Coordination/KeeperStateManager.cpp | 3 +- src/Coordination/Standalone/Context.cpp | 4 + src/Core/SettingsFields.cpp | 5 + .../MetadataStorageFromPlainObjectStorage.cpp | 1 + .../MetadataStorageFromPlainObjectStorage.h | 2 +- src/IO/ZstdDeflatingAppendableWriteBuffer.cpp | 11 +- src/IO/ZstdDeflatingAppendableWriteBuffer.h | 3 + src/Interpreters/Context.h | 2 +- 14 files changed, 277 insertions(+), 145 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index d5cf61daa6e..002ce413095 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -44,8 +44,6 @@ #include -#include - int mainEntryClickHouseKeeper(int argc, char ** argv) { diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 852a21c1c45..c94633d6dbd 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -3,17 +3,17 @@ #include #include #include +#include #include #include +#include #include #include #include -#include #include #include +#include #include -#include -#include namespace DB @@ -29,50 +29,58 @@ namespace ErrorCodes namespace { + void moveFileBetweenDisks(DiskPtr disk_from, ChangelogFileDescriptionPtr description, DiskPtr disk_to, const std::string & path_to) + { + disk_from->copyFile(description->path, *disk_to, path_to, {}); + disk_from->removeFile(description->path); + description->path = path_to; + description->disk = disk_to; + } -constexpr auto DEFAULT_PREFIX = "changelog"; + constexpr auto DEFAULT_PREFIX = "changelog"; -inline std::string formatChangelogPath(const std::string & name_prefix, uint64_t from_index, uint64_t to_index, const std::string & extension) -{ - return fmt::format("{}_{}_{}.{}", name_prefix, from_index, to_index, extension); -} + inline std::string + formatChangelogPath(const std::string & name_prefix, uint64_t from_index, uint64_t to_index, const std::string & extension) + { + return fmt::format("{}_{}_{}.{}", name_prefix, from_index, to_index, extension); + } -ChangelogFileDescriptionPtr getChangelogFileDescription(const std::filesystem::path & path) -{ - // we can have .bin.zstd so we cannot use std::filesystem stem and extension - std::string filename_with_extension = path.filename(); - std::string_view filename_with_extension_view = filename_with_extension; + ChangelogFileDescriptionPtr getChangelogFileDescription(const std::filesystem::path & path) + { + // we can have .bin.zstd so we cannot use std::filesystem stem and extension + std::string filename_with_extension = path.filename(); + std::string_view filename_with_extension_view = filename_with_extension; - auto first_dot = filename_with_extension.find('.'); - if (first_dot == std::string::npos) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid changelog file {}", path.generic_string()); + auto first_dot = filename_with_extension.find('.'); + if (first_dot == std::string::npos) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid changelog file {}", path.generic_string()); - Strings filename_parts; - boost::split(filename_parts, filename_with_extension_view.substr(0, first_dot), boost::is_any_of("_")); - if (filename_parts.size() < 3) - throw Exception(ErrorCodes::CORRUPTED_DATA, "Invalid changelog {}", path.generic_string()); + Strings filename_parts; + boost::split(filename_parts, filename_with_extension_view.substr(0, first_dot), boost::is_any_of("_")); + if (filename_parts.size() < 3) + throw Exception(ErrorCodes::CORRUPTED_DATA, "Invalid changelog {}", path.generic_string()); - auto result = std::make_shared(); - result->prefix = filename_parts[0]; - result->from_log_index = parse(filename_parts[1]); - result->to_log_index = parse(filename_parts[2]); - result->extension = std::string(filename_with_extension.substr(first_dot + 1)); - result->path = path.generic_string(); - return result; -} + auto result = std::make_shared(); + result->prefix = filename_parts[0]; + result->from_log_index = parse(filename_parts[1]); + result->to_log_index = parse(filename_parts[2]); + result->extension = std::string(filename_with_extension.substr(first_dot + 1)); + result->path = path.generic_string(); + return result; + } -Checksum computeRecordChecksum(const ChangelogRecord & record) -{ - SipHash hash; - hash.update(record.header.version); - hash.update(record.header.index); - hash.update(record.header.term); - hash.update(record.header.value_type); - hash.update(record.header.blob_size); - if (record.header.blob_size != 0) - hash.update(reinterpret_cast(record.blob->data_begin()), record.blob->size()); - return hash.get64(); -} + Checksum computeRecordChecksum(const ChangelogRecord & record) + { + SipHash hash; + hash.update(record.header.version); + hash.update(record.header.index); + hash.update(record.header.term); + hash.update(record.header.value_type); + hash.update(record.header.blob_size); + if (record.header.blob_size != 0) + hash.update(reinterpret_cast(record.blob->data_begin()), record.blob->size()); + return hash.get64(); + } } @@ -117,25 +125,55 @@ public: // if we wrote at least 1 log in the log file we can rename the file to reflect correctly the // contained logs // file can be deleted from disk earlier by compaction - if (!current_file_description->deleted && last_index_written - && *last_index_written != current_file_description->to_log_index) + if (!current_file_description->deleted) { - auto new_path = formatChangelogPath( - current_file_description->prefix, - current_file_description->from_log_index, - *last_index_written, - current_file_description->extension); - disk->moveFile(current_file_description->path, new_path); - current_file_description->path = std::move(new_path); + auto log_disk = current_file_description->disk; + const auto & path = current_file_description->path; + std::string new_path = path; + if (last_index_written && *last_index_written != current_file_description->to_log_index) + { + new_path = formatChangelogPath( + current_file_description->prefix, + current_file_description->from_log_index, + *last_index_written, + current_file_description->extension); + } + + if (disk == log_disk) + { + if (path != new_path) + { + try + { + disk->moveFile(path, new_path); + } + catch (...) + { + tryLogCurrentException(log, fmt::format("File rename failed on disk {}", disk->getName())); + } + current_file_description->path = std::move(new_path); + } + } + else + { + moveFileBetweenDisks(log_disk, current_file_description, disk, new_path); + } } } - file_buf = disk->writeFile(file_description->path, DBMS_DEFAULT_BUFFER_SIZE, mode); + auto current_log_disk = getCurrentLogDisk(); + assert(file_description->disk == current_log_disk); + file_buf = current_log_disk->writeFile(file_description->path, DBMS_DEFAULT_BUFFER_SIZE, mode); + assert(file_buf); last_index_written.reset(); current_file_description = std::move(file_description); if (log_file_settings.compress_logs) - compressed_buffer = std::make_unique(std::move(file_buf), /* compression level = */ 3, /* append_to_existing_file_ = */ mode == WriteMode::Append); + compressed_buffer = std::make_unique( + std::move(file_buf), + /* compressi)on level = */ 3, + /* append_to_existing_file_ = */ mode == WriteMode::Append, + [current_log_disk, path = current_file_description->path] { return current_log_disk->readFile(path); }); prealloc_done = false; } @@ -147,10 +185,7 @@ public: } /// There is bug when compressed_buffer has value, file_buf's ownership transfer to compressed_buffer - bool isFileSet() const - { - return compressed_buffer != nullptr || file_buf != nullptr; - } + bool isFileSet() const { return compressed_buffer != nullptr || file_buf != nullptr; } bool appendRecord(ChangelogRecord && record) { @@ -236,6 +271,7 @@ public: new_description->from_log_index = new_start_log_index; new_description->to_log_index = new_start_log_index + log_file_settings.rotate_interval - 1; new_description->extension = "bin"; + new_description->disk = getCurrentLogDisk(); if (log_file_settings.compress_logs) new_description->extension += "." + toContentEncodingName(CompressionMethod::Zstd); @@ -259,7 +295,6 @@ public: } private: - void finalizeCurrentFile() { assert(prealloc_done); @@ -279,14 +314,13 @@ private: const auto * file_buffer = tryGetFileBuffer(); - if (log_file_settings.max_size != 0 && isLocalDisk()) + if (log_file_settings.max_size != 0 && file_buffer) { int res = -1; do { res = ftruncate(file_buffer->getFD(), initial_file_size + file_buffer->count()); - } - while (res < 0 && errno == EINTR); + } while (res < 0 && errno == EINTR); if (res != 0) LOG_WARNING(log, "Could not ftruncate file. Error: {}, errno: {}", errnoToString(), errno); @@ -321,10 +355,7 @@ private: return *file_buffer; } - const WriteBufferFromFile * tryGetFileBuffer() const - { - return const_cast(this)->tryGetFileBuffer(); - } + const WriteBufferFromFile * tryGetFileBuffer() const { return const_cast(this)->tryGetFileBuffer(); } WriteBufferFromFile * tryGetFileBuffer() { @@ -344,30 +375,22 @@ private: void tryPreallocateForFile() { - if (log_file_settings.max_size == 0) - { - initial_file_size = 0; - prealloc_done = true; - return; - } - const auto * file_buffer = tryGetFileBuffer(); - if (!file_buffer) + if (log_file_settings.max_size == 0 || !file_buffer) { initial_file_size = 0; prealloc_done = true; - LOG_WARNING(log, "Could not preallocate space on disk {} using fallocate", getDisk()->getName()); return; } #ifdef OS_LINUX - if (isLocalDisk()) { int res = -1; do { - res = fallocate(file_buffer->getFD(), FALLOC_FL_KEEP_SIZE, 0, log_file_settings.max_size + log_file_settings.overallocate_size); + res = fallocate( + file_buffer->getFD(), FALLOC_FL_KEEP_SIZE, 0, log_file_settings.max_size + log_file_settings.overallocate_size); } while (res < 0 && errno == EINTR); if (res != 0) @@ -387,15 +410,11 @@ private: prealloc_done = true; } - DiskPtr getDisk() const - { - return keeper_context->getLogDisk(); - } + DiskPtr getCurrentLogDisk() const { return keeper_context->getCurrentLogDisk(); } - bool isLocalDisk() const - { - return dynamic_cast(getDisk().get()) != nullptr; - } + DiskPtr getDisk() const { return keeper_context->getLogDisk(); } + + bool isLocalDisk() const { return dynamic_cast(getDisk().get()) != nullptr; } std::map & existing_changelogs; @@ -440,9 +459,7 @@ struct ChangelogReadResult class ChangelogReader { public: - explicit ChangelogReader(DiskPtr disk_, const std::string & filepath_) - : disk(disk_) - , filepath(filepath_) + explicit ChangelogReader(DiskPtr disk_, const std::string & filepath_) : disk(disk_), filepath(filepath_) { auto compression_method = chooseCompressionMethod(filepath, ""); auto read_buffer_from_file = disk->readFile(filepath); @@ -546,10 +563,7 @@ private: std::unique_ptr read_buf; }; -Changelog::Changelog( - Poco::Logger * log_, - LogFileSettings log_file_settings, - KeeperContextPtr keeper_context_) +Changelog::Changelog(Poco::Logger * log_, LogFileSettings log_file_settings, KeeperContextPtr keeper_context_) : changelogs_detached_dir("detached") , rotate_interval(log_file_settings.rotate_interval) , log(log_) @@ -557,18 +571,30 @@ Changelog::Changelog( , append_completion_queue(std::numeric_limits::max()) , keeper_context(std::move(keeper_context_)) { - /// Load all files in changelog directory + /// Load all files on changelog disks + + const auto load_from_disk = [&](const auto & disk) + { + for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) + { + if (it->name() == changelogs_detached_dir) + continue; + + auto file_description = getChangelogFileDescription(it->path()); + file_description->disk = disk; + + auto [changelog_it, inserted] = existing_changelogs.insert_or_assign(file_description->from_log_index, std::move(file_description)); + + if (!inserted) + LOG_WARNING(log, "Found duplicate entries for {}, will use the entry from {}", changelog_it->second->path, disk->getName()); + } + }; auto disk = getDisk(); + load_from_disk(disk); - for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) - { - if (it->name() == changelogs_detached_dir) - continue; - - auto file_description = getChangelogFileDescription(it->path()); - existing_changelogs[file_description->from_log_index] = std::move(file_description); - } + auto current_log_disk = getCurrentLogDisk(); + load_from_disk(current_log_disk); if (existing_changelogs.empty()) LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", disk->getPath()); @@ -579,8 +605,7 @@ Changelog::Changelog( append_completion_thread = ThreadFromGlobalPool([this] { appendCompletionThread(); }); - current_writer = std::make_unique( - existing_changelogs, keeper_context, log_file_settings); + current_writer = std::make_unique(existing_changelogs, keeper_context, log_file_settings); } void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uint64_t logs_to_keep) @@ -652,7 +677,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin break; } - ChangelogReader reader(getDisk(), changelog_description.path); + ChangelogReader reader(changelog_description.disk, changelog_description.path); last_log_read_result = reader.readChangelog(logs, start_to_read_from, log); last_log_read_result->log_start_index = changelog_description.from_log_index; @@ -713,13 +738,13 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin assert(existing_changelogs.find(last_log_read_result->log_start_index) != existing_changelogs.end()); assert(existing_changelogs.find(last_log_read_result->log_start_index)->first == existing_changelogs.rbegin()->first); - /// Continue to write into incomplete existing log if it doesn't finished with error + /// Continue to write into incomplete existing log if it doesn't finish with error const auto & description = existing_changelogs[last_log_read_result->log_start_index]; if (last_log_read_result->last_read_index == 0 || last_log_read_result->error) /// If it's broken log then remove it { LOG_INFO(log, "Removing chagelog {} because it's empty or read finished with error", description->path); - getDisk()->removeFile(description->path); + description->disk->removeFile(description->path); existing_changelogs.erase(last_log_read_result->log_start_index); std::erase_if(logs, [last_log_read_result](const auto & item) { return item.first >= last_log_read_result->log_start_index; }); } @@ -728,6 +753,16 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin initWriter(description); } } + else if (last_log_read_result.has_value()) + { + /// check if we need to move it to another disk + auto current_log_disk = getCurrentLogDisk(); + auto disk = getDisk(); + + auto & description = existing_changelogs.at(last_log_read_result->log_start_index); + if (current_log_disk != disk && current_log_disk == description->disk) + moveFileBetweenDisks(current_log_disk, description, disk, description->path); + } /// Start new log if we don't initialize writer from previous log. All logs can be "complete". if (!current_writer->isFileSet()) @@ -740,27 +775,37 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin void Changelog::initWriter(ChangelogFileDescriptionPtr description) { if (description->expectedEntriesCountInLog() != rotate_interval) - LOG_TRACE(log, "Looks like rotate_logs_interval was changed, current {}, expected entries in last log {}", rotate_interval, description->expectedEntriesCountInLog()); + LOG_TRACE( + log, + "Looks like rotate_logs_interval was changed, current {}, expected entries in last log {}", + rotate_interval, + description->expectedEntriesCountInLog()); LOG_TRACE(log, "Continue to write into {}", description->path); + + auto log_disk = description->disk; + auto current_log_disk = getCurrentLogDisk(); + if (log_disk != current_log_disk) + moveFileBetweenDisks(log_disk, description, current_log_disk, description->path); + current_writer->setFile(std::move(description), WriteMode::Append); } namespace { -std::string getCurrentTimestampFolder() -{ - const auto timestamp = LocalDateTime{std::time(nullptr)}; - return fmt::format( - "{:02}{:02}{:02}T{:02}{:02}{:02}", - timestamp.year(), - timestamp.month(), - timestamp.day(), - timestamp.hour(), - timestamp.minute(), - timestamp.second()); -} + std::string getCurrentTimestampFolder() + { + const auto timestamp = LocalDateTime{std::time(nullptr)}; + return fmt::format( + "{:02}{:02}{:02}T{:02}{:02}{:02}", + timestamp.year(), + timestamp.month(), + timestamp.day(), + timestamp.hour(), + timestamp.minute(), + timestamp.second()); + } } @@ -769,6 +814,11 @@ DiskPtr Changelog::getDisk() const return keeper_context->getLogDisk(); } +DiskPtr Changelog::getCurrentLogDisk() const +{ + return keeper_context->getCurrentLogDisk(); +} + void Changelog::removeExistingLogs(ChangelogIter begin, ChangelogIter end) { auto disk = getDisk(); @@ -786,7 +836,23 @@ void Changelog::removeExistingLogs(ChangelogIter begin, ChangelogIter end) LOG_WARNING(log, "Removing changelog {}", itr->second->path); const std::filesystem::path & path = itr->second->path; const auto new_path = timestamp_folder / path.filename(); - disk->moveFile(path.generic_string(), new_path.generic_string()); + + auto changelog_disk = itr->second->disk; + if (changelog_disk == disk) + { + try + { + disk->moveFile(path.generic_string(), new_path.generic_string()); + } + catch (const DB::Exception & e) + { + if (e.code() == DB::ErrorCodes::NOT_IMPLEMENTED) + moveFileBetweenDisks(changelog_disk, itr->second, disk, new_path); + } + } + else + moveFileBetweenDisks(changelog_disk, itr->second, disk, new_path); + itr = existing_changelogs.erase(itr); } } @@ -921,7 +987,6 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry) throw Exception(ErrorCodes::LOGICAL_ERROR, "Changelog must be initialized before writing records"); { - std::lock_guard lock(writer_mutex); /// This write_at require to overwrite everything in this file and also in previous file(s) const bool go_to_previous_file = index < current_writer->getStartIndex(); @@ -937,13 +1002,18 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry) else description = std::prev(index_changelog)->second; + auto log_disk = description->disk; + auto current_log_disk = getCurrentLogDisk(); + if (log_disk != current_log_disk) + moveFileBetweenDisks(log_disk, description, current_log_disk, description->path); + current_writer->setFile(std::move(description), WriteMode::Append); /// Remove all subsequent files if overwritten something in previous one auto to_remove_itr = existing_changelogs.upper_bound(index); for (auto itr = to_remove_itr; itr != existing_changelogs.end();) { - getDisk()->removeFile(itr->second->path); + itr->second->disk->removeFile(itr->second->path); itr = existing_changelogs.erase(itr); } } @@ -993,16 +1063,17 @@ void Changelog::compact(uint64_t up_to_log_index) LOG_INFO(log, "Removing changelog {} because of compaction", changelog_description.path); /// If failed to push to queue for background removing, then we will remove it now - if (!log_files_to_delete_queue.tryPush(changelog_description.path, 1)) + if (!log_files_to_delete_queue.tryPush({changelog_description.path, changelog_description.disk}, 1)) { try { - getDisk()->removeFile(itr->second->path); - LOG_INFO(log, "Removed changelog {} because of compaction.", itr->second->path); + changelog_description.disk->removeFile(changelog_description.path); + LOG_INFO(log, "Removed changelog {} because of compaction.", changelog_description.path); } catch (Exception & e) { - LOG_WARNING(log, "Failed to remove changelog {} in compaction, error message: {}", itr->second->path, e.message()); + LOG_WARNING( + log, "Failed to remove changelog {} in compaction, error message: {}", changelog_description.path, e.message()); } catch (...) { @@ -1197,12 +1268,13 @@ Changelog::~Changelog() void Changelog::cleanLogThread() { - std::string path; - while (log_files_to_delete_queue.pop(path)) + std::pair path_with_disk; + while (log_files_to_delete_queue.pop(path_with_disk)) { + const auto & [path, disk] = path_with_disk; try { - getDisk()->removeFile(path); + disk->removeFile(path); LOG_INFO(log, "Removed changelog {} because of compaction.", path); } catch (Exception & e) diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 6f0c4e45605..4054829ef19 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -60,6 +60,7 @@ struct ChangelogFileDescription uint64_t to_log_index; std::string extension; + DiskPtr disk; std::string path; bool deleted = false; @@ -154,6 +155,7 @@ private: static ChangelogRecord buildRecord(uint64_t index, const LogEntryPtr & log_entry); DiskPtr getDisk() const; + DiskPtr getCurrentLogDisk() const; /// Currently existing changelogs std::map existing_changelogs; @@ -187,7 +189,7 @@ private: uint64_t max_log_id = 0; /// For compaction, queue of delete not used logs /// 128 is enough, even if log is not removed, it's not a problem - ConcurrentBoundedQueue log_files_to_delete_queue{128}; + ConcurrentBoundedQueue> log_files_to_delete_queue{128}; ThreadFromGlobalPool clean_log_thread; struct AppendLog diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index ff3f96f199b..bacafe75d13 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -21,6 +21,12 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) disk_selector->initialize(config, "storage_configuration.disks", Context::getGlobalContextInstance()); log_storage = getLogsPathFromConfig(config); + + if (config.has("keeper_server.current_log_storage_disk")) + current_log_storage = config.getString("keeper_server.current_log_storage_disk"); + else + current_log_storage = log_storage; + snapshot_storage = getSnapshotsPathFromConfig(config); state_file_storage = getStatePathFromConfig(config); @@ -57,7 +63,6 @@ DiskPtr KeeperContext::getDisk(const Storage & storage) const return *storage_disk; const auto & disk_name = std::get(storage); - return disk_selector->get(disk_name); } @@ -66,6 +71,11 @@ DiskPtr KeeperContext::getLogDisk() const return getDisk(log_storage); } +DiskPtr KeeperContext::getCurrentLogDisk() const +{ + return getDisk(current_log_storage); +} + DiskPtr KeeperContext::getSnapshotsDisk() const { return getDisk(snapshot_storage); diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 1fc01f12bba..aec2061bddf 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -32,6 +32,7 @@ public: bool digestEnabled() const; void setDigestEnabled(bool digest_enabled_); + DiskPtr getCurrentLogDisk() const; DiskPtr getLogDisk() const; DiskPtr getSnapshotsDisk() const; DiskPtr getStateFileDisk() const; @@ -53,6 +54,7 @@ private: std::shared_ptr disk_selector; Storage log_storage; + Storage current_log_storage; Storage snapshot_storage; Storage state_file_storage; diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index d47ea475c42..bfadf3af9aa 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -520,19 +520,41 @@ KeeperSnapshotManager::KeeperSnapshotManager( , keeper_context(keeper_context_) { auto disk = getDisk(); + + std::unordered_set invalid_snapshots; + /// collect invalid snapshots for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) { const auto & name = it->name(); if (name.empty()) continue; + if (startsWith(name, "tmp_")) + { + disk->removeFile(it->path()); + invalid_snapshots.insert(name.substr(4)); + continue; + } + + } + + /// process snapshots + for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) + { + const auto & name = it->name(); + if (name.empty()) + continue; + + /// Not snapshot file + if (!startsWith(name, "snapshot_")) + continue; + + if (invalid_snapshots.contains(name)) { disk->removeFile(it->path()); continue; } - /// Not snapshot file - if (!startsWith(name, "snapshot_")) - continue; + size_t snapshot_up_to = getSnapshotPathUpToLogIdx(name); existing_snapshots[snapshot_up_to] = it->path(); } @@ -549,11 +571,16 @@ std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; auto disk = getDisk(); - auto plain_buf = disk->writeFile(tmp_snapshot_file_name); + + { + disk->writeFile(tmp_snapshot_file_name); + } + + auto plain_buf = disk->writeFile(snapshot_file_name); copyData(reader, *plain_buf); plain_buf->sync(); - disk->moveFile(tmp_snapshot_file_name, snapshot_file_name); + disk->removeFile(tmp_snapshot_file_name); existing_snapshots.emplace(up_to_log_idx, snapshot_file_name); removeOutdatedSnapshotsIfNeeded(); @@ -673,7 +700,12 @@ std::pair KeeperSnapshotManager::serializeSnapshot auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; - auto writer = getDisk()->writeFile(tmp_snapshot_file_name); + auto disk = getDisk(); + { + disk->writeFile(tmp_snapshot_file_name); + } + + auto writer = disk->writeFile(snapshot_file_name); std::unique_ptr compressed_writer; if (compress_snapshots_zstd) compressed_writer = wrapWriteBufferWithCompressionMethod(std::move(writer), CompressionMethod::Zstd, 3); @@ -688,7 +720,8 @@ std::pair KeeperSnapshotManager::serializeSnapshot try { - getDisk()->moveFile(tmp_snapshot_file_name, snapshot_file_name); + std::cout << "Removing file " << tmp_snapshot_file_name << std::endl; + disk->removeFile(tmp_snapshot_file_name); } catch (fs::filesystem_error & e) { diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 352e67e7c43..6c1b14bc94f 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -385,7 +385,7 @@ nuraft::ptr KeeperStateManager::read_state() } auto state = nuraft::srv_state::deserialize(*state_buf); - LOG_INFO(logger, "Read state from {}", disk->getPath() + path); + LOG_INFO(logger, "Read state from {}", fs::path(disk->getPath()) / path); return state; } catch (const std::exception & e) @@ -408,7 +408,6 @@ nuraft::ptr KeeperStateManager::read_state() if (state) { disk->removeFileIfExists(old_path); - return state; } diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index 667fabf4d66..84083169df3 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -28,6 +28,10 @@ namespace CurrentMetrics namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} struct ContextSharedPart : boost::noncopyable { diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index c1f9fa00f2a..7711ed7465b 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -381,6 +381,11 @@ void SettingFieldMap::readBinary(ReadBuffer & in) #else +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + SettingFieldMap::SettingFieldMap(const Field &) : value(Map()) {} String SettingFieldMap::toString() const { diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index a680a344746..650fde7bcd1 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -120,6 +120,7 @@ const IMetadataStorage & MetadataStorageFromPlainObjectStorageTransaction::getSt void MetadataStorageFromPlainObjectStorageTransaction::unlinkFile(const std::string & path) { auto object = StoredObject(metadata_storage.getAbsolutePath(path)); + std::cout << "Removing from plain " << path << std::endl; metadata_storage.object_storage->removeObject(object); } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index 0beed65879b..fb5b6d0757c 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -63,7 +63,7 @@ public: uint32_t getHardlinkCount(const std::string & /* path */) const override { - return 1; + return 0; } bool supportsChmod() const override { return false; } diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp index 406de532216..81be8d8ce4d 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp @@ -1,5 +1,6 @@ #include #include +#include "IO/ReadBufferFromFileBase.h" #include namespace DB @@ -14,11 +15,13 @@ ZstdDeflatingAppendableWriteBuffer::ZstdDeflatingAppendableWriteBuffer( std::unique_ptr out_, int compression_level, bool append_to_existing_file_, + std::function()> read_buffer_creator_, size_t buf_size, char * existing_memory, size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment) , out(std::move(out_)) + , read_buffer_creator(std::move(read_buffer_creator_)) , append_to_existing_file(append_to_existing_file_) { cctx = ZSTD_createCCtx(); @@ -194,13 +197,13 @@ void ZstdDeflatingAppendableWriteBuffer::addEmptyBlock() bool ZstdDeflatingAppendableWriteBuffer::isNeedToAddEmptyBlock() { - ReadBufferFromFile reader(out->getFileName()); - auto fsize = reader.getFileSize(); + auto reader = read_buffer_creator(); + auto fsize = reader->getFileSize(); if (fsize > 3) { std::array result; - reader.seek(fsize - 3, SEEK_SET); - reader.readStrict(result.data(), 3); + reader->seek(fsize - 3, SEEK_SET); + reader->readStrict(result.data(), 3); /// If we don't have correct block in the end, then we need to add it manually. /// NOTE: maybe we can have the same bytes in case of data corruption/unfinished write. diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.h b/src/IO/ZstdDeflatingAppendableWriteBuffer.h index b64e5d5c4cf..d9c4f32d6da 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.h +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.h @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -32,6 +33,7 @@ public: std::unique_ptr out_, int compression_level, bool append_to_existing_file_, + std::function()> read_buffer_creator_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0); @@ -69,6 +71,7 @@ private: void addEmptyBlock(); std::unique_ptr out; + std::function()> read_buffer_creator; bool append_to_existing_file = false; ZSTD_CCtx * cctx; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 0ec39f18757..84e42830dee 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1235,7 +1235,7 @@ struct HTTPContext : public IHTTPContext } -#else +#else #include From 5db21607627cf244ce40cf12d8be3dd4430c218a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 May 2023 09:04:12 +0000 Subject: [PATCH 078/515] Fix build --- programs/keeper-converter/KeeperConverter.cpp | 3 +- src/Coordination/KeeperContext.cpp | 18 +- src/Coordination/KeeperContext.h | 7 +- src/Coordination/KeeperDispatcher.cpp | 44 +- src/Coordination/KeeperServer.cpp | 3 +- src/Coordination/KeeperServer.h | 1 + src/Coordination/KeeperSnapshotManager.cpp | 2 +- src/Coordination/tests/gtest_coordination.cpp | 463 ++++++++++++------ 8 files changed, 358 insertions(+), 183 deletions(-) diff --git a/programs/keeper-converter/KeeperConverter.cpp b/programs/keeper-converter/KeeperConverter.cpp index 58d090ca8b9..f2389021cb6 100644 --- a/programs/keeper-converter/KeeperConverter.cpp +++ b/programs/keeper-converter/KeeperConverter.cpp @@ -42,6 +42,7 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv) { auto keeper_context = std::make_shared(true); keeper_context->setDigestEnabled(true); + keeper_context->setSnapshotDisk(std::make_shared("Keeper-snapshots", options["output-dir"].as(), 0)); DB::KeeperStorage storage(/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false); @@ -52,7 +53,7 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv) DB::SnapshotMetadataPtr snapshot_meta = std::make_shared(storage.getZXID(), 1, std::make_shared()); DB::KeeperStorageSnapshot snapshot(&storage, snapshot_meta); - DB::KeeperSnapshotManager manager(std::make_shared("Keeper-snapshots", options["output-dir"].as(), 0), 1, keeper_context); + DB::KeeperSnapshotManager manager(1, keeper_context); auto snp = manager.serializeSnapshotToBuffer(snapshot); auto path = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID()); std::cout << "Snapshot serialized to path:" << path << std::endl; diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index bacafe75d13..3c6411a3a24 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -76,16 +76,32 @@ DiskPtr KeeperContext::getCurrentLogDisk() const return getDisk(current_log_storage); } -DiskPtr KeeperContext::getSnapshotsDisk() const +void KeeperContext::setLogDisk(DiskPtr disk) +{ + log_storage = disk; + current_log_storage = std::move(disk); +} + +DiskPtr KeeperContext::getSnapshotDisk() const { return getDisk(snapshot_storage); } +void KeeperContext::setSnapshotDisk(DiskPtr disk) +{ + snapshot_storage = std::move(disk); +} + DiskPtr KeeperContext::getStateFileDisk() const { return getDisk(state_file_storage); } +void KeeperContext::setStateFileDisk(DiskPtr disk) +{ + state_file_storage = std::move(disk); +} + KeeperContext::Storage KeeperContext::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const { const auto create_local_disk = [](const auto & path) diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index aec2061bddf..4e538c99649 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -34,8 +34,13 @@ public: DiskPtr getCurrentLogDisk() const; DiskPtr getLogDisk() const; - DiskPtr getSnapshotsDisk() const; + void setLogDisk(DiskPtr disk); + + DiskPtr getSnapshotDisk() const; + void setSnapshotDisk(DiskPtr disk); + DiskPtr getStateFileDisk() const; + void setStateFileDisk(DiskPtr disk); private: /// local disk defined using path or disk name using Storage = std::variant; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 6632e58782f..17a15067301 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -336,28 +336,36 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf snapshot_s3.startup(config, macros); - server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, snapshot_s3, [this](const KeeperStorage::RequestForSession & request_for_session) - { - /// check if we have queue of read requests depending on this request to be committed - std::lock_guard lock(read_request_queue_mutex); - if (auto it = read_request_queue.find(request_for_session.session_id); it != read_request_queue.end()) + server = std::make_unique( + configuration_and_settings, + config, + responses_queue, + snapshots_queue, + standalone_keeper, + snapshot_s3, + [this](const KeeperStorage::RequestForSession & request_for_session) { - auto & xid_to_request_queue = it->second; - - if (auto request_queue_it = xid_to_request_queue.find(request_for_session.request->xid); request_queue_it != xid_to_request_queue.end()) + /// check if we have queue of read requests depending on this request to be committed + std::lock_guard lock(read_request_queue_mutex); + if (auto it = read_request_queue.find(request_for_session.session_id); it != read_request_queue.end()) { - for (const auto & read_request : request_queue_it->second) - { - if (server->isLeaderAlive()) - server->putLocalReadRequest(read_request); - else - addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); - } + auto & xid_to_request_queue = it->second; - xid_to_request_queue.erase(request_queue_it); + if (auto request_queue_it = xid_to_request_queue.find(request_for_session.request->xid); + request_queue_it != xid_to_request_queue.end()) + { + for (const auto & read_request : request_queue_it->second) + { + if (server->isLeaderAlive()) + server->putLocalReadRequest(read_request); + else + addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); + } + + xid_to_request_queue.erase(request_queue_it); + } } - } - }); + }); try { diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 4cacf566df6..31c91e2de80 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -108,13 +108,14 @@ KeeperServer::KeeperServer( const Poco::Util::AbstractConfiguration & config, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, + bool standalone_keeper, KeeperSnapshotManagerS3 & snapshot_manager_s3, KeeperStateMachine::CommitCallback commit_callback) : server_id(configuration_and_settings_->server_id) , coordination_settings(configuration_and_settings_->coordination_settings) , log(&Poco::Logger::get("KeeperServer")) , is_recovering(config.getBool("keeper_server.force_recovery", false)) - , keeper_context{std::make_shared(true)} + , keeper_context{std::make_shared(standalone_keeper)} , create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true)) { if (coordination_settings->quorum_reads) diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index db4e9c1962e..63f9cc2bcea 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -72,6 +72,7 @@ public: const Poco::Util::AbstractConfiguration & config_, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, + bool standalone_keeper, KeeperSnapshotManagerS3 & snapshot_manager_s3, KeeperStateMachine::CommitCallback commit_callback); diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index bfadf3af9aa..7d808e88b3d 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -676,7 +676,7 @@ SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot() DiskPtr KeeperSnapshotManager::getDisk() const { - return keeper_context->getSnapshotsDisk(); + return keeper_context->getSnapshotDisk(); } void KeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded() diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index de5f2da262b..d3783a83bb0 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -2,35 +2,37 @@ #include #include "Common/ZooKeeper/IKeeper.h" -#include "Coordination/KeeperContext.h" -#include "Coordination/KeeperStorage.h" #include "Core/Defines.h" -#include "IO/WriteHelpers.h" #include "config.h" #if USE_NURAFT +#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 +#include #include #include -#include #include -#include -#include -#include -#include -#include -#include -#include + +#include #include @@ -39,9 +41,7 @@ struct ChangelogDirTest { std::string path; bool drop; - explicit ChangelogDirTest(std::string path_, bool drop_ = true) - : path(path_) - , drop(drop_) + explicit ChangelogDirTest(std::string path_, bool drop_ = true) : path(path_), drop(drop_) { if (fs::exists(path)) { @@ -66,8 +66,20 @@ struct CompressionParam class CoordinationTest : public ::testing::TestWithParam { protected: - DB::KeeperContextPtr keeper_context = std::make_shared(); + DB::KeeperContextPtr keeper_context = std::make_shared(true); Poco::Logger * log{&Poco::Logger::get("CoordinationTest")}; + + void setLogDirectory(const std::string & path) { keeper_context->setLogDisk(std::make_shared("LogDisk", path, 0)); } + + void setSnapshotDirectory(const std::string & path) + { + keeper_context->setSnapshotDisk(std::make_shared("SnapshotDisk", path, 0)); + } + + void setStateFileDirectory(const std::string & path) + { + keeper_context->setStateFileDisk(std::make_shared("StateFile", path, 0)); + } }; TEST_P(CoordinationTest, BuildTest) @@ -113,13 +125,14 @@ TEST_P(CoordinationTest, BufferSerde) template struct SimpliestRaftServer { - SimpliestRaftServer(int server_id_, const std::string & hostname_, int port_, const std::string & logs_path, const std::string & state_path) + SimpliestRaftServer( + int server_id_, const std::string & hostname_, int port_, DB::KeeperContextPtr keeper_context) : server_id(server_id_) , hostname(hostname_) , port(port_) , endpoint(hostname + ":" + std::to_string(port)) , state_machine(nuraft::cs_new()) - , state_manager(nuraft::cs_new(server_id, hostname, port, logs_path, state_path)) + , state_manager(nuraft::cs_new(server_id, hostname, port, keeper_context)) { state_manager->loadLogStore(1, 0); nuraft::raft_params params; @@ -135,8 +148,13 @@ struct SimpliestRaftServer nuraft::raft_server::init_options opts; opts.start_server_in_constructor_ = false; raft_instance = launcher.init( - state_machine, state_manager, nuraft::cs_new("ToyRaftLogger", DB::LogsLevel::trace), port, - nuraft::asio_service::options{}, params, opts); + state_machine, + state_manager, + nuraft::cs_new("ToyRaftLogger", DB::LogsLevel::trace), + port, + nuraft::asio_service::options{}, + params, + opts); if (!raft_instance) { @@ -195,15 +213,14 @@ nuraft::ptr getBuffer(int64_t number) return ret; } - TEST_P(CoordinationTest, TestSummingRaft1) { ChangelogDirTest test("./logs"); - SummingRaftServer s1(1, "localhost", 44444, "./logs", "./state"); - SCOPE_EXIT( - if (std::filesystem::exists("./state")) - std::filesystem::remove("./state"); - ); + setLogDirectory("./logs"); + setStateFileDirectory("."); + + SummingRaftServer s1(1, "localhost", 44444, keeper_context); + SCOPE_EXIT(if (std::filesystem::exists("./state")) std::filesystem::remove("./state");); /// Single node is leader EXPECT_EQ(s1.raft_instance->get_leader(), 1); @@ -235,8 +252,10 @@ TEST_P(CoordinationTest, ChangelogTestSimple) { auto params = GetParam(); ChangelogDirTest test("./logs"); + setLogDirectory("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello world", 77); changelog.append(entry); @@ -263,7 +282,10 @@ TEST_P(CoordinationTest, ChangelogTestFile) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello world", 77); changelog.append(entry); @@ -292,7 +314,10 @@ TEST_P(CoordinationTest, ChangelogReadWrite) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) @@ -306,7 +331,8 @@ TEST_P(CoordinationTest, ChangelogReadWrite) waitDurableLogs(changelog); - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), 10); EXPECT_EQ(changelog_reader.last_entry()->get_term(), changelog.last_entry()->get_term()); @@ -326,7 +352,10 @@ TEST_P(CoordinationTest, ChangelogWriteAt) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) { @@ -348,7 +377,8 @@ TEST_P(CoordinationTest, ChangelogWriteAt) EXPECT_EQ(changelog.entry_at(7)->get_term(), 77); EXPECT_EQ(changelog.next_slot(), 8); - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), changelog.size()); @@ -362,7 +392,10 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 7; ++i) { @@ -378,7 +411,8 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), 7); @@ -395,7 +429,7 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); size_t logs_count = 0; - for (const auto & _ [[maybe_unused]]: fs::directory_iterator("./logs")) + for (const auto & _ [[maybe_unused]] : fs::directory_iterator("./logs")) logs_count++; EXPECT_EQ(logs_count, 2); @@ -412,7 +446,7 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); logs_count = 0; - for (const auto & _ [[maybe_unused]]: fs::directory_iterator("./logs")) + for (const auto & _ [[maybe_unused]] : fs::directory_iterator("./logs")) logs_count++; EXPECT_EQ(logs_count, 3); @@ -440,7 +474,10 @@ TEST_P(CoordinationTest, ChangelogTestCompaction) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 3; ++i) @@ -489,7 +526,8 @@ TEST_P(CoordinationTest, ChangelogTestCompaction) EXPECT_EQ(changelog.next_slot(), 8); EXPECT_EQ(changelog.last_entry()->get_term(), 60); /// And we able to read it - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog_reader.init(7, 0); EXPECT_EQ(changelog_reader.size(), 1); @@ -502,7 +540,10 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperations) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) { @@ -517,7 +558,8 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperations) auto entries = changelog.pack(1, 5); - DB::KeeperLogStore apply_changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore apply_changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); apply_changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) @@ -549,23 +591,31 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperationsEmpty) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); - changelog.init(1, 0); - for (size_t i = 0; i < 10; ++i) + setLogDirectory("./logs"); + + nuraft::ptr entries; { - auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - changelog.append(entry); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + changelog.init(1, 0); + for (size_t i = 0; i < 10; ++i) + { + auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); + changelog.append(entry); + } + changelog.end_of_append_batch(0, 0); + + EXPECT_EQ(changelog.size(), 10); + + waitDurableLogs(changelog); + + entries = changelog.pack(5, 5); } - changelog.end_of_append_batch(0, 0); - - EXPECT_EQ(changelog.size(), 10); - - waitDurableLogs(changelog); - - auto entries = changelog.pack(5, 5); ChangelogDirTest test1("./logs1"); - DB::KeeperLogStore changelog_new("./logs1", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + setLogDirectory("./logs1"); + DB::KeeperLogStore changelog_new( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog_new.init(1, 0); EXPECT_EQ(changelog_new.size(), 0); @@ -587,7 +637,8 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperationsEmpty) EXPECT_EQ(changelog_new.start_index(), 5); EXPECT_EQ(changelog_new.next_slot(), 11); - DB::KeeperLogStore changelog_reader("./logs1", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog_reader.init(5, 0); } @@ -596,7 +647,10 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtPreviousFile) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 33; ++i) @@ -637,7 +691,8 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtPreviousFile) EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin" + params.extension)); EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); - DB::KeeperLogStore changelog_read("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog_read( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog_read.init(1, 0); EXPECT_EQ(changelog_read.size(), 7); EXPECT_EQ(changelog_read.start_index(), 1); @@ -649,7 +704,10 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtFileBorder) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 33; ++i) @@ -690,7 +748,8 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtFileBorder) EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin" + params.extension)); EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); - DB::KeeperLogStore changelog_read("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog_read( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog_read.init(1, 0); EXPECT_EQ(changelog_read.size(), 11); EXPECT_EQ(changelog_read.start_index(), 1); @@ -702,7 +761,10 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtAllFiles) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 33; ++i) { @@ -747,7 +809,10 @@ TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead) { auto params = GetParam(); ChangelogDirTest test("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -768,7 +833,8 @@ TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead) EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); EXPECT_FALSE(fs::exists("./logs/changelog_36_40.bin" + params.extension)); - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog_reader.init(1, 0); auto entry = getLogEntry("36_hello_world", 360); @@ -792,16 +858,16 @@ namespace { void assertBrokenLogRemoved(const fs::path & log_folder, const fs::path & filename) { - EXPECT_FALSE(fs::exists(log_folder / filename)); - // broken logs are sent to the detached/{timestamp} folder - // we don't know timestamp so we iterate all of them - for (const auto & dir_entry : fs::recursive_directory_iterator(log_folder / "detached")) - { - if (dir_entry.path().filename() == filename) - return; - } + EXPECT_FALSE(fs::exists(log_folder / filename)); + // broken logs are sent to the detached/{timestamp} folder + // we don't know timestamp so we iterate all of them + for (const auto & dir_entry : fs::recursive_directory_iterator(log_folder / "detached")) + { + if (dir_entry.path().filename() == filename) + return; + } - FAIL() << "Broken log " << filename << " was not moved to the detached folder"; + FAIL() << "Broken log " << filename << " was not moved to the detached folder"; } } @@ -812,8 +878,10 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) auto params = GetParam(); ChangelogDirTest test(log_folder); + setLogDirectory(log_folder); - DB::KeeperLogStore changelog(log_folder, DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -833,10 +901,12 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); - DB::WriteBufferFromFile plain_buf("./logs/changelog_11_15.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + DB::WriteBufferFromFile plain_buf( + "./logs/changelog_11_15.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); plain_buf.truncate(0); - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog_reader.init(1, 0); changelog_reader.end_of_append_batch(0, 0); @@ -869,7 +939,8 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) assertBrokenLogRemoved(log_folder, "changelog_26_30.bin" + params.extension); assertBrokenLogRemoved(log_folder, "changelog_31_35.bin" + params.extension); - DB::KeeperLogStore changelog_reader2("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog_reader2( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog_reader2.init(1, 0); EXPECT_EQ(changelog_reader2.size(), 11); EXPECT_EQ(changelog_reader2.last_entry()->get_term(), 7777); @@ -879,8 +950,10 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) { auto params = GetParam(); ChangelogDirTest test("./logs"); + setLogDirectory("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -894,10 +967,12 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin" + params.extension)); - DB::WriteBufferFromFile plain_buf("./logs/changelog_1_20.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + DB::WriteBufferFromFile plain_buf( + "./logs/changelog_1_20.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); plain_buf.truncate(30); - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), 0); @@ -912,7 +987,8 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) EXPECT_EQ(changelog_reader.size(), 1); EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777); - DB::KeeperLogStore changelog_reader2("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1}); + DB::KeeperLogStore changelog_reader2( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1}, keeper_context); changelog_reader2.init(1, 0); EXPECT_EQ(changelog_reader2.size(), 1); EXPECT_EQ(changelog_reader2.last_entry()->get_term(), 7777); @@ -922,8 +998,10 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles) { auto params = GetParam(); ChangelogDirTest test("./logs"); + setLogDirectory("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -939,7 +1017,8 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles) fs::remove("./logs/changelog_1_20.bin" + params.extension); - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, keeper_context); /// It should print error message, but still able to start changelog_reader.init(5, 0); assertBrokenLogRemoved("./logs", "changelog_21_40.bin" + params.extension); @@ -949,8 +1028,10 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles2) { auto params = GetParam(); ChangelogDirTest test("./logs"); + setLogDirectory("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}, keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -970,7 +1051,8 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles2) // we have a gap in our logs, we need to remove all the logs after the gap fs::remove("./logs/changelog_21_30.bin" + params.extension); - DB::KeeperLogStore changelog_reader("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}); + DB::KeeperLogStore changelog_reader( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}, keeper_context); /// It should print error message, but still able to start changelog_reader.init(5, 0); EXPECT_TRUE(fs::exists("./logs/changelog_1_10.bin" + params.extension)); @@ -1155,7 +1237,7 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) EXPECT_EQ(world.getApproximateDataSize(), 0); } -void addNode(DB::KeeperStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner=0) +void addNode(DB::KeeperStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner = 0) { using Node = DB::KeeperStorage::Node; Node node{}; @@ -1163,19 +1245,23 @@ void addNode(DB::KeeperStorage & storage, const std::string & path, const std::s node.stat.ephemeralOwner = ephemeral_owner; storage.container.insertOrReplace(path, node); auto child_it = storage.container.find(path); - auto child_path = DB::getBaseName(child_it->key); - storage.container.updateValue(DB::parentPath(StringRef{path}), [&](auto & parent) - { - parent.addChild(child_path); - parent.stat.numChildren++; - }); + auto child_path = DB::getBaseNodeName(child_it->key); + storage.container.updateValue( + DB::parentNodePath(StringRef{path}), + [&](auto & parent) + { + parent.addChild(child_path); + parent.stat.numChildren++; + }); } TEST_P(CoordinationTest, TestStorageSnapshotSimple) { auto params = GetParam(); ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3, keeper_context, params.enable_compression); + setSnapshotDirectory("./snapshots"); + + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); DB::KeeperStorage storage(500, "", keeper_context); addNode(storage, "/hello", "world", 1); @@ -1223,7 +1309,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) { auto params = GetParam(); ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3, keeper_context, params.enable_compression); + setSnapshotDirectory("./snapshots"); + + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); DB::KeeperStorage storage(500, "", keeper_context); storage.getSessionID(130); @@ -1264,7 +1352,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) { auto params = GetParam(); ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3, keeper_context, params.enable_compression); + setSnapshotDirectory("./snapshots"); + + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); DB::KeeperStorage storage(500, "", keeper_context); storage.getSessionID(130); @@ -1303,7 +1393,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) { auto params = GetParam(); ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3, keeper_context, params.enable_compression); + setSnapshotDirectory("./snapshots"); + + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); DB::KeeperStorage storage(500, "", keeper_context); for (size_t i = 0; i < 50; ++i) { @@ -1349,14 +1441,15 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) { EXPECT_EQ(restored_storage->container.getValue("/hello_" + std::to_string(i)).getData(), "world_" + std::to_string(i)); } - } TEST_P(CoordinationTest, TestStorageSnapshotBroken) { auto params = GetParam(); ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3, keeper_context, params.enable_compression); + setSnapshotDirectory("./snapshots"); + + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); DB::KeeperStorage storage(500, "", keeper_context); for (size_t i = 0; i < 50; ++i) { @@ -1370,7 +1463,8 @@ TEST_P(CoordinationTest, TestStorageSnapshotBroken) EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + params.extension)); /// Let's corrupt file - DB::WriteBufferFromFile plain_buf("./snapshots/snapshot_50.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + DB::WriteBufferFromFile plain_buf( + "./snapshots/snapshot_50.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); plain_buf.truncate(34); plain_buf.sync(); @@ -1390,25 +1484,35 @@ nuraft::ptr getBufferFromZKRequest(int64_t session_id, int64_t z return buf.getBuffer(); } -nuraft::ptr getLogEntryFromZKRequest(size_t term, int64_t session_id, int64_t zxid, const Coordination::ZooKeeperRequestPtr & request) +nuraft::ptr +getLogEntryFromZKRequest(size_t term, int64_t session_id, int64_t zxid, const Coordination::ZooKeeperRequestPtr & request) { auto buffer = getBufferFromZKRequest(session_id, zxid, request); return nuraft::cs_new(term, buffer); } -void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint64_t total_logs, bool enable_compression, Coordination::KeeperContextPtr keeper_context) +void testLogAndStateMachine( + Coordination::CoordinationSettingsPtr settings, + uint64_t total_logs, + bool enable_compression, + Coordination::KeeperContextPtr keeper_context) { using namespace Coordination; using namespace DB; ChangelogDirTest snapshots("./snapshots"); + keeper_context->setSnapshotDisk(std::make_shared("SnapshotDisk", "./snapshots", 0)); ChangelogDirTest logs("./logs"); + keeper_context->setLogDisk(std::make_shared("LogDisk", "./logs", 0)); ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared(queue, snapshots_queue, "./snapshots", settings, keeper_context, nullptr); + auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = enable_compression, .rotate_interval = settings->rotate_log_storage_interval}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{ + .force_sync = true, .compress_logs = enable_compression, .rotate_interval = settings->rotate_log_storage_interval}, + keeper_context); changelog.init(state_machine->last_commit_index() + 1, settings->reserved_log_items); for (size_t i = 1; i < total_logs + 1; ++i) { @@ -1426,7 +1530,8 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint if (i % settings->snapshot_distance == 0) { nuraft::snapshot s(i, 0, std::make_shared()); - nuraft::async_result::handler_type when_done = [&snapshot_created] (bool & ret, nuraft::ptr &/*exception*/) + nuraft::async_result::handler_type when_done + = [&snapshot_created](bool & ret, nuraft::ptr & /*exception*/) { snapshot_created = ret; LOG_INFO(&Poco::Logger::get("CoordinationTest"), "Snapshot finished"); @@ -1444,17 +1549,21 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint } SnapshotsQueue snapshots_queue1{1}; - auto restore_machine = std::make_shared(queue, snapshots_queue1, "./snapshots", settings, keeper_context, nullptr); + auto restore_machine = std::make_shared(queue, snapshots_queue1, settings, keeper_context, nullptr); restore_machine->init(); EXPECT_EQ(restore_machine->last_commit_index(), total_logs - total_logs % settings->snapshot_distance); - DB::KeeperLogStore restore_changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = enable_compression, .rotate_interval = settings->rotate_log_storage_interval}); + DB::KeeperLogStore restore_changelog( + DB::LogFileSettings{ + .force_sync = true, .compress_logs = enable_compression, .rotate_interval = settings->rotate_log_storage_interval}, + keeper_context); restore_changelog.init(restore_machine->last_commit_index() + 1, settings->reserved_log_items); EXPECT_EQ(restore_changelog.size(), std::min(settings->reserved_log_items + total_logs % settings->snapshot_distance, total_logs)); EXPECT_EQ(restore_changelog.next_slot(), total_logs + 1); if (total_logs > settings->reserved_log_items + 1) - EXPECT_EQ(restore_changelog.start_index(), total_logs - total_logs % settings->snapshot_distance - settings->reserved_log_items + 1); + EXPECT_EQ( + restore_changelog.start_index(), total_logs - total_logs % settings->snapshot_distance - settings->reserved_log_items + 1); else EXPECT_EQ(restore_changelog.start_index(), 1); @@ -1552,11 +1661,13 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove) using namespace DB; ChangelogDirTest snapshots("./snapshots"); + setSnapshotDirectory("./snapshots"); + CoordinationSettingsPtr settings = std::make_shared(); ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared(queue, snapshots_queue, "./snapshots", settings, keeper_context, nullptr); + auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); std::shared_ptr request_c = std::make_shared(); @@ -1585,11 +1696,12 @@ TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitte using namespace DB; ChangelogDirTest snapshots("./snapshots"); + setSnapshotDirectory("./snapshots"); CoordinationSettingsPtr settings = std::make_shared(); ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared(queue, snapshots_queue, "./snapshots", settings, keeper_context, nullptr); + auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); String user_auth_data = "test_user:test_password"; @@ -1635,11 +1747,13 @@ TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) using namespace DB; ChangelogDirTest snapshots("./snapshots"); + setSnapshotDirectory("./snapshots"); + CoordinationSettingsPtr settings = std::make_shared(); ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared(queue, snapshots_queue, "./snapshots", settings, keeper_context, nullptr); + auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); String user_auth_data = "test_user:test_password"; @@ -1691,8 +1805,10 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) using namespace Coordination; auto params = GetParam(); ChangelogDirTest snapshots("./logs"); + setLogDirectory("./logs"); { - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(0, 3); for (size_t i = 1; i < 55; ++i) @@ -1710,7 +1826,8 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) EXPECT_TRUE(fs::exists("./logs/changelog_1_100.bin" + params.extension)); - DB::KeeperLogStore changelog_1("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}); + DB::KeeperLogStore changelog_1( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}, keeper_context); changelog_1.init(0, 50); for (size_t i = 0; i < 55; ++i) { @@ -1726,7 +1843,8 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) EXPECT_TRUE(fs::exists("./logs/changelog_1_100.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_101_110.bin" + params.extension)); - DB::KeeperLogStore changelog_2("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 7}); + DB::KeeperLogStore changelog_2( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 7}, keeper_context); changelog_2.init(98, 55); for (size_t i = 0; i < 17; ++i) @@ -1749,7 +1867,8 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) EXPECT_TRUE(fs::exists("./logs/changelog_118_124.bin" + params.extension)); EXPECT_TRUE(fs::exists("./logs/changelog_125_131.bin" + params.extension)); - DB::KeeperLogStore changelog_3("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}); + DB::KeeperLogStore changelog_3( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); changelog_3.init(116, 3); for (size_t i = 0; i < 17; ++i) { @@ -1796,8 +1915,10 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) { using namespace Coordination; auto test_params = GetParam(); - ChangelogDirTest snapshots("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}); + ChangelogDirTest logs("./logs"); + setLogDirectory("./logs"); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(0, 3); for (size_t i = 1; i < 55; ++i) @@ -1811,7 +1932,8 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) waitDurableLogs(changelog); - DB::KeeperLogStore changelog1("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog1( + DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, keeper_context); changelog1.init(0, 3); for (size_t i = 55; i < 70; ++i) { @@ -1822,7 +1944,8 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) changelog1.end_of_append_batch(0, 0); } - DB::KeeperLogStore changelog2("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog2( + DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, keeper_context); changelog2.init(0, 3); for (size_t i = 70; i < 80; ++i) { @@ -1839,7 +1962,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) auto params = GetParam(); ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3, keeper_context, params.enable_compression); + setSnapshotDirectory("./snapshots"); + + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); DB::KeeperStorage storage(500, "", keeper_context); addNode(storage, "/hello", "world", 1); @@ -1857,7 +1982,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) manager.serializeSnapshotBufferToDisk(*buf, 2); EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin" + params.extension)); - DB::KeeperSnapshotManager new_manager("./snapshots", 3, keeper_context, !params.enable_compression); + DB::KeeperSnapshotManager new_manager(3, keeper_context, !params.enable_compression); auto debuf = new_manager.deserializeSnapshotBufferFromDisk(2); @@ -1883,9 +2008,10 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) { auto params = GetParam(); ChangelogDirTest test("./logs"); + setLogDirectory("./logs"); { LOG_INFO(log, "================First time====================="); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog(DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog.append(entry); @@ -1896,7 +2022,8 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) { LOG_INFO(log, "================Second time====================="); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog.append(entry); @@ -1907,7 +2034,8 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) { LOG_INFO(log, "================Third time====================="); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog.append(entry); @@ -1918,7 +2046,8 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) { LOG_INFO(log, "================Fourth time====================="); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog.append(entry); @@ -1933,10 +2062,12 @@ TEST_P(CoordinationTest, ChangelogInsertMultipleTimesSmooth) { auto params = GetParam(); ChangelogDirTest test("./logs"); + setLogDirectory("./logs"); for (size_t i = 0; i < 36; ++i) { LOG_INFO(log, "================First time====================="); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(1, 0); for (size_t j = 0; j < 7; ++j) { @@ -1947,7 +2078,8 @@ TEST_P(CoordinationTest, ChangelogInsertMultipleTimesSmooth) waitDurableLogs(changelog); } - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(1, 0); EXPECT_EQ(changelog.next_slot(), 36 * 7 + 1); } @@ -1956,9 +2088,11 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) { auto params = GetParam(); ChangelogDirTest test("./logs"); + setLogDirectory("./logs"); { LOG_INFO(log, "================First time====================="); - DB::KeeperLogStore changelog1("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog1( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog1.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog1.append(entry); @@ -1969,7 +2103,8 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) { LOG_INFO(log, "================Second time====================="); - DB::KeeperLogStore changelog2("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog2( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog2.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog2.append(entry); @@ -1980,7 +2115,8 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) { LOG_INFO(log, "================Third time====================="); - DB::KeeperLogStore changelog3("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog3( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog3.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog3.append(entry); @@ -1991,7 +2127,8 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) { LOG_INFO(log, "================Fourth time====================="); - DB::KeeperLogStore changelog4("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog4( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); changelog4.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog4.append(entry); @@ -2005,10 +2142,12 @@ TEST_P(CoordinationTest, TestStorageSnapshotEqual) { auto params = GetParam(); ChangelogDirTest test("./snapshots"); + setSnapshotDirectory("./snapshots"); + std::optional snapshot_hash; for (size_t i = 0; i < 15; ++i) { - DB::KeeperSnapshotManager manager("./snapshots", 3, keeper_context, params.enable_compression); + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); DB::KeeperStorage storage(500, "", keeper_context); addNode(storage, "/hello", ""); @@ -2048,7 +2187,10 @@ TEST_P(CoordinationTest, TestLogGap) using namespace Coordination; auto test_params = GetParam(); ChangelogDirTest logs("./logs"); - DB::KeeperLogStore changelog("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}); + setLogDirectory("./logs"); + + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, keeper_context); changelog.init(0, 3); for (size_t i = 1; i < 55; ++i) @@ -2060,7 +2202,8 @@ TEST_P(CoordinationTest, TestLogGap) changelog.end_of_append_batch(0, 0); } - DB::KeeperLogStore changelog1("./logs", DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}); + DB::KeeperLogStore changelog1( + DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, keeper_context); changelog1.init(61, 3); /// Logs discarded @@ -2220,13 +2363,13 @@ TEST_P(CoordinationTest, TestListRequestTypes) return create_response.path_created; }; - create_path(parentPath(StringRef{test_path}).toString(), false, false); + create_path(parentNodePath(StringRef{test_path}).toString(), false, false); static constexpr size_t persistent_num = 5; std::unordered_set expected_persistent_children; for (size_t i = 0; i < persistent_num; ++i) { - expected_persistent_children.insert(getBaseName(create_path(test_path, false)).toString()); + expected_persistent_children.insert(getBaseNodeName(create_path(test_path, false)).toString()); } ASSERT_EQ(expected_persistent_children.size(), persistent_num); @@ -2234,7 +2377,7 @@ TEST_P(CoordinationTest, TestListRequestTypes) std::unordered_set expected_ephemeral_children; for (size_t i = 0; i < ephemeral_num; ++i) { - expected_ephemeral_children.insert(getBaseName(create_path(test_path, true)).toString()); + expected_ephemeral_children.insert(getBaseNodeName(create_path(test_path, true)).toString()); } ASSERT_EQ(expected_ephemeral_children.size(), ephemeral_num); @@ -2242,7 +2385,7 @@ TEST_P(CoordinationTest, TestListRequestTypes) { const auto list_request = std::make_shared(); int new_zxid = ++zxid; - list_request->path = parentPath(StringRef{test_path}).toString(); + list_request->path = parentNodePath(StringRef{test_path}).toString(); list_request->list_request_type = list_request_type; storage.preprocessRequest(list_request, 1, 0, new_zxid); auto responses = storage.processRequest(list_request, 1, new_zxid); @@ -2270,20 +2413,23 @@ TEST_P(CoordinationTest, TestListRequestTypes) EXPECT_EQ(all_children.size(), ephemeral_num + persistent_num); for (const auto & child : all_children) { - EXPECT_TRUE(expected_ephemeral_children.contains(child) || expected_persistent_children.contains(child)) << "Missing child " << child; + EXPECT_TRUE(expected_ephemeral_children.contains(child) || expected_persistent_children.contains(child)) + << "Missing child " << child; } } TEST_P(CoordinationTest, TestDurableState) { ChangelogDirTest logs("./logs"); + setLogDirectory("./logs"); + setStateFileDirectory("."); auto state = nuraft::cs_new(); std::optional state_manager; const auto reload_state_manager = [&] { - state_manager.emplace(1, "localhost", 9181, "./logs", "./state"); + state_manager.emplace(1, "localhost", 9181, keeper_context); state_manager->loadLogStore(1, 0); }; @@ -2318,11 +2464,11 @@ TEST_P(CoordinationTest, TestDurableState) write_buf.sync(); write_buf.close(); reload_state_manager(); -#ifdef NDEBUG +# ifdef NDEBUG ASSERT_EQ(state_manager->read_state(), nullptr); -#else +# else ASSERT_THROW(state_manager->read_state(), DB::Exception); -#endif +# endif } { @@ -2396,15 +2542,16 @@ TEST_P(CoordinationTest, ChangelogTestMaxLogSize) { auto params = GetParam(); ChangelogDirTest test("./logs"); + setLogDirectory("./logs"); uint64_t last_entry_index{0}; size_t i{0}; { SCOPED_TRACE("Small rotation interval, big size limit"); DB::KeeperLogStore changelog( - "./logs", DB::LogFileSettings{ - .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20, .max_size = 50 * 1024 * 1024}); + .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20, .max_size = 50 * 1024 * 1024}, + keeper_context); changelog.init(1, 0); for (; i < 100; ++i) @@ -2421,9 +2568,9 @@ TEST_P(CoordinationTest, ChangelogTestMaxLogSize) { SCOPED_TRACE("Large rotation interval, small size limit"); DB::KeeperLogStore changelog( - "./logs", DB::LogFileSettings{ - .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100'000, .max_size = 4000}); + .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100'000, .max_size = 4000}, + keeper_context); changelog.init(1, 0); ASSERT_EQ(changelog.entry_at(last_entry_index)->get_term(), (i - 1 + 44) * 10); @@ -2442,13 +2589,12 @@ TEST_P(CoordinationTest, ChangelogTestMaxLogSize) { SCOPED_TRACE("Final verify all logs"); DB::KeeperLogStore changelog( - "./logs", DB::LogFileSettings{ - .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100'000, .max_size = 4000}); + .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100'000, .max_size = 4000}, + keeper_context); changelog.init(1, 0); ASSERT_EQ(changelog.entry_at(last_entry_index)->get_term(), (i - 1 + 44) * 10); } - } TEST_P(CoordinationTest, TestCheckNotExistsRequest) @@ -2524,13 +2670,10 @@ TEST_P(CoordinationTest, TestCheckNotExistsRequest) } } -INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite, +INSTANTIATE_TEST_SUITE_P( + CoordinationTestSuite, CoordinationTest, - ::testing::ValuesIn(std::initializer_list{ - CompressionParam{true, ".zstd"}, - CompressionParam{false, ""} - }) -); + ::testing::ValuesIn(std::initializer_list{CompressionParam{true, ".zstd"}, CompressionParam{false, ""}})); int main(int argc, char ** argv) { From 092cf99147293a8dbb45031cf361a03cbcf5777f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 May 2023 09:46:07 +0000 Subject: [PATCH 079/515] Fix build --- programs/keeper/CMakeLists.txt | 2 +- src/Coordination/Changelog.cpp | 134 ++++++++++++------ src/Coordination/KeeperContext.cpp | 20 +++ src/Coordination/KeeperContext.h | 3 + src/Coordination/KeeperSnapshotManager.cpp | 1 - .../MetadataStorageFromPlainObjectStorage.cpp | 1 - utils/keeper-data-dumper/main.cpp | 10 +- 7 files changed, 118 insertions(+), 53 deletions(-) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index a946ea06626..6e97ab324e3 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -143,9 +143,9 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ReadBufferFromRemoteFSGather.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/IOUringReader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteBufferFromTemporaryFile.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteBufferWithFinalizeCallback.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/getThreadPoolReader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolRemoteFSReader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolReader.cpp diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index c94633d6dbd..875b0758d27 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -24,63 +24,66 @@ namespace ErrorCodes extern const int CHECKSUM_DOESNT_MATCH; extern const int CORRUPTED_DATA; extern const int UNKNOWN_FORMAT_VERSION; + extern const int NOT_IMPLEMENTED; + extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; } namespace { - void moveFileBetweenDisks(DiskPtr disk_from, ChangelogFileDescriptionPtr description, DiskPtr disk_to, const std::string & path_to) - { - disk_from->copyFile(description->path, *disk_to, path_to, {}); - disk_from->removeFile(description->path); - description->path = path_to; - description->disk = disk_to; - } - constexpr auto DEFAULT_PREFIX = "changelog"; +void moveFileBetweenDisks(DiskPtr disk_from, ChangelogFileDescriptionPtr description, DiskPtr disk_to, const std::string & path_to) +{ + disk_from->copyFile(description->path, *disk_to, path_to, {}); + disk_from->removeFile(description->path); + description->path = path_to; + description->disk = disk_to; +} - inline std::string - formatChangelogPath(const std::string & name_prefix, uint64_t from_index, uint64_t to_index, const std::string & extension) - { - return fmt::format("{}_{}_{}.{}", name_prefix, from_index, to_index, extension); - } +constexpr auto DEFAULT_PREFIX = "changelog"; - ChangelogFileDescriptionPtr getChangelogFileDescription(const std::filesystem::path & path) - { - // we can have .bin.zstd so we cannot use std::filesystem stem and extension - std::string filename_with_extension = path.filename(); - std::string_view filename_with_extension_view = filename_with_extension; +inline std::string +formatChangelogPath(const std::string & name_prefix, uint64_t from_index, uint64_t to_index, const std::string & extension) +{ + return fmt::format("{}_{}_{}.{}", name_prefix, from_index, to_index, extension); +} - auto first_dot = filename_with_extension.find('.'); - if (first_dot == std::string::npos) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid changelog file {}", path.generic_string()); +ChangelogFileDescriptionPtr getChangelogFileDescription(const std::filesystem::path & path) +{ + // we can have .bin.zstd so we cannot use std::filesystem stem and extension + std::string filename_with_extension = path.filename(); + std::string_view filename_with_extension_view = filename_with_extension; - Strings filename_parts; - boost::split(filename_parts, filename_with_extension_view.substr(0, first_dot), boost::is_any_of("_")); - if (filename_parts.size() < 3) - throw Exception(ErrorCodes::CORRUPTED_DATA, "Invalid changelog {}", path.generic_string()); + auto first_dot = filename_with_extension.find('.'); + if (first_dot == std::string::npos) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid changelog file {}", path.generic_string()); - auto result = std::make_shared(); - result->prefix = filename_parts[0]; - result->from_log_index = parse(filename_parts[1]); - result->to_log_index = parse(filename_parts[2]); - result->extension = std::string(filename_with_extension.substr(first_dot + 1)); - result->path = path.generic_string(); - return result; - } + Strings filename_parts; + boost::split(filename_parts, filename_with_extension_view.substr(0, first_dot), boost::is_any_of("_")); + if (filename_parts.size() < 3) + throw Exception(ErrorCodes::CORRUPTED_DATA, "Invalid changelog {}", path.generic_string()); - Checksum computeRecordChecksum(const ChangelogRecord & record) - { - SipHash hash; - hash.update(record.header.version); - hash.update(record.header.index); - hash.update(record.header.term); - hash.update(record.header.value_type); - hash.update(record.header.blob_size); - if (record.header.blob_size != 0) - hash.update(reinterpret_cast(record.blob->data_begin()), record.blob->size()); - return hash.get64(); - } + auto result = std::make_shared(); + result->prefix = filename_parts[0]; + result->from_log_index = parse(filename_parts[1]); + result->to_log_index = parse(filename_parts[2]); + result->extension = std::string(filename_with_extension.substr(first_dot + 1)); + result->path = path.generic_string(); + return result; +} + +Checksum computeRecordChecksum(const ChangelogRecord & record) +{ + SipHash hash; + hash.update(record.header.version); + hash.update(record.header.index); + hash.update(record.header.term); + hash.update(record.header.value_type); + hash.update(record.header.blob_size); + if (record.header.blob_size != 0) + hash.update(reinterpret_cast(record.blob->data_begin()), record.blob->size()); + return hash.get64(); +} } @@ -571,6 +574,19 @@ Changelog::Changelog(Poco::Logger * log_, LogFileSettings log_file_settings, Kee , append_completion_queue(std::numeric_limits::max()) , keeper_context(std::move(keeper_context_)) { + if (auto current_log_disk = getCurrentLogDisk(); + log_file_settings.force_sync && dynamic_cast(current_log_disk.get()) == nullptr) + { + throw DB::Exception( + DB::ErrorCodes::BAD_ARGUMENTS, + "force_sync is set to true for logs but disk '{}' cannot satisfy such guarantee because it's not of type DiskLocal.\n" + "If you want to use force_sync and same disk for all logs, please set keeper_server.log_storage_disk to a local disk.\n" + "If you want to use force_sync and different disk only for old logs, please set 'keeper_server.log_storage_disk' to any " + "supported disk and 'keeper_server.current_log_storage_disk' to a local disk.\n" + "Otherwise, disable force_sync", + current_log_disk->getName()); + } + /// Load all files on changelog disks const auto load_from_disk = [&](const auto & disk) @@ -590,6 +606,12 @@ Changelog::Changelog(Poco::Logger * log_, LogFileSettings log_file_settings, Kee } }; + /// Load all files from old disks + for (const auto & disk : keeper_context->getOldLogDisks()) + { + load_from_disk(disk); + } + auto disk = getDisk(); load_from_disk(disk); @@ -738,7 +760,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin assert(existing_changelogs.find(last_log_read_result->log_start_index) != existing_changelogs.end()); assert(existing_changelogs.find(last_log_read_result->log_start_index)->first == existing_changelogs.rbegin()->first); - /// Continue to write into incomplete existing log if it doesn't finish with error + /// Continue to write into incomplete existing log if it didn't finish with error const auto & description = existing_changelogs[last_log_read_result->log_start_index]; if (last_log_read_result->last_read_index == 0 || last_log_read_result->error) /// If it's broken log then remove it @@ -755,7 +777,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin } else if (last_log_read_result.has_value()) { - /// check if we need to move it to another disk + /// check if we need to move completed log to another disk auto current_log_disk = getCurrentLogDisk(); auto disk = getDisk(); @@ -768,6 +790,24 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin if (!current_writer->isFileSet()) current_writer->rotate(max_log_id + 1); + /// Move files to correct disks + auto latest_start_index = current_writer->getStartIndex(); + auto current_log_disk = getCurrentLogDisk(); + auto disk = getDisk(); + for (const auto & [start_index, description] : existing_changelogs) + { + /// latest log should already be on current_log_disk + if (start_index == latest_start_index) + { + chassert(description->disk == current_log_disk); + continue; + } + + if (description->disk != disk) + moveFileBetweenDisks(description->disk, description, disk, description->path); + } + + initialized = true; } diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 3c6411a3a24..e27cfc60cff 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -27,6 +28,14 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) else current_log_storage = log_storage; + Poco::Util::AbstractConfiguration::Keys old_log_disk_name_keys; + config.keys("keeper_server", old_log_disk_name_keys); + for (const auto & key : old_log_disk_name_keys) + { + if (key.starts_with("old_log_storage_disk")) + old_log_disk_names.push_back(config.getString("keeper_server." + key)); + } + snapshot_storage = getSnapshotsPathFromConfig(config); state_file_storage = getStatePathFromConfig(config); @@ -71,6 +80,17 @@ DiskPtr KeeperContext::getLogDisk() const return getDisk(log_storage); } +std::vector KeeperContext::getOldLogDisks() const +{ + std::vector old_log_disks; + old_log_disks.reserve(old_log_disk_names.size()); + + for (const auto & disk_name : old_log_disk_names) + old_log_disks.push_back(disk_selector->get(disk_name)); + + return old_log_disks; +} + DiskPtr KeeperContext::getCurrentLogDisk() const { return getDisk(current_log_storage); diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 4e538c99649..e04d1cd6b3d 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -34,6 +34,7 @@ public: DiskPtr getCurrentLogDisk() const; DiskPtr getLogDisk() const; + std::vector getOldLogDisks() const; void setLogDisk(DiskPtr disk); DiskPtr getSnapshotDisk() const; @@ -63,6 +64,8 @@ private: Storage snapshot_storage; Storage state_file_storage; + std::vector old_log_disk_names; + bool standalone_keeper; }; diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 7d808e88b3d..7f097c182a1 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -720,7 +720,6 @@ std::pair KeeperSnapshotManager::serializeSnapshot try { - std::cout << "Removing file " << tmp_snapshot_file_name << std::endl; disk->removeFile(tmp_snapshot_file_name); } catch (fs::filesystem_error & e) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 650fde7bcd1..a680a344746 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -120,7 +120,6 @@ const IMetadataStorage & MetadataStorageFromPlainObjectStorageTransaction::getSt void MetadataStorageFromPlainObjectStorageTransaction::unlinkFile(const std::string & path) { auto object = StoredObject(metadata_storage.getAbsolutePath(path)); - std::cout << "Removing from plain " << path << std::endl; metadata_storage.object_storage->removeObject(object); } diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index e82b21079fe..5a6fd15d72c 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -8,6 +8,7 @@ #include #include #include +#include using namespace Coordination; using namespace DB; @@ -62,15 +63,18 @@ int main(int argc, char *argv[]) ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; CoordinationSettingsPtr settings = std::make_shared(); - KeeperContextPtr keeper_context = std::make_shared(); - auto state_machine = std::make_shared(queue, snapshots_queue, argv[1], settings, keeper_context, nullptr); + KeeperContextPtr keeper_context = std::make_shared(true); + keeper_context->setLogDisk(std::make_shared("LogDisk", argv[2], 0)); + keeper_context->setSnapshotDisk(std::make_shared("LogDisk", argv[1], 0)); + + auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); size_t last_commited_index = state_machine->last_commit_index(); LOG_INFO(logger, "Last committed index: {}", last_commited_index); DB::KeeperLogStore changelog( - argv[2], LogFileSettings{.force_sync = true, .compress_logs = settings->compress_logs, .rotate_interval = 10000000}); + LogFileSettings{.force_sync = true, .compress_logs = settings->compress_logs, .rotate_interval = 10000000}, keeper_context); changelog.init(last_commited_index, 10000000000UL); /// collect all logs if (changelog.size() == 0) LOG_INFO(logger, "Changelog empty"); From d8f39b8df1ef608b9f78844a6665495be3cbec33 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 24 May 2023 17:53:37 +0000 Subject: [PATCH 080/515] Fixing more tests. --- src/Interpreters/ActionsVisitor.cpp | 8 ++++++ .../ClusterProxy/SelectStreamFactory.cpp | 8 +++--- .../ClusterProxy/SelectStreamFactory.h | 6 ++++- .../ClusterProxy/executeQuery.cpp | 1 + src/Interpreters/GlobalSubqueriesVisitor.h | 3 ++- src/Interpreters/PreparedSets.cpp | 6 ++++- .../QueryPlan/DistributedCreateLocalPlan.cpp | 3 ++- .../QueryPlan/DistributedCreateLocalPlan.h | 4 +++ src/Processors/QueryPlan/ReadFromRemote.cpp | 6 +++-- src/Processors/QueryPlan/ReadFromRemote.h | 2 ++ src/Storages/StorageDistributed.cpp | 3 ++- src/Storages/StorageMergeTree.cpp | 3 ++- src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- src/Storages/VirtualColumnUtils.cpp | 27 +++++++++++++++++++ 14 files changed, 71 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index dcf6c4a461f..1405568aa71 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -3,6 +3,7 @@ #include #include #include +#include "Parsers/queryToString.h" #include #include @@ -1392,8 +1393,15 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool { if (no_subqueries) return {}; + //std::cerr << queryToString(right_in_operand) << std::endl; auto set_key = PreparedSetKey::forSubquery(right_in_operand->getTreeHash()); + // std::cerr << set_key.toString() << std::endl; + // std::cerr << data.prepared_sets->getSets().size() << std::endl; + // std::cerr << reinterpret_cast(data.prepared_sets.get()) << std::endl; + // for (const auto & [k, v] : data.prepared_sets->getSets()) + // std::cerr << "... " << k.toString(); + if (auto set = data.prepared_sets->getFuture(set_key)) return set; diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 0cf3f360994..0f8a725e144 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -92,11 +92,13 @@ SelectStreamFactory::SelectStreamFactory( const Block & header_, const ColumnsDescriptionByShardNum & objects_by_shard_, const StorageSnapshotPtr & storage_snapshot_, - QueryProcessingStage::Enum processed_stage_) + QueryProcessingStage::Enum processed_stage_, + PreparedSetsPtr prepared_sets_) : header(header_), objects_by_shard(objects_by_shard_), storage_snapshot(storage_snapshot_), - processed_stage(processed_stage_) + processed_stage(processed_stage_), + prepared_sets(std::move(prepared_sets_)) { } @@ -117,7 +119,7 @@ void SelectStreamFactory::createForShard( auto emplace_local_stream = [&]() { local_plans.emplace_back(createLocalPlan( - query_ast, header, context, processed_stage, shard_info.shard_num, shard_count, /*replica_num=*/0, /*replica_count=*/0, /*coordinator=*/nullptr)); + query_ast, header, context, processed_stage, prepared_sets, shard_info.shard_num, shard_count, /*replica_num=*/0, /*replica_count=*/0, /*coordinator=*/nullptr)); }; auto emplace_remote_stream = [&](bool lazy = false, time_t local_delay = 0) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index f1a8b3e0984..b19012ddba6 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -26,6 +26,8 @@ using QueryPlanPtr = std::unique_ptr; struct StorageID; +class PreparedSets; +using PreparedSetsPtr = std::shared_ptr; namespace ClusterProxy { @@ -67,7 +69,8 @@ public: const Block & header_, const ColumnsDescriptionByShardNum & objects_by_shard_, const StorageSnapshotPtr & storage_snapshot_, - QueryProcessingStage::Enum processed_stage_); + QueryProcessingStage::Enum processed_stage_, + PreparedSetsPtr prepared_sets_); void createForShard( const Cluster::ShardInfo & shard_info, @@ -92,6 +95,7 @@ public: const ColumnsDescriptionByShardNum objects_by_shard; const StorageSnapshotPtr storage_snapshot; QueryProcessingStage::Enum processed_stage; + PreparedSetsPtr prepared_sets; }; } diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index e2f1dfe8ba7..a9cf3d55392 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -226,6 +226,7 @@ void executeQuery( std::move(remote_shards), header, processed_stage, + stream_factory.prepared_sets, main_table, table_func_ptr, new_context, diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 3a846bb4bc3..a872af529aa 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -181,7 +181,7 @@ public: // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); // subquery_for_set.createSource(*interpreter, external_storage); auto key = subquery_or_table_name->getColumnName(); - auto set_key = PreparedSetKey::forSubquery(subquery_or_table_name->getTreeHash()); + auto set_key = PreparedSetKey::forSubquery(database_and_table_name->getTreeHash()); if (!prepared_sets->getFuture(set_key)) { @@ -190,6 +190,7 @@ public: subquery_for_set.table = std::move(external_storage); subquery_for_set.createSource(*interpreter); + //std::cerr << reinterpret_cast(prepared_sets.get()) << std::endl; prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set)); } else diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index cd6b2a81ba0..3b63d942404 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -140,6 +140,10 @@ FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryF if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + // std::cerr << key.toString() << std::endl; + // std::cerr << "========= PreparedSets::addFromSubquery\n"; + // std::cerr << StackTrace().toString() << std::endl; + subqueries.emplace(id, std::move(from_subquery)); return it->second; } @@ -226,7 +230,7 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c if (set) return nullptr; - //std::cerr << StackTrace().toString() << std::endl; + // std::cerr << StackTrace().toString() << std::endl; auto set_cache = context->getPreparedSetsCache(); if (set_cache) diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index 9b9cc221ca8..62e369659d1 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -43,6 +43,7 @@ std::unique_ptr createLocalPlan( const Block & header, ContextPtr context, QueryProcessingStage::Enum processed_stage, + PreparedSetsPtr prepared_sets, size_t shard_num, size_t shard_count, size_t replica_num, @@ -98,7 +99,7 @@ std::unique_ptr createLocalPlan( } else { - auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options); + auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options, prepared_sets); interpreter.buildQueryPlan(*query_plan); } diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h index 1afdc07fa4d..cf59027a33f 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h @@ -10,11 +10,15 @@ namespace DB { +class PreparedSets; +using PreparedSetsPtr = std::shared_ptr; + std::unique_ptr createLocalPlan( const ASTPtr & query_ast, const Block & header, ContextPtr context, QueryProcessingStage::Enum processed_stage, + PreparedSetsPtr prepared_sets, size_t shard_num, size_t shard_count, size_t replica_num, diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index ed740e3e242..16cb06a94d6 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -97,6 +97,7 @@ ReadFromRemote::ReadFromRemote( ClusterProxy::SelectStreamFactory::Shards shards_, Block header_, QueryProcessingStage::Enum stage_, + PreparedSetsPtr prepared_sets_, StorageID main_table_, ASTPtr table_func_ptr_, ContextMutablePtr context_, @@ -109,6 +110,7 @@ ReadFromRemote::ReadFromRemote( : ISourceStep(DataStream{.header = std::move(header_)}) , shards(std::move(shards_)) , stage(stage_) + , prepared_sets(std::move(prepared_sets_)) , main_table(std::move(main_table_)) , table_func_ptr(std::move(table_func_ptr_)) , context(std::move(context_)) @@ -150,7 +152,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream my_context = context, my_throttler = throttler, my_main_table = main_table, my_table_func_ptr = table_func_ptr, my_scalars = scalars, my_external_tables = external_tables, - my_stage = stage, local_delay = shard.local_delay, + my_stage = stage, my_prepared_sets = prepared_sets, local_delay = shard.local_delay, add_agg_info, add_totals, add_extremes, async_read, async_query_sending]() mutable -> QueryPipelineBuilder { @@ -185,7 +187,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream if (try_results.empty() || local_delay < max_remote_delay) { auto plan = createLocalPlan( - query, header, my_context, my_stage, my_shard.shard_info.shard_num, my_shard_count, 0, 0, /*coordinator=*/nullptr); + query, header, my_context, my_stage, my_prepared_sets, my_shard.shard_info.shard_num, my_shard_count, 0, 0, /*coordinator=*/nullptr); return std::move(*plan->buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(my_context), diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index d4005d81f1b..405b5727ff2 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -26,6 +26,7 @@ public: ClusterProxy::SelectStreamFactory::Shards shards_, Block header_, QueryProcessingStage::Enum stage_, + PreparedSetsPtr prepared_sets_, StorageID main_table_, ASTPtr table_func_ptr_, ContextMutablePtr context_, @@ -46,6 +47,7 @@ public: private: ClusterProxy::SelectStreamFactory::Shards shards; QueryProcessingStage::Enum stage; + PreparedSetsPtr prepared_sets; StorageID main_table; ASTPtr table_func_ptr; ContextMutablePtr context; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 86a2599b49c..f45e247f8e7 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1098,7 +1098,8 @@ void StorageDistributed::read( header, snapshot_data.objects_by_shard, storage_snapshot, - processed_stage); + processed_stage, + query_info.prepared_sets); auto settings = local_context->getSettingsRef(); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 2c19d3ba122..4600532231f 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -231,7 +231,8 @@ void StorageMergeTree::read( header, {}, storage_snapshot, - processed_stage); + processed_stage, + query_info.prepared_sets); ClusterProxy::executeQueryWithParallelReplicas( query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d9c8f09ccf1..ab78ea2f0a4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4663,7 +4663,8 @@ void StorageReplicatedMergeTree::read( header, {}, storage_snapshot, - processed_stage); + processed_stage, + query_info.prepared_sets); ClusterProxy::executeQueryWithParallelReplicas( query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr, diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 31fa1cc6a7e..be2206a78e9 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -23,6 +23,8 @@ #include #include #include +#include +#include #include @@ -202,6 +204,31 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex //buildSets(expression_ast, analyzer); ExpressionActionsPtr actions = analyzer.getActions(false /* add alises */, true /* project result */, CompileExpressions::yes); + for (const auto & node : actions->getNodes()) + { + if (node.type == ActionsDAG::ActionType::COLUMN) + { + const ColumnSet * column_set = checkAndGetColumnConstData(node.column.get()); + if (!column_set) + column_set = checkAndGetColumn(node.column.get()); + + if (column_set) + { + auto future_set = column_set->getData(); + if (!future_set->isFilled()) + { + auto plan = future_set->build(context); + auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + pipeline.complete(std::make_shared(Block())); + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); + } + } + } + } + Block block_with_filter = block; actions->execute(block_with_filter); From 7e68f61df1671b0be0b30c8cef33f34c8c971d3b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 May 2023 08:43:11 +0000 Subject: [PATCH 081/515] Fix snapshot recovery --- programs/keeper-converter/KeeperConverter.cpp | 4 +- programs/server/config.d/users.xml | 26 ----- src/Coordination/FourLetterCommand.cpp | 1 + src/Coordination/KeeperContext.cpp | 35 ++++++- src/Coordination/KeeperContext.h | 4 + src/Coordination/KeeperDiskSelector.h | 37 -------- src/Coordination/KeeperDispatcher.cpp | 11 ++- src/Coordination/KeeperDispatcher.h | 7 ++ src/Coordination/KeeperServer.cpp | 4 +- src/Coordination/KeeperServer.h | 2 +- src/Coordination/KeeperSnapshotManager.cpp | 20 +--- src/Coordination/KeeperSnapshotManager.h | 21 ++-- src/Coordination/KeeperSnapshotManagerS3.cpp | 24 ++--- src/Coordination/KeeperSnapshotManagerS3.h | 8 +- src/Coordination/KeeperStateMachine.cpp | 95 ++++++++++++++----- src/Coordination/KeeperStateMachine.h | 2 +- .../test_keeper_four_word_command/test.py | 2 + 17 files changed, 165 insertions(+), 138 deletions(-) delete mode 100644 programs/server/config.d/users.xml delete mode 100644 src/Coordination/KeeperDiskSelector.h diff --git a/programs/keeper-converter/KeeperConverter.cpp b/programs/keeper-converter/KeeperConverter.cpp index f2389021cb6..a049e6bc2b3 100644 --- a/programs/keeper-converter/KeeperConverter.cpp +++ b/programs/keeper-converter/KeeperConverter.cpp @@ -55,8 +55,8 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv) DB::KeeperSnapshotManager manager(1, keeper_context); auto snp = manager.serializeSnapshotToBuffer(snapshot); - auto path = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID()); - std::cout << "Snapshot serialized to path:" << path << std::endl; + auto file_info = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID()); + std::cout << "Snapshot serialized to path:" << fs::path(file_info.disk->getPath()) / file_info.path << std::endl; } catch (...) { diff --git a/programs/server/config.d/users.xml b/programs/server/config.d/users.xml deleted file mode 100644 index d8a62b45baa..00000000000 --- a/programs/server/config.d/users.xml +++ /dev/null @@ -1,26 +0,0 @@ - - - - 10000000000 - 0 - 2 - - - 5000000000 - 20000000000 - - - - - - - - - - - - - default - - - diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 7077e792fd8..b7419bcaccc 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -292,6 +292,7 @@ String ConfCommand::run() StringBuffer buf; keeper_dispatcher.getKeeperConfigurationAndSettings()->dump(buf); + keeper_dispatcher.getKeeperContext()->dumpConfiguration(buf); return buf.str(); } diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index e27cfc60cff..da49868f706 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -122,6 +122,35 @@ void KeeperContext::setStateFileDisk(DiskPtr disk) state_file_storage = std::move(disk); } +void KeeperContext::dumpConfiguration(WriteBufferFromOwnString & buf) const +{ + auto dump_disk_info = [&](const std::string_view prefix, const IDisk & disk) + { + writeText(fmt::format("{}_path=", prefix), buf); + writeText(disk.getPath(), buf); + buf.write('\n'); + + writeText(fmt::format("{}_disk=", prefix), buf); + writeText(disk.getName(), buf); + buf.write('\n'); + + }; + + { + auto log_disk = getDisk(log_storage); + dump_disk_info("log_storage", *log_disk); + + auto current_log_disk = getDisk(current_log_storage); + if (log_disk != current_log_disk) + dump_disk_info("current_log_storage", *current_log_disk); + } + + { + auto snapshot_disk = getDisk(snapshot_storage); + dump_disk_info("snapshot_storage", *snapshot_disk); + } +} + KeeperContext::Storage KeeperContext::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const { const auto create_local_disk = [](const auto & path) @@ -129,7 +158,7 @@ KeeperContext::Storage KeeperContext::getLogsPathFromConfig(const Poco::Util::Ab if (!fs::exists(path)) fs::create_directories(path); - return std::make_shared("LogDisk", path, 0); + return std::make_shared("LocalLogDisk", path, 0); }; /// the most specialized path @@ -155,7 +184,7 @@ KeeperContext::Storage KeeperContext::getSnapshotsPathFromConfig(const Poco::Uti if (!fs::exists(path)) fs::create_directories(path); - return std::make_shared("SnapshotDisk", path, 0); + return std::make_shared("LocalSnapshotDisk", path, 0); }; /// the most specialized path @@ -181,7 +210,7 @@ KeeperContext::Storage KeeperContext::getStatePathFromConfig(const Poco::Util::A if (!fs::exists(path)) fs::create_directories(path); - return std::make_shared("SnapshotDisk", path, 0); + return std::make_shared("LocalStateFileDisk", path, 0); }; if (config.has("keeper_server.state_storage_disk")) diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index e04d1cd6b3d..6b7af3a60db 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -2,6 +2,8 @@ #include +#include + #include #include @@ -42,6 +44,8 @@ public: DiskPtr getStateFileDisk() const; void setStateFileDisk(DiskPtr disk); + + void dumpConfiguration(WriteBufferFromOwnString & buf) const; private: /// local disk defined using path or disk name using Storage = std::variant; diff --git a/src/Coordination/KeeperDiskSelector.h b/src/Coordination/KeeperDiskSelector.h deleted file mode 100644 index fa78fec1952..00000000000 --- a/src/Coordination/KeeperDiskSelector.h +++ /dev/null @@ -1,37 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -class KeeperDiskSelector -{ -public: - void initialize(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); - - DiskSelectorPtr updateFromConfig( - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - ContextPtr context) const; - - /// Get disk by name - DiskPtr get(const String & name) const; - - DiskPtr tryGet(const String & name) const; - - /// Get all disks with names - const DisksMap & getDisksMap() const; - - void shutdown(); - -private: - mutable std::mutex disk_selector_mutex; - DiskSelectorPtr disk_selector; -}; - -using KeeperDiskSelectorPtr = std::shared_ptr; - - -} diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 17a15067301..4a460777621 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -238,13 +238,13 @@ void KeeperDispatcher::snapshotThread() try { - auto snapshot_path = task.create_snapshot(std::move(task.snapshot)); + auto snapshot_file_info = task.create_snapshot(std::move(task.snapshot)); - if (snapshot_path.empty()) + if (snapshot_file_info.path.empty()) continue; if (isLeader()) - snapshot_s3.uploadSnapshot(snapshot_path); + snapshot_s3.uploadSnapshot(snapshot_file_info); } catch (...) { @@ -336,12 +336,15 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf snapshot_s3.startup(config, macros); + keeper_context = std::make_shared(standalone_keeper); + keeper_context->initialize(config); + server = std::make_unique( configuration_and_settings, config, responses_queue, snapshots_queue, - standalone_keeper, + keeper_context, snapshot_s3, [this](const KeeperStorage::RequestForSession & request_for_session) { diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 4b8b134cf8f..1b44f0f6ced 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -81,6 +81,8 @@ private: KeeperSnapshotManagerS3 snapshot_s3; + KeeperContextPtr keeper_context; + /// Thread put requests to raft void requestThread(); /// Thread put responses for subscribed sessions @@ -198,6 +200,11 @@ public: return configuration_and_settings; } + const KeeperContextPtr & getKeeperContext() const + { + return keeper_context; + } + void incrementPacketsSent() { keeper_stats.incrementPacketsSent(); diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 31c91e2de80..989455a5a79 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -108,14 +108,14 @@ KeeperServer::KeeperServer( const Poco::Util::AbstractConfiguration & config, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, - bool standalone_keeper, + KeeperContextPtr keeper_context_, KeeperSnapshotManagerS3 & snapshot_manager_s3, KeeperStateMachine::CommitCallback commit_callback) : server_id(configuration_and_settings_->server_id) , coordination_settings(configuration_and_settings_->coordination_settings) , log(&Poco::Logger::get("KeeperServer")) , is_recovering(config.getBool("keeper_server.force_recovery", false)) - , keeper_context{std::make_shared(standalone_keeper)} + , keeper_context{std::move(keeper_context_)} , create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true)) { if (coordination_settings->quorum_reads) diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 63f9cc2bcea..8f416b1f48c 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -72,7 +72,7 @@ public: const Poco::Util::AbstractConfiguration & config_, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, - bool standalone_keeper, + KeeperContextPtr keeper_context_, KeeperSnapshotManagerS3 & snapshot_manager_s3, KeeperStateMachine::CommitCallback commit_callback); diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 7f097c182a1..3bfe700bcd5 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -563,7 +563,7 @@ KeeperSnapshotManager::KeeperSnapshotManager( } -std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx) +SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx) { ReadBufferFromNuraftBuffer reader(buffer); @@ -585,7 +585,7 @@ std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer existing_snapshots.emplace(up_to_log_idx, snapshot_file_name); removeOutdatedSnapshotsIfNeeded(); - return snapshot_file_name; + return {snapshot_file_name, disk}; } nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBufferFromDisk() @@ -694,7 +694,7 @@ void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) existing_snapshots.erase(itr); } -std::pair KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot) +SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot) { auto up_to_log_idx = snapshot.snapshot_meta->get_last_log_idx(); auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); @@ -716,22 +716,12 @@ std::pair KeeperSnapshotManager::serializeSnapshot compressed_writer->finalize(); compressed_writer->sync(); - std::error_code ec; - - try - { - disk->removeFile(tmp_snapshot_file_name); - } - catch (fs::filesystem_error & e) - { - ec = e.code(); - return {snapshot_file_name, ec}; - } + disk->removeFile(tmp_snapshot_file_name); existing_snapshots.emplace(up_to_log_idx, snapshot_file_name); removeOutdatedSnapshotsIfNeeded(); - return {snapshot_file_name, ec}; + return {snapshot_file_name, disk}; } } diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 7b1129018d8..036c0cab62b 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -87,8 +87,14 @@ public: uint64_t nodes_digest; }; +struct SnapshotFileInfo +{ + std::string path; + DiskPtr disk; +}; + using KeeperStorageSnapshotPtr = std::shared_ptr; -using CreateSnapshotCallback = std::function; +using CreateSnapshotCallback = std::function; using SnapshotMetaAndStorage = std::pair; @@ -112,10 +118,10 @@ public: nuraft::ptr serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const; /// Serialize already compressed snapshot to disk (return path) - std::string serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx); + SnapshotFileInfo serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx); /// Serialize snapshot directly to disk - std::pair serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot); + SnapshotFileInfo serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot); SnapshotDeserializationResult deserializeSnapshotFromBuffer(nuraft::ptr buffer) const; @@ -139,7 +145,7 @@ public: return 0; } - std::string getLatestSnapshotPath() const + SnapshotFileInfo getLatestSnapshotInfo() const { if (!existing_snapshots.empty()) { @@ -147,14 +153,15 @@ public: try { - if (getDisk()->exists(path)) - return path; + auto disk = getDisk(); + if (disk->exists(path)) + return {path, disk}; } catch (...) { } } - return ""; + return {"", nullptr}; } private: diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 1afe0b352c5..580e166e302 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -132,8 +132,9 @@ std::shared_ptr KeeperSnapshotManagerS return snapshot_s3_client; } -void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_path) +void KeeperSnapshotManagerS3::uploadSnapshotImpl(const SnapshotFileInfo & snapshot_file_info) { + const auto & [snapshot_path, snapshot_disk] = snapshot_file_info; try { auto s3_client = getSnapshotS3Client(); @@ -154,8 +155,9 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_pa }; }; - LOG_INFO(log, "Will try to upload snapshot on {} to S3", snapshot_path); - ReadBufferFromFile snapshot_file(snapshot_path); + LOG_INFO(log, "Will try to upload snapshot on {} to S3", snapshot_file_info.path); + + auto snapshot_file = snapshot_disk->readFile(snapshot_file_info.path); auto snapshot_name = fs::path(snapshot_path).filename().string(); auto lock_file = fmt::format(".{}_LOCK", snapshot_name); @@ -222,7 +224,7 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_pa }); WriteBufferFromS3 snapshot_writer = create_writer(snapshot_name); - copyData(snapshot_file, snapshot_writer); + copyData(*snapshot_file, snapshot_writer); snapshot_writer.finalize(); LOG_INFO(log, "Successfully uploaded {} to S3", snapshot_path); @@ -240,31 +242,31 @@ void KeeperSnapshotManagerS3::snapshotS3Thread() while (!shutdown_called) { - std::string snapshot_path; - if (!snapshots_s3_queue.pop(snapshot_path)) + SnapshotFileInfo snapshot_file_info; + if (!snapshots_s3_queue.pop(snapshot_file_info)) break; if (shutdown_called) break; - uploadSnapshotImpl(snapshot_path); + uploadSnapshotImpl(snapshot_file_info); } } -void KeeperSnapshotManagerS3::uploadSnapshot(const std::string & path, bool async_upload) +void KeeperSnapshotManagerS3::uploadSnapshot(const SnapshotFileInfo & file_info, bool async_upload) { if (getSnapshotS3Client() == nullptr) return; if (async_upload) { - if (!snapshots_s3_queue.push(path)) - LOG_WARNING(log, "Failed to add snapshot {} to S3 queue", path); + if (!snapshots_s3_queue.push(file_info)) + LOG_WARNING(log, "Failed to add snapshot {} to S3 queue", file_info.path); return; } - uploadSnapshotImpl(path); + uploadSnapshotImpl(file_info); } void KeeperSnapshotManagerS3::startup(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros) diff --git a/src/Coordination/KeeperSnapshotManagerS3.h b/src/Coordination/KeeperSnapshotManagerS3.h index eff7868bba9..908deb76851 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.h +++ b/src/Coordination/KeeperSnapshotManagerS3.h @@ -10,6 +10,8 @@ #include #include +#include + #include #endif @@ -24,13 +26,13 @@ public: /// 'macros' are used to substitute macros in endpoint of disks void updateS3Configuration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); - void uploadSnapshot(const std::string & path, bool async_upload = true); + void uploadSnapshot(const SnapshotFileInfo & file_info, bool async_upload = true); /// 'macros' are used to substitute macros in endpoint of disks void startup(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); void shutdown(); private: - using SnapshotS3Queue = ConcurrentBoundedQueue; + using SnapshotS3Queue = ConcurrentBoundedQueue; SnapshotS3Queue snapshots_s3_queue; /// Upload new snapshots to S3 @@ -48,7 +50,7 @@ private: std::shared_ptr getSnapshotS3Client() const; - void uploadSnapshotImpl(const std::string & snapshot_path); + void uploadSnapshotImpl(const SnapshotFileInfo & snapshot_file_info); /// Thread upload snapshots to S3 in the background void snapshotS3Thread(); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 69c15db51da..49243541bc8 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -14,6 +14,8 @@ #include #include "Coordination/KeeperStorage.h" +#include + namespace ProfileEvents { @@ -66,6 +68,16 @@ KeeperStateMachine::KeeperStateMachine( { } +namespace +{ + +bool isLocalDisk(const IDisk & disk) +{ + return dynamic_cast(&disk) != nullptr; +} + +} + void KeeperStateMachine::init() { /// Do everything without mutexes, no other threads exist. @@ -80,9 +92,13 @@ void KeeperStateMachine::init() try { - auto snapshot_deserialization_result - = snapshot_manager.deserializeSnapshotFromBuffer(snapshot_manager.deserializeSnapshotBufferFromDisk(latest_log_index)); - latest_snapshot_path = snapshot_manager.getLatestSnapshotPath(); + latest_snapshot_buf = snapshot_manager.deserializeSnapshotBufferFromDisk(latest_log_index); + auto snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_buf); + latest_snapshot_info = snapshot_manager.getLatestSnapshotInfo(); + + if (isLocalDisk(*latest_snapshot_info.disk)) + latest_snapshot_buf = nullptr; + storage = std::move(snapshot_deserialization_result.storage); latest_snapshot_meta = snapshot_deserialization_result.snapshot_meta; cluster_config = snapshot_deserialization_result.cluster_config; @@ -306,8 +322,14 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) { /// deserialize and apply snapshot to storage std::lock_guard lock(storage_and_responses_lock); - auto snapshot_deserialization_result - = snapshot_manager.deserializeSnapshotFromBuffer(snapshot_manager.deserializeSnapshotBufferFromDisk(s.get_last_log_idx())); + + SnapshotDeserializationResult snapshot_deserialization_result; + if (latest_snapshot_ptr) + snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_ptr); + else + snapshot_deserialization_result + = snapshot_manager.deserializeSnapshotFromBuffer(snapshot_manager.deserializeSnapshotBufferFromDisk(s.get_last_log_idx())); + storage = std::move(snapshot_deserialization_result.storage); latest_snapshot_meta = snapshot_deserialization_result.snapshot_meta; cluster_config = snapshot_deserialization_result.cluster_config; @@ -387,19 +409,22 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res } else { - auto [path, error_code] = snapshot_manager.serializeSnapshotToDisk(*snapshot); - if (error_code) + /// we rely on the fact that the snapshot disk cannot be changed during runtime + if (isLocalDisk(*keeper_context->getSnapshotDisk())) { - throw Exception( - ErrorCodes::SYSTEM_ERROR, - "Snapshot {} was created failed, error: {}", - snapshot->snapshot_meta->get_last_log_idx(), - error_code.message()); + latest_snapshot_info = snapshot_manager.serializeSnapshotToDisk(*snapshot); + latest_snapshot_meta = snapshot->snapshot_meta; + latest_snapshot_buf = nullptr; } - latest_snapshot_path = path; - latest_snapshot_meta = snapshot->snapshot_meta; + else + { + auto snapshot_buf = snapshot_manager.serializeSnapshotToBuffer(*snapshot); + latest_snapshot_info = snapshot_manager.serializeSnapshotBufferToDisk(*snapshot_buf, snapshot->snapshot_meta->get_last_log_idx()); + latest_snapshot_buf = std::move(snapshot_buf); + } + ProfileEvents::increment(ProfileEvents::KeeperSnapshotCreations); - LOG_DEBUG(log, "Created persistent snapshot {} with path {}", latest_snapshot_meta->get_last_log_idx(), path); + LOG_DEBUG(log, "Created persistent snapshot {} with path {}", latest_snapshot_meta->get_last_log_idx(), latest_snapshot_info.path); } } @@ -423,19 +448,19 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res when_done(ret, exception); - return ret ? latest_snapshot_path : ""; + return ret ? latest_snapshot_info : SnapshotFileInfo{}; }; if (keeper_context->getServerState() == KeeperContext::Phase::SHUTDOWN) { LOG_INFO(log, "Creating a snapshot during shutdown because 'create_snapshot_on_exit' is enabled."); - auto snapshot_path = snapshot_task.create_snapshot(std::move(snapshot_task.snapshot)); + auto snapshot_file_info = snapshot_task.create_snapshot(std::move(snapshot_task.snapshot)); - if (!snapshot_path.empty() && snapshot_manager_s3) + if (!snapshot_file_info.path.empty() && snapshot_manager_s3) { - LOG_INFO(log, "Uploading snapshot {} during shutdown because 'upload_snapshot_on_exit' is enabled.", snapshot_path); - snapshot_manager_s3->uploadSnapshot(snapshot_path, /* asnyc_upload */ false); + LOG_INFO(log, "Uploading snapshot {} during shutdown because 'upload_snapshot_on_exit' is enabled.", snapshot_file_info.path); + snapshot_manager_s3->uploadSnapshot(snapshot_file_info, /* asnyc_upload */ false); } return; @@ -456,14 +481,20 @@ void KeeperStateMachine::save_logical_snp_obj( nuraft::ptr snp_buf = s.serialize(); nuraft::ptr cloned_meta = nuraft::snapshot::deserialize(*snp_buf); + nuraft::ptr cloned_buffer; + + /// we rely on the fact that the snapshot disk cannot be changed during runtime + if (!isLocalDisk(*keeper_context->getSnapshotDisk())) + cloned_buffer = nuraft::buffer::clone(data); + try { std::lock_guard lock(snapshots_lock); /// Serialize snapshot to disk - auto result_path = snapshot_manager.serializeSnapshotBufferToDisk(data, s.get_last_log_idx()); - latest_snapshot_path = result_path; + latest_snapshot_info = snapshot_manager.serializeSnapshotBufferToDisk(data, s.get_last_log_idx()); latest_snapshot_meta = cloned_meta; - LOG_DEBUG(log, "Saved snapshot {} to path {}", s.get_last_log_idx(), result_path); + latest_snapshot_buf = std::move(cloned_buffer); + LOG_DEBUG(log, "Saved snapshot {} to path {}", s.get_last_log_idx(), latest_snapshot_info.path); obj_id++; ProfileEvents::increment(ProfileEvents::KeeperSaveSnapshot); } @@ -523,11 +554,23 @@ int KeeperStateMachine::read_logical_snp_obj( latest_snapshot_meta->get_last_log_idx()); return -1; } - if (bufferFromFile(log, latest_snapshot_path, data_out)) + + const auto & [path, disk] = latest_snapshot_info; + if (isLocalDisk(*disk)) { - LOG_WARNING(log, "Error reading snapshot {} from {}", s.get_last_log_idx(), latest_snapshot_path); - return -1; + auto full_path = fs::path(disk->getPath()) / path; + if (bufferFromFile(log, full_path, data_out)) + { + LOG_WARNING(log, "Error reading snapshot {} from {}", s.get_last_log_idx(), full_path); + return -1; + } } + else + { + chassert(latest_snapshot_buf); + data_out = nuraft::buffer::clone(*latest_snapshot_buf); + } + is_last_obj = true; ProfileEvents::increment(ProfileEvents::KeeperReadSnapshot); diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index afe11150f36..834837314df 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -110,7 +110,7 @@ private: /// In our state machine we always have a single snapshot which is stored /// in memory in compressed (serialized) format. SnapshotMetadataPtr latest_snapshot_meta = nullptr; - std::string latest_snapshot_path; + SnapshotFileInfo latest_snapshot_info; nuraft::ptr latest_snapshot_buf = nullptr; CoordinationSettingsPtr coordination_settings; diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 2098daea5fe..aab8a2ccb68 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -252,10 +252,12 @@ def test_cmd_conf(started_cluster): assert result["four_letter_word_allow_list"] == "*" assert result["log_storage_path"] == "/var/lib/clickhouse/coordination/log" + assert result["log_storage_disk"] == "LocalLogDisk" assert ( result["snapshot_storage_path"] == "/var/lib/clickhouse/coordination/snapshots" ) + assert result["snapshot_storage_disk"] == "LocalSnapshotDisk" assert result["session_timeout_ms"] == "30000" assert result["min_session_timeout_ms"] == "10000" From 4483602c50918e69d5a4d79c1628412b2c667c0d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 May 2023 08:52:14 +0000 Subject: [PATCH 082/515] Remove double initialization --- src/Coordination/KeeperServer.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 989455a5a79..363e7f9bef1 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -121,8 +121,6 @@ KeeperServer::KeeperServer( if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); - keeper_context->initialize(config); - state_machine = nuraft::cs_new( responses_queue_, snapshots_queue_, From f519aa4613a7f82b30bb39686412343d5a5a939d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 May 2023 09:10:45 +0000 Subject: [PATCH 083/515] Calculate size --- .../KeeperAsynchronousMetrics.cpp | 12 +++---- src/Coordination/KeeperDispatcher.cpp | 32 +++++++++---------- src/Coordination/KeeperSnapshotManagerS3.h | 5 +-- src/Coordination/KeeperStateMachine.cpp | 5 --- 4 files changed, 25 insertions(+), 29 deletions(-) diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index 1427130b184..890079e98f7 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -28,8 +28,8 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM size_t zxid = 0; size_t session_with_watches = 0; size_t paths_watched = 0; - size_t snapshot_dir_size = 0; - size_t log_dir_size = 0; + //size_t snapshot_dir_size = 0; + //size_t log_dir_size = 0; if (keeper_dispatcher.isServerActive()) { @@ -49,8 +49,8 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM latest_snapshot_size = state_machine.getLatestSnapshotBufSize(); session_with_watches = state_machine.getSessionsWithWatchesCount(); paths_watched = state_machine.getWatchedPathsCount(); - snapshot_dir_size = keeper_dispatcher.getSnapDirSize(); - log_dir_size = keeper_dispatcher.getLogDirSize(); + //snapshot_dir_size = keeper_dispatcher.getSnapDirSize(); + //log_dir_size = keeper_dispatcher.getLogDirSize(); # if defined(__linux__) || defined(__APPLE__) open_file_descriptor_count = getCurrentProcessFDCount(); @@ -85,8 +85,8 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM new_values["KeeperZxid"] = { zxid, "The current transaction id number (zxid) in ClickHouse Keeper." }; new_values["KeeperSessionWithWatches"] = { session_with_watches, "The number of client sessions of ClickHouse Keeper having watches." }; new_values["KeeperPathsWatched"] = { paths_watched, "The number of different paths watched by the clients of ClickHouse Keeper." }; - new_values["KeeperSnapshotDirSize"] = { snapshot_dir_size, "The size of the snapshots directory of ClickHouse Keeper, in bytes." }; - new_values["KeeperLogDirSize"] = { log_dir_size, "The size of the logs directory of ClickHouse Keeper, in bytes." }; + //new_values["KeeperSnapshotDirSize"] = { snapshot_dir_size, "The size of the snapshots directory of ClickHouse Keeper, in bytes." }; + //new_values["KeeperLogDirSize"] = { log_dir_size, "The size of the logs directory of ClickHouse Keeper, in bytes." }; auto keeper_log_info = keeper_dispatcher.getKeeperLogInfo(); diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 4a460777621..6d1239c9210 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -777,37 +777,37 @@ void KeeperDispatcher::updateKeeperStatLatency(uint64_t process_time_ms) keeper_stats.updateLatency(process_time_ms); } -static uint64_t getDirSize(const fs::path & dir) +static uint64_t getTotalSize(const DiskPtr & disk, const std::string & path = "") { checkStackSize(); - if (!fs::exists(dir)) - return 0; - fs::directory_iterator it(dir); - fs::directory_iterator end; - - uint64_t size{0}; - while (it != end) + uint64_t size = 0; + for (auto it = disk->iterateDirectory(path); it->isValid(); it->next()) { - if (it->is_regular_file()) - size += fs::file_size(*it); + if (disk->isFile(it->path())) + size += disk->getFileSize(it->path()); else - size += getDirSize(it->path()); - ++it; + size += getTotalSize(disk, it->path()); } + return size; } uint64_t KeeperDispatcher::getLogDirSize() const { - //return getDirSize(configuration_and_settings->log_storage_path); - return 0; + auto log_disk = keeper_context->getLogDisk(); + auto size = getTotalSize(log_disk); + + auto current_log_disk = keeper_context->getCurrentLogDisk(); + if (log_disk != current_log_disk) + size += getTotalSize(current_log_disk); + + return size; } uint64_t KeeperDispatcher::getSnapDirSize() const { - //return getDirSize(configuration_and_settings->snapshot_storage_path); - return 0; + return getTotalSize(keeper_context->getSnapshotDisk()); } Keeper4LWInfo KeeperDispatcher::getKeeper4LWInfo() const diff --git a/src/Coordination/KeeperSnapshotManagerS3.h b/src/Coordination/KeeperSnapshotManagerS3.h index 908deb76851..e17cf5a1cfb 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.h +++ b/src/Coordination/KeeperSnapshotManagerS3.h @@ -6,11 +6,12 @@ #include #include +#include + #if USE_AWS_S3 #include #include -#include #include #endif @@ -62,7 +63,7 @@ public: KeeperSnapshotManagerS3() = default; void updateS3Configuration(const Poco::Util::AbstractConfiguration &, const MultiVersion::Version &) {} - void uploadSnapshot(const std::string &, [[maybe_unused]] bool async_upload = true) {} + void uploadSnapshot(const SnapshotFileInfo &, [[maybe_unused]] bool async_upload = true) {} void startup(const Poco::Util::AbstractConfiguration &, const MultiVersion::Version &) {} diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 49243541bc8..5cfc9333a66 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -35,11 +35,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int SYSTEM_ERROR; -} - -namespace -{ } KeeperStateMachine::KeeperStateMachine( From bb77441acb488bb2a32f691ce724b2950ec0d9ba Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 May 2023 13:31:11 +0000 Subject: [PATCH 084/515] Add support for old storage disks --- src/Coordination/KeeperContext.cpp | 30 ++++-- src/Coordination/KeeperContext.h | 2 + src/Coordination/KeeperSnapshotManager.cpp | 118 +++++++++++++-------- src/Coordination/KeeperSnapshotManager.h | 5 +- src/Coordination/KeeperStateMachine.cpp | 2 +- 5 files changed, 103 insertions(+), 54 deletions(-) diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index da49868f706..1d6f1be9bfb 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -8,7 +8,6 @@ namespace DB { - KeeperContext::KeeperContext(bool standalone_keeper_) : disk_selector(std::make_shared()) , standalone_keeper(standalone_keeper_) @@ -28,13 +27,19 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) else current_log_storage = log_storage; - Poco::Util::AbstractConfiguration::Keys old_log_disk_name_keys; - config.keys("keeper_server", old_log_disk_name_keys); - for (const auto & key : old_log_disk_name_keys) + const auto collect_old_disk_names = [&](const std::string_view key_prefix, std::vector & disk_names) { - if (key.starts_with("old_log_storage_disk")) - old_log_disk_names.push_back(config.getString("keeper_server." + key)); - } + Poco::Util::AbstractConfiguration::Keys disk_name_keys; + config.keys("keeper_server", disk_name_keys); + for (const auto & key : disk_name_keys) + { + if (key.starts_with(key_prefix)) + disk_names.push_back(config.getString(fmt::format("keeper_server.{}", key_prefix))); + } + }; + + collect_old_disk_names("old_log_storage_disk", old_log_disk_names); + collect_old_disk_names("old_snapshot_storage_disk", old_snapshot_disk_names); snapshot_storage = getSnapshotsPathFromConfig(config); @@ -107,6 +112,17 @@ DiskPtr KeeperContext::getSnapshotDisk() const return getDisk(snapshot_storage); } +std::vector KeeperContext::getOldSnapshotDisks() const +{ + std::vector old_snapshot_disks; + old_snapshot_disks.reserve(old_snapshot_disk_names.size()); + + for (const auto & disk_name : old_snapshot_disk_names) + old_snapshot_disks.push_back(disk_selector->get(disk_name)); + + return old_snapshot_disks; +} + void KeeperContext::setSnapshotDisk(DiskPtr disk) { snapshot_storage = std::move(disk); diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 6b7af3a60db..e41d8e35032 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -40,6 +40,7 @@ public: void setLogDisk(DiskPtr disk); DiskPtr getSnapshotDisk() const; + std::vector getOldSnapshotDisks() const; void setSnapshotDisk(DiskPtr disk); DiskPtr getStateFileDisk() const; @@ -69,6 +70,7 @@ private: Storage state_file_storage; std::vector old_log_disk_names; + std::vector old_snapshot_disk_names; bool standalone_keeper; }; diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 3bfe700bcd5..77abbfb2054 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -519,49 +519,79 @@ KeeperSnapshotManager::KeeperSnapshotManager( , storage_tick_time(storage_tick_time_) , keeper_context(keeper_context_) { + const auto load_snapshot_from_disk = [&](DiskPtr disk) + { + std::unordered_set invalid_snapshots; + /// collect invalid snapshots + for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) + { + const auto & name = it->name(); + if (name.empty()) + continue; + + if (startsWith(name, "tmp_")) + { + disk->removeFile(it->path()); + invalid_snapshots.insert(name.substr(4)); + continue; + } + } + + /// process snapshots + for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) + { + const auto & name = it->name(); + if (name.empty()) + continue; + + /// Not snapshot file + if (!startsWith(name, "snapshot_")) + continue; + + if (invalid_snapshots.contains(name)) + { + disk->removeFile(it->path()); + continue; + } + + size_t snapshot_up_to = getSnapshotPathUpToLogIdx(name); + auto [_, inserted] = existing_snapshots.insert_or_assign(snapshot_up_to, SnapshotFileInfo{it->path(), disk}); + + if (!inserted) + LOG_WARNING( + &Poco::Logger::get("KeeperSnapshotManager"), + "Found another snapshots with last log idx {}, will use snapshot from disk {}", + snapshot_up_to, + disk->getName()); + } + }; + + for (const auto & disk : keeper_context->getOldSnapshotDisks()) + load_snapshot_from_disk(disk); + auto disk = getDisk(); - - std::unordered_set invalid_snapshots; - /// collect invalid snapshots - for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) - { - const auto & name = it->name(); - if (name.empty()) - continue; - - if (startsWith(name, "tmp_")) - { - disk->removeFile(it->path()); - invalid_snapshots.insert(name.substr(4)); - continue; - } - - } - - /// process snapshots - for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) - { - const auto & name = it->name(); - if (name.empty()) - continue; - - /// Not snapshot file - if (!startsWith(name, "snapshot_")) - continue; - - if (invalid_snapshots.contains(name)) - { - disk->removeFile(it->path()); - continue; - } - - size_t snapshot_up_to = getSnapshotPathUpToLogIdx(name); - existing_snapshots[snapshot_up_to] = it->path(); - } + load_snapshot_from_disk(disk); removeOutdatedSnapshotsIfNeeded(); -} + /// move snapshots from old disks to new one + for (auto & [_, file_info] : existing_snapshots) + { + if (file_info.disk == disk) + continue; + + auto file_path = fs::path(file_info.path); + auto tmp_snapshot_path = file_path.parent_path() / ("tmp_" + file_path.filename().generic_string()); + + { + disk->writeFile(tmp_snapshot_path); + } + + file_info.disk->copyFile(file_info.path, *disk, file_info.path, {}); + disk->removeFile(tmp_snapshot_path); + file_info.disk = disk; + } +} SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx) { @@ -599,7 +629,8 @@ nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBuff } catch (const DB::Exception &) { - getDisk()->removeFile(latest_itr->second); + const auto & [path, disk] = latest_itr->second; + disk->removeFile(path); existing_snapshots.erase(latest_itr->first); tryLogCurrentException(__PRETTY_FUNCTION__); } @@ -610,9 +641,9 @@ nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBuff nuraft::ptr KeeperSnapshotManager::deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const { - const std::string & snapshot_path = existing_snapshots.at(up_to_log_idx); + const auto & [snapshot_path, snapshot_disk] = existing_snapshots.at(up_to_log_idx); WriteBufferFromNuraftBuffer writer; - auto reader = getDisk()->readFile(snapshot_path); + auto reader = snapshot_disk->readFile(snapshot_path); copyData(*reader, writer); return writer.getBuffer(); } @@ -690,7 +721,8 @@ void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) auto itr = existing_snapshots.find(log_idx); if (itr == existing_snapshots.end()) throw Exception(ErrorCodes::UNKNOWN_SNAPSHOT, "Unknown snapshot with log index {}", log_idx); - getDisk()->removeFile(itr->second); + const auto & [path, disk] = itr->second; + disk->removeFile(path); existing_snapshots.erase(itr); } diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 036c0cab62b..0afe582ef59 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -149,11 +149,10 @@ public: { if (!existing_snapshots.empty()) { - const auto & path = existing_snapshots.at(getLatestSnapshotIndex()); + const auto & [path, disk] = existing_snapshots.at(getLatestSnapshotIndex()); try { - auto disk = getDisk(); if (disk->exists(path)) return {path, disk}; } @@ -176,7 +175,7 @@ private: /// How many snapshots to keep before remove const size_t snapshots_to_keep; /// All existing snapshots in our path (log_index -> path) - std::map existing_snapshots; + std::map existing_snapshots; /// Compress snapshots in common ZSTD format instead of custom ClickHouse block LZ4 format const bool compress_snapshots_zstd; /// Superdigest for deserialization of storage diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 5cfc9333a66..9a1cb1941b7 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -404,11 +404,11 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res } else { + latest_snapshot_meta = snapshot->snapshot_meta; /// we rely on the fact that the snapshot disk cannot be changed during runtime if (isLocalDisk(*keeper_context->getSnapshotDisk())) { latest_snapshot_info = snapshot_manager.serializeSnapshotToDisk(*snapshot); - latest_snapshot_meta = snapshot->snapshot_meta; latest_snapshot_buf = nullptr; } else From 30ff5113d98141ab4adb05ad08b7ffb48888c33d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 May 2023 13:33:52 +0000 Subject: [PATCH 085/515] Fixing more tests. --- src/Functions/in.cpp | 8 +- src/Interpreters/ActionsVisitor.cpp | 2 +- src/Interpreters/PreparedSets.cpp | 70 +++++++++++------ src/Interpreters/PreparedSets.h | 17 +++-- src/Interpreters/Set.cpp | 76 +++++++++++-------- src/Interpreters/Set.h | 18 ++++- src/Planner/CollectSets.cpp | 2 +- .../CreateSetAndFilterOnTheFlyTransform.cpp | 2 +- src/Storages/KVStorageUtils.cpp | 4 +- src/Storages/MergeTree/KeyCondition.cpp | 8 +- .../MergeTreeIndexConditionBloomFilter.cpp | 2 +- .../MergeTree/MergeTreeIndexFullText.cpp | 2 +- .../MergeTree/MergeTreeIndexInverted.cpp | 2 +- .../System/StorageSystemZooKeeper.cpp | 4 +- 14 files changed, 136 insertions(+), 81 deletions(-) diff --git a/src/Functions/in.cpp b/src/Functions/in.cpp index eb623951bf3..6a88a413c63 100644 --- a/src/Functions/in.cpp +++ b/src/Functions/in.cpp @@ -127,12 +127,12 @@ public: } auto future_set = column_set->getData(); - if (!future_set || !future_set->isFilled()) + if (!future_set || !future_set->isReady()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Not-ready Set passed as the second argument for function '{}'", getName()); - if (auto * for_tuple = typeid_cast(future_set.get())) - if (!for_tuple->isReady()) - for_tuple->buildForTuple(size_limits, transform_null_in); + // if (auto * for_tuple = typeid_cast(future_set.get())) + // if (!for_tuple->isReady()) + // for_tuple->buildForTuple(size_limits, transform_null_in); auto set = future_set->get(); auto set_types = set->getDataTypes(); diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 1405568aa71..da8666466fc 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -458,7 +458,7 @@ FutureSetPtr makeExplicitSet( else block = createBlockForSet(left_arg_type, right_arg, set_element_types, context); - return prepared_sets.addFromTuple(set_key, block); + return prepared_sets.addFromTuple(set_key, block, context->getSettings()); } class ScopeStack::Index diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 3b63d942404..eebc59e85d5 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -14,6 +14,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int SET_SIZE_LIMIT_EXCEEDED; } PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) @@ -120,9 +121,9 @@ FutureSetPtr PreparedSets::addFromStorage(const PreparedSetKey & key, SetPtr set return it->second; } -FutureSetPtr PreparedSets::addFromTuple(const PreparedSetKey & key, Block block) +FutureSetPtr PreparedSets::addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings) { - auto from_tuple = std::make_shared(std::move(block)); + auto from_tuple = std::make_shared(std::move(block), settings); auto [it, inserted] = sets.emplace(key, std::move(from_tuple)); if (!inserted) @@ -292,7 +293,25 @@ SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordere return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); } -FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) {} +FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) +{ + bool create_ordered_set = false; + auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); + set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); + set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); + + Columns columns; + columns.reserve(block.columns()); + for (const auto & column : block) + columns.emplace_back(column.column); + + set_key_columns.filter = ColumnUInt8::create(block.rows()); + + set->initSetElements(); + set->insertFromColumns(columns, set_key_columns); + set->finishInsert(); + //block(std::move(block_)) +} FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquery(std::move(subquery_)) {} @@ -301,35 +320,40 @@ FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) { SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) { const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, true); - fill(size_limits, settings.transform_null_in, true); + auto limits = getSizeLimitsForSet(settings, true); + + if (!limits.check(set->getTotalRowCount(), set->getTotalByteCount(), "IN-set", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED)) + return nullptr; + + set->initSetElements(); + set->appendSetElements(set_key_columns); return set; } -std::unique_ptr FutureSetFromTuple::build(const ContextPtr & context) +std::unique_ptr FutureSetFromTuple::build(const ContextPtr &) { - const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, false); - fill(size_limits, settings.transform_null_in, false); + // const auto & settings = context->getSettingsRef(); + // auto size_limits = getSizeLimitsForSet(settings, false); + // fill(size_limits, settings.transform_null_in, false); return nullptr; } -void FutureSetFromTuple::buildForTuple(SizeLimits size_limits, bool transform_null_in) -{ - fill(size_limits, transform_null_in, false); -} +// void FutureSetFromTuple::buildForTuple(SizeLimits size_limits, bool transform_null_in) +// { +// fill(size_limits, transform_null_in, false); +// } -void FutureSetFromTuple::fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set) -{ - //std::cerr << StackTrace().toString() << std::endl; +// void FutureSetFromTuple::fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set) +// { +// //std::cerr << StackTrace().toString() << std::endl; - if (set) - return; +// if (set) +// return; - set = std::make_shared(size_limits, create_ordered_set, transform_null_in); - set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); - set->insertFromBlock(block.getColumnsWithTypeAndName()); - set->finishInsert(); -} +// set = std::make_shared(size_limits, create_ordered_set, transform_null_in); +// set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); +// set->insertFromBlock(block.getColumnsWithTypeAndName()); +// set->finishInsert(); +// } }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index ef7aba38f24..35bbc1e12fb 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -80,24 +80,23 @@ using FutureSetPtr = std::shared_ptr; class FutureSetFromTuple final : public FutureSet { public: - FutureSetFromTuple(Block block_); + FutureSetFromTuple(Block block, const Settings & settings); - bool isReady() const override { return set != nullptr; } + bool isReady() const override { return true; } bool isFilled() const override { return true; } SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override; - std::unique_ptr build(const ContextPtr & context) override; + std::unique_ptr build(const ContextPtr &) override; - void buildForTuple(SizeLimits size_limits, bool transform_null_in); +/// void buildForTuple(SizeLimits size_limits, bool transform_null_in); private: - Block block; - SetPtr set; + Set::SetKeyColumns set_key_columns; - void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set); + //void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set); }; /// Information on how to build set for the [GLOBAL] IN section. @@ -145,6 +144,8 @@ public: return nullptr; auto plan = buildPlan(context, true); + if (!plan) + return nullptr; auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); @@ -249,7 +250,7 @@ public: // SizeLimits set_size_limit, bool transform_null_in); FutureSetPtr addFromStorage(const PreparedSetKey & key, SetPtr set_); - FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block); + FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings); FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery); void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage); diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index a7bea63bd99..5adf3d07353 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -156,25 +156,46 @@ void Set::setHeader(const ColumnsWithTypeAndName & header) { /// Create empty columns with set values in advance. /// It is needed because set may be empty, so method 'insertFromBlock' will be never called. - set_elements.reserve(keys_size); - for (const auto & type : set_elements_types) - set_elements.emplace_back(type->createColumn()); + initSetElements(); } /// Choose data structure to use for the set. data.init(data.chooseMethod(key_columns, key_sizes)); } +void Set::initSetElements() +{ + set_elements.reserve(keys_size); + for (const auto & type : set_elements_types) + set_elements.emplace_back(type->createColumn()); +} + bool Set::insertFromBlock(const ColumnsWithTypeAndName & columns) { Columns cols; cols.reserve(columns.size()); for (const auto & column : columns) cols.emplace_back(column.column); - return insertFromBlock(cols); + return insertFromColumns(cols); } -bool Set::insertFromBlock(const Columns & columns) +bool Set::insertFromColumns(const Columns & columns) +{ + size_t rows = columns.at(0)->size(); + + SetKeyColumns holder; + /// Filter to extract distinct values from the block. + if (fill_set_elements) + holder.filter = ColumnUInt8::create(rows); + + bool inserted = insertFromColumns(columns, holder); + if (inserted && fill_set_elements) + appendSetElements(holder); + + return inserted; +} + +bool Set::insertFromColumns(const Columns & columns, SetKeyColumns & holder) { std::lock_guard lock(rwlock); @@ -183,15 +204,13 @@ bool Set::insertFromBlock(const Columns & columns) ColumnRawPtrs key_columns; key_columns.reserve(keys_size); - - /// The constant columns to the right of IN are not supported directly. For this, they first materialize. - Columns materialized_columns; + holder.key_columns.reserve(keys_size); /// Remember the columns we will work with for (size_t i = 0; i < keys_size; ++i) { - materialized_columns.emplace_back(columns.at(i)->convertToFullIfNeeded()); - key_columns.emplace_back(materialized_columns.back().get()); + holder.key_columns.emplace_back(columns.at(i)->convertToFullIfNeeded()); + key_columns.emplace_back(holder.key_columns.back().get()); } size_t rows = columns.at(0)->size(); @@ -202,40 +221,37 @@ bool Set::insertFromBlock(const Columns & columns) if (!transform_null_in) null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); - /// Filter to extract distinct values from the block. - ColumnUInt8::MutablePtr filter; - if (fill_set_elements) - filter = ColumnUInt8::create(rows); - switch (data.type) { case SetVariants::Type::EMPTY: break; #define M(NAME) \ case SetVariants::Type::NAME: \ - insertFromBlockImpl(*data.NAME, key_columns, rows, data, null_map, filter ? &filter->getData() : nullptr); \ + insertFromBlockImpl(*data.NAME, key_columns, rows, data, null_map, holder.filter ? &holder.filter->getData() : nullptr); \ break; APPLY_FOR_SET_VARIANTS(M) #undef M } - if (fill_set_elements) - { - for (size_t i = 0; i < keys_size; ++i) - { - auto filtered_column = key_columns[i]->filter(filter->getData(), rows); - if (set_elements[i]->empty()) - set_elements[i] = filtered_column; - else - set_elements[i]->insertRangeFrom(*filtered_column, 0, filtered_column->size()); - if (transform_null_in && null_map_holder) - set_elements[i]->insert(Null{}); - } - } - return limits.check(data.getTotalRowCount(), data.getTotalByteCount(), "IN-set", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); } +void Set::appendSetElements(SetKeyColumns & holder) +{ + //std::cerr << "========= " << keys_size << ' ' << holder.key_columns.size() << std::endl; + size_t rows = holder.key_columns.at(0)->size(); + for (size_t i = 0; i < keys_size; ++i) + { + auto filtered_column = holder.key_columns[i]->filter(holder.filter->getData(), rows); + if (set_elements[i]->empty()) + set_elements[i] = filtered_column; + else + set_elements[i]->insertRangeFrom(*filtered_column, 0, filtered_column->size()); + if (transform_null_in && holder.null_map_holder) + set_elements[i]->insert(Null{}); + } +} + void Set::checkIsCreated() const { if (!is_created.load()) diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index fff5fa4e1b1..cb47fde7f7d 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -45,9 +45,23 @@ public: void setHeader(const ColumnsWithTypeAndName & header); /// Returns false, if some limit was exceeded and no need to insert more data. - bool insertFromBlock(const Columns & columns); + bool insertFromColumns(const Columns & columns); bool insertFromBlock(const ColumnsWithTypeAndName & columns); + + struct SetKeyColumns + { + //ColumnRawPtrs key_columns; + /// The constant columns to the right of IN are not supported directly. For this, they first materialize. + Columns key_columns; + ColumnPtr null_map_holder; + ColumnUInt8::MutablePtr filter; + }; + + void initSetElements(); + bool insertFromColumns(const Columns & columns, SetKeyColumns & holder); + void appendSetElements(SetKeyColumns & holder); + /// Call after all blocks were inserted. To get the information that set is already created. void finishInsert() { is_created = true; } @@ -68,7 +82,7 @@ public: const DataTypes & getDataTypes() const { return data_types; } const DataTypes & getElementsTypes() const { return set_elements_types; } - bool hasExplicitSetElements() const { return fill_set_elements; } + bool hasExplicitSetElements() const { return fill_set_elements || (!set_elements.empty() && set_elements.front()->size() == data.getTotalRowCount()); } Columns getSetElements() const { checkIsCreated(); return { set_elements.begin(), set_elements.end() }; } void checkColumnsNumber(size_t num_key_columns) const; diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 6c970e0e91b..d66424f89ec 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -82,7 +82,7 @@ public: auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); - sets.addFromTuple(set_key, std::move(set)); + sets.addFromTuple(set_key, std::move(set), settings); //planner_context.registerSet(set_key, PlannerSet(FutureSet(std::move(set)))); } diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp index 4278eb8e8b2..59c4b9a6a87 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp @@ -106,7 +106,7 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) if (chunk.getNumRows()) { Columns key_columns = getColumnsByIndices(chunk, key_column_indices); - bool limit_exceeded = !set->insertFromBlock(key_columns); + bool limit_exceeded = !set->insertFromColumns(key_columns); if (limit_exceeded) { auto prev_state = set->state.exchange(SetWithState::State::Suspended); diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index 281236e631e..f4a4b6e9af1 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -76,8 +76,8 @@ bool traverseASTFilter( if (!future_set) return false; - if (!future_set->isReady()) - future_set->buildOrderedSetInplace(context); + //if (!future_set->isReady()) + future_set->buildOrderedSetInplace(context); auto set = future_set->get(); if (!set) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 4b053a37aa2..7fcc111ced9 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1212,11 +1212,11 @@ bool KeyCondition::tryPrepareSetIndex( // LOG_TRACE(&Poco::Logger::get("KK"), "Found set for {}", right_arg.getColumnName()); - if (!future_set->isReady()) - { + //if (!future_set->isReady()) + //{ // LOG_TRACE(&Poco::Logger::get("KK"), "Building set inplace for {}", right_arg.getColumnName()); - future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); - } + future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); + //} auto prepared_set = future_set->get(); if (!prepared_set) diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 2bd9db12b93..3d0883a1241 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -316,7 +316,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNo //std::cerr << "==== Finding set for MergeTreeBF " << bool(future_set) << std::endl; - if (future_set && !future_set->isReady()) + if (future_set) // && !future_set->isReady()) { //std::cerr << "==== not ready, building " << std::endl; future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 5e1d23df3c7..d14ff4fa20b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -625,7 +625,7 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( return false; auto future_set = right_argument.tryGetPreparedSet(data_types); - if (future_set && !future_set->isReady()) + if (future_set) // && !future_set->isReady()) future_set->buildOrderedSetInplace(right_argument.getTreeContext().getQueryContext()); ConstSetPtr prepared_set; diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index a64f81807ae..1d3b923056a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -661,7 +661,7 @@ bool MergeTreeConditionInverted::tryPrepareSetGinFilter( //std::cerr << "==== Set for MergeTreeConditionInverted" << bool(future_set) << std::endl; - if (future_set && !future_set->isReady()) + if (future_set) // && !future_set->isReady()) future_set->buildOrderedSetInplace(rhs.getTreeContext().getQueryContext()); ConstSetPtr prepared_set; diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 0f45f6825f6..34b463eadee 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -317,8 +317,8 @@ static void extractPathImpl(const ActionsDAG::Node & node, Paths & res, ContextP if (!future_set) return; - if (!future_set->isReady()) - future_set->buildOrderedSetInplace(context); + //if (!future_set->isReady()) + future_set->buildOrderedSetInplace(context); auto set = future_set->get(); if (!set) From 6a8a21e09a49303919d2357f0b7f9ca38c79684b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 May 2023 16:01:40 +0000 Subject: [PATCH 086/515] Add test for s3 disk restore --- src/Coordination/Changelog.cpp | 3 +- src/Coordination/KeeperSnapshotManager.cpp | 4 +- .../configs/disk_s3_storage.xml | 26 +++ .../configs/enable_keeper1.xml | 2 - .../configs/enable_keeper2.xml | 2 - .../configs/enable_keeper3.xml | 2 - .../configs/keeper1_snapshot_disk.xml | 5 + .../configs/keeper2_snapshot_disk.xml | 5 + .../configs/keeper3_snapshot_disk.xml | 5 + .../configs/local_storage_path.xml | 6 + .../test_keeper_restore_from_snapshot/test.py | 12 +- .../test_disk_s3.py | 152 ++++++++++++++++++ 12 files changed, 212 insertions(+), 12 deletions(-) create mode 100644 tests/integration/test_keeper_restore_from_snapshot/configs/disk_s3_storage.xml create mode 100644 tests/integration/test_keeper_restore_from_snapshot/configs/keeper1_snapshot_disk.xml create mode 100644 tests/integration/test_keeper_restore_from_snapshot/configs/keeper2_snapshot_disk.xml create mode 100644 tests/integration/test_keeper_restore_from_snapshot/configs/keeper3_snapshot_disk.xml create mode 100644 tests/integration/test_keeper_restore_from_snapshot/configs/local_storage_path.xml create mode 100644 tests/integration/test_keeper_restore_from_snapshot/test_disk_s3.py diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 875b0758d27..b289e1dc9f2 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -616,7 +616,8 @@ Changelog::Changelog(Poco::Logger * log_, LogFileSettings log_file_settings, Kee load_from_disk(disk); auto current_log_disk = getCurrentLogDisk(); - load_from_disk(current_log_disk); + if (disk != current_log_disk) + load_from_disk(current_log_disk); if (existing_changelogs.empty()) LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", disk->getPath()); diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 77abbfb2054..c3371501976 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -612,7 +612,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::bu disk->removeFile(tmp_snapshot_file_name); - existing_snapshots.emplace(up_to_log_idx, snapshot_file_name); + existing_snapshots.emplace(up_to_log_idx, SnapshotFileInfo{snapshot_file_name, disk}); removeOutdatedSnapshotsIfNeeded(); return {snapshot_file_name, disk}; @@ -750,7 +750,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStor disk->removeFile(tmp_snapshot_file_name); - existing_snapshots.emplace(up_to_log_idx, snapshot_file_name); + existing_snapshots.emplace(up_to_log_idx, SnapshotFileInfo{snapshot_file_name, disk}); removeOutdatedSnapshotsIfNeeded(); return {snapshot_file_name, disk}; diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/disk_s3_storage.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/disk_s3_storage.xml new file mode 100644 index 00000000000..2ec4bcd77fb --- /dev/null +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/disk_s3_storage.xml @@ -0,0 +1,26 @@ + + + + + s3_plain + http://minio1:9001/root/data/snapshots1/ + minio + minio123 + + + s3_plain + http://minio1:9001/root/data/snapshots2/ + minio + minio123 + + + s3_plain + http://minio1:9001/root/data/snapshots3/ + minio + minio123 + + + + + + diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper1.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper1.xml index 4ea543e6f31..510913b5dc4 100644 --- a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper1.xml @@ -2,8 +2,6 @@ 9181 1 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots 5000 diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper2.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper2.xml index 4bf3083c1fa..0d50573df01 100644 --- a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper2.xml @@ -2,8 +2,6 @@ 9181 2 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots 5000 diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper3.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper3.xml index b9e2a2d0422..8fa322cb9e7 100644 --- a/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/enable_keeper3.xml @@ -2,8 +2,6 @@ 9181 3 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots 5000 diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/keeper1_snapshot_disk.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/keeper1_snapshot_disk.xml new file mode 100644 index 00000000000..01759e2771a --- /dev/null +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/keeper1_snapshot_disk.xml @@ -0,0 +1,5 @@ + + + snapshot_s3_plain1 + + \ No newline at end of file diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/keeper2_snapshot_disk.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/keeper2_snapshot_disk.xml new file mode 100644 index 00000000000..6c33c5fb300 --- /dev/null +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/keeper2_snapshot_disk.xml @@ -0,0 +1,5 @@ + + + snapshot_s3_plain2 + + \ No newline at end of file diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/keeper3_snapshot_disk.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/keeper3_snapshot_disk.xml new file mode 100644 index 00000000000..5016ccd581b --- /dev/null +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/keeper3_snapshot_disk.xml @@ -0,0 +1,5 @@ + + + snapshot_s3_plain3 + + \ No newline at end of file diff --git a/tests/integration/test_keeper_restore_from_snapshot/configs/local_storage_path.xml b/tests/integration/test_keeper_restore_from_snapshot/configs/local_storage_path.xml new file mode 100644 index 00000000000..63cb958c88e --- /dev/null +++ b/tests/integration/test_keeper_restore_from_snapshot/configs/local_storage_path.xml @@ -0,0 +1,6 @@ + + + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + \ No newline at end of file diff --git a/tests/integration/test_keeper_restore_from_snapshot/test.py b/tests/integration/test_keeper_restore_from_snapshot/test.py index bc33689dd20..e4d5793bb17 100644 --- a/tests/integration/test_keeper_restore_from_snapshot/test.py +++ b/tests/integration/test_keeper_restore_from_snapshot/test.py @@ -9,13 +9,19 @@ import time cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/enable_keeper1.xml"], stay_alive=True + "node1", + main_configs=["configs/enable_keeper1.xml", "configs/local_storage_path.xml"], + stay_alive=True, ) node2 = cluster.add_instance( - "node2", main_configs=["configs/enable_keeper2.xml"], stay_alive=True + "node2", + main_configs=["configs/enable_keeper2.xml", "configs/local_storage_path.xml"], + stay_alive=True, ) node3 = cluster.add_instance( - "node3", main_configs=["configs/enable_keeper3.xml"], stay_alive=True + "node3", + main_configs=["configs/enable_keeper3.xml", "configs/local_storage_path.xml"], + stay_alive=True, ) from kazoo.client import KazooClient, KazooState diff --git a/tests/integration/test_keeper_restore_from_snapshot/test_disk_s3.py b/tests/integration/test_keeper_restore_from_snapshot/test_disk_s3.py new file mode 100644 index 00000000000..1226df75203 --- /dev/null +++ b/tests/integration/test_keeper_restore_from_snapshot/test_disk_s3.py @@ -0,0 +1,152 @@ +#!/usr/bin/env python3 +import pytest +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as keeper_utils +import random +import string +import os +import time + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", + main_configs=[ + "configs/enable_keeper1.xml", + "configs/disk_s3_storage.xml", + "configs/keeper1_snapshot_disk.xml", + ], + stay_alive=True, + with_minio=True, +) +node2 = cluster.add_instance( + "node2", + main_configs=[ + "configs/enable_keeper2.xml", + "configs/disk_s3_storage.xml", + "configs/keeper2_snapshot_disk.xml", + ], + stay_alive=True, + with_minio=True, +) +node3 = cluster.add_instance( + "node3", + main_configs=[ + "configs/enable_keeper3.xml", + "configs/disk_s3_storage.xml", + "configs/keeper3_snapshot_disk.xml", + ], + stay_alive=True, + with_minio=True, +) + +from kazoo.client import KazooClient, KazooState + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def get_fake_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient( + hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout + ) + _fake_zk_instance.start() + return _fake_zk_instance + + +def stop_zk(zk): + try: + if zk: + zk.stop() + zk.close() + except: + pass + + +def test_recover_from_snapshot_with_disk_s3(started_cluster): + try: + node1_zk = node2_zk = node3_zk = None + node1_zk = get_fake_zk("node1") + node2_zk = get_fake_zk("node2") + node3_zk = get_fake_zk("node3") + + node1_zk.create("/test_snapshot_multinode_recover", "somedata".encode()) + + node2_zk.sync("/test_snapshot_multinode_recover") + node3_zk.sync("/test_snapshot_multinode_recover") + + assert node1_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata" + assert node2_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata" + assert node3_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata" + + node3.stop_clickhouse(kill=True) + + # at least we will have 2 snapshots + for i in range(435): + node1_zk.create( + "/test_snapshot_multinode_recover" + str(i), + ("somedata" + str(i)).encode(), + ) + + for i in range(435): + if i % 10 == 0: + node1_zk.delete("/test_snapshot_multinode_recover" + str(i)) + + finally: + for zk in [node1_zk, node2_zk, node3_zk]: + stop_zk(zk) + + # stale node should recover from leader's snapshot + # with some sanitizers can start longer than 5 seconds + node3.start_clickhouse(20) + keeper_utils.wait_until_connected(cluster, node3) + print("Restarted") + + try: + node1_zk = node2_zk = node3_zk = None + node1_zk = get_fake_zk("node1") + node2_zk = get_fake_zk("node2") + node3_zk = get_fake_zk("node3") + + node1_zk.sync("/test_snapshot_multinode_recover") + node2_zk.sync("/test_snapshot_multinode_recover") + node3_zk.sync("/test_snapshot_multinode_recover") + + assert node1_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata" + assert node2_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata" + assert node3_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata" + + for i in range(435): + if i % 10 != 0: + assert ( + node1_zk.get("/test_snapshot_multinode_recover" + str(i))[0] + == ("somedata" + str(i)).encode() + ) + assert ( + node2_zk.get("/test_snapshot_multinode_recover" + str(i))[0] + == ("somedata" + str(i)).encode() + ) + assert ( + node3_zk.get("/test_snapshot_multinode_recover" + str(i))[0] + == ("somedata" + str(i)).encode() + ) + else: + assert ( + node1_zk.exists("/test_snapshot_multinode_recover" + str(i)) is None + ) + assert ( + node2_zk.exists("/test_snapshot_multinode_recover" + str(i)) is None + ) + assert ( + node3_zk.exists("/test_snapshot_multinode_recover" + str(i)) is None + ) + finally: + for zk in [node1_zk, node2_zk, node3_zk]: + stop_zk(zk) From 7ebe19f5fb1390bc5f2dac3a5cca7db9c76bdd98 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 May 2023 19:18:11 +0000 Subject: [PATCH 087/515] Fixing more tests --- src/Interpreters/ActionsVisitor.cpp | 7 ++- src/Interpreters/Context.cpp | 15 +++++ src/Interpreters/Context.h | 1 + src/Interpreters/DatabaseCatalog.h | 3 + src/Interpreters/GlobalSubqueriesVisitor.h | 19 ++++--- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/PreparedSets.cpp | 55 +++++++++++++++---- src/Interpreters/PreparedSets.h | 35 ++++-------- src/Planner/CollectSets.cpp | 3 +- src/Planner/Planner.cpp | 2 +- src/Planner/PlannerActionsVisitor.cpp | 8 ++- src/Processors/QueryPlan/CreatingSetsStep.cpp | 4 +- .../QueryPlan/DistributedCreateLocalPlan.cpp | 4 +- .../QueryPlan/ReadFromMemoryStorageStep.cpp | 6 +- .../QueryPlan/ReadFromMemoryStorageStep.h | 2 + .../Transforms/CreatingSetsTransform.cpp | 13 +++-- .../Transforms/CreatingSetsTransform.h | 5 +- src/QueryPipeline/QueryPipelineBuilder.cpp | 4 +- src/QueryPipeline/QueryPipelineBuilder.h | 5 +- src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageMemory.h | 2 + 21 files changed, 132 insertions(+), 65 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index da8666466fc..133fcbbfe87 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1405,6 +1405,8 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool if (auto set = data.prepared_sets->getFuture(set_key)) return set; + FutureSetPtr external_table_set; + /// A special case is if the name of the table is specified on the right side of the IN statement, /// and the table has the type Set (a previously prepared set). if (identifier) @@ -1417,6 +1419,9 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool if (StorageSet * storage_set = dynamic_cast(table.get())) return data.prepared_sets->addFromStorage(set_key, storage_set->getSet()); } + + if (auto tmp_table = data.getContext()->findExternalTable(table_id.getShortName())) + external_table_set = tmp_table->future_set; } /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. @@ -1438,7 +1443,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool subquery_for_set.createSource(*interpreter); } - return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set)); + return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), std::move(external_table_set)); } else { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d9f450191bc..f8fe99fbcc6 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1319,6 +1319,21 @@ void Context::addExternalTable(const String & table_name, TemporaryTableHolder & external_tables_mapping.emplace(table_name, std::make_shared(std::move(temporary_table))); } +std::shared_ptr Context::findExternalTable(const String & table_name) const +{ + if (isGlobalContext()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have external tables"); + + std::shared_ptr holder; + { + auto lock = getLock(); + auto iter = external_tables_mapping.find(table_name); + if (iter == external_tables_mapping.end()) + return {}; + holder = iter->second; + } + return holder; +} std::shared_ptr Context::removeExternalTable(const String & table_name) { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 1be662e0958..1a0ee891699 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -611,6 +611,7 @@ public: Tables getExternalTables() const; void addExternalTable(const String & table_name, TemporaryTableHolder && temporary_table); + std::shared_ptr findExternalTable(const String & table_name) const; std::shared_ptr removeExternalTable(const String & table_name); const Scalars & getScalars() const; diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 51e9fbdb936..258ea2dee7c 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -79,6 +79,8 @@ private: using DDLGuardPtr = std::unique_ptr; +class FutureSet; +using FutureSetPtr = std::shared_ptr; /// Creates temporary table in `_temporary_and_external_tables` with randomly generated unique StorageID. /// Such table can be accessed from everywhere by its ID. @@ -111,6 +113,7 @@ struct TemporaryTableHolder : boost::noncopyable, WithContext IDatabase * temporary_tables = nullptr; UUID id = UUIDHelpers::Nil; + FutureSetPtr future_set; }; ///TODO maybe remove shared_ptr from here? diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index a872af529aa..195839da04f 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -64,6 +64,7 @@ public: void addExternalStorage(ASTPtr & ast, bool set_alias = false) { + // std::cerr << "=============== addExternalStorage is remote " << is_remote << std::endl; /// With nondistributed queries, creating temporary tables does not make sense. if (!is_remote) return; @@ -163,10 +164,10 @@ public: /// We need to materialize external tables immediately because reading from distributed /// tables might generate local plans which can refer to external tables during index /// analysis. It's too late to populate the external table via CreatingSetsTransform. - if (is_explain) - { - /// Do not materialize external tables if it's explain statement. - } + // if (is_explain) + // { + // /// Do not materialize external tables if it's explain statement. + // } // else if (getContext()->getSettingsRef().use_index_for_in_with_subqueries) // { // auto external_table = external_storage_holder->getTable(); @@ -176,13 +177,15 @@ public: // CompletedPipelineExecutor executor(io.pipeline); // executor.execute(); // } - else + // else { // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); // subquery_for_set.createSource(*interpreter, external_storage); auto key = subquery_or_table_name->getColumnName(); auto set_key = PreparedSetKey::forSubquery(database_and_table_name->getTreeHash()); + // std::cerr << "====== Adding key " << set_key.toString() << std::endl; + if (!prepared_sets->getFuture(set_key)) { SubqueryForSet subquery_for_set; @@ -191,10 +194,12 @@ public: subquery_for_set.createSource(*interpreter); //std::cerr << reinterpret_cast(prepared_sets.get()) << std::endl; - prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set)); + auto future_set = prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), nullptr); + external_storage_holder->future_set = std::move(future_set); } else - prepared_sets->addStorageToSubquery(key, std::move(external_storage)); + throw Exception(ErrorCodes::LOGICAL_ERROR, "!!!!!!!!"); + //prepared_sets->addStorageToSubquery(key, std::move(external_storage)); } /** NOTE If it was written IN tmp_table - the existing temporary (but not external) table, diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8d305c07ce9..28a0e27384a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -3088,7 +3088,7 @@ void InterpreterSelectQuery::executeExtremes(QueryPlan & query_plan) void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_plan) { - auto subqueries = prepared_sets->detachSubqueries(context); + auto subqueries = prepared_sets->detachSubqueries(); if (!subqueries.empty()) { diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index eebc59e85d5..f2b70462227 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -101,14 +101,14 @@ String PreparedSetKey::toString() const /// If the subquery is not associated with any set, create default-constructed SubqueryForSet. /// It's aimed to fill external table passed to SubqueryForSet::createSource. -void PreparedSets::addStorageToSubquery(const String & subquery_id, StoragePtr storage) -{ - auto it = subqueries.find(subquery_id); - if (it == subqueries.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find subquery {}", subquery_id); +// void PreparedSets::addStorageToSubquery(const String & subquery_id, StoragePtr storage) +// { +// auto it = subqueries.find(subquery_id); +// if (it == subqueries.end()) +// throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find subquery {}", subquery_id); - it->second->addStorage(std::move(storage)); -} +// it->second->addStorage(std::move(storage)); +// } FutureSetPtr PreparedSets::addFromStorage(const PreparedSetKey & key, SetPtr set_) { @@ -132,10 +132,10 @@ FutureSetPtr PreparedSets::addFromTuple(const PreparedSetKey & key, Block block, return it->second; } -FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery) +FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, FutureSetPtr external_table_set) { auto id = subquery.key; - auto from_subquery = std::make_shared(std::move(subquery)); + auto from_subquery = std::make_shared(std::move(subquery), std::move(external_table_set)); auto [it, inserted] = sets.emplace(key, from_subquery); if (!inserted) @@ -145,7 +145,7 @@ FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryF // std::cerr << "========= PreparedSets::addFromSubquery\n"; // std::cerr << StackTrace().toString() << std::endl; - subqueries.emplace(id, std::move(from_subquery)); + subqueries.emplace_back(SetAndName{.name = id, .set = std::move(from_subquery)}); return it->second; } @@ -176,7 +176,7 @@ FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const // return res; // } -PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries(const ContextPtr &) +PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries() { auto res = std::move(subqueries); subqueries = SubqueriesForSets(); @@ -226,6 +226,36 @@ std::variant, SharedSet> PreparedSetsCache::findOrPromiseTo return promise_to_fill_set; } +SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) +{ + if (!context->getSettingsRef().use_index_for_in_with_subqueries) + return nullptr; + + if (set) + { + if (set->hasExplicitSetElements()) + return set; + + return nullptr; + } + + if (external_table_set) + return set = external_table_set->buildOrderedSetInplace(context); + + auto plan = buildPlan(context, true); + if (!plan) + return nullptr; + + auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + pipeline.complete(std::make_shared(Block())); + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); + + return set; +} + std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & context, bool create_ordered_set) { if (set) @@ -313,7 +343,8 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) //block(std::move(block_)) } -FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquery(std::move(subquery_)) {} +FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_) + : subquery(std::move(subquery_)), external_table_set(std::move(external_table_set_)) {} FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 35bbc1e12fb..c510e690e25 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -132,30 +132,13 @@ public: class FutureSetFromSubquery : public FutureSet { public: - FutureSetFromSubquery(SubqueryForSet subquery_); + FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_); bool isReady() const override { return set != nullptr; } bool isFilled() const override { return isReady(); } SetPtr get() const override { return set; } - SetPtr buildOrderedSetInplace(const ContextPtr & context) override - { - if (!context->getSettingsRef().use_index_for_in_with_subqueries) - return nullptr; - - auto plan = buildPlan(context, true); - if (!plan) - return nullptr; - - auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); - pipeline.complete(std::make_shared(Block())); - - CompletedPipelineExecutor executor(pipeline); - executor.execute(); - - return set; - } + SetPtr buildOrderedSetInplace(const ContextPtr & context) override; std::unique_ptr build(const ContextPtr & context) override { @@ -167,6 +150,7 @@ public: private: SetPtr set; SubqueryForSet subquery; + FutureSetPtr external_table_set; std::unique_ptr buildPlan(const ContextPtr & context, bool create_ordered_set); }; @@ -244,16 +228,21 @@ struct PreparedSetKey class PreparedSets { public: - using SubqueriesForSets = std::unordered_map>; + struct SetAndName + { + String name; + std::shared_ptr set; + }; + using SubqueriesForSets = std::vector; // SubqueryForSet & createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, // SizeLimits set_size_limit, bool transform_null_in); FutureSetPtr addFromStorage(const PreparedSetKey & key, SetPtr set_); FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings); - FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery); + FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, FutureSetPtr external_table_set); - void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage); + //void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage); FutureSetPtr getFuture(const PreparedSetKey & key) const; //SubqueryForSet & getSubquery(const String & subquery_id); @@ -262,7 +251,7 @@ public: /// Get subqueries and clear them. /// We need to build a plan for subqueries just once. That's why we can clear them after accessing them. /// SetPtr would still be available for consumers of PreparedSets. - SubqueriesForSets detachSubqueries(const ContextPtr &); + SubqueriesForSets detachSubqueries(); /// Returns all sets that match the given ast hash not checking types /// Used in KeyCondition and MergeTreeIndexConditionBloomFilter to make non exact match for types in PreparedSetKey diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index d66424f89ec..19dc569ff0c 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -107,7 +107,8 @@ public: subquery_for_set.key = planner_context.createSetKey(in_second_argument); subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); - sets.addFromSubquery(set_key, std::move(subquery_for_set)); + /// TODO + sets.addFromSubquery(set_key, std::move(subquery_for_set), nullptr); //planner_context.registerSet(set_key, PlannerSet(in_second_argument)); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index b1780212e51..5c3fc82a80b 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1468,7 +1468,7 @@ void Planner::buildPlanForQueryNode() if (!select_query_options.only_analyze) { - auto subqueries = planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()); + auto subqueries = planner_context->getPreparedSets().detachSubqueries(); if (!subqueries.empty()) { diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index e0844a6d2b1..bbac9bf5c4b 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -632,7 +632,13 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma DataTypes set_element_types; auto in_second_argument_node_type = in_second_argument->getNodeType(); - if (!(in_second_argument_node_type == QueryTreeNodeType::QUERY || in_second_argument_node_type == QueryTreeNodeType::UNION)) + // std::cerr << "=========== " << in_second_argument->getNodeTypeName() << std::endl; + bool subquery_or_table = + in_second_argument_node_type == QueryTreeNodeType::QUERY || + in_second_argument_node_type == QueryTreeNodeType::UNION || + in_second_argument_node_type == QueryTreeNodeType::TABLE; + + if (!subquery_or_table) { set_element_types = {in_first_argument->getResultType()}; const auto * left_tuple_type = typeid_cast(set_element_types.front().get()); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 34018b2144c..38506412531 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -48,7 +48,7 @@ CreatingSetStep::CreatingSetStep( void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, context); + pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, context->getPreparedSetsCache()); } void CreatingSetStep::updateOutputStream() @@ -189,7 +189,7 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, if (!prepared_sets || prepared_sets->empty()) return; - addCreatingSetsStep(query_plan, prepared_sets->detachSubqueries(context), context); + addCreatingSetsStep(query_plan, prepared_sets->detachSubqueries(), context); } DelayedCreatingSetsStep::DelayedCreatingSetsStep( diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index 62e369659d1..8e2221d564d 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -43,7 +43,7 @@ std::unique_ptr createLocalPlan( const Block & header, ContextPtr context, QueryProcessingStage::Enum processed_stage, - PreparedSetsPtr prepared_sets, + [[maybe_unused]] PreparedSetsPtr prepared_sets, size_t shard_num, size_t shard_count, size_t replica_num, @@ -99,7 +99,7 @@ std::unique_ptr createLocalPlan( } else { - auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options, prepared_sets); + auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options); //, prepared_sets); interpreter.buildQueryPlan(*query_plan); } diff --git a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp index 2080c31d253..4ad3cc7373a 100644 --- a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp @@ -96,11 +96,13 @@ private: }; ReadFromMemoryStorageStep::ReadFromMemoryStorageStep(const Names & columns_to_read_, + StoragePtr storage_, const StorageSnapshotPtr & storage_snapshot_, const size_t num_streams_, const bool delay_read_for_global_sub_queries_) : SourceStepWithFilter(DataStream{.header=storage_snapshot_->getSampleBlockForColumns(columns_to_read_)}), columns_to_read(columns_to_read_), + storage(std::move(storage_)), storage_snapshot(storage_snapshot_), num_streams(num_streams_), delay_read_for_global_sub_queries(delay_read_for_global_sub_queries_) @@ -142,9 +144,9 @@ Pipe ReadFromMemoryStorageStep::makePipe() storage_snapshot, nullptr /* data */, nullptr /* parallel execution index */, - [current_data](std::shared_ptr & data_to_initialize) + [storage = storage](std::shared_ptr & data_to_initialize) { - data_to_initialize = current_data; + data_to_initialize = static_cast(*storage).data.get(); })); } diff --git a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h index cec523ed58b..1122bfbb2a5 100644 --- a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h +++ b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h @@ -16,6 +16,7 @@ class ReadFromMemoryStorageStep final : public SourceStepWithFilter { public: ReadFromMemoryStorageStep(const Names & columns_to_read_, + StoragePtr storage_, const StorageSnapshotPtr & storage_snapshot_, size_t num_streams_, bool delay_read_for_global_sub_queries_); @@ -35,6 +36,7 @@ private: static constexpr auto name = "ReadFromMemoryStorage"; Names columns_to_read; + StoragePtr storage; StorageSnapshotPtr storage_snapshot; size_t num_streams; bool delay_read_for_global_sub_queries; diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index 6626d4b9795..24e17ff210f 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -27,11 +27,11 @@ CreatingSetsTransform::CreatingSetsTransform( Block out_header_, SubqueryForSet subquery_for_set_, SizeLimits network_transfer_limits_, - ContextPtr context_) + PreparedSetsCachePtr prepared_sets_cache_) : IAccumulatingTransform(std::move(in_header_), std::move(out_header_)) - , WithContext(context_) , subquery(std::move(subquery_for_set_)) , network_transfer_limits(std::move(network_transfer_limits_)) + , prepared_sets_cache(std::move(prepared_sets_cache_)) { } @@ -52,14 +52,13 @@ void CreatingSetsTransform::work() void CreatingSetsTransform::startSubquery() { /// Lookup the set in the cache if we don't need to build table. - auto ctx = context.lock(); - if (ctx && ctx->getPreparedSetsCache() && !subquery.table) + if (prepared_sets_cache && !subquery.table) { /// Try to find the set in the cache and wait for it to be built. /// Retry if the set from cache fails to be built. while (true) { - auto from_cache = ctx->getPreparedSetsCache()->findOrPromiseToBuild(subquery.key); + auto from_cache = prepared_sets_cache->findOrPromiseToBuild(subquery.key); if (from_cache.index() == 0) { promise_to_build = std::move(std::get<0>(from_cache)); @@ -89,9 +88,11 @@ void CreatingSetsTransform::startSubquery() if (subquery.table) LOG_TRACE(log, "Filling temporary table."); + // std::cerr << StackTrace().toString() << std::endl; + if (subquery.table) /// TODO: make via port - table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), getContext())); + table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), nullptr)); done_with_set = !subquery.set; done_with_table = !subquery.table; diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index 26bbc45933d..ef586b0655c 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -23,7 +23,7 @@ class PushingPipelineExecutor; /// Don't return any data. Sets are created when Finish status is returned. /// In general, several work() methods need to be called to finish. /// Independent processors is created for each subquery. -class CreatingSetsTransform : public IAccumulatingTransform, WithContext +class CreatingSetsTransform : public IAccumulatingTransform { public: CreatingSetsTransform( @@ -31,7 +31,7 @@ public: Block out_header_, SubqueryForSet subquery_for_set_, SizeLimits network_transfer_limits_, - ContextPtr context_); + PreparedSetsCachePtr prepared_sets_cache_); ~CreatingSetsTransform() override; @@ -55,6 +55,7 @@ private: bool done_with_table = true; SizeLimits network_transfer_limits; + PreparedSetsCachePtr prepared_sets_cache; size_t rows_to_transfer = 0; size_t bytes_to_transfer = 0; diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 764997e7b7e..4898501b980 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -569,7 +569,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe return left; } -void QueryPipelineBuilder::addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, ContextPtr context) +void QueryPipelineBuilder::addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, PreparedSetsCachePtr prepared_sets_cache) { resize(1); @@ -578,7 +578,7 @@ void QueryPipelineBuilder::addCreatingSetsTransform(const Block & res_header, Su res_header, std::move(subquery_for_set), limits, - context); + std::move(prepared_sets_cache)); InputPort * totals_port = nullptr; diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 3a5d65d4388..85b6f5c6772 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -33,6 +33,9 @@ class TableJoin; class QueryPipelineBuilder; using QueryPipelineBuilderPtr = std::unique_ptr; +class PreparedSetsCache; +using PreparedSetsCachePtr = std::shared_ptr; + class QueryPipelineBuilder { public: @@ -138,7 +141,7 @@ public: /// This is used for CreatingSets. void addPipelineBefore(QueryPipelineBuilder pipeline); - void addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, ContextPtr context); + void addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, PreparedSetsCachePtr prepared_sets_cache); PipelineExecutorPtr execute(); diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 31e45db55cb..1392ed5c684 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -155,7 +155,7 @@ void StorageMemory::read( size_t /*max_block_size*/, size_t num_streams) { - query_plan.addStep(std::make_unique(column_names, storage_snapshot, num_streams, delay_read_for_global_subqueries)); + query_plan.addStep(std::make_unique(column_names, shared_from_this(), storage_snapshot, num_streams, delay_read_for_global_subqueries)); } diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index ce8a59b8bcd..31c8ee31f8b 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -132,6 +132,8 @@ private: std::atomic total_size_rows = 0; bool compress; + + friend class ReadFromMemoryStorageStep; }; } From 0740bfbe4bfb3d186c8803e15aaffd2042c7f1fa Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 26 May 2023 14:31:09 +0000 Subject: [PATCH 088/515] Better disks moving --- programs/keeper/CMakeLists.txt | 3 +- src/Coordination/Changelog.cpp | 111 +++++++++++++++++++------- src/Coordination/Changelog.h | 2 +- src/Coordination/KeeperContext.cpp | 20 ++--- src/Coordination/KeeperContext.h | 4 +- src/Coordination/KeeperDispatcher.cpp | 6 +- 6 files changed, 98 insertions(+), 48 deletions(-) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 6e97ab324e3..f775e8a5a22 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -131,6 +131,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/StoredObject.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/registerDiskS3.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -144,7 +145,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/IOUringReader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteBufferFromTemporaryFile.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteBufferWithFinalizeCallback.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/AsynchronousBoundedReadBuffer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/getThreadPoolReader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolRemoteFSReader.cpp diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index b289e1dc9f2..bcce9982e04 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -32,9 +32,21 @@ namespace ErrorCodes namespace { +constexpr std::string_view tmp_prefix = "tmp_"; + void moveFileBetweenDisks(DiskPtr disk_from, ChangelogFileDescriptionPtr description, DiskPtr disk_to, const std::string & path_to) { - disk_from->copyFile(description->path, *disk_to, path_to, {}); + /// we use empty file with prefix tmp_ to detect incomplete copies + /// if a copy is complete we don't care from which disk we use the same file + /// so it's okay if a failure happens after removing of tmp file but before we remove + /// the changelog from the source disk + auto from_path = fs::path(description->path); + auto tmp_changelog_name = from_path.parent_path() / (std::string{tmp_prefix} + from_path.filename().string()); + { + disk_to->writeFile(tmp_changelog_name); + } + disk_from->copyFile(from_path, *disk_to, path_to, {}); + disk_to->removeFile(tmp_changelog_name); disk_from->removeFile(description->path); description->path = path_to; description->disk = disk_to; @@ -164,9 +176,9 @@ public: } } - auto current_log_disk = getCurrentLogDisk(); - assert(file_description->disk == current_log_disk); - file_buf = current_log_disk->writeFile(file_description->path, DBMS_DEFAULT_BUFFER_SIZE, mode); + auto latest_log_disk = getLatestLogDisk(); + assert(file_description->disk == latest_log_disk); + file_buf = latest_log_disk->writeFile(file_description->path, DBMS_DEFAULT_BUFFER_SIZE, mode); assert(file_buf); last_index_written.reset(); current_file_description = std::move(file_description); @@ -176,7 +188,7 @@ public: std::move(file_buf), /* compressi)on level = */ 3, /* append_to_existing_file_ = */ mode == WriteMode::Append, - [current_log_disk, path = current_file_description->path] { return current_log_disk->readFile(path); }); + [latest_log_disk, path = current_file_description->path] { return latest_log_disk->readFile(path); }); prealloc_done = false; } @@ -274,7 +286,7 @@ public: new_description->from_log_index = new_start_log_index; new_description->to_log_index = new_start_log_index + log_file_settings.rotate_interval - 1; new_description->extension = "bin"; - new_description->disk = getCurrentLogDisk(); + new_description->disk = getLatestLogDisk(); if (log_file_settings.compress_logs) new_description->extension += "." + toContentEncodingName(CompressionMethod::Zstd); @@ -413,7 +425,7 @@ private: prealloc_done = true; } - DiskPtr getCurrentLogDisk() const { return keeper_context->getCurrentLogDisk(); } + DiskPtr getLatestLogDisk() const { return keeper_context->getLatestLogDisk(); } DiskPtr getDisk() const { return keeper_context->getLogDisk(); } @@ -574,50 +586,87 @@ Changelog::Changelog(Poco::Logger * log_, LogFileSettings log_file_settings, Kee , append_completion_queue(std::numeric_limits::max()) , keeper_context(std::move(keeper_context_)) { - if (auto current_log_disk = getCurrentLogDisk(); - log_file_settings.force_sync && dynamic_cast(current_log_disk.get()) == nullptr) + if (auto latest_log_disk = getLatestLogDisk(); + log_file_settings.force_sync && dynamic_cast(latest_log_disk.get()) == nullptr) { throw DB::Exception( DB::ErrorCodes::BAD_ARGUMENTS, "force_sync is set to true for logs but disk '{}' cannot satisfy such guarantee because it's not of type DiskLocal.\n" "If you want to use force_sync and same disk for all logs, please set keeper_server.log_storage_disk to a local disk.\n" "If you want to use force_sync and different disk only for old logs, please set 'keeper_server.log_storage_disk' to any " - "supported disk and 'keeper_server.current_log_storage_disk' to a local disk.\n" + "supported disk and 'keeper_server.latest_log_storage_disk' to a local disk.\n" "Otherwise, disable force_sync", - current_log_disk->getName()); + latest_log_disk->getName()); } /// Load all files on changelog disks const auto load_from_disk = [&](const auto & disk) { + LOG_TRACE(log, "Reading from disk {}", disk->getName()); + std::unordered_map incomplete_files; + + const auto clean_incomplete_file = [&](const auto & file_path) + { + if (auto incomplete_it = incomplete_files.find(fs::path(file_path).filename()); incomplete_it != incomplete_files.end()) + { + LOG_TRACE(log, "Removing {} from {}", file_path, disk->getName()); + disk->removeFile(file_path); + disk->removeFile(incomplete_it->second); + incomplete_files.erase(incomplete_it); + return true; + } + + return false; + }; + + std::vector changelog_files; for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) { if (it->name() == changelogs_detached_dir) continue; - auto file_description = getChangelogFileDescription(it->path()); + if (it->name().starts_with(tmp_prefix)) + { + incomplete_files.emplace(it->name().substr(tmp_prefix.size()), it->path()); + continue; + } + + if (clean_incomplete_file(it->path())) + continue; + + changelog_files.push_back(it->path()); + } + + for (const auto & changelog_file : changelog_files) + { + if (clean_incomplete_file(fs::path(changelog_file).filename())) + continue; + + auto file_description = getChangelogFileDescription(changelog_file); file_description->disk = disk; + LOG_TRACE(log, "Found {} on {}", changelog_file, disk->getName()); auto [changelog_it, inserted] = existing_changelogs.insert_or_assign(file_description->from_log_index, std::move(file_description)); if (!inserted) LOG_WARNING(log, "Found duplicate entries for {}, will use the entry from {}", changelog_it->second->path, disk->getName()); } + + for (const auto & [name, path] : incomplete_files) + disk->removeFile(path); }; /// Load all files from old disks for (const auto & disk : keeper_context->getOldLogDisks()) - { load_from_disk(disk); - } auto disk = getDisk(); load_from_disk(disk); - auto current_log_disk = getCurrentLogDisk(); - if (disk != current_log_disk) - load_from_disk(current_log_disk); + auto latest_log_disk = getLatestLogDisk(); + if (disk != latest_log_disk) + load_from_disk(latest_log_disk); if (existing_changelogs.empty()) LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", disk->getPath()); @@ -779,12 +828,12 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin else if (last_log_read_result.has_value()) { /// check if we need to move completed log to another disk - auto current_log_disk = getCurrentLogDisk(); + auto latest_log_disk = getLatestLogDisk(); auto disk = getDisk(); auto & description = existing_changelogs.at(last_log_read_result->log_start_index); - if (current_log_disk != disk && current_log_disk == description->disk) - moveFileBetweenDisks(current_log_disk, description, disk, description->path); + if (latest_log_disk != disk && latest_log_disk == description->disk) + moveFileBetweenDisks(latest_log_disk, description, disk, description->path); } /// Start new log if we don't initialize writer from previous log. All logs can be "complete". @@ -793,14 +842,14 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin /// Move files to correct disks auto latest_start_index = current_writer->getStartIndex(); - auto current_log_disk = getCurrentLogDisk(); + auto latest_log_disk = getLatestLogDisk(); auto disk = getDisk(); for (const auto & [start_index, description] : existing_changelogs) { - /// latest log should already be on current_log_disk + /// latest log should already be on latest_log_disk if (start_index == latest_start_index) { - chassert(description->disk == current_log_disk); + chassert(description->disk == latest_log_disk); continue; } @@ -825,9 +874,9 @@ void Changelog::initWriter(ChangelogFileDescriptionPtr description) LOG_TRACE(log, "Continue to write into {}", description->path); auto log_disk = description->disk; - auto current_log_disk = getCurrentLogDisk(); - if (log_disk != current_log_disk) - moveFileBetweenDisks(log_disk, description, current_log_disk, description->path); + auto latest_log_disk = getLatestLogDisk(); + if (log_disk != latest_log_disk) + moveFileBetweenDisks(log_disk, description, latest_log_disk, description->path); current_writer->setFile(std::move(description), WriteMode::Append); } @@ -855,9 +904,9 @@ DiskPtr Changelog::getDisk() const return keeper_context->getLogDisk(); } -DiskPtr Changelog::getCurrentLogDisk() const +DiskPtr Changelog::getLatestLogDisk() const { - return keeper_context->getCurrentLogDisk(); + return keeper_context->getLatestLogDisk(); } void Changelog::removeExistingLogs(ChangelogIter begin, ChangelogIter end) @@ -1044,9 +1093,9 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry) description = std::prev(index_changelog)->second; auto log_disk = description->disk; - auto current_log_disk = getCurrentLogDisk(); - if (log_disk != current_log_disk) - moveFileBetweenDisks(log_disk, description, current_log_disk, description->path); + auto latest_log_disk = getLatestLogDisk(); + if (log_disk != latest_log_disk) + moveFileBetweenDisks(log_disk, description, latest_log_disk, description->path); current_writer->setFile(std::move(description), WriteMode::Append); diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 4054829ef19..d20eaa1666c 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -155,7 +155,7 @@ private: static ChangelogRecord buildRecord(uint64_t index, const LogEntryPtr & log_entry); DiskPtr getDisk() const; - DiskPtr getCurrentLogDisk() const; + DiskPtr getLatestLogDisk() const; /// Currently existing changelogs std::map existing_changelogs; diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 1d6f1be9bfb..9d7e62ffae7 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -22,10 +22,10 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) log_storage = getLogsPathFromConfig(config); - if (config.has("keeper_server.current_log_storage_disk")) - current_log_storage = config.getString("keeper_server.current_log_storage_disk"); + if (config.has("keeper_server.latest_log_storage_disk")) + latest_log_storage = config.getString("keeper_server.latest_log_storage_disk"); else - current_log_storage = log_storage; + latest_log_storage = log_storage; const auto collect_old_disk_names = [&](const std::string_view key_prefix, std::vector & disk_names) { @@ -34,7 +34,7 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) for (const auto & key : disk_name_keys) { if (key.starts_with(key_prefix)) - disk_names.push_back(config.getString(fmt::format("keeper_server.{}", key_prefix))); + disk_names.push_back(config.getString(fmt::format("keeper_server.{}", key))); } }; @@ -96,15 +96,15 @@ std::vector KeeperContext::getOldLogDisks() const return old_log_disks; } -DiskPtr KeeperContext::getCurrentLogDisk() const +DiskPtr KeeperContext::getLatestLogDisk() const { - return getDisk(current_log_storage); + return getDisk(latest_log_storage); } void KeeperContext::setLogDisk(DiskPtr disk) { log_storage = disk; - current_log_storage = std::move(disk); + latest_log_storage = std::move(disk); } DiskPtr KeeperContext::getSnapshotDisk() const @@ -156,9 +156,9 @@ void KeeperContext::dumpConfiguration(WriteBufferFromOwnString & buf) const auto log_disk = getDisk(log_storage); dump_disk_info("log_storage", *log_disk); - auto current_log_disk = getDisk(current_log_storage); - if (log_disk != current_log_disk) - dump_disk_info("current_log_storage", *current_log_disk); + auto latest_log_disk = getDisk(latest_log_storage); + if (log_disk != latest_log_disk) + dump_disk_info("latest_log_storage", *latest_log_disk); } { diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index e41d8e35032..00607fd904e 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -34,7 +34,7 @@ public: bool digestEnabled() const; void setDigestEnabled(bool digest_enabled_); - DiskPtr getCurrentLogDisk() const; + DiskPtr getLatestLogDisk() const; DiskPtr getLogDisk() const; std::vector getOldLogDisks() const; void setLogDisk(DiskPtr disk); @@ -65,7 +65,7 @@ private: std::shared_ptr disk_selector; Storage log_storage; - Storage current_log_storage; + Storage latest_log_storage; Storage snapshot_storage; Storage state_file_storage; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 6d1239c9210..a5b0f0b2c01 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -798,9 +798,9 @@ uint64_t KeeperDispatcher::getLogDirSize() const auto log_disk = keeper_context->getLogDisk(); auto size = getTotalSize(log_disk); - auto current_log_disk = keeper_context->getCurrentLogDisk(); - if (log_disk != current_log_disk) - size += getTotalSize(current_log_disk); + auto latest_log_disk = keeper_context->getLatestLogDisk(); + if (log_disk != latest_log_disk) + size += getTotalSize(latest_log_disk); return size; } From 6ed7a3b73feadcf5c3d72ed4a137fccd55e10fbb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 May 2023 19:25:33 +0000 Subject: [PATCH 089/515] Fixing more tests. --- src/Interpreters/ActionsVisitor.cpp | 6 +++- src/Interpreters/DatabaseCatalog.cpp | 2 +- src/Interpreters/GlobalSubqueriesVisitor.h | 4 +++ src/Interpreters/PreparedSets.cpp | 7 +++- src/Interpreters/Set.cpp | 17 ++++++---- src/Interpreters/Set.h | 3 +- .../QueryPlan/ReadFromMergeTree.cpp | 27 ++++++++++++--- src/Processors/QueryPlan/ReadFromMergeTree.h | 3 ++ src/Storages/MergeTree/KeyCondition.cpp | 18 +++++++++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 20 ++++------- .../MergeTree/MergeTreeDataSelectExecutor.h | 3 +- .../MergeTreeIndexConditionBloomFilter.cpp | 11 ++++++ src/Storages/MergeTree/PartitionPruner.cpp | 16 +++++++++ src/Storages/MergeTree/PartitionPruner.h | 1 + .../System/StorageSystemZooKeeper.cpp | 34 +++++++++++-------- 15 files changed, 127 insertions(+), 45 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 133fcbbfe87..23c6867a868 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1420,8 +1420,12 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool return data.prepared_sets->addFromStorage(set_key, storage_set->getSet()); } - if (auto tmp_table = data.getContext()->findExternalTable(table_id.getShortName())) + // std::cerr << ".... checking for " << identifier->getColumnName() << std::endl; + if (auto tmp_table = data.getContext()->findExternalTable(identifier->getColumnName())) + { external_table_set = tmp_table->future_set; + // std::cerr << "Found " << reinterpret_cast(tmp_table.get()) << " " << reinterpret_cast(external_table_set.get()) << std::endl; + } } /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 8d3fa91a7fe..dccf2978f49 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -110,7 +110,7 @@ TemporaryTableHolder::TemporaryTableHolder( } TemporaryTableHolder::TemporaryTableHolder(TemporaryTableHolder && rhs) noexcept - : WithContext(rhs.context), temporary_tables(rhs.temporary_tables), id(rhs.id) + : WithContext(rhs.context), temporary_tables(rhs.temporary_tables), id(rhs.id), future_set(std::move(rhs.future_set)) { rhs.id = UUIDHelpers::Nil; } diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 195839da04f..8d2f2204e84 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -30,6 +30,7 @@ namespace DB namespace ErrorCodes { extern const int WRONG_GLOBAL_SUBQUERY; + extern const int LOGICAL_ERROR; } class GlobalSubqueriesMatcher @@ -159,6 +160,8 @@ public: /*create_for_global_subquery*/ true); StoragePtr external_storage = external_storage_holder->getTable(); + // std::cerr << "......... adding external table " << external_table_name << std::endl; + external_tables.emplace(external_table_name, external_storage_holder); /// We need to materialize external tables immediately because reading from distributed @@ -195,6 +198,7 @@ public: //std::cerr << reinterpret_cast(prepared_sets.get()) << std::endl; auto future_set = prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), nullptr); + // std::cerr << "... Future set " << reinterpret_cast(external_storage_holder.get()) << " " << reinterpret_cast(future_set.get()) << std::endl; external_storage_holder->future_set = std::move(future_set); } else diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index f2b70462227..c71273e0baa 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -239,6 +239,8 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) return nullptr; } + // std::cerr << "... external_table_set " << reinterpret_cast(external_table_set.get()) << std::endl; + if (external_table_set) return set = external_table_set->buildOrderedSetInplace(context); @@ -337,7 +339,7 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) set_key_columns.filter = ColumnUInt8::create(block.rows()); - set->initSetElements(); + //set->initSetElements(); set->insertFromColumns(columns, set_key_columns); set->finishInsert(); //block(std::move(block_)) @@ -350,6 +352,9 @@ FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) { SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) { + if (set->hasExplicitSetElements()) + return set; + const auto & settings = context->getSettingsRef(); auto limits = getSizeLimitsForSet(settings, true); diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 5adf3d07353..bd9cafc66eb 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -202,15 +202,14 @@ bool Set::insertFromColumns(const Columns & columns, SetKeyColumns & holder) if (data.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Method Set::setHeader must be called before Set::insertFromBlock"); - ColumnRawPtrs key_columns; - key_columns.reserve(keys_size); holder.key_columns.reserve(keys_size); + holder.materialized_columns.reserve(keys_size); /// Remember the columns we will work with for (size_t i = 0; i < keys_size; ++i) { - holder.key_columns.emplace_back(columns.at(i)->convertToFullIfNeeded()); - key_columns.emplace_back(holder.key_columns.back().get()); + holder.materialized_columns.emplace_back(columns.at(i)->convertToFullIfNeeded()); + holder.key_columns.emplace_back(holder.materialized_columns.back().get()); } size_t rows = columns.at(0)->size(); @@ -219,7 +218,7 @@ bool Set::insertFromColumns(const Columns & columns, SetKeyColumns & holder) ConstNullMapPtr null_map{}; ColumnPtr null_map_holder; if (!transform_null_in) - null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); + null_map_holder = extractNestedColumnsAndNullMap(holder.key_columns, null_map); switch (data.type) { @@ -227,7 +226,7 @@ bool Set::insertFromColumns(const Columns & columns, SetKeyColumns & holder) break; #define M(NAME) \ case SetVariants::Type::NAME: \ - insertFromBlockImpl(*data.NAME, key_columns, rows, data, null_map, holder.filter ? &holder.filter->getData() : nullptr); \ + insertFromBlockImpl(*data.NAME, holder.key_columns, rows, data, null_map, holder.filter ? &holder.filter->getData() : nullptr); \ break; APPLY_FOR_SET_VARIANTS(M) #undef M @@ -445,6 +444,11 @@ void Set::checkTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) c MergeTreeSetIndex::MergeTreeSetIndex(const Columns & set_elements, std::vector && indexes_mapping_) : has_all_keys(set_elements.size() == indexes_mapping_.size()), indexes_mapping(std::move(indexes_mapping_)) { + // std::cerr << "MergeTreeSetIndex::MergeTreeSetIndex " + // << set_elements.size() << ' ' << indexes_mapping.size() << std::endl; + // for (const auto & vv : indexes_mapping) + // std::cerr << vv.key_index << ' ' << vv.tuple_index << std::endl; + ::sort(indexes_mapping.begin(), indexes_mapping.end(), [](const KeyTuplePositionMapping & l, const KeyTuplePositionMapping & r) { @@ -487,6 +491,7 @@ MergeTreeSetIndex::MergeTreeSetIndex(const Columns & set_elements, std::vector & key_ranges, const DataTypes & data_types, bool single_point) const { size_t tuple_size = indexes_mapping.size(); + // std::cerr << "MergeTreeSetIndex::checkInRange " << single_point << ' ' << tuple_size << ' ' << has_all_keys << std::endl; FieldValues left_point; FieldValues right_point; diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index cb47fde7f7d..c2931d79de0 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -53,7 +53,8 @@ public: { //ColumnRawPtrs key_columns; /// The constant columns to the right of IN are not supported directly. For this, they first materialize. - Columns key_columns; + ColumnRawPtrs key_columns; + Columns materialized_columns; ColumnPtr null_map_holder; ColumnUInt8::MutablePtr filter; }; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index acd9147b613..4940cbd032c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1174,6 +1174,7 @@ static ActionsDAGPtr buildFilterDAG( static void buildIndexes( std::optional & indexes, ActionsDAGPtr filter_actions_dag, + const MergeTreeData & data, const ContextPtr & context, const SelectQueryInfo & query_info, const StorageMetadataPtr & metadata_snapshot) @@ -1196,7 +1197,7 @@ static void buildIndexes( context, primary_key_column_names, primary_key.expression, - array_join_name_set}, {}, false}); + array_join_name_set}, {}, {}, {}, false}); } else { @@ -1204,7 +1205,22 @@ static void buildIndexes( query_info, context, primary_key_column_names, - primary_key.expression}, {}, false}); + primary_key.expression}, {}, {}, {}, false}); + } + + if (metadata_snapshot->hasPartitionKey()) + { + const auto & partition_key = metadata_snapshot->getPartitionKey(); + auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); + auto minmax_expression_actions = data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context)); + // minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); + + // if (context->getSettingsRef().allow_experimental_analyzer) + indexes->minmax_idx_condition.emplace(filter_actions_dag, context, minmax_columns_names, minmax_expression_actions, NameSet()); + // else + // indexes->minmax_idx_condition.emplace(query_info, context, minmax_columns_names, minmax_expression_actions); + + indexes->partition_pruner.emplace(metadata_snapshot, filter_actions_dag, context, false /* strict */); } indexes->use_skip_indexes = settings.use_skip_indexes; @@ -1250,7 +1266,7 @@ void ReadFromMergeTree::onAddFilterFinish() if (!filter_nodes.nodes.empty()) { auto filter_actions_dag = buildFilterDAG(context, prewhere_info, filter_nodes, query_info); - buildIndexes(indexes, filter_actions_dag, context, query_info, metadata_for_reading); + buildIndexes(indexes, filter_actions_dag, data, context, query_info, metadata_for_reading); } } @@ -1366,7 +1382,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( // } if (!indexes) - buildIndexes(indexes, query_info.filter_actions_dag, context, query_info, metadata_snapshot); + buildIndexes(indexes, query_info.filter_actions_dag, data, context, query_info, metadata_snapshot); if (settings.force_primary_key && indexes->key_condition.alwaysUnknownOrTrue()) { @@ -1386,11 +1402,12 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( try { MergeTreeDataSelectExecutor::filterPartsByPartition( + indexes->partition_pruner, + indexes->minmax_idx_condition, parts, part_values, metadata_snapshot_base, data, - query_info, context, max_block_numbers_to_read.get(), log, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 6610b463726..5ff9d2f046c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -164,6 +165,8 @@ public: struct Indexes { KeyCondition key_condition; + std::optional partition_pruner; + std::optional minmax_idx_condition; UsefulSkipIndexes skip_indexes; bool use_skip_indexes; }; diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 7fcc111ced9..fe87198dcf5 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1157,6 +1157,8 @@ bool KeyCondition::tryPrepareSetIndex( RPNElement & out, size_t & out_key_column_num) { + // std::cerr << "::: tryPrepareSetIndex for " << func.getColumnName() << std::endl; + // std::cerr << StackTrace().toString() << std::endl; const auto & left_arg = func.getArgumentAt(0); out_key_column_num = 0; @@ -1200,7 +1202,10 @@ bool KeyCondition::tryPrepareSetIndex( } if (indexes_mapping.empty()) + { + // std::cerr << ".. index mapping is empty\n"; return false; + } const auto right_arg = func.getArgumentAt(1); @@ -1208,7 +1213,10 @@ bool KeyCondition::tryPrepareSetIndex( auto future_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types); if (!future_set) + { + // std::cerr << ".. no future set\n"; return false; + } // LOG_TRACE(&Poco::Logger::get("KK"), "Found set for {}", right_arg.getColumnName()); @@ -1220,13 +1228,21 @@ bool KeyCondition::tryPrepareSetIndex( auto prepared_set = future_set->get(); if (!prepared_set) + { + + // std::cerr << ".. no prepared set\n"; return false; + } // LOG_TRACE(&Poco::Logger::get("KK"), "Set if ready for {}", right_arg.getColumnName()); /// The index can be prepared if the elements of the set were saved in advance. if (!prepared_set->hasExplicitSetElements()) + { + + // std::cerr << ".. no explicit elements\n"; return false; + } // LOG_TRACE(&Poco::Logger::get("KK"), "Has explicit elements for {}", right_arg.getColumnName()); @@ -1235,7 +1251,7 @@ bool KeyCondition::tryPrepareSetIndex( prepared_set->checkTypesEqual(indexes_mapping[i].tuple_index, data_types[i]); out.set_index = std::make_shared(prepared_set->getSetElements(), std::move(indexes_mapping)); - + // std::cerr << ".. can use\n"; return true; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index f99e15c0fc1..f42fada4222 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -793,38 +793,28 @@ std::optional> MergeTreeDataSelectExecutor::filterPar } void MergeTreeDataSelectExecutor::filterPartsByPartition( + std::optional & partition_pruner, + std::optional & minmax_idx_condition, MergeTreeData::DataPartsVector & parts, const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, - const SelectQueryInfo & query_info, const ContextPtr & context, const PartitionIdToMaxBlock * max_block_numbers_to_read, Poco::Logger * log, ReadFromMergeTree::IndexStats & index_stats) { const Settings & settings = context->getSettingsRef(); - - std::optional partition_pruner; - std::optional minmax_idx_condition; DataTypes minmax_columns_types; if (metadata_snapshot->hasPartitionKey()) { const auto & partition_key = metadata_snapshot->getPartitionKey(); - auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); - auto minmax_expression_actions = data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context)); minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); - if (context->getSettingsRef().allow_experimental_analyzer) - minmax_idx_condition.emplace(query_info.filter_actions_dag, context, minmax_columns_names, minmax_expression_actions, NameSet()); - else - minmax_idx_condition.emplace(query_info, context, minmax_columns_names, minmax_expression_actions); - - partition_pruner.emplace(metadata_snapshot, query_info, context, false /* strict */); - if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless())) { + auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); throw Exception(ErrorCodes::INDEX_NOT_USED, "Neither MinMax index by columns ({}) nor partition expr is used and setting 'force_index_by_date' is set", fmt::join(minmax_columns_names, ", ")); @@ -1835,7 +1825,9 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( if (partition_pruner) { - if (partition_pruner->canBePruned(*part)) + auto val = partition_pruner->canBePruned(*part); + // std::cerr << "... part " << part->getNameWithState() << " cbp ? " << val << std::endl; + if (val) continue; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 8c8ce59bebe..f1efbdf0310 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -173,11 +173,12 @@ public: /// Filter parts using minmax index and partition key. static void filterPartsByPartition( + std::optional & partition_pruner, + std::optional & minmax_idx_condition, MergeTreeData::DataPartsVector & parts, const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, - const SelectQueryInfo & query_info, const ContextPtr & context, const PartitionIdToMaxBlock * max_block_numbers_to_read, Poco::Logger * log, diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 3d0883a1241..e2bf9bde674 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -31,12 +31,21 @@ namespace ColumnWithTypeAndName getPreparedSetInfo(const ConstSetPtr & prepared_set) { + // std::cerr << "====== " << prepared_set->getDataTypes().size() << std::endl; if (prepared_set->getDataTypes().size() == 1) return {prepared_set->getSetElements()[0], prepared_set->getElementsTypes()[0], "dummy"}; Columns set_elements; for (auto & set_element : prepared_set->getSetElements()) + { + // std::cerr << set_element->dumpStructure() << std::endl; set_elements.emplace_back(set_element->convertToFullColumnIfConst()); + } + + // for (auto & set_element : prepared_set->getElementsTypes()) + // { + // // std::cerr << set_element->getName() << std::endl; + // } return {ColumnTuple::create(set_elements), std::make_shared(prepared_set->getElementsTypes()), "dummy"}; } @@ -331,6 +340,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNo if (prepared_set && prepared_set->hasExplicitSetElements()) { const auto prepared_info = getPreparedSetInfo(prepared_set); + // std::cerr << "...... " << prepared_info.dumpStructure() << std::endl; if (traverseTreeIn(function_name, lhs_argument, prepared_set, prepared_info.type, prepared_info.column, out)) maybe_useful = true; } @@ -377,6 +387,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeIn( size_t row_size = column->size(); size_t position = header.getPositionByName(key_node_column_name); const DataTypePtr & index_type = header.getByPosition(position).type; + // std::cerr << "::::: " << ColumnWithTypeAndName{column, type, ""}.dumpStructure() << " -> " << index_type->getName() << std::endl; const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, index_type); out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(index_type, converted_column, 0, row_size))); diff --git a/src/Storages/MergeTree/PartitionPruner.cpp b/src/Storages/MergeTree/PartitionPruner.cpp index 35b2d5db3b5..a397a1475d1 100644 --- a/src/Storages/MergeTree/PartitionPruner.cpp +++ b/src/Storages/MergeTree/PartitionPruner.cpp @@ -24,6 +24,19 @@ PartitionPruner::PartitionPruner(const StorageMetadataPtr & metadata, const Sele { } +PartitionPruner::PartitionPruner(const StorageMetadataPtr & metadata, ActionsDAGPtr filter_actions_dag, ContextPtr context, bool strict) + : partition_key(MergeTreePartition::adjustPartitionKey(metadata, context)) + , partition_condition(filter_actions_dag, context, partition_key.column_names, partition_key.expression, {}, true /* single_point */, strict) + , useless(strict ? partition_condition.anyUnknownOrAlwaysTrue() : partition_condition.alwaysUnknownOrTrue()) +{ + // auto description = getKeyCondition().getDescription(); + // std::cerr << ".... " << description.condition << std::endl; + // std::cerr << filter_actions_dag->dumpDAG() << std::endl; + // for (const auto & name : partition_key.column_names) + // std::cerr << ". " << name << std::endl; + // std::cerr << partition_key.expression->dumpActions() << std::endl; +} + bool PartitionPruner::canBePruned(const IMergeTreeDataPart & part) { if (part.isEmpty()) @@ -39,6 +52,8 @@ bool PartitionPruner::canBePruned(const IMergeTreeDataPart & part) else { const auto & partition_value = part.partition.value; + // for (const auto & val : partition_value) + // std::cerr << val.dump() << std::endl; std::vector index_value(partition_value.begin(), partition_value.end()); for (auto & field : index_value) { @@ -49,6 +64,7 @@ bool PartitionPruner::canBePruned(const IMergeTreeDataPart & part) is_valid = partition_condition.mayBeTrueInRange( partition_value.size(), index_value.data(), index_value.data(), partition_key.data_types); + // std::cerr << "may be true " << is_valid << std::endl; partition_filter_map.emplace(partition_id, is_valid); if (!is_valid) diff --git a/src/Storages/MergeTree/PartitionPruner.h b/src/Storages/MergeTree/PartitionPruner.h index 3a986923321..7f1b74795c4 100644 --- a/src/Storages/MergeTree/PartitionPruner.h +++ b/src/Storages/MergeTree/PartitionPruner.h @@ -14,6 +14,7 @@ class PartitionPruner { public: PartitionPruner(const StorageMetadataPtr & metadata, const SelectQueryInfo & query_info, ContextPtr context, bool strict); + PartitionPruner(const StorageMetadataPtr & metadata, ActionsDAGPtr filter_actions_dag, ContextPtr context, bool strict); bool canBePruned(const IMergeTreeDataPart & part); diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 34b463eadee..1e89427071c 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -161,6 +161,17 @@ public: } }; +/// Type of path to be fetched +enum class ZkPathType +{ + Exact, /// Fetch all nodes under this path + Prefix, /// Fetch all nodes starting with this prefix, recursively (multiple paths may match prefix) + Recurse, /// Fatch all nodes under this path, recursively +}; + +/// List of paths to be feched from zookeeper +using Paths = std::deque>; + class ReadFromSystemZooKeeper final : public SourceStepWithFilter { public: @@ -170,11 +181,14 @@ public: void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; + void onAddFilterFinish() override; + private: - void fillData(MutableColumns & res_columns) const; + void fillData(MutableColumns & res_columns); std::shared_ptr storage_limits; ContextPtr context; + Paths paths; }; StorageSystemZooKeeper::StorageSystemZooKeeper(const StorageID & table_id_) @@ -246,17 +260,6 @@ NamesAndTypesList StorageSystemZooKeeper::getNamesAndTypes() }; } -/// Type of path to be fetched -enum class ZkPathType -{ - Exact, /// Fetch all nodes under this path - Prefix, /// Fetch all nodes starting with this prefix, recursively (multiple paths may match prefix) - Recurse, /// Fatch all nodes under this path, recursively -}; - -/// List of paths to be feched from zookeeper -using Paths = std::deque>; - static String pathCorrected(const String & path) { String path_corrected; @@ -421,10 +424,13 @@ static Paths extractPath(const ActionsDAG::NodeRawConstPtrs & filter_nodes, Cont } -void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns) const +void ReadFromSystemZooKeeper::onAddFilterFinish() { - Paths paths = extractPath(getFilterNodes().nodes, context, context->getSettingsRef().allow_unrestricted_reads_from_keeper); + paths = extractPath(getFilterNodes().nodes, context, context->getSettingsRef().allow_unrestricted_reads_from_keeper); +} +void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns) +{ zkutil::ZooKeeperPtr zookeeper = context->getZooKeeper(); if (paths.empty()) From d4cec1f0e0fa114f5e04dead6b90f51b3e292310 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 30 May 2023 01:30:01 +0200 Subject: [PATCH 090/515] fix client thread attachment + make better tests --- programs/client/Client.cpp | 1 + src/Client/ClientBase.cpp | 3 --- src/Client/Connection.cpp | 4 --- src/Common/ThreadStatus.h | 3 ++- src/DataTypes/DataTypeDateTime.h | 4 ++- src/Server/TCPHandler.cpp | 2 +- .../02681_timezone_setting.reference | 5 ---- .../0_stateless/02681_timezone_setting.sql | 11 -------- .../02737_timezone_setting.reference | 7 +++++ .../0_stateless/02737_timezone_setting.sql | 27 +++++++++++++++++++ 10 files changed, 41 insertions(+), 26 deletions(-) delete mode 100644 tests/queries/0_stateless/02681_timezone_setting.reference delete mode 100644 tests/queries/0_stateless/02681_timezone_setting.sql create mode 100644 tests/queries/0_stateless/02737_timezone_setting.reference create mode 100644 tests/queries/0_stateless/02737_timezone_setting.sql diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 6b34bdbc5bb..231b7fd6d61 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -326,6 +326,7 @@ try // All that just to set DB::CurrentThread::get().getGlobalContext() // which is required for client timezone (pushed from server) to work. auto thread_group = std::make_shared(); + thread_group->global_context = global_context; thread_status.attachToGroup(thread_group, false); } diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index fad9494ba4b..562c11680a1 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -448,9 +448,7 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) /// output_format, do not output it. /// Also do not output too much data if we're fuzzing. if (block.rows() == 0 || (query_fuzzer_runs != 0 && processed_rows >= 100)) - { return; - } /// If results are written INTO OUTFILE, we can avoid clearing progress to avoid flicker. if (need_render_progress && tty_buf && (!select_into_file || select_into_file_and_stdout)) @@ -902,7 +900,6 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa if (send_external_tables) sendExternalTables(parsed_query); - receiveResult(parsed_query, signals_before_stop, settings.partial_result_on_first_cancel); break; diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 86585d805d9..457d90c5bd4 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -355,10 +355,6 @@ void Connection::receiveHello() nonce.emplace(read_nonce); } } -// else if (packet_type == Protocol::Server::TimezoneUpdate) -// { -// // skip this packet at hello, will receive and process it later -// } else if (packet_type == Protocol::Server::Exception) receiveException()->rethrow(); else diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 600dfc56d2b..400b55c2409 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -72,7 +72,8 @@ public: /// Set up at creation, no race when reading const ContextWeakPtr query_context; - const ContextWeakPtr global_context; + /// Cannot make it const -- we need to modify it in ch-client to process timezone from server + ContextWeakPtr global_context; const FatalErrorCallback fatal_error_callback; diff --git a/src/DataTypes/DataTypeDateTime.h b/src/DataTypes/DataTypeDateTime.h index 91a09ff7cb9..a4a05917ba5 100644 --- a/src/DataTypes/DataTypeDateTime.h +++ b/src/DataTypes/DataTypeDateTime.h @@ -21,7 +21,9 @@ namespace DB * all types with different time zones are equivalent and may be used interchangingly. * Time zone only affects parsing and displaying in text formats. * - * If time zone is not specified (example: DateTime without parameter), then default time zone is used. + * If time zone is not specified (example: DateTime without parameter), + * then `session_timezone` setting value is used. + * If `session_timezone` is not set (or empty string), server default time zone is used. * Default time zone is server time zone, if server is doing transformations * and if client is doing transformations, unless 'use_client_time_zone' setting is passed to client; * Server time zone is the time zone specified in 'timezone' parameter in configuration file, diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index d57a1c93dd7..c41eace68ba 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1068,7 +1068,7 @@ void TCPHandler::sendTimezone() if (client_tcp_protocol_version < DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES) return; - const String & tz = query_context->getSettingsRef().session_timezone.toString(); + const String & tz = query_context->getSettingsRef().session_timezone.value; LOG_DEBUG(log, "TCPHandler::sendTimezone(): {}", tz); writeVarUInt(Protocol::Server::TimezoneUpdate, *out); diff --git a/tests/queries/0_stateless/02681_timezone_setting.reference b/tests/queries/0_stateless/02681_timezone_setting.reference deleted file mode 100644 index 8850d77ab03..00000000000 --- a/tests/queries/0_stateless/02681_timezone_setting.reference +++ /dev/null @@ -1,5 +0,0 @@ -2022-12-12 17:23:23.123 -2022-12-12 23:23:23.123 -2022-12-12 22:23:23.123 -Europe/Zurich Europe/Zurich -Pacific/Pitcairn Pacific/Pitcairn diff --git a/tests/queries/0_stateless/02681_timezone_setting.sql b/tests/queries/0_stateless/02681_timezone_setting.sql deleted file mode 100644 index f66e8d2b646..00000000000 --- a/tests/queries/0_stateless/02681_timezone_setting.sql +++ /dev/null @@ -1,11 +0,0 @@ -SET session_timezone = 'Абырвалг'; -- { serverError BAD_ARGUMENTS} - -SET session_timezone = 'Asia/Novosibirsk'; -SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich'); -SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'Europe/Zurich'; - -SET session_timezone = 'Asia/Manila'; -SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Asia/Novosibirsk'); - -SELECT timezone(), timezoneOf(now()) SETTINGS session_timezone = 'Europe/Zurich' FORMAT TSV; -SELECT timezone(), timezoneOf(now()) SETTINGS session_timezone = 'Pacific/Pitcairn' FORMAT TSV; diff --git a/tests/queries/0_stateless/02737_timezone_setting.reference b/tests/queries/0_stateless/02737_timezone_setting.reference new file mode 100644 index 00000000000..578aec4e316 --- /dev/null +++ b/tests/queries/0_stateless/02737_timezone_setting.reference @@ -0,0 +1,7 @@ +Pacific/Pitcairn Pacific/Pitcairn +Asia/Novosibirsk Asia/Novosibirsk +2022-12-12 17:23:23 +2022-12-13 07:23:23.123 +2002-12-12 23:23:23 2002-12-12 23:23:23 +2002-12-12 23:23:23.123 2002-12-12 23:23:23.123 +2000-01-01 01:00:00 diff --git a/tests/queries/0_stateless/02737_timezone_setting.sql b/tests/queries/0_stateless/02737_timezone_setting.sql new file mode 100644 index 00000000000..87eeec0779b --- /dev/null +++ b/tests/queries/0_stateless/02737_timezone_setting.sql @@ -0,0 +1,27 @@ +SET session_timezone = 'Абырвалг'; -- { serverError BAD_ARGUMENTS} + +SELECT timezone(), timezoneOf(now()) SETTINGS session_timezone = 'Pacific/Pitcairn'; + +SET session_timezone = 'Asia/Novosibirsk'; +SELECT timezone(), timezoneOf(now()); + +-- test simple queries +SELECT toDateTime(toDateTime('2022-12-12 23:23:23'), 'Europe/Zurich'); +SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'America/Denver'; + +-- test proper serialization +SELECT toDateTime('2002-12-12 23:23:23'), toString(toDateTime('2002-12-12 23:23:23')) SETTINGS session_timezone = 'Asia/Phnom_Penh'; +SELECT toDateTime64('2002-12-12 23:23:23.123', 3), toString(toDateTime64('2002-12-12 23:23:23.123', 3)) SETTINGS session_timezone = 'Asia/Phnom_Penh'; + +-- Create a table and test that DateTimes are processed correctly on insert +SET session_timezone='Asia/Novosibirsk'; +CREATE TABLE test_tz_setting (d DateTime('UTC')) Engine=Memory AS SELECT toDateTime('2000-01-01 00:00:00'); +INSERT INTO test_tz_setting VALUES ('2000-01-01 01:00:00'); -- this is parsed using timezone from `d` column +INSERT INTO test_tz_setting VALUES (toDateTime('2000-01-02 02:00:00')); -- this is parsed using `session_timezone` + +-- Test parsing in WHERE filter, shall have the same logic as insert +SELECT d FROM test_tz_setting WHERE d == '2000-01-01 01:00:00'; -- 1 row expected +SELECT d FROM test_tz_setting WHERE d == toDateTime('2000-01-01 02:00:00'); -- 0 rows expected + +-- Cleanup table +DROP TABLE test_tz_setting SYNC; \ No newline at end of file From 21aba94909e2ab7fe357c30d694af8674f81dbd0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 May 2023 13:22:40 +0000 Subject: [PATCH 091/515] Add support for latest snapshot disk --- src/Coordination/KeeperContext.cpp | 11 ++ src/Coordination/KeeperContext.h | 2 + src/Coordination/KeeperSnapshotManager.cpp | 143 ++++++++++++++------- src/Coordination/KeeperSnapshotManager.h | 4 + src/Coordination/KeeperStateMachine.cpp | 8 +- 5 files changed, 119 insertions(+), 49 deletions(-) diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 9d7e62ffae7..5ec86c827b8 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -43,6 +43,11 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) snapshot_storage = getSnapshotsPathFromConfig(config); + if (config.has("keeper_server.latest_snapshot_storage_disk")) + latest_snapshot_storage = config.getString("keeper_server.latest_snapshot_storage_disk"); + else + latest_snapshot_storage = snapshot_storage; + state_file_storage = getStatePathFromConfig(config); } @@ -107,6 +112,11 @@ void KeeperContext::setLogDisk(DiskPtr disk) latest_log_storage = std::move(disk); } +DiskPtr KeeperContext::getLatestSnapshotDisk() const +{ + return getDisk(latest_snapshot_storage); +} + DiskPtr KeeperContext::getSnapshotDisk() const { return getDisk(snapshot_storage); @@ -126,6 +136,7 @@ std::vector KeeperContext::getOldSnapshotDisks() const void KeeperContext::setSnapshotDisk(DiskPtr disk) { snapshot_storage = std::move(disk); + latest_snapshot_storage = snapshot_storage; } DiskPtr KeeperContext::getStateFileDisk() const diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 00607fd904e..b04afd2a57b 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -39,6 +39,7 @@ public: std::vector getOldLogDisks() const; void setLogDisk(DiskPtr disk); + DiskPtr getLatestSnapshotDisk() const; DiskPtr getSnapshotDisk() const; std::vector getOldSnapshotDisks() const; void setSnapshotDisk(DiskPtr disk); @@ -67,6 +68,7 @@ private: Storage log_storage; Storage latest_log_storage; Storage snapshot_storage; + Storage latest_snapshot_storage; Storage state_file_storage; std::vector old_log_disk_names; diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index c3371501976..4b9a34c07e9 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -16,6 +16,7 @@ #include #include #include +#include "Core/Field.h" #include @@ -31,6 +32,24 @@ namespace ErrorCodes namespace { + constexpr std::string_view tmp_prefix = "tmp_"; + + void moveFileBetweenDisks(DiskPtr disk_from, const std::string & path_from, DiskPtr disk_to, const std::string & path_to) + { + /// we use empty file with prefix tmp_ to detect incomplete copies + /// if a copy is complete we don't care from which disk we use the same file + /// so it's okay if a failure happens after removing of tmp file but before we remove + /// the snapshot from the source disk + auto from_path = fs::path(path_from); + auto tmp_snapshot_name = from_path.parent_path() / (std::string{tmp_prefix} + from_path.filename().string()); + { + disk_to->writeFile(tmp_snapshot_name); + } + disk_from->copyFile(from_path, *disk_to, path_to, {}); + disk_to->removeFile(tmp_snapshot_name); + disk_from->removeFile(path_from); + } + uint64_t getSnapshotPathUpToLogIdx(const String & snapshot_path) { std::filesystem::path path(snapshot_path); @@ -519,43 +538,48 @@ KeeperSnapshotManager::KeeperSnapshotManager( , storage_tick_time(storage_tick_time_) , keeper_context(keeper_context_) { - const auto load_snapshot_from_disk = [&](DiskPtr disk) + const auto load_snapshot_from_disk = [&](const auto & disk) { - std::unordered_set invalid_snapshots; - /// collect invalid snapshots + LOG_TRACE(log, "Reading from disk {}", disk->getName()); + std::unordered_map incomplete_files; + + const auto clean_incomplete_file = [&](const auto & file_path) + { + if (auto incomplete_it = incomplete_files.find(fs::path(file_path).filename()); incomplete_it != incomplete_files.end()) + { + LOG_TRACE(log, "Removing {} from {}", file_path, disk->getName()); + disk->removeFile(file_path); + disk->removeFile(incomplete_it->second); + incomplete_files.erase(incomplete_it); + return true; + } + + return false; + }; + + std::vector snapshot_files; for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) { - const auto & name = it->name(); - if (name.empty()) - continue; - - if (startsWith(name, "tmp_")) + if (it->name().starts_with(tmp_prefix)) { - disk->removeFile(it->path()); - invalid_snapshots.insert(name.substr(4)); + incomplete_files.emplace(it->name().substr(tmp_prefix.size()), it->path()); continue; } + + if (clean_incomplete_file(it->path())) + continue; + + snapshot_files.push_back(it->path()); } - /// process snapshots - for (auto it = disk->iterateDirectory(""); it->isValid(); it->next()) + for (const auto & snapshot_file : snapshot_files) { - const auto & name = it->name(); - if (name.empty()) + if (clean_incomplete_file(fs::path(snapshot_file).filename())) continue; - /// Not snapshot file - if (!startsWith(name, "snapshot_")) - continue; - - if (invalid_snapshots.contains(name)) - { - disk->removeFile(it->path()); - continue; - } - - size_t snapshot_up_to = getSnapshotPathUpToLogIdx(name); - auto [_, inserted] = existing_snapshots.insert_or_assign(snapshot_up_to, SnapshotFileInfo{it->path(), disk}); + LOG_TRACE(log, "Found {} on {}", snapshot_file, disk->getName()); + size_t snapshot_up_to = getSnapshotPathUpToLogIdx(snapshot_file); + auto [_, inserted] = existing_snapshots.insert_or_assign(snapshot_up_to, SnapshotFileInfo{snapshot_file, disk}); if (!inserted) LOG_WARNING( @@ -564,6 +588,9 @@ KeeperSnapshotManager::KeeperSnapshotManager( snapshot_up_to, disk->getName()); } + + for (const auto & [name, path] : incomplete_files) + disk->removeFile(path); }; for (const auto & disk : keeper_context->getOldSnapshotDisks()) @@ -572,25 +599,12 @@ KeeperSnapshotManager::KeeperSnapshotManager( auto disk = getDisk(); load_snapshot_from_disk(disk); + auto latest_snapshot_disk = getLatestSnapshotDisk(); + if (latest_snapshot_disk != disk) + load_snapshot_from_disk(latest_snapshot_disk); + removeOutdatedSnapshotsIfNeeded(); - - /// move snapshots from old disks to new one - for (auto & [_, file_info] : existing_snapshots) - { - if (file_info.disk == disk) - continue; - - auto file_path = fs::path(file_info.path); - auto tmp_snapshot_path = file_path.parent_path() / ("tmp_" + file_path.filename().generic_string()); - - { - disk->writeFile(tmp_snapshot_path); - } - - file_info.disk->copyFile(file_info.path, *disk, file_info.path, {}); - disk->removeFile(tmp_snapshot_path); - file_info.disk = disk; - } + moveSnapshotsIfNeeded(); } SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx) @@ -600,7 +614,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::bu auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; - auto disk = getDisk(); + auto disk = getLatestSnapshotDisk(); { disk->writeFile(tmp_snapshot_file_name); @@ -614,6 +628,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::bu existing_snapshots.emplace(up_to_log_idx, SnapshotFileInfo{snapshot_file_name, disk}); removeOutdatedSnapshotsIfNeeded(); + moveSnapshotsIfNeeded(); return {snapshot_file_name, disk}; } @@ -710,12 +725,47 @@ DiskPtr KeeperSnapshotManager::getDisk() const return keeper_context->getSnapshotDisk(); } +DiskPtr KeeperSnapshotManager::getLatestSnapshotDisk() const +{ + return keeper_context->getLatestSnapshotDisk(); +} + void KeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded() { while (existing_snapshots.size() > snapshots_to_keep) removeSnapshot(existing_snapshots.begin()->first); } +void KeeperSnapshotManager::moveSnapshotsIfNeeded() +{ + /// move snapshots to correct disks + + auto disk = getDisk(); + auto latest_snapshot_disk = getLatestSnapshotDisk(); + auto latest_snapshot_idx = getLatestSnapshotIndex(); + + for (auto & [idx, file_info] : existing_snapshots) + { + if (idx == latest_snapshot_idx) + { + if (file_info.disk != latest_snapshot_disk) + { + moveFileBetweenDisks(file_info.disk, file_info.path, latest_snapshot_disk, file_info.path); + file_info.disk = latest_snapshot_disk; + } + } + else + { + if (file_info.disk != disk) + { + moveFileBetweenDisks(file_info.disk, file_info.path, disk, file_info.path); + file_info.disk = disk; + } + } + } + +} + void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) { auto itr = existing_snapshots.find(log_idx); @@ -732,7 +782,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStor auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; - auto disk = getDisk(); + auto disk = getLatestSnapshotDisk(); { disk->writeFile(tmp_snapshot_file_name); } @@ -752,6 +802,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStor existing_snapshots.emplace(up_to_log_idx, SnapshotFileInfo{snapshot_file_name, disk}); removeOutdatedSnapshotsIfNeeded(); + moveSnapshotsIfNeeded(); return {snapshot_file_name, disk}; } diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 0afe582ef59..9bb287b9276 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -165,8 +165,10 @@ public: private: void removeOutdatedSnapshotsIfNeeded(); + void moveSnapshotsIfNeeded(); DiskPtr getDisk() const; + DiskPtr getLatestSnapshotDisk() const; /// Checks first 4 buffer bytes to became sure that snapshot compressed with /// ZSTD codec. @@ -184,6 +186,8 @@ private: size_t storage_tick_time; KeeperContextPtr keeper_context; + + Poco::Logger * log = &Poco::Logger::get("KeeperSnapshotManager"); }; /// Keeper create snapshots in background thread. KeeperStateMachine just create diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index e7cd409f569..a8c0d8d1518 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -468,15 +468,17 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res { latest_snapshot_meta = snapshot->snapshot_meta; /// we rely on the fact that the snapshot disk cannot be changed during runtime - if (isLocalDisk(*keeper_context->getSnapshotDisk())) + if (isLocalDisk(*keeper_context->getLatestSnapshotDisk())) { - latest_snapshot_info = snapshot_manager.serializeSnapshotToDisk(*snapshot); + auto snapshot_info = snapshot_manager.serializeSnapshotToDisk(*snapshot); + latest_snapshot_info = std::move(snapshot_info); latest_snapshot_buf = nullptr; } else { auto snapshot_buf = snapshot_manager.serializeSnapshotToBuffer(*snapshot); - latest_snapshot_info = snapshot_manager.serializeSnapshotBufferToDisk(*snapshot_buf, snapshot->snapshot_meta->get_last_log_idx()); + auto snapshot_info = snapshot_manager.serializeSnapshotBufferToDisk(*snapshot_buf, snapshot->snapshot_meta->get_last_log_idx()); + latest_snapshot_info = std::move(snapshot_info); latest_snapshot_buf = std::move(snapshot_buf); } From e91934bceaea3809feb0e2e52532b9eeb35ad7f2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 30 May 2023 16:02:14 +0000 Subject: [PATCH 092/515] Fixing style. --- src/Interpreters/GlobalSubqueriesVisitor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 8d2f2204e84..fa4fc2c82df 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -202,7 +202,7 @@ public: external_storage_holder->future_set = std::move(future_set); } else - throw Exception(ErrorCodes::LOGICAL_ERROR, "!!!!!!!!"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Set is already created for GLOBAL IN"); //prepared_sets->addStorageToSubquery(key, std::move(external_storage)); } From c7403284260992c296fbb34782bd1007a18ba28f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 30 May 2023 18:51:18 +0000 Subject: [PATCH 093/515] Fix more tests. --- src/Interpreters/ActionsVisitor.cpp | 11 +++++++---- src/Planner/CollectSets.cpp | 2 ++ src/Processors/QueryPlan/ReadFromMergeTree.cpp | 13 +++++++++++-- 3 files changed, 20 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 23c6867a868..142b6f73b75 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1420,11 +1420,14 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool return data.prepared_sets->addFromStorage(set_key, storage_set->getSet()); } - // std::cerr << ".... checking for " << identifier->getColumnName() << std::endl; - if (auto tmp_table = data.getContext()->findExternalTable(identifier->getColumnName())) + if (!data.getContext()->isGlobalContext()) { - external_table_set = tmp_table->future_set; - // std::cerr << "Found " << reinterpret_cast(tmp_table.get()) << " " << reinterpret_cast(external_table_set.get()) << std::endl; + // std::cerr << ".... checking for " << identifier->getColumnName() << std::endl; + if (auto tmp_table = data.getContext()->findExternalTable(identifier->getColumnName())) + { + external_table_set = tmp_table->future_set; + // std::cerr << "Found " << reinterpret_cast(tmp_table.get()) << " " << reinterpret_cast(external_table_set.get()) << std::endl; + } } } diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 19dc569ff0c..913cf1c6ee3 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -90,6 +90,8 @@ public: in_second_argument_node_type == QueryTreeNodeType::UNION) { auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash()); + if (sets.getFuture(set_key)) + return; auto subquery_options = select_query_options.subquery(); Planner subquery_planner( diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 367d3dcb525..8483df797ef 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1236,6 +1236,15 @@ static void buildIndexes( if (!indexes->use_skip_indexes) return; + const SelectQueryInfo * info = &query_info; + std::optional info_copy; + if (settings.allow_experimental_analyzer) + { + info_copy.emplace(query_info); + info_copy->filter_actions_dag = filter_actions_dag; + info = &*info_copy; + } + UsefulSkipIndexes skip_indexes; using Key = std::pair; std::map merged; @@ -1249,14 +1258,14 @@ static void buildIndexes( if (inserted) { skip_indexes.merged_indices.emplace_back(); - skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); + skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(*info, metadata_snapshot); } skip_indexes.merged_indices[it->second].addIndex(index_helper); } else { - auto condition = index_helper->createIndexCondition(query_info, context); + auto condition = index_helper->createIndexCondition(*info, context); if (!condition->alwaysUnknownOrTrue()) skip_indexes.useful_indices.emplace_back(index_helper, condition); } From 0d4ed32baca8eb2d897bcfd66eed0d04781af166 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Wed, 31 May 2023 11:25:33 -0400 Subject: [PATCH 094/515] better exception message --- src/Functions/FunctionsCodingIP.h | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionsCodingIP.h b/src/Functions/FunctionsCodingIP.h index bd53fa7e043..9d090abb736 100644 --- a/src/Functions/FunctionsCodingIP.h +++ b/src/Functions/FunctionsCodingIP.h @@ -341,7 +341,11 @@ ColumnPtr convertIPv6ToIPv4(ColumnPtr column, const PaddedPODArray * null { if constexpr (exception_mode == IPStringToNumExceptionMode::Throw) { - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 in column {} is not in IPv4 mapping block", column->getName()); + char addr[IPV6_MAX_TEXT_LENGTH + 1] {}; + char * paddr = addr; + formatIPv6(src, paddr); + + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 {} in column {} is not in IPv4 mapping block", addr, column->getName()); } else if constexpr (exception_mode == IPStringToNumExceptionMode::Default) { From 3af7e0a6fa21d570f78fcf9366c299e3199d2b77 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Wed, 31 May 2023 11:26:58 -0400 Subject: [PATCH 095/515] better exception message --- src/Functions/FunctionsConversion.h | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 4d4efc84df1..6d22fb661c3 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -310,7 +310,13 @@ struct ConvertImpl const uint8_t ip4_cidr[] {0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xff, 0xff, 0x00, 0x00, 0x00, 0x00}; const uint8_t * src = reinterpret_cast(&vec_from[i].toUnderType()); if (!matchIPv6Subnet(src, ip4_cidr, 96)) - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 in column {} is not in IPv4 mapping block", named_from.column->getName()); + { + char addr[IPV6_MAX_TEXT_LENGTH + 1] {}; + char * paddr = addr; + formatIPv6(src, paddr); + + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 {} in column {} is not in IPv4 mapping block", addr, named_from.column->getName()); + } uint8_t * dst = reinterpret_cast(&vec_to[i].toUnderType()); if constexpr (std::endian::native == std::endian::little) From 4ba08a5cbc960cace3dfcf32b3497855b3ffe6fd Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 31 May 2023 23:21:39 +0200 Subject: [PATCH 096/515] remove unused import --- src/DataTypes/Serializations/SerializationDate.h | 2 +- src/DataTypes/Serializations/SerializationDate32.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationDate.h b/src/DataTypes/Serializations/SerializationDate.h index 4d6a6fa36ec..f751b06fba6 100644 --- a/src/DataTypes/Serializations/SerializationDate.h +++ b/src/DataTypes/Serializations/SerializationDate.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/DataTypes/Serializations/SerializationDate32.h b/src/DataTypes/Serializations/SerializationDate32.h index 6b6e5442240..49560fb6c7d 100644 --- a/src/DataTypes/Serializations/SerializationDate32.h +++ b/src/DataTypes/Serializations/SerializationDate32.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { From 0b4d29ff98916d3ab1f2ea24f4254897b18351cf Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Jun 2023 12:19:00 +0000 Subject: [PATCH 097/515] Add tests for disks --- .../integration/test_keeper_disks/__init__.py | 0 .../configs/enable_keeper.xml | 53 ++++ tests/integration/test_keeper_disks/test.py | 260 ++++++++++++++++++ 3 files changed, 313 insertions(+) create mode 100644 tests/integration/test_keeper_disks/__init__.py create mode 100644 tests/integration/test_keeper_disks/configs/enable_keeper.xml create mode 100644 tests/integration/test_keeper_disks/test.py diff --git a/tests/integration/test_keeper_disks/__init__.py b/tests/integration/test_keeper_disks/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_disks/configs/enable_keeper.xml b/tests/integration/test_keeper_disks/configs/enable_keeper.xml new file mode 100644 index 00000000000..5814979229c --- /dev/null +++ b/tests/integration/test_keeper_disks/configs/enable_keeper.xml @@ -0,0 +1,53 @@ + + + + + local + /var/lib/clickhouse/coordination/logs/ + + + s3_plain + http://minio1:9001/root/logs/ + minio + minio123 + + + local + /var/lib/clickhouse/coordination/snapshots/ + + + s3_plain + http://minio1:9001/root/snapshots/ + minio + minio123 + + + + + + false + 9181 + 1 + false + + + 5000 + 10000 + trace + 10 + 10 + 1 + 3 + + + + + + + 1 + node + 9234 + + + + \ No newline at end of file diff --git a/tests/integration/test_keeper_disks/test.py b/tests/integration/test_keeper_disks/test.py new file mode 100644 index 00000000000..e978df18bab --- /dev/null +++ b/tests/integration/test_keeper_disks/test.py @@ -0,0 +1,260 @@ +#!/usr/bin/env python3 +import pytest +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as keeper_utils +from minio.deleteobjects import DeleteObject + +import os + +CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", main_configs=["configs/enable_keeper.xml"], stay_alive=True, with_minio=True +) + +from kazoo.client import KazooClient, KazooState + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def get_fake_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient( + hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout + ) + _fake_zk_instance.start() + return _fake_zk_instance + + +def stop_zk(zk): + try: + if zk: + zk.stop() + zk.close() + except: + pass + + +def stop_clickhouse(cluster, cleanup_disks): + node.stop_clickhouse() + + if not cleanup_disks: + return + + node.exec_in_container(["rm", "-rf", "/var/lib/clickhouse/coordination/logs"]) + node.exec_in_container( + ["rm", "-rf", "/var/lib/clickhouse/coordination/snapshots"] + ) + + s3_objects = list_s3_objects(cluster, prefix="") + if len(s3_objects) == 0: + return + + assert ( + len( + list( + cluster.minio_client.remove_objects( + cluster.minio_bucket, + [DeleteObject(obj) for obj in s3_objects], + ) + ) + ) + == 0 + ) + + +def setup_storage(cluster, storage_config, cleanup_disks): + stop_clickhouse(cluster, cleanup_disks) + node.copy_file_to_container( + os.path.join(CURRENT_TEST_DIR, "configs/enable_keeper.xml"), + "/etc/clickhouse-server/config.d/enable_keeper.xml", + ) + node.replace_in_config( + "/etc/clickhouse-server/config.d/enable_keeper.xml", + "", + storage_config, + ) + node.start_clickhouse() + keeper_utils.wait_until_connected(cluster, node) + + +def setup_local_storage(cluster): + setup_storage( + cluster, + "log_local<\\/log_storage_disk>" + "snapshot_local<\\/snapshot_storage_disk>", + cleanup_disks=True, + ) + + +def list_s3_objects(cluster, prefix=""): + minio = cluster.minio_client + prefix_len = len(prefix) + return [ + obj.object_name[prefix_len:] + for obj in minio.list_objects( + cluster.minio_bucket, prefix=prefix, recursive=True + ) + ] + + +def get_local_files(path): + files = node.exec_in_container(["ls", path]).strip().split("\n") + files.sort() + return files + + +def get_local_logs(): + return get_local_files("/var/lib/clickhouse/coordination/logs") + + +def get_local_snapshots(): + return get_local_files("/var/lib/clickhouse/coordination/snapshots") + + +def test_logs_with_disks(started_cluster): + setup_local_storage(started_cluster) + + node_zk = get_fake_zk("node") + try: + node_zk.create("/test") + for _ in range(30): + node_zk.create("/test/somenode", b"somedata", sequence=True) + + stop_zk(node_zk) + + previous_log_files = get_local_logs() + + setup_storage( + started_cluster, + "log_s3_plain<\\/log_storage_disk>" + "log_local<\\/latest_log_storage_disk>" + "snapshot_local<\\/snapshot_storage_disk>", + cleanup_disks=False, + ) + + # all but the latest log should be on S3 + s3_log_files = list_s3_objects(started_cluster, "logs/") + assert set(s3_log_files) == set(previous_log_files[:-1]) + local_log_files = get_local_logs() + assert len(local_log_files) == 1 + assert local_log_files[0] == previous_log_files[-1] + + previous_log_files = s3_log_files + local_log_files + + node_zk = get_fake_zk("node") + + for _ in range(30): + node_zk.create("/test/somenode", b"somedata", sequence=True) + + stop_zk(node_zk) + + log_files = list_s3_objects(started_cluster, "logs/") + local_log_files = get_local_logs() + assert len(local_log_files) == 1 + + log_files.extend(local_log_files) + assert set(log_files) != previous_log_files + + previous_log_files = log_files + + setup_storage( + started_cluster, + "log_s3_plain<\\/old_log_storage_disk>" + "log_local<\\/log_storage_disk>" + "snapshot_local<\\/snapshot_storage_disk>", + cleanup_disks=False, + ) + + local_log_files = get_local_logs() + assert set(local_log_files) == set(previous_log_files) + + node_zk = get_fake_zk("node") + + for child in node_zk.get_children("/test"): + assert node_zk.get(f"/test/{child}")[0] == b"somedata" + + finally: + stop_zk(node_zk) + + +def test_snapshots_with_disks(started_cluster): + setup_local_storage(started_cluster) + + node_zk = get_fake_zk("node") + try: + node_zk.create("/test2") + for _ in range(30): + node_zk.create("/test2/somenode", b"somedata", sequence=True) + + stop_zk(node_zk) + + snapshot_idx = keeper_utils.send_4lw_cmd(cluster, node, "csnp") + node.wait_for_log_line( + f"Created persistent snapshot {snapshot_idx}", look_behind_lines=1000 + ) + + previous_snapshot_files = get_local_snapshots() + + setup_storage( + started_cluster, + "snapshot_s3_plain<\\/snapshot_storage_disk>" + "snapshot_local<\\/latest_snapshot_storage_disk>" + "log_local<\\/log_storage_disk>", + cleanup_disks=False, + ) + + ## all but the latest log should be on S3 + s3_snapshot_files = list_s3_objects(started_cluster, "snapshots/") + assert set(s3_snapshot_files) == set(previous_snapshot_files[:-1]) + local_snapshot_files = get_local_snapshots() + assert len(local_snapshot_files) == 1 + assert local_snapshot_files[0] == previous_snapshot_files[-1] + + previous_snapshot_files = s3_snapshot_files + local_snapshot_files + + node_zk = get_fake_zk("node") + + for _ in range(30): + node_zk.create("/test2/somenode", b"somedata", sequence=True) + + stop_zk(node_zk) + + snapshot_idx = keeper_utils.send_4lw_cmd(cluster, node, "csnp") + node.wait_for_log_line( + f"Created persistent snapshot {snapshot_idx}", look_behind_lines=1000 + ) + + snapshot_files = list_s3_objects(started_cluster, "snapshots/") + local_snapshot_files = get_local_snapshots() + assert len(local_snapshot_files) == 1 + + snapshot_files.extend(local_snapshot_files) + + previous_snapshot_files = snapshot_files + + setup_storage( + started_cluster, + "snapshot_s3_plain<\\/old_snapshot_storage_disk>" + "snapshot_local<\\/snapshot_storage_disk>" + "log_local<\\/log_storage_disk>", + cleanup_disks=False, + ) + + local_snapshot_files = get_local_snapshots() + assert set(local_snapshot_files) == set(previous_snapshot_files) + + node_zk = get_fake_zk("node") + + for child in node_zk.get_children("/test2"): + assert node_zk.get(f"/test2/{child}")[0] == b"somedata" + + finally: + stop_zk(node_zk) From a9eb2c3c4eb8354753e611c6eed71fa9b672077c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Jun 2023 12:31:06 +0000 Subject: [PATCH 098/515] use multiple disks for jepsne --- .../resources/keeper_config.xml | 22 +++++++++++++++++++ .../src/jepsen/clickhouse/constants.clj | 2 ++ .../src/jepsen/clickhouse/keeper/utils.clj | 6 +++-- 3 files changed, 28 insertions(+), 2 deletions(-) diff --git a/tests/jepsen.clickhouse/resources/keeper_config.xml b/tests/jepsen.clickhouse/resources/keeper_config.xml index 2ab747fbd71..52f2a0dbdc2 100644 --- a/tests/jepsen.clickhouse/resources/keeper_config.xml +++ b/tests/jepsen.clickhouse/resources/keeper_config.xml @@ -8,6 +8,28 @@ never + + + + + local + /var/lib/clickhouse/coordination/logs/ + + + local + /var/lib/clickhouse/coordination/latest_log/ + + + local + /var/lib/clickhouse/coordination/snapshots/ + + + local + /var/lib/clickhouse/coordination/latest_snapshot/ + + + + 9181 {id} diff --git a/tests/jepsen.clickhouse/src/jepsen/clickhouse/constants.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/constants.clj index 887fc04265c..2a9a7ef6d27 100644 --- a/tests/jepsen.clickhouse/src/jepsen/clickhouse/constants.clj +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/constants.clj @@ -14,7 +14,9 @@ (def coordination-data-dir (str data-dir "/coordination")) (def coordination-snapshots-dir (str coordination-data-dir "/snapshots")) +(def coordination-latest-snapshot-dir (str coordination-data-dir "/latest_snapshot")) (def coordination-logs-dir (str coordination-data-dir "/logs")) +(def coordination-latest_log-dir (str coordination-data-dir "/latest_log")) (def stderr-file (str logs-dir "/stderr.log")) diff --git a/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/utils.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/utils.clj index b882af77758..869313079d0 100644 --- a/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/utils.clj +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/utils.clj @@ -146,6 +146,8 @@ :-- :--logger.log (str logs-dir "/clickhouse-keeper.log") :--logger.errorlog (str logs-dir "/clickhouse-keeper.err.log") - :--keeper_server.snapshot_storage_path coordination-snapshots-dir - :--keeper_server.log_storage_path coordination-logs-dir + :--keeper_server.snapshot_storage_disk "snapshot_local" + :--keeper_server.latest_snapshot_storage_disk "latest_snapshot_local" + :--keeper_server.log_storage_disk "log_local" + :--keeper_server.latest_log_storage_disk "latest_log_local" :--path coordination-data-dir)) From e8442b7a88467aa68a87c0abfa3983131c7b8c75 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Jun 2023 12:54:00 +0000 Subject: [PATCH 099/515] Add finalize calls --- src/Coordination/Changelog.cpp | 9 ++++++++- src/Coordination/KeeperSnapshotManager.cpp | 6 ++++-- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index bcce9982e04..97f2b437377 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -43,7 +43,8 @@ void moveFileBetweenDisks(DiskPtr disk_from, ChangelogFileDescriptionPtr descrip auto from_path = fs::path(description->path); auto tmp_changelog_name = from_path.parent_path() / (std::string{tmp_prefix} + from_path.filename().string()); { - disk_to->writeFile(tmp_changelog_name); + auto buf = disk_to->writeFile(tmp_changelog_name); + buf->finalize(); } disk_from->copyFile(from_path, *disk_to, path_to, {}); disk_to->removeFile(tmp_changelog_name); @@ -342,9 +343,15 @@ private: } if (log_file_settings.compress_logs) + { compressed_buffer.reset(); + } else + { + chassert(file_buf); + file_buf->finalize(); file_buf.reset(); + } } WriteBuffer & getBuffer() diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 4b9a34c07e9..8dd6d27c379 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -43,7 +43,8 @@ namespace auto from_path = fs::path(path_from); auto tmp_snapshot_name = from_path.parent_path() / (std::string{tmp_prefix} + from_path.filename().string()); { - disk_to->writeFile(tmp_snapshot_name); + auto buf = disk_to->writeFile(tmp_snapshot_name); + buf->finalize(); } disk_from->copyFile(from_path, *disk_to, path_to, {}); disk_to->removeFile(tmp_snapshot_name); @@ -784,7 +785,8 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStor auto disk = getLatestSnapshotDisk(); { - disk->writeFile(tmp_snapshot_file_name); + auto buf = disk->writeFile(tmp_snapshot_file_name); + buf->finalize(); } auto writer = disk->writeFile(snapshot_file_name); From 92ee24acd33dc6f62c369dcbd10c90f126b72613 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 1 Jun 2023 13:10:50 +0000 Subject: [PATCH 100/515] Automatic style fix --- tests/integration/test_keeper_disks/test.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/test_keeper_disks/test.py b/tests/integration/test_keeper_disks/test.py index e978df18bab..11bb215be54 100644 --- a/tests/integration/test_keeper_disks/test.py +++ b/tests/integration/test_keeper_disks/test.py @@ -49,9 +49,7 @@ def stop_clickhouse(cluster, cleanup_disks): return node.exec_in_container(["rm", "-rf", "/var/lib/clickhouse/coordination/logs"]) - node.exec_in_container( - ["rm", "-rf", "/var/lib/clickhouse/coordination/snapshots"] - ) + node.exec_in_container(["rm", "-rf", "/var/lib/clickhouse/coordination/snapshots"]) s3_objects = list_s3_objects(cluster, prefix="") if len(s3_objects) == 0: From ffd4f7f196db8f827ea3f8cc5edeb6b0a1082d72 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Jun 2023 14:39:01 +0000 Subject: [PATCH 101/515] Add better support for state disks --- src/Coordination/KeeperDispatcher.cpp | 2 -- src/Coordination/KeeperStateManager.cpp | 33 ++++++++++++++++++++----- 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index a5b0f0b2c01..9d9df5c7f30 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -38,8 +38,6 @@ namespace ProfileEvents extern const Event MemoryAllocatorPurgeTimeMicroseconds; } -namespace fs = std::filesystem; - namespace DB { diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 33e62684d46..450fd04b61d 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -23,6 +23,8 @@ namespace ErrorCodes namespace { +const std::string copy_lock_file = "STATE_COPY_LOCK"; + bool isLocalhost(const std::string & hostname) { try @@ -324,7 +326,13 @@ void KeeperStateManager::save_state(const nuraft::srv_state & state) auto disk = getStateFileDisk(); if (disk->exists(server_state_file_name)) - disk->moveFile(server_state_file_name, old_path); + { + auto buf = disk->writeFile(copy_lock_file); + buf->finalize(); + disk->copyFile(server_state_file_name, *disk, old_path); + disk->removeFile(copy_lock_file); + disk->removeFile(old_path); + } auto server_state_file = disk->writeFile(server_state_file_name); auto buf = state.serialize(); @@ -339,6 +347,7 @@ void KeeperStateManager::save_state(const nuraft::srv_state & state) server_state_file->write(reinterpret_cast(buf->data_begin()), buf->size()); server_state_file->sync(); + server_state_file->finalize(); disk->removeFileIfExists(old_path); } @@ -417,13 +426,25 @@ nuraft::ptr KeeperStateManager::read_state() if (disk->exists(old_path)) { - auto state = try_read_file(old_path); - if (state) + if (disk->exists(copy_lock_file)) { - disk->moveFile(old_path, server_state_file_name); - return state; + disk->removeFile(old_path); + disk->removeFile(copy_lock_file); } - disk->removeFile(old_path); + else + { + auto state = try_read_file(old_path); + if (state) + { + disk->moveFile(old_path, server_state_file_name); + return state; + } + disk->removeFile(old_path); + } + } + else if (disk->exists(copy_lock_file)) + { + disk->removeFile(copy_lock_file); } LOG_WARNING(logger, "No state was read"); From 92859ebb3baad37e6538118f5e2c5c8016754b7f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 1 Jun 2023 21:15:15 +0000 Subject: [PATCH 102/515] Fixing more tests. --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- src/Functions/in.cpp | 12 +- src/Interpreters/ActionsVisitor.cpp | 4 +- src/Interpreters/GlobalSubqueriesVisitor.h | 2 +- src/Interpreters/PreparedSets.cpp | 119 ++++++++++-------- src/Interpreters/PreparedSets.h | 25 ++-- src/Interpreters/Set.cpp | 30 ++++- src/Interpreters/Set.h | 7 +- src/Planner/CollectSets.cpp | 2 +- .../CreateSetAndFilterOnTheFlyStep.cpp | 2 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 8 +- src/Processors/QueryPlan/CreatingSetsStep.h | 6 +- .../Transforms/CreatingSetsTransform.cpp | 11 +- .../Transforms/CreatingSetsTransform.h | 6 +- src/QueryPipeline/QueryPipelineBuilder.cpp | 10 +- src/QueryPipeline/QueryPipelineBuilder.h | 10 +- src/Storages/MergeTree/RPNBuilder.cpp | 53 ++++---- src/Storages/StorageSet.cpp | 4 +- .../01786_explain_merge_tree.reference | 4 +- 19 files changed, 195 insertions(+), 122 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 05b3a545dca..b39aff86d32 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5141,7 +5141,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; - auto set = std::make_shared(size_limits_for_set, true /*fill_set_elements*/, settings.transform_null_in); + auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, 0, settings.transform_null_in); set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName()); set->insertFromBlock(result_block.getColumnsWithTypeAndName()); diff --git a/src/Functions/in.cpp b/src/Functions/in.cpp index 6a88a413c63..0e576b92aad 100644 --- a/src/Functions/in.cpp +++ b/src/Functions/in.cpp @@ -55,13 +55,9 @@ public: /// It is needed to perform type analysis without creation of set. static constexpr auto name = FunctionInName::name; - FunctionIn(SizeLimits size_limits_, bool transform_null_in_) - : size_limits(std::move(size_limits_)), transform_null_in(transform_null_in_) {} - - static FunctionPtr create(ContextPtr context) + static FunctionPtr create(ContextPtr) { - const auto & settings = context->getSettingsRef(); - return std::make_shared(FutureSet::getSizeLimitsForSet(settings, false), settings.transform_null_in); + return std::make_shared(); } String getName() const override @@ -182,10 +178,6 @@ public: return res; } - -private: - SizeLimits size_limits; - bool transform_null_in; }; template diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 142b6f73b75..59bbc74ca3a 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1435,7 +1435,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool // String set_id = right_in_operand->getColumnName(); //bool transform_null_in = data.getContext()->getSettingsRef().transform_null_in; SubqueryForSet subquery_for_set; // = data.prepared_sets->createOrGetSubquery(set_id, set_key, data.set_size_limit, transform_null_in); - subquery_for_set.key = right_in_operand->getColumnName(); + subquery_for_set.key = set_key.toString(); //right_in_operand->getColumnName(); /** The following happens for GLOBAL INs or INs: * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, @@ -1450,7 +1450,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool subquery_for_set.createSource(*interpreter); } - return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), std::move(external_table_set)); + return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), data.getContext()->getSettingsRef(), std::move(external_table_set)); } else { diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 624b33ea66b..cbdfb826f85 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -197,7 +197,7 @@ public: subquery_for_set.createSource(*interpreter); //std::cerr << reinterpret_cast(prepared_sets.get()) << std::endl; - auto future_set = prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), nullptr); + auto future_set = prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), getContext()->getSettingsRef(), nullptr); // std::cerr << "... Future set " << reinterpret_cast(external_storage_holder.get()) << " " << reinterpret_cast(future_set.get()) << std::endl; external_storage_holder->future_set = std::move(future_set); } diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index c71273e0baa..1e475e8403d 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -132,10 +132,10 @@ FutureSetPtr PreparedSets::addFromTuple(const PreparedSetKey & key, Block block, return it->second; } -FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, FutureSetPtr external_table_set) +FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set) { auto id = subquery.key; - auto from_subquery = std::make_shared(std::move(subquery), std::move(external_table_set)); + auto from_subquery = std::make_shared(std::move(subquery), std::move(external_table_set), settings.transform_null_in); auto [it, inserted] = sets.emplace(key, from_subquery); if (!inserted) @@ -210,6 +210,8 @@ std::variant, SharedSet> PreparedSetsCache::findOrPromiseTo { std::lock_guard lock(cache_mutex); + // std::cerr << "PreparedSetsCache::findOrPromiseToBuild " << key << "\n" << StackTrace().toString() << std::endl; + auto it = cache.find(key); if (it != cache.end()) { @@ -231,10 +233,10 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) if (!context->getSettingsRef().use_index_for_in_with_subqueries) return nullptr; - if (set) + if (subquery.set) { - if (set->hasExplicitSetElements()) - return set; + if (subquery.set->hasExplicitSetElements()) + return subquery.set; return nullptr; } @@ -242,7 +244,7 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) // std::cerr << "... external_table_set " << reinterpret_cast(external_table_set.get()) << std::endl; if (external_table_set) - return set = external_table_set->buildOrderedSetInplace(context); + return subquery.set = external_table_set->buildOrderedSetInplace(context); auto plan = buildPlan(context, true); if (!plan) @@ -255,37 +257,44 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) CompletedPipelineExecutor executor(pipeline); executor.execute(); - return set; + subquery.set->checkIsCreated(); + + return subquery.set; +} + +static SizeLimits getSizeLimitsForSet(const Settings & settings) +{ + return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); } std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & context, bool create_ordered_set) { - if (set) + if (subquery.set) return nullptr; // std::cerr << StackTrace().toString() << std::endl; - auto set_cache = context->getPreparedSetsCache(); - if (set_cache) - { - auto from_cache = set_cache->findOrPromiseToBuild(subquery.key); - if (from_cache.index() == 0) - { - subquery.promise_to_fill_set = std::move(std::get<0>(from_cache)); - } - else - { - LOG_TRACE(&Poco::Logger::get("FutureSetFromSubquery"), "Waiting for set, key: {}", subquery.key); - set = std::get<1>(from_cache).get(); - return nullptr; - } - } + // auto set_cache = context->getPreparedSetsCache(); + // if (set_cache) + // { + // auto from_cache = set_cache->findOrPromiseToBuild(subquery.key); + // if (from_cache.index() == 0) + // { + // subquery.promise_to_fill_set = std::move(std::get<0>(from_cache)); + // } + // else + // { + // LOG_TRACE(&Poco::Logger::get("FutureSetFromSubquery"), "Waiting for set, key: {}", subquery.key); + // set = std::get<1>(from_cache).get(); + // return nullptr; + // } + // } const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); + auto size_limits = getSizeLimitsForSet(settings); - subquery.set = set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); + subquery.set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); auto plan = subquery.detachSource(); auto description = subquery.key; @@ -297,7 +306,8 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c auto creating_set = std::make_unique( plan->getCurrentDataStream(), description, - std::move(subquery), + subquery, + shared_from_this(), SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), context); creating_set->setStepDescription("Create set for subquery"); @@ -305,31 +315,25 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c return plan; } +// static SizeLimits getSizeLimitsForOrderedSet(const Settings & settings) +// { +// if (settings.use_index_for_in_with_subqueries_max_values && +// settings.use_index_for_in_with_subqueries_max_values < settings.max_rows_in_set) +// return getSizeLimitsForUnorderedSet(settings); -static SizeLimits getSizeLimitsForUnorderedSet(const Settings & settings) -{ - return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); -} +// return SizeLimits(settings.use_index_for_in_with_subqueries_max_values, settings.max_bytes_in_set, OverflowMode::BREAK); +// } -static SizeLimits getSizeLimitsForOrderedSet(const Settings & settings) -{ - if (settings.use_index_for_in_with_subqueries_max_values && - settings.use_index_for_in_with_subqueries_max_values < settings.max_rows_in_set) - return getSizeLimitsForUnorderedSet(settings); - - return SizeLimits(settings.use_index_for_in_with_subqueries_max_values, settings.max_bytes_in_set, OverflowMode::BREAK); -} - -SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordered_set) -{ - return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); -} +// SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordered_set) +// { +// return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); +// } FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) { bool create_ordered_set = false; - auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); - set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); + auto size_limits = getSizeLimitsForSet(settings); + set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); Columns columns; @@ -345,8 +349,16 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) //block(std::move(block_)) } -FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_) - : subquery(std::move(subquery_)), external_table_set(std::move(external_table_set_)) {} +FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, bool transform_null_in_) + : subquery(std::move(subquery_)), external_table_set(std::move(external_table_set_)), transform_null_in(transform_null_in_) {} + +DataTypes FutureSetFromSubquery::getTypes() const +{ + if (subquery.set) + return subquery.set->getElementsTypes(); + + return Set::getElementTypes(subquery.source->getCurrentDataStream().header.getColumnsWithTypeAndName(), transform_null_in); +} FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} @@ -356,13 +368,14 @@ SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) return set; const auto & settings = context->getSettingsRef(); - auto limits = getSizeLimitsForSet(settings, true); + size_t max_values = settings.use_index_for_in_with_subqueries_max_values; + bool too_many_values = max_values && max_values < set->getTotalRowCount(); + if (!too_many_values) + { + set->initSetElements(); + set->appendSetElements(set_key_columns); + } - if (!limits.check(set->getTotalRowCount(), set->getTotalByteCount(), "IN-set", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED)) - return nullptr; - - set->initSetElements(); - set->appendSetElements(set_key_columns); return set; } diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index c510e690e25..fa7f7069994 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -72,7 +72,9 @@ public: virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; virtual std::unique_ptr build(const ContextPtr & context) = 0; - static SizeLimits getSizeLimitsForSet(const Settings & settings, bool ordered_set); + virtual DataTypes getTypes() const = 0; + + // static SizeLimits getSizeLimitsForSet(const Settings & settings, bool ordered_set); }; using FutureSetPtr = std::shared_ptr; @@ -90,6 +92,8 @@ public: std::unique_ptr build(const ContextPtr &) override; + DataTypes getTypes() const override { return set->getElementsTypes(); } + /// void buildForTuple(SizeLimits size_limits, bool transform_null_in); private: @@ -129,14 +133,14 @@ public: std::unique_ptr source; }; -class FutureSetFromSubquery : public FutureSet +class FutureSetFromSubquery : public FutureSet, public std::enable_shared_from_this { public: - FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_); + FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, bool transform_null_in_); - bool isReady() const override { return set != nullptr; } + bool isReady() const override { return subquery.set != nullptr && subquery.set->isCreated(); } bool isFilled() const override { return isReady(); } - SetPtr get() const override { return set; } + SetPtr get() const override { return subquery.set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override; @@ -145,12 +149,15 @@ public: return buildPlan(context, false); } - void addStorage(StoragePtr storage) { subquery.table = std::move(storage); } + DataTypes getTypes() const override; + + // void addStorage(StoragePtr storage) { subquery.table = std::move(storage); } private: - SetPtr set; + //SetPtr set; SubqueryForSet subquery; FutureSetPtr external_table_set; + bool transform_null_in; std::unique_ptr buildPlan(const ContextPtr & context, bool create_ordered_set); }; @@ -169,6 +176,8 @@ public: return set->hasExplicitSetElements() ? set : nullptr; } + DataTypes getTypes() const override { return set->getElementsTypes(); } + std::unique_ptr build(const ContextPtr &) override { return nullptr; } private: @@ -240,7 +249,7 @@ public: FutureSetPtr addFromStorage(const PreparedSetKey & key, SetPtr set_); FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings); - FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, FutureSetPtr external_table_set); + FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set); //void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage); diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index bd9cafc66eb..52e9a3a9bcb 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -103,6 +103,25 @@ void NO_INLINE Set::insertFromBlockImplCase( } +DataTypes Set::getElementTypes(const ColumnsWithTypeAndName & header, bool transform_null_in) +{ + DataTypes data_types; + data_types.reserve(header.size()); + + for (const auto & column : header) + { + data_types.push_back(column.type); + if (const auto * low_cardinality_type = typeid_cast(data_types.back().get())) + data_types.back() = low_cardinality_type->getDictionaryType(); + + if (!transform_null_in) + data_types.back() = removeNullable(data_types.back()); + } + + return data_types; +} + + void Set::setHeader(const ColumnsWithTypeAndName & header) { std::lock_guard lock(rwlock); @@ -190,7 +209,16 @@ bool Set::insertFromColumns(const Columns & columns) bool inserted = insertFromColumns(columns, holder); if (inserted && fill_set_elements) - appendSetElements(holder); + { + if (max_elements_to_fill && max_elements_to_fill < data.getTotalRowCount()) + { + /// Drop filled elementes + fill_set_elements = false; + set_elements.clear(); + } + else + appendSetElements(holder); + } return inserted; } diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index c2931d79de0..9b76ab30a1b 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -30,9 +30,9 @@ public: /// (that is useful only for checking that some value is in the set and may not store the original values), /// store all set elements in explicit form. /// This is needed for subsequent use for index. - Set(const SizeLimits & limits_, bool fill_set_elements_, bool transform_null_in_) + Set(const SizeLimits & limits_, bool fill_set_elements_, size_t max_elements_to_fill_, bool transform_null_in_) : log(&Poco::Logger::get("Set")), - limits(limits_), fill_set_elements(fill_set_elements_), transform_null_in(transform_null_in_) + limits(limits_), fill_set_elements(fill_set_elements_), max_elements_to_fill(max_elements_to_fill_), transform_null_in(transform_null_in_) { } @@ -90,6 +90,8 @@ public: bool areTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const; void checkTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const; + static DataTypes getElementTypes(const ColumnsWithTypeAndName & header, bool transform_null_in); + private: size_t keys_size = 0; Sizes key_sizes; @@ -126,6 +128,7 @@ private: /// Do we need to additionally store all elements of the set in explicit form for subsequent use for index. bool fill_set_elements; + size_t max_elements_to_fill; /// If true, insert NULL values to set. bool transform_null_in; diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 913cf1c6ee3..d264810645a 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -110,7 +110,7 @@ public: subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); /// TODO - sets.addFromSubquery(set_key, std::move(subquery_for_set), nullptr); + sets.addFromSubquery(set_key, std::move(subquery_for_set), settings, nullptr); //planner_context.registerSet(set_key, PlannerSet(in_second_argument)); } diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index fe362f64b96..ebd58c3dc95 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -105,7 +105,7 @@ CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( : ITransformingStep(input_stream_, input_stream_.header, getTraits()) , column_names(column_names_) , max_rows_in_set(max_rows_in_set_) - , own_set(std::make_shared(SizeLimits(max_rows_in_set, 0, OverflowMode::BREAK), false, true)) + , own_set(std::make_shared(SizeLimits(max_rows_in_set, 0, OverflowMode::BREAK), false, 0, true)) , filtering_set(nullptr) , crosswise_connection(crosswise_connection_) , position(position_) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 38506412531..0909ee9f1eb 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -35,12 +35,14 @@ static ITransformingStep::Traits getTraits() CreatingSetStep::CreatingSetStep( const DataStream & input_stream_, String description_, - SubqueryForSet subquery_for_set_, + SubqueryForSet & subquery_for_set_, + FutureSetPtr set_, SizeLimits network_transfer_limits_, ContextPtr context_) : ITransformingStep(input_stream_, Block{}, getTraits()) , description(std::move(description_)) - , subquery_for_set(std::move(subquery_for_set_)) + , subquery_for_set(subquery_for_set_) + , set(std::move(set_)) , network_transfer_limits(std::move(network_transfer_limits_)) , context(std::move(context_)) { @@ -48,7 +50,7 @@ CreatingSetStep::CreatingSetStep( void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, context->getPreparedSetsCache()); + pipeline.addCreatingSetsTransform(getOutputStream().header, subquery_for_set, std::move(set), network_transfer_limits, context->getPreparedSetsCache()); } void CreatingSetStep::updateOutputStream() diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index 244bb27ba78..2a6cb43c45a 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -15,7 +15,8 @@ public: CreatingSetStep( const DataStream & input_stream_, String description_, - SubqueryForSet subquery_for_set_, + SubqueryForSet & subquery_for_set_, + FutureSetPtr set_, SizeLimits network_transfer_limits_, ContextPtr context_); @@ -30,7 +31,8 @@ private: void updateOutputStream() override; String description; - SubqueryForSet subquery_for_set; + SubqueryForSet & subquery_for_set; + FutureSetPtr set; SizeLimits network_transfer_limits; ContextPtr context; }; diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index 24e17ff210f..f6e6f7a191e 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -25,11 +25,13 @@ CreatingSetsTransform::~CreatingSetsTransform() = default; CreatingSetsTransform::CreatingSetsTransform( Block in_header_, Block out_header_, - SubqueryForSet subquery_for_set_, + SubqueryForSet & subquery_for_set_, + FutureSetPtr set_, SizeLimits network_transfer_limits_, PreparedSetsCachePtr prepared_sets_cache_) : IAccumulatingTransform(std::move(in_header_), std::move(out_header_)) - , subquery(std::move(subquery_for_set_)) + , subquery(subquery_for_set_) + , set(std::move(set_)) , network_transfer_limits(std::move(network_transfer_limits_)) , prepared_sets_cache(std::move(prepared_sets_cache_)) { @@ -61,6 +63,7 @@ void CreatingSetsTransform::startSubquery() auto from_cache = prepared_sets_cache->findOrPromiseToBuild(subquery.key); if (from_cache.index() == 0) { + LOG_TRACE(log, "Building set, key: {}", subquery.key); promise_to_build = std::move(std::get<0>(from_cache)); } else @@ -74,8 +77,8 @@ void CreatingSetsTransform::startSubquery() continue; } - subquery.promise_to_fill_set.set_value(ready_set); - subquery.set.reset(); + //subquery.promise_to_fill_set.set_value(ready_set); + subquery.set = ready_set; //.reset(); done_with_set = true; set_from_cache = true; } diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index ef586b0655c..27c330bdbc3 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -29,7 +29,8 @@ public: CreatingSetsTransform( Block in_header_, Block out_header_, - SubqueryForSet subquery_for_set_, + SubqueryForSet & subquery_for_set_, + FutureSetPtr set_, SizeLimits network_transfer_limits_, PreparedSetsCachePtr prepared_sets_cache_); @@ -42,7 +43,8 @@ public: Chunk generate() override; private: - SubqueryForSet subquery; + SubqueryForSet & subquery; + FutureSetPtr set; std::optional> promise_to_build; QueryPipeline table_out; diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 4898501b980..08d90899f4a 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -569,14 +569,20 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe return left; } -void QueryPipelineBuilder::addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, PreparedSetsCachePtr prepared_sets_cache) +void QueryPipelineBuilder::addCreatingSetsTransform( + const Block & res_header, + SubqueryForSet & subquery_for_set, + FutureSetPtr set, + const SizeLimits & limits, + PreparedSetsCachePtr prepared_sets_cache) { resize(1); auto transform = std::make_shared( getHeader(), res_header, - std::move(subquery_for_set), + subquery_for_set, + std::move(set), limits, std::move(prepared_sets_cache)); diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 85b6f5c6772..da8443a7e33 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -33,6 +33,9 @@ class TableJoin; class QueryPipelineBuilder; using QueryPipelineBuilderPtr = std::unique_ptr; +class FutureSet; +using FutureSetPtr = std::shared_ptr; + class PreparedSetsCache; using PreparedSetsCachePtr = std::shared_ptr; @@ -141,7 +144,12 @@ public: /// This is used for CreatingSets. void addPipelineBefore(QueryPipelineBuilder pipeline); - void addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, PreparedSetsCachePtr prepared_sets_cache); + void addCreatingSetsTransform( + const Block & res_header, + SubqueryForSet & subquery_for_set, + FutureSetPtr set, + const SizeLimits & limits, + PreparedSetsCachePtr prepared_sets_cache); PipelineExecutorPtr execute(); diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index cc7ec45be6a..c9feb533ea3 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -353,34 +353,34 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( { const auto & prepared_sets = getTreeContext().getPreparedSets(); + /// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information + /// about types in left argument of the IN operator. Instead, we manually iterate through all the sets + /// and find the one for the right arg based on the AST structure (getTreeHash), after that we check + /// that the types it was prepared with are compatible with the types of the primary key. + auto types_match = [&indexes_mapping, &data_types](const DataTypes & set_types) + { + assert(indexes_mapping.size() == data_types.size()); + + for (size_t i = 0; i < indexes_mapping.size(); ++i) + { + if (indexes_mapping[i].tuple_index >= set_types.size()) + return false; + + auto lhs = recursiveRemoveLowCardinality(data_types[i]); + auto rhs = recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index]); + + if (!lhs->equals(*rhs)) + return false; + } + + return true; + }; + if (prepared_sets && ast_node) { if (ast_node->as() || ast_node->as()) return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); - /// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information - /// about types in left argument of the IN operator. Instead, we manually iterate through all the sets - /// and find the one for the right arg based on the AST structure (getTreeHash), after that we check - /// that the types it was prepared with are compatible with the types of the primary key. - auto types_match = [&indexes_mapping, &data_types](const DataTypes & set_types) - { - assert(indexes_mapping.size() == data_types.size()); - - for (size_t i = 0; i < indexes_mapping.size(); ++i) - { - if (indexes_mapping[i].tuple_index >= set_types.size()) - return false; - - auto lhs = recursiveRemoveLowCardinality(data_types[i]); - auto rhs = recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index]); - - if (!lhs->equals(*rhs)) - return false; - } - - return true; - }; - auto tree_hash = ast_node->getTreeHash(); for (const auto & [key, future_set] : prepared_sets->getSets()) { @@ -392,7 +392,12 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( { const auto * node_without_alias = getNodeWithoutAlias(dag_node); if (node_without_alias->column) - return tryGetSetFromDAGNode(node_without_alias); + { + auto future_set = tryGetSetFromDAGNode(node_without_alias); + auto set_types = future_set->getTypes(); + if (types_match(future_set->getTypes())) + return future_set; + } } return nullptr; diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 00b5dbfc5e3..da94c87b69e 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -147,7 +147,7 @@ StorageSet::StorageSet( const String & comment, bool persistent_) : StorageSetOrJoinBase{disk_, relative_path_, table_id_, columns_, constraints_, comment, persistent_} - , set(std::make_shared(SizeLimits(), false, true)) + , set(std::make_shared(SizeLimits(), false, 0, true)) { Block header = getInMemoryMetadataPtr()->getSampleBlock(); set->setHeader(header.getColumnsWithTypeAndName()); @@ -176,7 +176,7 @@ void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_sn Block header = metadata_snapshot->getSampleBlock(); increment = 0; - set = std::make_shared(SizeLimits(), false, true); + set = std::make_shared(SizeLimits(), false, 0, true); set->setHeader(header.getColumnsWithTypeAndName()); } diff --git a/tests/queries/0_stateless/01786_explain_merge_tree.reference b/tests/queries/0_stateless/01786_explain_merge_tree.reference index e6628813dbd..8d3954484dd 100644 --- a/tests/queries/0_stateless/01786_explain_merge_tree.reference +++ b/tests/queries/0_stateless/01786_explain_merge_tree.reference @@ -10,7 +10,7 @@ Keys: y bitAnd(z, 3) - Condition: and((bitAnd(z, 3) not in [1, 1]), and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1]))) + Condition: and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1])) Parts: 3/4 Granules: 10/11 PrimaryKey @@ -46,7 +46,7 @@ { "Type": "Partition", "Keys": ["y", "bitAnd(z, 3)"], - "Condition": "and((bitAnd(z, 3) not in [1, 1]), and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1])))", + "Condition": "and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1]))", "Initial Parts": 4, "Selected Parts": 3, "Initial Granules": 11, From 36129c3d20bef74c4502126b2c37001e2c80b8d2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 2 Jun 2023 13:36:14 +0000 Subject: [PATCH 103/515] Try to fix last tests. --- src/Storages/MergeTree/RPNBuilder.cpp | 6 ++++-- .../0_stateless/00981_in_subquery_with_tuple.reference | 3 --- tests/queries/0_stateless/01651_bugs_from_15889.reference | 1 + tests/queries/0_stateless/01651_bugs_from_15889.sql | 5 ++--- 4 files changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index c9feb533ea3..b82c350fa1a 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -366,8 +366,10 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( if (indexes_mapping[i].tuple_index >= set_types.size()) return false; - auto lhs = recursiveRemoveLowCardinality(data_types[i]); - auto rhs = recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index]); + auto lhs = removeNullable(recursiveRemoveLowCardinality(data_types[i])); + auto rhs = removeNullable(recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index])); + + // std::cerr << "============ " << lhs->getName() << ' ' << rhs->getName() << std::endl; if (!lhs->equals(*rhs)) return false; diff --git a/tests/queries/0_stateless/00981_in_subquery_with_tuple.reference b/tests/queries/0_stateless/00981_in_subquery_with_tuple.reference index 673d035ede6..833a8c93d24 100644 --- a/tests/queries/0_stateless/00981_in_subquery_with_tuple.reference +++ b/tests/queries/0_stateless/00981_in_subquery_with_tuple.reference @@ -1,7 +1,4 @@ -OK1 OK2 OK3 -OK4 -OK5 2019-08-11 world 2019-08-12 hello diff --git a/tests/queries/0_stateless/01651_bugs_from_15889.reference b/tests/queries/0_stateless/01651_bugs_from_15889.reference index 8b137891791..77ac542d4fb 100644 --- a/tests/queries/0_stateless/01651_bugs_from_15889.reference +++ b/tests/queries/0_stateless/01651_bugs_from_15889.reference @@ -1 +1,2 @@ +0 diff --git a/tests/queries/0_stateless/01651_bugs_from_15889.sql b/tests/queries/0_stateless/01651_bugs_from_15889.sql index 4717a8dcc0d..6a8c6d35911 100644 --- a/tests/queries/0_stateless/01651_bugs_from_15889.sql +++ b/tests/queries/0_stateless/01651_bugs_from_15889.sql @@ -8,10 +8,9 @@ INSERT INTO xp SELECT '2020-01-01', number, '' FROM numbers(100000); CREATE TABLE xp_d AS xp ENGINE = Distributed(test_shard_localhost, currentDatabase(), xp); --- FIXME: this query spontaneously returns either 8 or 20 error code. Looks like it's potentially flaky. --- SELECT count(7 = (SELECT number FROM numbers(0) ORDER BY number ASC NULLS FIRST LIMIT 7)) FROM xp_d PREWHERE toYYYYMM(A) GLOBAL IN (SELECT NULL = (SELECT number FROM numbers(1) ORDER BY number DESC NULLS LAST LIMIT 1), toYYYYMM(min(A)) FROM xp_d) WHERE B > NULL; -- { serverError 8 } +SELECT count(7 = (SELECT number FROM numbers(0) ORDER BY number ASC NULLS FIRST LIMIT 7)) FROM xp_d PREWHERE toYYYYMM(A) GLOBAL IN (SELECT NULL = (SELECT number FROM numbers(1) ORDER BY number DESC NULLS LAST LIMIT 1), toYYYYMM(min(A)) FROM xp_d) WHERE B > NULL; -- { serverError 8 } -SELECT count() FROM xp_d WHERE A GLOBAL IN (SELECT NULL); -- { serverError 53 } +SELECT count() FROM xp_d WHERE A GLOBAL IN (SELECT NULL); DROP TABLE IF EXISTS xp; DROP TABLE IF EXISTS xp_d; From 47966c4e8129c3777679e42966cfec9c7a2aa383 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Jun 2023 14:10:52 +0000 Subject: [PATCH 104/515] Finalize bufferg --- src/Coordination/KeeperSnapshotManager.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 8dd6d27c379..44e990c7b95 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -618,12 +618,14 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::bu auto disk = getLatestSnapshotDisk(); { - disk->writeFile(tmp_snapshot_file_name); + auto buf = disk->writeFile(tmp_snapshot_file_name); + buf->finalize(); } auto plain_buf = disk->writeFile(snapshot_file_name); copyData(reader, *plain_buf); plain_buf->sync(); + plain_buf->finalize(); disk->removeFile(tmp_snapshot_file_name); From 975e58c56d5cf7038052e0be9699c094c7203161 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 2 Jun 2023 15:15:44 +0000 Subject: [PATCH 105/515] Fix style. --- src/Interpreters/PreparedSets.cpp | 1 - src/Planner/CollectSets.cpp | 2 ++ src/Processors/QueryPlan/CreatingSetsStep.cpp | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 1e475e8403d..6df7c748e60 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -14,7 +14,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int SET_SIZE_LIMIT_EXCEEDED; } PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index d264810645a..5f44994c14b 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -81,6 +81,8 @@ public: element_type = low_cardinality_type->getDictionaryType(); auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); + if (sets.getFuture(set_key)) + return; sets.addFromTuple(set_key, std::move(set), settings); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 0909ee9f1eb..264c4b9ef47 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -46,6 +46,7 @@ CreatingSetStep::CreatingSetStep( , network_transfer_limits(std::move(network_transfer_limits_)) , context(std::move(context_)) { + std::cerr << StackTrace().toString() << std::endl; } void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) From c19866f72d3b356f2c4fed334c2beeecd441ef0e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 2 Jun 2023 15:16:36 +0000 Subject: [PATCH 106/515] Fix style. --- src/Processors/QueryPlan/CreatingSetsStep.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 264c4b9ef47..0909ee9f1eb 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -46,7 +46,6 @@ CreatingSetStep::CreatingSetStep( , network_transfer_limits(std::move(network_transfer_limits_)) , context(std::move(context_)) { - std::cerr << StackTrace().toString() << std::endl; } void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) From 34c4b89b161df4427912d402fa6aee4a6821dde1 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 4 Jun 2023 16:23:34 +0000 Subject: [PATCH 107/515] fix backward compatibility for IP types hashing in aggregate functions --- base/base/IPv4andIPv6.h | 24 +++++++++++++------ .../AggregateFunctionMap.cpp | 3 +++ src/AggregateFunctions/AggregateFunctionMap.h | 3 +++ .../AggregateFunctionUniq.cpp | 9 +++++++ .../AggregateFunctionUniq.h | 12 +++++----- src/Core/Types_fwd.h | 2 +- src/DataTypes/DataTypeMap.cpp | 3 ++- 7 files changed, 41 insertions(+), 15 deletions(-) diff --git a/base/base/IPv4andIPv6.h b/base/base/IPv4andIPv6.h index 7b745ec7b84..4aee2329572 100644 --- a/base/base/IPv4andIPv6.h +++ b/base/base/IPv4andIPv6.h @@ -7,16 +7,17 @@ namespace DB { - using IPv4 = StrongTypedef; + struct IPv4 : StrongTypedef + { + using StrongTypedef::StrongTypedef; + using StrongTypedef::operator=; + constexpr explicit IPv4(UInt64 value): StrongTypedef(static_cast(value)) {} + }; struct IPv6 : StrongTypedef { - constexpr IPv6() = default; - constexpr explicit IPv6(const UInt128 & x) : StrongTypedef(x) {} - constexpr explicit IPv6(UInt128 && x) : StrongTypedef(std::move(x)) {} - - IPv6 & operator=(const UInt128 & rhs) { StrongTypedef::operator=(rhs); return *this; } - IPv6 & operator=(UInt128 && rhs) { StrongTypedef::operator=(std::move(rhs)); return *this; } + using StrongTypedef::StrongTypedef; + using StrongTypedef::operator=; bool operator<(const IPv6 & rhs) const { @@ -62,4 +63,13 @@ namespace std return std::hash()(x.toUnderType()); } }; + + template <> + struct hash + { + size_t operator()(const DB::IPv4 & x) const + { + return std::hash()(x.toUnderType()); + } + }; } diff --git a/src/AggregateFunctions/AggregateFunctionMap.cpp b/src/AggregateFunctions/AggregateFunctionMap.cpp index f6100602f3f..38e4f49d9a2 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionMap.cpp @@ -100,6 +100,9 @@ public: return std::make_shared>(nested_function, arguments); case TypeIndex::UUID: return std::make_shared>(nested_function, arguments); + case TypeIndex::IPv4: + return std::make_shared>(nested_function, arguments); + case TypeIndex::IPv6: case TypeIndex::FixedString: case TypeIndex::String: return std::make_shared>(nested_function, arguments); diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 55f6611974e..4a4ae92735b 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -19,6 +19,7 @@ #include #include #include "DataTypes/Serializations/ISerialization.h" +#include #include "base/types.h" #include #include "AggregateFunctions/AggregateFunctionFactory.h" @@ -147,6 +148,8 @@ public: StringRef key_ref; if (key_type->getTypeId() == TypeIndex::FixedString) key_ref = assert_cast(key_column).getDataAt(offset + i); + else if (key_type->getTypeId() == TypeIndex::IPv6) + key_ref = assert_cast(key_column).getDataAt(offset + i); else key_ref = assert_cast(key_column).getDataAt(offset + i); diff --git a/src/AggregateFunctions/AggregateFunctionUniq.cpp b/src/AggregateFunctions/AggregateFunctionUniq.cpp index bf998c5ee9e..f5147daa97b 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniq.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -60,6 +61,10 @@ createAggregateFunctionUniq(const std::string & name, const DataTypes & argument return std::make_shared>(argument_types); else if (which.isUUID()) return std::make_shared>(argument_types); + else if (which.isIPv4()) + return std::make_shared>(argument_types); + else if (which.isIPv6()) + return std::make_shared>(argument_types); else if (which.isTuple()) { if (use_exact_hash_function) @@ -109,6 +114,10 @@ createAggregateFunctionUniq(const std::string & name, const DataTypes & argument return std::make_shared>>(argument_types); else if (which.isUUID()) return std::make_shared>>(argument_types); + else if (which.isIPv4()) + return std::make_shared>>(argument_types); + else if (which.isIPv6()) + return std::make_shared>>(argument_types); else if (which.isTuple()) { if (use_exact_hash_function) diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index c782b9314fd..0524dd53ec0 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -248,17 +248,17 @@ struct Adder AggregateFunctionUniqUniquesHashSetData> || std::is_same_v>) { const auto & column = *columns[0]; - if constexpr (!std::is_same_v) + if constexpr (std::is_same_v || std::is_same_v) + { + StringRef value = column.getDataAt(row_num); + data.set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size)); + } + else { using ValueType = typename decltype(data.set)::value_type; const auto & value = assert_cast &>(column).getElement(row_num); data.set.insert(static_cast(AggregateFunctionUniqTraits::hash(value))); } - else - { - StringRef value = column.getDataAt(row_num); - data.set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size)); - } } else if constexpr (std::is_same_v>) { diff --git a/src/Core/Types_fwd.h b/src/Core/Types_fwd.h index 3db1127fe5a..a59e4b6eab8 100644 --- a/src/Core/Types_fwd.h +++ b/src/Core/Types_fwd.h @@ -27,7 +27,7 @@ namespace DB using UUID = StrongTypedef; -using IPv4 = StrongTypedef; +struct IPv4; struct IPv6; diff --git a/src/DataTypes/DataTypeMap.cpp b/src/DataTypes/DataTypeMap.cpp index 0f5d97e6761..90561857fad 100644 --- a/src/DataTypes/DataTypeMap.cpp +++ b/src/DataTypes/DataTypeMap.cpp @@ -69,7 +69,7 @@ void DataTypeMap::assertKeyType() const if (!checkKeyType(key_type)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type of Map key must be a type, that can be represented by integer " - "or String or FixedString (possibly LowCardinality) or UUID," + "or String or FixedString (possibly LowCardinality) or UUID or IPv6," " but {} given", key_type->getName()); } @@ -120,6 +120,7 @@ bool DataTypeMap::checkKeyType(DataTypePtr key_type) else if (!key_type->isValueRepresentedByInteger() && !isStringOrFixedString(*key_type) && !WhichDataType(key_type).isNothing() + && !WhichDataType(key_type).isIPv6() && !WhichDataType(key_type).isUUID()) { return false; From ece96f54e96b526693e58b859c9c835f17eff5f4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 5 Jun 2023 09:47:58 +0000 Subject: [PATCH 108/515] Fix tests --- src/Coordination/KeeperSnapshotManager.cpp | 8 +++----- src/Coordination/tests/gtest_coordination.cpp | 2 ++ 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 44e990c7b95..d10df0fd785 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -62,7 +62,7 @@ namespace std::string getSnapshotFileName(uint64_t up_to_log_idx, bool compress_zstd) { - auto base = std::string{"snapshot_"} + std::to_string(up_to_log_idx) + ".bin"; + auto base = fmt::format("snapshot_{}.bin", up_to_log_idx); if (compress_zstd) base += ".zstd"; return base; @@ -567,10 +567,8 @@ KeeperSnapshotManager::KeeperSnapshotManager( continue; } - if (clean_incomplete_file(it->path())) - continue; - - snapshot_files.push_back(it->path()); + if (it->name().starts_with("snapshot_") && !clean_incomplete_file(it->path())) + snapshot_files.push_back(it->path()); } for (const auto & snapshot_file : snapshot_files) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index ff7d545ecdd..50e81eca8ca 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1944,6 +1944,8 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) changelog1.end_of_append_batch(0, 0); } + waitDurableLogs(changelog1); + DB::KeeperLogStore changelog2( DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, keeper_context); changelog2.init(0, 3); From 760483d8d3a012b4b6456b9bdf14afd2d052a514 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 6 Jun 2023 04:18:51 +0000 Subject: [PATCH 109/515] multiple fixes --- base/base/IPv4andIPv6.h | 4 ++- .../AggregateFunctionMap.cpp | 1 + src/AggregateFunctions/AggregateFunctionMap.h | 26 +++++++++++++++++++ .../AggregateFunctionUniq.cpp | 2 +- .../AggregateFunctionUniq.h | 12 +++++++++ src/IO/ReadHelpers.h | 14 ++++++++++ src/IO/WriteHelpers.h | 8 ++++++ 7 files changed, 65 insertions(+), 2 deletions(-) diff --git a/base/base/IPv4andIPv6.h b/base/base/IPv4andIPv6.h index 4aee2329572..e2f93b54124 100644 --- a/base/base/IPv4andIPv6.h +++ b/base/base/IPv4andIPv6.h @@ -2,6 +2,7 @@ #include #include +#include #include namespace DB @@ -55,12 +56,13 @@ namespace DB namespace std { + /// For historical reasons we hash IPv6 as a FixedString(16) template <> struct hash { size_t operator()(const DB::IPv6 & x) const { - return std::hash()(x.toUnderType()); + return std::hash{}(std::string_view(reinterpret_cast(&x.toUnderType()), IPV6_BINARY_LENGTH)); } }; diff --git a/src/AggregateFunctions/AggregateFunctionMap.cpp b/src/AggregateFunctions/AggregateFunctionMap.cpp index 38e4f49d9a2..b957b541fe1 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionMap.cpp @@ -103,6 +103,7 @@ public: case TypeIndex::IPv4: return std::make_shared>(nested_function, arguments); case TypeIndex::IPv6: + return std::make_shared>(nested_function, arguments); case TypeIndex::FixedString: case TypeIndex::String: return std::make_shared>(nested_function, arguments); diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 4a4ae92735b..7b9bb088d8f 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -21,6 +21,7 @@ #include "DataTypes/Serializations/ISerialization.h" #include #include "base/types.h" +#include #include #include "AggregateFunctions/AggregateFunctionFactory.h" @@ -70,6 +71,31 @@ struct AggregateFunctionMapCombinatorData } }; +/// Specialization for IPv6 - for historical reasons it should be stored as FixedString(16) +template <> +struct AggregateFunctionMapCombinatorData +{ + struct IPv6Hash + { + using hash_type = std::hash; + using is_transparent = void; + + size_t operator()(const IPv6 & ip) const { return hash_type{}(ip); } + }; + + using SearchType = IPv6; + std::unordered_map> merged_maps; + + static void writeKey(const IPv6 & key, WriteBuffer & buf) + { + writeIPv6Binary(key, buf); + } + static void readKey(IPv6 & key, ReadBuffer & buf) + { + readIPv6Binary(key, buf); + } +}; + template class AggregateFunctionMap final : public IAggregateFunctionDataHelper, AggregateFunctionMap> diff --git a/src/AggregateFunctions/AggregateFunctionUniq.cpp b/src/AggregateFunctions/AggregateFunctionUniq.cpp index f5147daa97b..748a232641e 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniq.cpp @@ -117,7 +117,7 @@ createAggregateFunctionUniq(const std::string & name, const DataTypes & argument else if (which.isIPv4()) return std::make_shared>>(argument_types); else if (which.isIPv6()) - return std::make_shared>>(argument_types); + return std::make_shared>>(argument_types); else if (which.isTuple()) { if (use_exact_hash_function) diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index 0524dd53ec0..03d999b47e2 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -101,6 +101,18 @@ struct AggregateFunctionUniqHLL12Data static String getName() { return "uniqHLL12"; } }; +template <> +struct AggregateFunctionUniqHLL12Data +{ + using Set = HyperLogLogWithSmallSetOptimization; + Set set; + + constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_variadic = false; + + static String getName() { return "uniqHLL12"; } +}; + template struct AggregateFunctionUniqHLL12DataForVariadic { diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 32338552b66..1aa294f76bf 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -61,6 +61,7 @@ namespace ErrorCodes extern const int INCORRECT_DATA; extern const int TOO_LARGE_STRING_SIZE; extern const int TOO_LARGE_ARRAY_SIZE; + extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH; } /// Helper functions for formatted input. @@ -136,6 +137,19 @@ inline void readStringBinary(std::string & s, ReadBuffer & buf, size_t max_strin buf.readStrict(s.data(), size); } +/// For historical reasons we store IPv6 as a String +inline void readIPv6Binary(IPv6 & ip, ReadBuffer & buf) +{ + size_t size = 0; + readVarUInt(size, buf); + + if (size != IPV6_BINARY_LENGTH) + throw Exception(ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH, + "Size of the string {} doesn't match size of binary IPv6 {}", size, IPV6_BINARY_LENGTH); + + buf.readStrict(reinterpret_cast(&ip.toUnderType()), size); +} + template void readVectorBinary(std::vector & v, ReadBuffer & buf) { diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index cdbc952690c..505a2f988f0 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -10,6 +10,7 @@ #include +#include "Common/formatIPv6.h" #include #include #include @@ -104,6 +105,13 @@ inline void writeStringBinary(const std::string & s, WriteBuffer & buf) buf.write(s.data(), s.size()); } +/// For historical reasons we store IPv6 as a String +inline void writeIPv6Binary(const IPv6 & ip, WriteBuffer & buf) +{ + writeVarUInt(IPV6_BINARY_LENGTH, buf); + buf.write(reinterpret_cast(&ip.toUnderType()), IPV6_BINARY_LENGTH); +} + inline void writeStringBinary(StringRef s, WriteBuffer & buf) { writeVarUInt(s.size, buf); From ad2b926248a4fd464f6f278045c9103a75d92ca7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 6 Jun 2023 07:17:30 +0000 Subject: [PATCH 110/515] Fix jepsen runs in PRs --- tests/ci/jepsen_check.py | 8 ++------ tests/jepsen.clickhouse/resources/keeper_config.xml | 8 ++++---- 2 files changed, 6 insertions(+), 10 deletions(-) diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index 9d35d2d6e35..c21fafa2605 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -25,6 +25,7 @@ from stopwatch import Stopwatch from tee_popen import TeePopen from upload_result_helper import upload_results from version_helper import get_version_from_repo +from build_check import get_release_or_pr JEPSEN_GROUP_NAME = "jepsen_group" @@ -210,12 +211,7 @@ if __name__ == "__main__": build_name = get_build_name_for_check(check_name) - if pr_info.number == 0: - version = get_version_from_repo() - release_or_pr = f"{version.major}.{version.minor}" - else: - # PR number for anything else - release_or_pr = str(pr_info.number) + release_or_pr, _ = get_release_or_pr(pr_info, get_version_from_repo()) # This check run separately from other checks because it requires exclusive # run (see .github/workflows/jepsen.yml) So we cannot add explicit diff --git a/tests/jepsen.clickhouse/resources/keeper_config.xml b/tests/jepsen.clickhouse/resources/keeper_config.xml index 52f2a0dbdc2..1972ef6b917 100644 --- a/tests/jepsen.clickhouse/resources/keeper_config.xml +++ b/tests/jepsen.clickhouse/resources/keeper_config.xml @@ -13,19 +13,19 @@ local - /var/lib/clickhouse/coordination/logs/ + /home/robot-clickhouse/db/coordination/logs/ local - /var/lib/clickhouse/coordination/latest_log/ + /home/robot-clickhouse/db/coordination/latest_log/ local - /var/lib/clickhouse/coordination/snapshots/ + /home/robot-clickhouse/db/coordination/snapshots/ local - /var/lib/clickhouse/coordination/latest_snapshot/ + /home/robot-clickhouse/db/coordination/latest_snapshot/ From bea4e8e81f6a782edc3dc2a672ae3011843de305 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 7 Jun 2023 06:45:14 +0000 Subject: [PATCH 111/515] Add new files --- programs/keeper/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index f775e8a5a22..90f4f870df6 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -131,6 +131,8 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/ObjectStorageIterator.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/StoredObject.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/registerDiskS3.cpp From fa877f456185b5de5b5f2e36e775acdb8dec7f31 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 8 Jun 2023 16:05:14 +0200 Subject: [PATCH 112/515] cosmetic changes --- src/Common/DateLUT.h | 2 +- src/Common/LocalDate.h | 13 +++++-------- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index 23698331afe..833b2291254 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -39,7 +39,7 @@ public: /// Timezone is passed in query_context, but on CH-Client we have no query context, /// and each time we modify client's global context - const auto global_context = DB::CurrentThread::get().getGlobalContext(); + const DB::ContextPtr global_context = DB::CurrentThread::get().getGlobalContext(); if (global_context) { context_timezone = extractTimezoneFromContext(global_context); diff --git a/src/Common/LocalDate.h b/src/Common/LocalDate.h index 4a383129ae4..2331a40fd12 100644 --- a/src/Common/LocalDate.h +++ b/src/Common/LocalDate.h @@ -24,9 +24,8 @@ private: unsigned char m_month; unsigned char m_day; - void init(time_t time) + void init(time_t time, const DateLUTImpl & date_lut) { - const auto & date_lut = DateLUT::instance(); const auto & values = date_lut.getValues(time); m_year = values.year; @@ -56,9 +55,9 @@ private: } public: - explicit LocalDate(time_t time) + explicit LocalDate(time_t time, const DateLUTImpl & time_zone = DateLUT::instance()) { - init(time); + init(time, time_zone); } LocalDate(DayNum day_num, const DateLUTImpl & time_zone = DateLUT::instance()) /// NOLINT @@ -99,15 +98,13 @@ public: LocalDate(const LocalDate &) noexcept = default; LocalDate & operator= (const LocalDate &) noexcept = default; - DayNum getDayNum() const + DayNum getDayNum(const DateLUTImpl & lut = DateLUT::instance()) const { - const auto & lut = DateLUT::instance(); return DayNum(lut.makeDayNum(m_year, m_month, m_day).toUnderType()); } - ExtendedDayNum getExtenedDayNum() const + ExtendedDayNum getExtenedDayNum(const DateLUTImpl & lut = DateLUT::instance()) const { - const auto & lut = DateLUT::instance(); return ExtendedDayNum (lut.makeDayNum(m_year, m_month, m_day).toUnderType()); } From 4d4e5c690e446db23f8a1ef7fc1e577df93e9373 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 8 Jun 2023 17:10:51 +0200 Subject: [PATCH 113/515] update docs spelling check failed --- docs/en/operations/settings/settings.md | 2 +- docs/en/sql-reference/functions/date-time-functions.md | 4 +--- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 77c9238e4c7..6c9c8349519 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4116,7 +4116,7 @@ SELECT *, timezone() FROM test_tz WHERE d = '2000-01-01 00:00:00' SETTINGS sessi This happens due to different parsing pipelines: - `toDateTime('2000-01-01 00:00:00')` creates a new DateTime in a usual way, and thus `session_timezone` setting from query context is applied. - - `2000-01-01 00:00:00` is parsed to a DateTime inheriting type of `d` column, including DateTime's time zone, and `session_timezone` has no impact on this value. + - `2000-01-01 00:00:00` is parsed to a DateTime inheriting type of `d` column, including its time zone, and `session_timezone` has no impact on this value. Possible values: diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 1a5b0dcabf9..89ac6d438ff 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -163,7 +163,7 @@ Type: [String](../../sql-reference/data-types/string.md). ## serverTimeZone Returns the default timezone of the server, i.e. the value of setting [timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). -If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. +If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise, it produces a constant value. **Syntax** @@ -171,8 +171,6 @@ If it is executed in the context of a distributed table, then it generates a nor serverTimeZone() ``` -Alias: `ServerTimezone`, `servertimezone`. - **Returned value** - Timezone. diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index ded7a4643a9..8301579b6a8 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -484,6 +484,7 @@ russian rw sasl schemas +servertimezone simdjson skippingerrors sparsehash From a1b1e12e5bb5d6a1937f9081eb43374afef60f9b Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 8 Jun 2023 18:38:51 +0200 Subject: [PATCH 114/515] upd spell --- docs/en/sql-reference/functions/date-time-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 89ac6d438ff..62bbb84053a 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -158,9 +158,9 @@ Type: [String](../../sql-reference/data-types/string.md). **See also** -- [serverTimeZone](#serverTimeZone) +- [serverTimezone](#serverTimeZone) -## serverTimeZone +## serverTimezone Returns the default timezone of the server, i.e. the value of setting [timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise, it produces a constant value. From f4bf42cc42a6a2d8966d711874286427956d51d7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 8 Jun 2023 17:29:51 +0000 Subject: [PATCH 115/515] more fixes --- .../AggregateFunctionGroupArray.cpp | 1 + .../AggregateFunctionGroupUniqArray.cpp | 11 ++ .../AggregateFunctionTopK.cpp | 19 ++ .../AggregateFunctionUniq.h | 31 +++- .../AggregateFunctionUniqCombined.cpp | 5 + .../AggregateFunctionUniqCombined.h | 16 +- ...es_aggregate_functions_states.reference.j2 | 172 ++++++++++++++++++ ...ip_types_aggregate_functions_states.sql.j2 | 123 +++++++++++++ 8 files changed, 366 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 create mode 100644 tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index 15f500b8bb6..bb1368b9ff8 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -25,6 +25,7 @@ IAggregateFunction * createWithNumericOrTimeType(const IDataType & argument_type WhichDataType which(argument_type); if (which.idx == TypeIndex::Date) return new AggregateFunctionTemplate(std::forward(args)...); if (which.idx == TypeIndex::DateTime) return new AggregateFunctionTemplate(std::forward(args)...); + if (which.idx == TypeIndex::IPv4) return new AggregateFunctionTemplate(std::forward(args)...); return createWithNumericType(argument_type, std::forward(args)...); } diff --git a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp index 16f2feb71bf..9e8060d44cc 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -39,12 +40,22 @@ public: static DataTypePtr createResultType() { return std::make_shared(std::make_shared()); } }; +template +class AggregateFunctionGroupUniqArrayIPv4 : public AggregateFunctionGroupUniqArray +{ +public: + explicit AggregateFunctionGroupUniqArrayIPv4(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits::max()) + : AggregateFunctionGroupUniqArray(argument_type, parameters_, createResultType(), max_elems_) {} + static DataTypePtr createResultType() { return std::make_shared(std::make_shared()); } +}; + template IAggregateFunction * createWithExtraTypes(const DataTypePtr & argument_type, TArgs && ... args) { WhichDataType which(argument_type); if (which.idx == TypeIndex::Date) return new AggregateFunctionGroupUniqArrayDate(argument_type, std::forward(args)...); else if (which.idx == TypeIndex::DateTime) return new AggregateFunctionGroupUniqArrayDateTime(argument_type, std::forward(args)...); + else if (which.idx == TypeIndex::IPv4) return new AggregateFunctionGroupUniqArrayIPv4(argument_type, std::forward(args)...); else { /// Check that we can use plain version of AggregateFunctionGroupUniqArrayGeneric diff --git a/src/AggregateFunctions/AggregateFunctionTopK.cpp b/src/AggregateFunctions/AggregateFunctionTopK.cpp index e568694df02..8f6652223cc 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.cpp +++ b/src/AggregateFunctions/AggregateFunctionTopK.cpp @@ -5,6 +5,7 @@ #include #include #include +#include static inline constexpr UInt64 TOP_K_MAX_SIZE = 0xFFFFFF; @@ -60,6 +61,22 @@ public: {} }; +template +class AggregateFunctionTopKIPv4 : public AggregateFunctionTopK +{ +public: + using AggregateFunctionTopK::AggregateFunctionTopK; + + AggregateFunctionTopKIPv4(UInt64 threshold_, UInt64 load_factor, const DataTypes & argument_types_, const Array & params) + : AggregateFunctionTopK( + threshold_, + load_factor, + argument_types_, + params, + std::make_shared(std::make_shared())) + {} +}; + template IAggregateFunction * createWithExtraTypes(const DataTypes & argument_types, UInt64 threshold, UInt64 load_factor, const Array & params) @@ -72,6 +89,8 @@ IAggregateFunction * createWithExtraTypes(const DataTypes & argument_types, UInt return new AggregateFunctionTopKDate(threshold, load_factor, argument_types, params); if (which.idx == TypeIndex::DateTime) return new AggregateFunctionTopKDateTime(threshold, load_factor, argument_types, params); + if (which.idx == TypeIndex::IPv4) + return new AggregateFunctionTopKIPv4(threshold, load_factor, argument_types, params); /// Check that we can use plain version of AggregateFunctionTopKGeneric if (argument_types[0]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion()) diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index 03d999b47e2..de68e9076a0 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -167,6 +167,25 @@ struct AggregateFunctionUniqExactData static String getName() { return "uniqExact"; } }; +/// For historical reasons IPv6 is treated as FixedString(16) +template +struct AggregateFunctionUniqExactData +{ + using Key = UInt128; + + /// When creating, the hash table must be small. + using SingleLevelSet = HashSet, HashTableAllocatorWithStackMemory>; + using TwoLevelSet = TwoLevelHashSet; + using Set = UniqExactSet; + + Set set; + + constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_variadic = false; + + static String getName() { return "uniqExact"; } +}; + template struct AggregateFunctionUniqExactDataForVariadic : AggregateFunctionUniqExactData { @@ -275,12 +294,7 @@ struct Adder else if constexpr (std::is_same_v>) { const auto & column = *columns[0]; - if constexpr (!std::is_same_v) - { - data.set.template insert( - assert_cast &>(column).getData()[row_num]); - } - else + if constexpr (std::is_same_v || std::is_same_v) { StringRef value = column.getDataAt(row_num); @@ -291,6 +305,11 @@ struct Adder data.set.template insert(key); } + else + { + data.set.template insert( + assert_cast &>(column).getData()[row_num]); + } } #if USE_DATASKETCHES else if constexpr (std::is_same_v) diff --git a/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp b/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp index 1c59da59e83..8c2cb6ea0de 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -60,6 +61,10 @@ namespace return std::make_shared::template AggregateFunction>(argument_types, params); else if (which.isUUID()) return std::make_shared::template AggregateFunction>(argument_types, params); + else if (which.isIPv4()) + return std::make_shared::template AggregateFunction>(argument_types, params); + else if (which.isIPv6()) + return std::make_shared::template AggregateFunction>(argument_types, params); else if (which.isTuple()) { if (use_exact_hash_function) diff --git a/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/src/AggregateFunctions/AggregateFunctionUniqCombined.h index d879e3b3dde..5e8fa69f9de 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqCombined.h +++ b/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -119,6 +119,10 @@ struct AggregateFunctionUniqCombinedData : public Aggr { }; +template +struct AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey +{ +}; template class AggregateFunctionUniqCombined final @@ -141,16 +145,16 @@ public: void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { - if constexpr (!std::is_same_v) - { - const auto & value = assert_cast &>(*columns[0]).getElement(row_num); - this->data(place).set.insert(detail::AggregateFunctionUniqCombinedTraits::hash(value)); - } - else + if constexpr (std::is_same_v || std::is_same_v) { StringRef value = columns[0]->getDataAt(row_num); this->data(place).set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size)); } + else + { + const auto & value = assert_cast &>(*columns[0]).getElement(row_num); + this->data(place).set.insert(detail::AggregateFunctionUniqCombinedTraits::hash(value)); + } } void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 new file mode 100644 index 00000000000..481dd723b66 --- /dev/null +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 @@ -0,0 +1,172 @@ +----- hash / State / ip4 ----- +Row 1: +────── +minState: 12535288824949910799 +maxState: 18210943739258811465 +first_valueState: 12535288824949910799 +last_valueState: 18210943739258811465 +topKState: 1594227852744382511 +groupArrayState: 8025417272361615478 +groupUniqArrayState: 919082878249747568 +uniqState: 14828781561416784358 +uniqExactState: 11391659146320471795 +uniqCombinedState: 9631896280254268221 +uniqCombined64State: 5156097536649078816 +uniqHLL12State: 9696624347265201099 +uniqThetaState: 10464560810701154023 +----- hash / State / ip6 ----- +Row 1: +────── +minState: 9428555662807296659 +maxState: 18253481702148601156 +first_valueState: 9428555662807296659 +last_valueState: 18253481702148601156 +topKState: 8045294331733869941 +groupArrayState: 10451014709837753966 +groupUniqArrayState: 1954028114836070615 +uniqState: 14986562136250471284 +uniqExactState: 10032843621916709112 +uniqCombinedState: 6379274083567016598 +uniqCombined64State: 6379274083567016598 +uniqHLL12State: 9181286681186915812 +uniqThetaState: 2415188383468008881 +----- finalizeAggregation / State / ip4 ----- +Row 1: +────── +min: 59.154.201.255 +max: 59.154.202.48 +first_value: 59.154.201.255 +last_value: 59.154.202.48 +topK: ['59.154.202.48','59.154.202.5','59.154.202.26','59.154.202.25','59.154.202.24','59.154.202.23','59.154.202.22','59.154.202.21','59.154.202.27','59.154.202.19'] +groupArray: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8','59.154.202.9','59.154.202.10','59.154.202.11','59.154.202.12','59.154.202.13','59.154.202.14','59.154.202.15','59.154.202.16','59.154.202.17','59.154.202.18','59.154.202.19','59.154.202.20','59.154.202.21','59.154.202.22','59.154.202.23','59.154.202.24','59.154.202.25','59.154.202.26','59.154.202.27','59.154.202.28','59.154.202.29','59.154.202.30','59.154.202.31','59.154.202.32','59.154.202.33','59.154.202.34','59.154.202.35','59.154.202.36','59.154.202.37','59.154.202.38','59.154.202.39','59.154.202.40','59.154.202.41','59.154.202.42','59.154.202.43','59.154.202.44','59.154.202.45','59.154.202.46','59.154.202.47','59.154.202.48'] +groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.48','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] +uniq: 50 +uniqExact: 50 +uniqCombined: 50 +uniqCombined64: 50 +uniqHLL12: 49 +uniqTheta: 50 +----- finalizeAggregation / State / ip6 ----- +Row 1: +────── +min: 8c:333c::8c:333c:0:0 +max: ff8b:333c::ff8b:333c:0:0 +first_value: 8c:333c::8c:333c:0:0 +last_value: ff8b:333c::ff8b:333c:0:0 +topK: ['ff8b:333c::ff8b:333c:0:0','68c:333c::68c:333c:0:0','e98b:333c::e98b:333c:0:0','e88b:333c::e88b:333c:0:0','e78b:333c::e78b:333c:0:0','e68b:333c::e68b:333c:0:0','e58b:333c::e58b:333c:0:0','e48b:333c::e48b:333c:0:0','ea8b:333c::ea8b:333c:0:0','e28b:333c::e28b:333c:0:0'] +groupArray: ['8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','e8c:333c::e8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e8c:333c::e8c:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +uniq: 50 +uniqExact: 50 +uniqCombined: 50 +uniqCombined64: 50 +uniqHLL12: 50 +uniqTheta: 50 +----- hash / IfState / ip4 ----- +Row 1: +────── +minIfState: 12535288824949910799 +maxIfState: 18210943739258811465 +first_valueIfState: 12535288824949910799 +last_valueIfState: 18210943739258811465 +topKIfState: 1594227852744382511 +groupArrayIfState: 8025417272361615478 +groupUniqArrayIfState: 919082878249747568 +uniqIfState: 14828781561416784358 +uniqExactIfState: 11391659146320471795 +uniqCombinedIfState: 9631896280254268221 +uniqCombined64IfState: 5156097536649078816 +uniqHLL12IfState: 9696624347265201099 +uniqThetaIfState: 10464560810701154023 +----- hash / IfState / ip6 ----- +Row 1: +────── +minIfState: 9428555662807296659 +maxIfState: 18253481702148601156 +first_valueIfState: 9428555662807296659 +last_valueIfState: 18253481702148601156 +topKIfState: 8045294331733869941 +groupArrayIfState: 10451014709837753966 +groupUniqArrayIfState: 1954028114836070615 +uniqIfState: 14986562136250471284 +uniqExactIfState: 10032843621916709112 +uniqCombinedIfState: 6379274083567016598 +uniqCombined64IfState: 6379274083567016598 +uniqHLL12IfState: 9181286681186915812 +uniqThetaIfState: 2415188383468008881 +----- finalizeAggregation / IfState / ip4 ----- +Row 1: +────── +min: 59.154.201.255 +max: 59.154.202.48 +first_value: 59.154.201.255 +last_value: 59.154.202.48 +topK: ['59.154.202.48','59.154.202.5','59.154.202.26','59.154.202.25','59.154.202.24','59.154.202.23','59.154.202.22','59.154.202.21','59.154.202.27','59.154.202.19'] +groupArray: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8','59.154.202.9','59.154.202.10','59.154.202.11','59.154.202.12','59.154.202.13','59.154.202.14','59.154.202.15','59.154.202.16','59.154.202.17','59.154.202.18','59.154.202.19','59.154.202.20','59.154.202.21','59.154.202.22','59.154.202.23','59.154.202.24','59.154.202.25','59.154.202.26','59.154.202.27','59.154.202.28','59.154.202.29','59.154.202.30','59.154.202.31','59.154.202.32','59.154.202.33','59.154.202.34','59.154.202.35','59.154.202.36','59.154.202.37','59.154.202.38','59.154.202.39','59.154.202.40','59.154.202.41','59.154.202.42','59.154.202.43','59.154.202.44','59.154.202.45','59.154.202.46','59.154.202.47','59.154.202.48'] +groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.48','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] +uniq: 50 +uniqExact: 50 +uniqCombined: 50 +uniqCombined64: 50 +uniqHLL12: 49 +uniqTheta: 50 +----- finalizeAggregation / IfState / ip6 ----- +Row 1: +────── +min: 8c:333c::8c:333c:0:0 +max: ff8b:333c::ff8b:333c:0:0 +first_value: 8c:333c::8c:333c:0:0 +last_value: ff8b:333c::ff8b:333c:0:0 +topK: ['ff8b:333c::ff8b:333c:0:0','68c:333c::68c:333c:0:0','e98b:333c::e98b:333c:0:0','e88b:333c::e88b:333c:0:0','e78b:333c::e78b:333c:0:0','e68b:333c::e68b:333c:0:0','e58b:333c::e58b:333c:0:0','e48b:333c::e48b:333c:0:0','ea8b:333c::ea8b:333c:0:0','e28b:333c::e28b:333c:0:0'] +groupArray: ['8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','e8c:333c::e8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e8c:333c::e8c:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +uniq: 50 +uniqExact: 50 +uniqCombined: 50 +uniqCombined64: 50 +uniqHLL12: 50 +uniqTheta: 50 +----- Arg / hash / State / ip4 ----- +Row 1: +────── +argMinState: 13774589216353164344 +argMaxState: 9177365218111013695 +----- Arg / hash / State / ip6 ----- +Row 1: +────── +argMinState: 7320668278649617037 +argMaxState: 16598449636475438091 +----- Arg / finalizeAggregation / State / ip4 ----- +Row 1: +────── +argMinState: 59.154.201.255 +argMaxState: 59.154.202.48 +----- Arg / finalizeAggregation / State / ip6 ----- +Row 1: +────── +argMinState: 8c:333c::8c:333c:0:0 +argMaxState: ff8b:333c::ff8b:333c:0:0 +----- hash / State / ip4 ----- +Row 1: +────── +anyState: 12535288824949910799 +anyHeavyState: 9327034461443333306 +anyLastState: 12535288824949910799 +----- hash / State / ip6 ----- +Row 1: +────── +anyState: 1383994153676807399 +anyHeavyState: 15555709096566410627 +anyLastState: 1383994153676807399 +----- finalizeAggregation / State / ip4 ----- +Row 1: +────── +any: 59.154.201.255 +anyHeavy: 59.154.201.255 +anyLast: 59.154.201.255 +----- finalizeAggregation / State / ip6 ----- +Row 1: +────── +any: dd8b:333c::dd8b:333c:0:0 +anyHeavy: dd8b:333c::dd8b:333c:0:0 +anyLast: dd8b:333c::dd8b:333c:0:0 diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 new file mode 100644 index 00000000000..133d5287fdb --- /dev/null +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 @@ -0,0 +1,123 @@ +{# this test checks backward compatibility of aggregate functions States against IPv4, IPv6 types #} + +{% set ip4_generator = "select number::UInt32::IPv4 ip from numbers(999999999,50) order by ip" %} +{% set ip6_generator = "SELECT toIPv6(IPv6NumToString(toFixedString(reinterpretAsFixedString(number)||reinterpretAsFixedString(number), 16))) AS ip FROM numbers(1010011101, 50) order by ip" %} +{% set ip_generators = {'ip4': ip4_generator, 'ip6': ip6_generator} %} + + +{% set agg_func_list = [ "min", "max", "first_value", "last_value", "topK", "groupArray", "groupUniqArray", "uniq", "uniqExact", "uniqCombined", "uniqCombined64", "uniqHLL12", "uniqTheta" ] %} + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- hash / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}State(ip) )) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- finalizeAggregation / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}State(ip) ) AS {{ func }}{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- hash / IfState / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}IfState(ip, 1) )) AS {{ func }}IfState{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- finalizeAggregation / IfState / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}IfState(ip, 1) ) AS {{ func }}{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + + +{% set agg_func_list = [ "argMin", "argMax" ] %} + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Arg / hash / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}State(ip, ip) )) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Arg / finalizeAggregation / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}State(ip, ip) ) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + +{# let's test functions with not deterministic result against 1 row, to make it deterministic #} +{% set ip4_generator = "select number::UInt32::IPv4 ip from numbers(999999999,1) order by ip" %} +{% set ip6_generator = "SELECT toIPv6(IPv6NumToString(toFixedString(reinterpretAsFixedString(number)||reinterpretAsFixedString(number), 16))) AS ip FROM numbers(1010011101, 1) order by ip" %} + +{% set ip_generators = {'ip4': ip4_generator, 'ip6': ip6_generator} %} + +{% set agg_func_list = [ "any", "anyHeavy", "anyLast" ] %} + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- hash / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}State(ip) )) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- finalizeAggregation / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}State(ip) ) AS {{ func }}{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + From c96989ca14415707e5dc37958ca36093e8292f46 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 8 Jun 2023 19:37:52 +0000 Subject: [PATCH 116/515] no-fasttest because uniqTheta --- .../02751_ip_types_aggregate_functions_states.sql.j2 | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 index 133d5287fdb..708eeab7724 100644 --- a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest because uniqTheta + {# this test checks backward compatibility of aggregate functions States against IPv4, IPv6 types #} {% set ip4_generator = "select number::UInt32::IPv4 ip from numbers(999999999,50) order by ip" %} From 07582d56f32d72a3f13d9e7303310ff1753c97f5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Jun 2023 13:18:13 +0000 Subject: [PATCH 117/515] Add config for pinning api version --- src/Coordination/FourLetterCommand.cpp | 2 +- src/Coordination/KeeperConstants.h | 4 +- src/Coordination/KeeperContext.cpp | 37 ++++++++ src/Coordination/KeeperContext.h | 6 ++ src/Coordination/KeeperServer.cpp | 2 + src/Coordination/KeeperSnapshotManager.cpp | 2 +- src/Coordination/KeeperStorage.cpp | 2 +- src/Coordination/tests/gtest_coordination.cpp | 2 +- .../__init__.py | 0 .../configs/enable_keeper.xml | 31 +++++++ .../test_keeper_api_version_config/test.py | 87 +++++++++++++++++++ 11 files changed, 169 insertions(+), 6 deletions(-) create mode 100644 src/Coordination/KeeperContext.cpp create mode 100644 tests/integration/test_keeper_api_version_config/__init__.py create mode 100644 tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml create mode 100644 tests/integration/test_keeper_api_version_config/test.py diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 7077e792fd8..a64969e3d31 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -486,7 +486,7 @@ String RecoveryCommand::run() String ApiVersionCommand::run() { - return toString(static_cast(Coordination::current_keeper_api_version)); + return toString(static_cast(Coordination::latest_keeper_api_version)); } String CreateSnapshotCommand::run() diff --git a/src/Coordination/KeeperConstants.h b/src/Coordination/KeeperConstants.h index 4b5a5b54be0..42161eee908 100644 --- a/src/Coordination/KeeperConstants.h +++ b/src/Coordination/KeeperConstants.h @@ -13,7 +13,7 @@ enum class KeeperApiVersion : uint8_t WITH_CHECK_NOT_EXISTS, }; -inline constexpr auto current_keeper_api_version = KeeperApiVersion::WITH_CHECK_NOT_EXISTS; +inline constexpr auto latest_keeper_api_version = KeeperApiVersion::WITH_CHECK_NOT_EXISTS; const std::string keeper_system_path = "/keeper"; const std::string keeper_api_version_path = keeper_system_path + "/api_version"; @@ -21,7 +21,7 @@ const std::string keeper_api_version_path = keeper_system_path + "/api_version"; using PathWithData = std::pair; const std::vector child_system_paths_with_data { - {keeper_api_version_path, toString(static_cast(current_keeper_api_version))} + {keeper_api_version_path, toString(static_cast(latest_keeper_api_version))} }; } diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp new file mode 100644 index 00000000000..d420242e670 --- /dev/null +++ b/src/Coordination/KeeperContext.cpp @@ -0,0 +1,37 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +DECLARE_SETTING_ENUM(KeeperApiVersion); +IMPLEMENT_SETTING_ENUM(KeeperApiVersion, ErrorCodes::BAD_ARGUMENTS, + {{"ZOOKEEPER_COMPATIBLE", KeeperApiVersion::ZOOKEEPER_COMPATIBLE}, + {"WITH_FILTERED_LIST", KeeperApiVersion::WITH_FILTERED_LIST}, + {"WITH_MULTI_READ", KeeperApiVersion::WITH_MULTI_READ}, + {"WITH_CHECK_NOT_EXISTS", KeeperApiVersion::WITH_CHECK_NOT_EXISTS}}); + +void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) +{ + for (const auto & [path, data] : child_system_paths_with_data) + system_nodes_with_data[std::string{path}] = data; + + if (config.has("keeper_server.api_version")) + { + auto version_string = config.getString("keeper_server.api_version"); + auto api_version = SettingFieldKeeperApiVersionTraits::fromString(version_string); + LOG_INFO(&Poco::Logger::get("KeeperContext"), "API version override used: {}", version_string); + system_nodes_with_data[keeper_api_version_path] = toString(static_cast(api_version)); + } +} + +} diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 64fa8cea6ec..f26009c9af0 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -1,10 +1,14 @@ #pragma once +#include + namespace DB { struct KeeperContext { + void initialize(const Poco::Util::AbstractConfiguration & config); + enum class Phase : uint8_t { INIT, @@ -16,6 +20,8 @@ struct KeeperContext bool ignore_system_path_on_startup{false}; bool digest_enabled{true}; + + std::unordered_map system_nodes_with_data; }; using KeeperContextPtr = std::shared_ptr; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 45db9e85fa5..897d7e05671 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -119,6 +119,8 @@ KeeperServer::KeeperServer( if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); + keeper_context->initialize(config); + keeper_context->digest_enabled = config.getBool("keeper_server.digest_enabled", false); keeper_context->ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 8b80db3e520..a2d9d8136cd 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -185,7 +185,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr } /// Serialize data tree - writeBinary(snapshot.snapshot_container_size - child_system_paths_with_data.size(), out); + writeBinary(snapshot.snapshot_container_size - keeper_context->system_nodes_with_data.size(), out); size_t counter = 0; for (auto it = snapshot.begin; counter < snapshot.snapshot_container_size; ++counter) { diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 66d6b0f5843..8abcd062b7f 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -283,7 +283,7 @@ void KeeperStorage::initializeSystemNodes() } // insert child system nodes - for (const auto & [path, data] : child_system_paths_with_data) + for (const auto & [path, data] : keeper_context->system_nodes_with_data) { assert(keeper_api_version_path.starts_with(keeper_system_path)); Node child_system_node; diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 453fd0f2e60..005c67ad261 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -2357,7 +2357,7 @@ TEST_P(CoordinationTest, TestCurrentApiVersion) uint8_t keeper_version{0}; DB::ReadBufferFromOwnString buf(get_response.data); DB::readIntText(keeper_version, buf); - EXPECT_EQ(keeper_version, static_cast(current_keeper_api_version)); + EXPECT_EQ(keeper_version, static_cast(latest_keeper_api_version)); } TEST_P(CoordinationTest, TestSystemNodeModify) diff --git a/tests/integration/test_keeper_api_version_config/__init__.py b/tests/integration/test_keeper_api_version_config/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml b/tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml new file mode 100644 index 00000000000..c153d025598 --- /dev/null +++ b/tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml @@ -0,0 +1,31 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 10 + 5 + 5000 + 10000 + trace + + + 0 + 0 + 0 + + + + + + + 1 + localhost + 9234 + + + + diff --git a/tests/integration/test_keeper_api_version_config/test.py b/tests/integration/test_keeper_api_version_config/test.py new file mode 100644 index 00000000000..34d3acc4f04 --- /dev/null +++ b/tests/integration/test_keeper_api_version_config/test.py @@ -0,0 +1,87 @@ +#!/usr/bin/env python3 + +import pytest +import os +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as keeper_utils +from kazoo.client import KazooClient, KazooState + +CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) +cluster = ClickHouseCluster(__file__) + +# clickhouse itself will use external zookeeper +node = cluster.add_instance( + "node", + main_configs=["configs/enable_keeper.xml"], + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def get_connection_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient( + hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout + ) + _fake_zk_instance.start() + return _fake_zk_instance + + +def restart_clickhouse(api_version=None, expect_fail=True): + node.stop_clickhouse() + node.copy_file_to_container( + os.path.join(CURRENT_TEST_DIR, "configs/enable_keeper.xml"), + "/etc/clickhouse-server/config.d/enable_keeper.xml", + ) + + if api_version: + node.replace_in_config( + "/etc/clickhouse-server/config.d/enable_keeper.xml", + "", + f"{api_version}<\\/api_version>", + ) + + node.start_clickhouse(retry_start=not expect_fail) + keeper_utils.wait_until_connected(cluster, node) + + +def test_keeper_api_version(started_cluster): + restart_clickhouse() + + def assert_version(string_version, version_number): + node.wait_for_log_line( + f"Detected server's API version: {string_version}", look_behind_lines=1000 + ) + + try: + node_zk = get_connection_zk(node.name) + assert node_zk.get("/keeper/api_version")[0] == str(version_number).encode() + finally: + if node_zk: + node_zk.stop() + node_zk.close() + + assert_version("WITH_CHECK_NOT_EXISTS", 3) + + for i, version in enumerate( + [ + "ZOOKEEPER_COMPATIBLE", + "WITH_FILTERED_LIST", + "WITH_MULTI_READ", + "WITH_CHECK_NOT_EXISTS", + ] + ): + restart_clickhouse(version) + assert_version(version, i) + + with pytest.raises(Exception): + restart_clickhouse("INVALID_VERSION", expect_fail=True) From b8b70d78e15948dd278d298d42bc1e4ef2d4fc0d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Jun 2023 13:34:27 +0000 Subject: [PATCH 118/515] Build fix --- contrib/azure-cmake/CMakeLists.txt | 2 +- programs/keeper/CMakeLists.txt | 1 + src/Coordination/KeeperContext.cpp | 7 +++++-- src/Coordination/KeeperContext.h | 2 ++ 4 files changed, 9 insertions(+), 3 deletions(-) diff --git a/contrib/azure-cmake/CMakeLists.txt b/contrib/azure-cmake/CMakeLists.txt index 9c361db47ca..23e38e6b63d 100644 --- a/contrib/azure-cmake/CMakeLists.txt +++ b/contrib/azure-cmake/CMakeLists.txt @@ -1,6 +1,6 @@ option (ENABLE_AZURE_BLOB_STORAGE "Enable Azure blob storage" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_AZURE_BLOB_STORAGE OR BUILD_STANDALONE_KEEPER OR OS_FREEBSD OR ARCH_PPC64LE) +if (NOT ENABLE_AZURE_BLOB_STORAGE OR OS_FREEBSD OR ARCH_PPC64LE) message(STATUS "Not using Azure blob storage") return() endif() diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 1f1138f49eb..4f74cc06801 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -43,6 +43,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperDispatcher.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperLogStore.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperServer.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperContext.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManager.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManagerS3.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateMachine.cpp diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index d420242e670..a750f2e1860 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -20,11 +20,14 @@ IMPLEMENT_SETTING_ENUM(KeeperApiVersion, ErrorCodes::BAD_ARGUMENTS, {"WITH_MULTI_READ", KeeperApiVersion::WITH_MULTI_READ}, {"WITH_CHECK_NOT_EXISTS", KeeperApiVersion::WITH_CHECK_NOT_EXISTS}}); -void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) +KeeperContext::KeeperContext() { for (const auto & [path, data] : child_system_paths_with_data) system_nodes_with_data[std::string{path}] = data; - +} + +void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) +{ if (config.has("keeper_server.api_version")) { auto version_string = config.getString("keeper_server.api_version"); diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index f26009c9af0..de502b6c566 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -7,6 +7,8 @@ namespace DB struct KeeperContext { + KeeperContext(); + void initialize(const Poco::Util::AbstractConfiguration & config); enum class Phase : uint8_t From 9a4043a4b4c97bcfb7eb345e0753b27228c2f4f7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Jun 2023 17:51:59 +0000 Subject: [PATCH 119/515] Fixing more tests. --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 1 - src/Interpreters/PreparedSets.h | 6 ++ src/Planner/CollectSets.cpp | 25 +++--- src/Planner/CollectSets.h | 2 +- src/Planner/Planner.cpp | 103 +++++++++++----------- src/Planner/Utils.cpp | 4 +- src/Storages/StorageDistributed.cpp | 11 ++- 7 files changed, 82 insertions(+), 70 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index b39aff86d32..bab64480901 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2333,7 +2333,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con auto storage_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); auto storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context); - auto result = std::make_shared(std::move(storage), std::move(storage_lock), std::move(storage_snapshot)); if (is_temporary_table) result->setTemporaryTableName(table_name); diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index fa7f7069994..a119c24ad10 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -29,6 +29,9 @@ class Set; using SetPtr = std::shared_ptr; class InterpreterSelectWithUnionQuery; +class IQueryTreeNode; +using QueryTreeNodePtr = std::shared_ptr; + /// Represents a set in a query that might be referenced at analysis time and built later during execution. /// Also it can represent a constant set that is ready to use. /// At analysis stage the FutureSets are created but not necessarily filled. Then for non-constant sets there @@ -131,6 +134,7 @@ public: /// The source is obtained using the InterpreterSelectQuery subquery. std::unique_ptr source; + QueryTreeNodePtr query_tree; }; class FutureSetFromSubquery : public FutureSet, public std::enable_shared_from_this @@ -153,6 +157,8 @@ public: // void addStorage(StoragePtr storage) { subquery.table = std::move(storage); } + SubqueryForSet & getSubquery() { return subquery; } + private: //SetPtr set; SubqueryForSet subquery; diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 5f44994c14b..b3e3f5f472a 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -28,9 +28,9 @@ namespace class CollectSetsVisitor : public ConstInDepthQueryTreeVisitor { public: - explicit CollectSetsVisitor(PlannerContext & planner_context_, const SelectQueryOptions & select_query_options_) + explicit CollectSetsVisitor(PlannerContext & planner_context_) //, const SelectQueryOptions & select_query_options_) : planner_context(planner_context_) - , select_query_options(select_query_options_) + //, select_query_options(select_query_options_) {} void visitImpl(const QueryTreeNodePtr & node) @@ -95,12 +95,12 @@ public: if (sets.getFuture(set_key)) return; - auto subquery_options = select_query_options.subquery(); - Planner subquery_planner( - in_second_argument, - subquery_options, - planner_context.getGlobalPlannerContext()); - subquery_planner.buildQueryPlanIfNeeded(); + // auto subquery_options = select_query_options.subquery(); + // Planner subquery_planner( + // in_second_argument, + // subquery_options, + // planner_context.getGlobalPlannerContext()); + // subquery_planner.buildQueryPlanIfNeeded(); // const auto & settings = planner_context.getQueryContext()->getSettingsRef(); // SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; @@ -109,7 +109,8 @@ public: SubqueryForSet subquery_for_set; subquery_for_set.key = planner_context.createSetKey(in_second_argument); - subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); + subquery_for_set.query_tree = in_second_argument; + //subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); /// TODO sets.addFromSubquery(set_key, std::move(subquery_for_set), settings, nullptr); @@ -132,14 +133,14 @@ public: private: PlannerContext & planner_context; - const SelectQueryOptions & select_query_options; + //const SelectQueryOptions & select_query_options; }; } -void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context, const SelectQueryOptions & select_query_options) +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context) //, const SelectQueryOptions & select_query_options) { - CollectSetsVisitor visitor(planner_context, select_query_options); + CollectSetsVisitor visitor(planner_context); //, select_query_options); visitor.visit(node); } diff --git a/src/Planner/CollectSets.h b/src/Planner/CollectSets.h index 57e662a392e..e0db802d3b4 100644 --- a/src/Planner/CollectSets.h +++ b/src/Planner/CollectSets.h @@ -12,6 +12,6 @@ struct SelectQueryOptions; /** Collect prepared sets and sets for subqueries that are necessary to execute IN function and its variations. * Collected sets are registered in planner context. */ -void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context, const SelectQueryOptions & select_query_options); +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context); //, const SelectQueryOptions & select_query_options); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 3184e229c15..30510d05840 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -893,50 +894,62 @@ void addOffsetStep(QueryPlan & query_plan, const QueryAnalysisResult & query_ana query_plan.addStep(std::move(offsets_step)); } -// void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, -// const SelectQueryOptions & select_query_options, -// const PlannerContextPtr & planner_context, -// const std::vector & result_actions_to_execute) -// { -// PreparedSets::SubqueriesForSets subqueries_for_sets; +void addBuildSubqueriesForSetsStepIfNeeded( + QueryPlan & query_plan, + const SelectQueryOptions & select_query_options, + const PlannerContextPtr & planner_context, + const std::vector & result_actions_to_execute) +{ + auto subqueries = planner_context->getPreparedSets().detachSubqueries(); + std::unordered_set useful_sets; -// for (const auto & actions_to_execute : result_actions_to_execute) -// { -// for (const auto & node : actions_to_execute->getNodes()) -// { -// const auto & set_key = node.result_name; -// auto * planner_set = planner_context->getSetOrNull(set_key); -// if (!planner_set) -// continue; + PreparedSets::SubqueriesForSets subqueries_for_sets; -// if (planner_set->getSet().isCreated() || !planner_set->getSubqueryNode()) -// continue; + for (const auto & actions_to_execute : result_actions_to_execute) + { + for (const auto & node : actions_to_execute->getNodes()) + { + if (node.column) + { + const IColumn * column = node.column.get(); + if (const auto * column_const = typeid_cast(column)) + column = &column_const->getDataColumn(); -// auto subquery_options = select_query_options.subquery(); -// Planner subquery_planner( -// planner_set->getSubqueryNode(), -// subquery_options, -// planner_context->getGlobalPlannerContext()); -// subquery_planner.buildQueryPlanIfNeeded(); + if (const auto * column_set = typeid_cast(column)) + useful_sets.insert(column_set->getData().get()); + } + } + } -// const auto & settings = planner_context->getQueryContext()->getSettingsRef(); -// SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; -// bool tranform_null_in = settings.transform_null_in; -// auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); + auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.set.get()); }; + auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate)); + subqueries.erase(it, subqueries.end()); -// SubqueryForSet subquery_for_set; -// subquery_for_set.key = set_key; -// subquery_for_set.set_in_progress = set; -// subquery_for_set.set = planner_set->getSet(); -// subquery_for_set.promise_to_fill_set = planner_set->extractPromiseToBuildSet(); -// subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); + for (auto & subquery : subqueries) + { + auto & subquery_for_set = subquery.set->getSubquery(); + auto subquery_options = select_query_options.subquery(); + Planner subquery_planner( + subquery_for_set.query_tree, + subquery_options, + planner_context->getGlobalPlannerContext()); + subquery_planner.buildQueryPlanIfNeeded(); -// subqueries_for_sets.emplace(set_key, std::move(subquery_for_set)); -// } -// } + subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); + } -// addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); -// } + //addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); + + if (!subqueries.empty()) + { + auto step = std::make_unique( + query_plan.getCurrentDataStream(), + std::move(subqueries), + planner_context->getQueryContext()); + + query_plan.addStep(std::move(step)); + } +} /// Support for `additional_result_filter` setting void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan, @@ -1197,7 +1210,7 @@ void Planner::buildPlanForQueryNode() } checkStoragesSupportTransactions(planner_context); - collectSets(query_tree, *planner_context, select_query_options); + collectSets(query_tree, *planner_context); //, select_query_options); collectTableExpressionData(query_tree, planner_context); const auto & settings = query_context->getSettingsRef(); @@ -1497,20 +1510,8 @@ void Planner::buildPlanForQueryNode() if (!select_query_options.only_analyze) { - auto subqueries = planner_context->getPreparedSets().detachSubqueries(); - - if (!subqueries.empty()) - { - auto step = std::make_unique( - query_plan.getCurrentDataStream(), - std::move(subqueries), - planner_context->getQueryContext()); - - query_plan.addStep(std::move(step)); - } - //addCreatingSetsStep(query_plan, planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()), planner_context->getQueryContext()); - //addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); + addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); } } diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 2b4febf58ea..372bb15822a 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -426,7 +426,7 @@ SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, const QueryTreeNodePtr & table_expression, PlannerContextPtr & planner_context, - const SelectQueryOptions & select_query_options, + [[maybe_unused]] const SelectQueryOptions & select_query_options, NameSet table_expression_required_names_without_filter) { const auto & query_context = planner_context->getQueryContext(); @@ -444,7 +444,7 @@ FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, } collectSourceColumns(filter_query_tree, planner_context); - collectSets(filter_query_tree, *planner_context, select_query_options); + collectSets(filter_query_tree, *planner_context); //, select_query_options); auto filter_actions_dag = std::make_shared(); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b0fb07d69ce..64a621e5710 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include @@ -898,9 +899,13 @@ QueryTreeNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, temporary_table_expression_node->setTemporaryTableName(temporary_table_name); auto table_out = external_storage->write({}, external_storage->getInMemoryMetadataPtr(), mutable_context); - auto io = interpreter.execute(); - io.pipeline.complete(std::move(table_out)); - CompletedPipelineExecutor executor(io.pipeline); + + auto optimization_settings = QueryPlanOptimizationSettings::fromContext(mutable_context); + auto build_pipeline_settings = BuildQueryPipelineSettings::fromContext(mutable_context); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*query_plan.buildQueryPipeline(optimization_settings, build_pipeline_settings))); + + pipeline.complete(std::move(table_out)); + CompletedPipelineExecutor executor(pipeline); executor.execute(); mutable_context->addExternalTable(temporary_table_name, std::move(external_storage_holder)); From e094bf3247b82c94098567a4f8f172e57fcdc017 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Jun 2023 18:30:47 +0000 Subject: [PATCH 120/515] Resolving conflicts. --- src/Planner/CollectSets.cpp | 33 ++++++++++- .../QueryPlan/ReadFromMergeTree.cpp | 58 ++++++++++++++----- 2 files changed, 76 insertions(+), 15 deletions(-) diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index be4cb0e2e2b..68ad1ab78d3 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -96,6 +97,36 @@ public: if (sets.getFuture(set_key)) return; + auto subquery_to_execute = in_second_argument; + + if (auto * table_node = in_second_argument->as()) + { + auto storage_snapshot = table_node->getStorageSnapshot(); + auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary)); + + size_t columns_to_select_size = columns_to_select.size(); + + auto column_nodes_to_select = std::make_shared(); + column_nodes_to_select->getNodes().reserve(columns_to_select_size); + + NamesAndTypes projection_columns; + projection_columns.reserve(columns_to_select_size); + + for (auto & column : columns_to_select) + { + column_nodes_to_select->getNodes().emplace_back(std::make_shared(column, subquery_to_execute)); + projection_columns.emplace_back(column.name, column.type); + } + + auto subquery_for_table = std::make_shared(Context::createCopy(planner_context.getQueryContext())); + subquery_for_table->setIsSubquery(true); + subquery_for_table->getProjectionNode() = std::move(column_nodes_to_select); + subquery_for_table->getJoinTree() = std::move(subquery_to_execute); + subquery_for_table->resolveProjectionColumns(std::move(projection_columns)); + + subquery_to_execute = std::move(subquery_for_table); + } + // auto subquery_options = select_query_options.subquery(); // Planner subquery_planner( // in_second_argument, @@ -110,7 +141,7 @@ public: SubqueryForSet subquery_for_set; subquery_for_set.key = planner_context.createSetKey(in_second_argument); - subquery_for_set.query_tree = in_second_argument; + subquery_for_set.query_tree = std::move(subquery_to_execute); //subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); /// TODO diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 8483df797ef..8f610eb4380 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -37,6 +37,8 @@ #include #include #include +#include +#include #include #include @@ -100,6 +102,7 @@ namespace ErrorCodes extern const int INDEX_NOT_USED; extern const int LOGICAL_ERROR; extern const int TOO_MANY_ROWS; + extern const int CANNOT_PARSE_TEXT; } static MergeTreeReaderSettings getMergeTreeReaderSettings( @@ -1245,29 +1248,56 @@ static void buildIndexes( info = &*info_copy; } + std::unordered_set ignored_index_names; + + if (settings.ignore_data_skipping_indices.changed) + { + const auto & indices = settings.ignore_data_skipping_indices.toString(); + Tokens tokens(indices.data(), indices.data() + indices.size(), settings.max_query_size); + IParser::Pos pos(tokens, static_cast(settings.max_parser_depth)); + Expected expected; + + /// Use an unordered list rather than string vector + auto parse_single_id_or_literal = [&] + { + String str; + if (!parseIdentifierOrStringLiteral(pos, expected, str)) + return false; + + ignored_index_names.insert(std::move(str)); + return true; + }; + + if (!ParserList::parseUtil(pos, expected, parse_single_id_or_literal, false)) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse ignore_data_skipping_indices ('{}')", indices); + } + UsefulSkipIndexes skip_indexes; using Key = std::pair; std::map merged; for (const auto & index : metadata_snapshot->getSecondaryIndices()) { - auto index_helper = MergeTreeIndexFactory::instance().get(index); - if (index_helper->isMergeable()) + if (!ignored_index_names.contains(index.name)) { - auto [it, inserted] = merged.emplace(Key{index_helper->index.type, index_helper->getGranularity()}, skip_indexes.merged_indices.size()); - if (inserted) + auto index_helper = MergeTreeIndexFactory::instance().get(index); + if (index_helper->isMergeable()) { - skip_indexes.merged_indices.emplace_back(); - skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(*info, metadata_snapshot); - } + auto [it, inserted] = merged.emplace(Key{index_helper->index.type, index_helper->getGranularity()}, skip_indexes.merged_indices.size()); + if (inserted) + { + skip_indexes.merged_indices.emplace_back(); + skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(*info, metadata_snapshot); + } - skip_indexes.merged_indices[it->second].addIndex(index_helper); - } - else - { - auto condition = index_helper->createIndexCondition(*info, context); - if (!condition->alwaysUnknownOrTrue()) - skip_indexes.useful_indices.emplace_back(index_helper, condition); + skip_indexes.merged_indices[it->second].addIndex(index_helper); + } + else + { + auto condition = index_helper->createIndexCondition(*info, context); + if (!condition->alwaysUnknownOrTrue()) + skip_indexes.useful_indices.emplace_back(index_helper, condition); + } } } From b634012feb40445079145639d23c44967c00547e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 12 Jun 2023 10:57:03 +0000 Subject: [PATCH 121/515] Add feature flags for Keeper API --- src/Common/ZooKeeper/IKeeper.h | 4 +- src/Common/ZooKeeper/TestKeeper.h | 5 +- src/Common/ZooKeeper/ZooKeeper.cpp | 4 +- src/Common/ZooKeeper/ZooKeeper.h | 7 +- src/Common/ZooKeeper/ZooKeeperArgs.cpp | 1 + src/Common/ZooKeeper/ZooKeeperImpl.cpp | 60 +++++++---- src/Common/ZooKeeper/ZooKeeperImpl.h | 7 +- .../ZooKeeper/ZooKeeperWithFaultInjection.h | 4 +- src/Coordination/CoordinationSettings.cpp | 2 +- src/Coordination/FourLetterCommand.cpp | 27 ++++- src/Coordination/FourLetterCommand.h | 12 +++ src/Coordination/KeeperConstants.h | 9 +- src/Coordination/KeeperContext.cpp | 46 ++++---- src/Coordination/KeeperContext.h | 3 + src/Coordination/KeeperDispatcher.cpp | 47 ++++---- src/Coordination/KeeperDispatcher.h | 8 ++ src/Coordination/KeeperFeatureFlags.cpp | 102 ++++++++++++++++++ src/Coordination/KeeperFeatureFlags.h | 49 +++++++++ src/Coordination/KeeperServer.cpp | 8 +- src/Coordination/KeeperServer.h | 1 + src/Coordination/KeeperStorage.cpp | 4 +- src/Coordination/tests/gtest_coordination.cpp | 15 +-- .../StorageSystemZooKeeperConnection.cpp | 4 +- .../test_keeper_api_version_config/test.py | 87 --------------- .../__init__.py | 0 .../configs/enable_keeper.xml | 2 +- .../test_keeper_feature_flags_config/test.py | 92 ++++++++++++++++ 27 files changed, 422 insertions(+), 188 deletions(-) create mode 100644 src/Coordination/KeeperFeatureFlags.cpp create mode 100644 src/Coordination/KeeperFeatureFlags.h delete mode 100644 tests/integration/test_keeper_api_version_config/test.py rename tests/integration/{test_keeper_api_version_config => test_keeper_feature_flags_config}/__init__.py (100%) rename tests/integration/{test_keeper_api_version_config => test_keeper_feature_flags_config}/configs/enable_keeper.xml (97%) create mode 100644 tests/integration/test_keeper_feature_flags_config/test.py diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 3eb5819df90..369aacf16c7 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include @@ -530,7 +530,7 @@ public: const Requests & requests, MultiCallback callback) = 0; - virtual DB::KeeperApiVersion getApiVersion() const = 0; + virtual bool isFeatureEnabled(DB::KeeperFeatureFlag feature_flag) const = 0; /// Expire session and finish all pending requests virtual void finalize(const String & reason) = 0; diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 4bffa4e1d4f..9bbd018cfb1 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace Coordination @@ -92,9 +93,9 @@ public: void finalize(const String & reason) override; - DB::KeeperApiVersion getApiVersion() const override + bool isFeatureEnabled(DB::KeeperFeatureFlag) const override { - return KeeperApiVersion::ZOOKEEPER_COMPATIBLE; + return false; } struct Node diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index a587ad6caf4..826032fc56b 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -865,9 +865,9 @@ bool ZooKeeper::expired() return impl->isExpired(); } -DB::KeeperApiVersion ZooKeeper::getApiVersion() const +bool ZooKeeper::isFeatureEnabled(DB::KeeperFeatureFlag feature_flag) const { - return impl->getApiVersion(); + return impl->isFeatureEnabled(feature_flag); } Int64 ZooKeeper::getClientID() diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 96f9914b597..a4a631b9d5a 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -15,6 +15,7 @@ #include #include #include +#include #include #include @@ -215,7 +216,7 @@ public: /// Returns true, if the session has expired. bool expired(); - DB::KeeperApiVersion getApiVersion() const; + bool isFeatureEnabled(DB::KeeperFeatureFlag feature_flag) const; /// Create a znode. /// Throw an exception if something went wrong. @@ -553,7 +554,7 @@ private: template MultiReadResponses multiRead(TIter start, TIter end, RequestFactory request_factory, AsyncFunction async_fun) { - if (getApiVersion() >= DB::KeeperApiVersion::WITH_MULTI_READ) + if (isFeatureEnabled(DB::KeeperFeatureFlag::MULTI_READ)) { Coordination::Requests requests; for (auto it = start; it != end; ++it) @@ -685,7 +686,7 @@ String getZooKeeperConfigName(const Poco::Util::AbstractConfiguration & config); template void addCheckNotExistsRequest(Coordination::Requests & requests, const Client & client, const std::string & path) { - if (client.getApiVersion() >= DB::KeeperApiVersion::WITH_CHECK_NOT_EXISTS) + if (client.isFeatureEnabled(DB::KeeperFeatureFlag::CHECK_NOT_EXISTS)) { auto request = std::make_shared(); request->path = path; diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp index 0ebc7f667cb..e99285b0056 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.cpp +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include namespace DB diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 7f23ac00efe..bf0d1871244 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -354,7 +354,8 @@ ZooKeeper::ZooKeeper( send_thread = ThreadFromGlobalPool([this] { sendThread(); }); receive_thread = ThreadFromGlobalPool([this] { receiveThread(); }); - initApiVersion(); + initFeatureFlags(); + keeper_feature_flags.logFlags(log); ProfileEvents::increment(ProfileEvents::ZooKeeperInit); } @@ -1089,41 +1090,58 @@ void ZooKeeper::pushRequest(RequestInfo && info) ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); } -KeeperApiVersion ZooKeeper::getApiVersion() const +bool ZooKeeper::isFeatureEnabled(KeeperFeatureFlag feature_flag) const { - return keeper_api_version; + return keeper_feature_flags.isEnabled(feature_flag); } -void ZooKeeper::initApiVersion() +void ZooKeeper::initFeatureFlags() { - auto promise = std::make_shared>(); - auto future = promise->get_future(); - - auto callback = [promise](const Coordination::GetResponse & response) mutable + const auto try_get = [&](const std::string & path, const std::string & description) -> std::optional { - promise->set_value(response); + auto promise = std::make_shared>(); + auto future = promise->get_future(); + + auto callback = [promise](const Coordination::GetResponse & response) mutable + { + promise->set_value(response); + }; + + get(path, std::move(callback), {}); + if (future.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) + { + LOG_TRACE(log, "Failed to get {}: timeout", description); + return std::nullopt; + } + + auto response = future.get(); + + if (response.error != Coordination::Error::ZOK) + { + LOG_TRACE(log, "Failed to get {}", description); + return std::nullopt; + } + + return std::move(response.data); }; - get(keeper_api_version_path, std::move(callback), {}); - if (future.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) + if (auto feature_flags = try_get(keeper_api_feature_flags_path, "feature flags"); feature_flags.has_value()) { - LOG_TRACE(log, "Failed to get API version: timeout"); + keeper_feature_flags.setFeatureFlags(std::move(*feature_flags)); return; } - auto response = future.get(); + auto keeper_api_version_string = try_get(keeper_api_version_path, "API version"); - if (response.error != Coordination::Error::ZOK) - { - LOG_TRACE(log, "Failed to get API version"); + if (!keeper_api_version_string.has_value()) return; - } + DB::ReadBufferFromOwnString buf(*keeper_api_version_string); uint8_t keeper_version{0}; - DB::ReadBufferFromOwnString buf(response.data); DB::readIntText(keeper_version, buf); - keeper_api_version = static_cast(keeper_version); + auto keeper_api_version = static_cast(keeper_version); LOG_TRACE(log, "Detected server's API version: {}", keeper_api_version); + keeper_feature_flags.fromApiVersion(keeper_api_version); } @@ -1243,7 +1261,7 @@ void ZooKeeper::list( WatchCallback watch) { std::shared_ptr request{nullptr}; - if (keeper_api_version < Coordination::KeeperApiVersion::WITH_FILTERED_LIST) + if (!isFeatureEnabled(KeeperFeatureFlag::FILTERED_LIST)) { if (list_request_type != ListRequestType::ALL) throw Exception(Error::ZBADARGUMENTS, "Filtered list request type cannot be used because it's not supported by the server"); @@ -1308,7 +1326,7 @@ void ZooKeeper::multi( { ZooKeeperMultiRequest request(requests, default_acls); - if (request.getOpNum() == OpNum::MultiRead && keeper_api_version < Coordination::KeeperApiVersion::WITH_MULTI_READ) + if (request.getOpNum() == OpNum::MultiRead && !isFeatureEnabled(KeeperFeatureFlag::MULTI_READ)) throw Exception(Error::ZBADARGUMENTS, "MultiRead request type cannot be used because it's not supported by the server"); RequestInfo request_info; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 944c5032fac..ae6bef067e3 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -181,7 +182,7 @@ public: const Requests & requests, MultiCallback callback) override; - DB::KeeperApiVersion getApiVersion() const override; + bool isFeatureEnabled(KeeperFeatureFlag feature_flag) const override; /// Without forcefully invalidating (finalizing) ZooKeeper session before /// establishing a new one, there was a possibility that server is using @@ -312,12 +313,12 @@ private: void logOperationIfNeeded(const ZooKeeperRequestPtr & request, const ZooKeeperResponsePtr & response = nullptr, bool finalize = false, UInt64 elapsed_ms = 0); - void initApiVersion(); + void initFeatureFlags(); CurrentMetrics::Increment active_session_metric_increment{CurrentMetrics::ZooKeeperSession}; std::shared_ptr zk_log; - DB::KeeperApiVersion keeper_api_version{DB::KeeperApiVersion::ZOOKEEPER_COMPATIBLE}; + DB::KeeperFeatureFlags keeper_feature_flags; }; } diff --git a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h index bf99cb76798..9d02d674010 100644 --- a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h +++ b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h @@ -402,9 +402,9 @@ public: ephemeral_nodes.clear(); } - KeeperApiVersion getApiVersion() const + bool isFeatureEnabled(KeeperFeatureFlag feature_flag) const { - return keeper->getApiVersion(); + return keeper->isFeatureEnabled(feature_flag); } private: diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 7a66134f43f..edbdfd4f9fc 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -36,7 +36,7 @@ void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco } -const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld,rclc,clrs"; +const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld,rclc,clrs,ftfl"; KeeperConfigurationAndSettings::KeeperConfigurationAndSettings() : server_id(NOT_EXIST) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index a64969e3d31..55120e70d99 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -9,6 +9,7 @@ #include #include #include +#include "Coordination/KeeperFeatureFlags.h" #include #include #include @@ -153,6 +154,9 @@ void FourLetterCommandFactory::registerCommands(KeeperDispatcher & keeper_dispat FourLetterCommandPtr clean_resources_command = std::make_shared(keeper_dispatcher); factory.registerCommand(clean_resources_command); + FourLetterCommandPtr feature_flags_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(feature_flags_command); + factory.initializeAllowList(keeper_dispatcher); factory.setInitialize(true); } @@ -486,7 +490,7 @@ String RecoveryCommand::run() String ApiVersionCommand::run() { - return toString(static_cast(Coordination::latest_keeper_api_version)); + return toString(0); } String CreateSnapshotCommand::run() @@ -535,4 +539,25 @@ String CleanResourcesCommand::run() return "ok"; } +String FeatureFlagsCommand::run() +{ + const auto & feature_flags = keeper_dispatcher.getKeeperContext()->feature_flags; + + StringBuffer ret; + + auto append = [&ret] (String key, uint8_t value) -> void + { + writeText(key, ret); + writeText('\t', ret); + writeText(std::to_string(value), ret); + writeText('\n', ret); + }; + + for (const auto feature : all_keeper_feature_flags) + append(SettingFieldKeeperFeatureFlagTraits::toString(feature), feature_flags.isEnabled(feature)); + + return ret.str(); + +} + } diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index c1a91303c05..eb2cf9419ae 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -401,4 +401,16 @@ struct CleanResourcesCommand : public IFourLetterCommand ~CleanResourcesCommand() override = default; }; +struct FeatureFlagsCommand : public IFourLetterCommand +{ + explicit FeatureFlagsCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } + + String name() override { return "ftfl"; } + String run() override; + ~FeatureFlagsCommand() override = default; +}; + } diff --git a/src/Coordination/KeeperConstants.h b/src/Coordination/KeeperConstants.h index 42161eee908..6d9c03ca8a9 100644 --- a/src/Coordination/KeeperConstants.h +++ b/src/Coordination/KeeperConstants.h @@ -5,6 +5,7 @@ namespace DB { +/// left for backwards compatibility enum class KeeperApiVersion : uint8_t { ZOOKEEPER_COMPATIBLE = 0, @@ -13,15 +14,9 @@ enum class KeeperApiVersion : uint8_t WITH_CHECK_NOT_EXISTS, }; -inline constexpr auto latest_keeper_api_version = KeeperApiVersion::WITH_CHECK_NOT_EXISTS; const std::string keeper_system_path = "/keeper"; const std::string keeper_api_version_path = keeper_system_path + "/api_version"; - -using PathWithData = std::pair; -const std::vector child_system_paths_with_data -{ - {keeper_api_version_path, toString(static_cast(latest_keeper_api_version))} -}; +const std::string keeper_api_feature_flags_path = keeper_system_path + "/feature_flags"; } diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index a750f2e1860..dc408afc19a 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -1,40 +1,42 @@ #include #include -#include -#include #include -#include namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -DECLARE_SETTING_ENUM(KeeperApiVersion); -IMPLEMENT_SETTING_ENUM(KeeperApiVersion, ErrorCodes::BAD_ARGUMENTS, - {{"ZOOKEEPER_COMPATIBLE", KeeperApiVersion::ZOOKEEPER_COMPATIBLE}, - {"WITH_FILTERED_LIST", KeeperApiVersion::WITH_FILTERED_LIST}, - {"WITH_MULTI_READ", KeeperApiVersion::WITH_MULTI_READ}, - {"WITH_CHECK_NOT_EXISTS", KeeperApiVersion::WITH_CHECK_NOT_EXISTS}}); - KeeperContext::KeeperContext() { - for (const auto & [path, data] : child_system_paths_with_data) - system_nodes_with_data[std::string{path}] = data; + /// enable by default some feature flags + feature_flags.enableFeatureFlag(KeeperFeatureFlag::FILTERED_LIST); + feature_flags.enableFeatureFlag(KeeperFeatureFlag::MULTI_READ); + system_nodes_with_data[keeper_api_feature_flags_path] = feature_flags.getFeatureFlags(); } void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) { - if (config.has("keeper_server.api_version")) + digest_enabled = config.getBool("keeper_server.digest_enabled", false); + ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); + + static const std::string feature_flags_key = "keeper_server.feature_flags"; + if (config.has(feature_flags_key)) { - auto version_string = config.getString("keeper_server.api_version"); - auto api_version = SettingFieldKeeperApiVersionTraits::fromString(version_string); - LOG_INFO(&Poco::Logger::get("KeeperContext"), "API version override used: {}", version_string); - system_nodes_with_data[keeper_api_version_path] = toString(static_cast(api_version)); + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(feature_flags_key, keys); + for (const auto & key : keys) + { + auto feature_flag = SettingFieldKeeperFeatureFlagTraits::fromString(key); + auto is_enabled = config.getBool(feature_flags_key + "." + key); + if (is_enabled) + feature_flags.enableFeatureFlag(feature_flag); + else + feature_flags.disableFeatureFlag(feature_flag); + } + + system_nodes_with_data[keeper_api_feature_flags_path] = feature_flags.getFeatureFlags(); } + + feature_flags.logFlags(&Poco::Logger::get("KeeperContext")); } } diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index de502b6c566..9b7000fa726 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -24,6 +25,8 @@ struct KeeperContext bool digest_enabled{true}; std::unordered_map system_nodes_with_data; + + KeeperFeatureFlags feature_flags; }; using KeeperContextPtr = std::shared_ptr; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index d64134f3024..7318a492b35 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -336,28 +336,39 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf snapshot_s3.startup(config, macros); - server = std::make_unique(configuration_and_settings, config, responses_queue, snapshots_queue, snapshot_s3, [this](const KeeperStorage::RequestForSession & request_for_session) - { - /// check if we have queue of read requests depending on this request to be committed - std::lock_guard lock(read_request_queue_mutex); - if (auto it = read_request_queue.find(request_for_session.session_id); it != read_request_queue.end()) + keeper_context = std::make_shared(); + keeper_context->initialize(config); + + server = std::make_unique( + configuration_and_settings, + config, + responses_queue, + snapshots_queue, + keeper_context, + snapshot_s3, + [this](const KeeperStorage::RequestForSession & request_for_session) { - auto & xid_to_request_queue = it->second; - - if (auto request_queue_it = xid_to_request_queue.find(request_for_session.request->xid); request_queue_it != xid_to_request_queue.end()) + /// check if we have queue of read requests depending on this request to be committed + std::lock_guard lock(read_request_queue_mutex); + if (auto it = read_request_queue.find(request_for_session.session_id); it != read_request_queue.end()) { - for (const auto & read_request : request_queue_it->second) - { - if (server->isLeaderAlive()) - server->putLocalReadRequest(read_request); - else - addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); - } + auto & xid_to_request_queue = it->second; - xid_to_request_queue.erase(request_queue_it); + if (auto request_queue_it = xid_to_request_queue.find(request_for_session.request->xid); + request_queue_it != xid_to_request_queue.end()) + { + for (const auto & read_request : request_queue_it->second) + { + if (server->isLeaderAlive()) + server->putLocalReadRequest(read_request); + else + addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); + } + + xid_to_request_queue.erase(request_queue_it); + } } - } - }); + }); try { diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 4b8b134cf8f..1759f55d981 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -81,6 +81,8 @@ private: KeeperSnapshotManagerS3 snapshot_s3; + KeeperContextPtr keeper_context; + /// Thread put requests to raft void requestThread(); /// Thread put responses for subscribed sessions @@ -198,6 +200,12 @@ public: return configuration_and_settings; } + const KeeperContextPtr & getKeeperContext() const + { + return keeper_context; + } + + void incrementPacketsSent() { keeper_stats.incrementPacketsSent(); diff --git a/src/Coordination/KeeperFeatureFlags.cpp b/src/Coordination/KeeperFeatureFlags.cpp new file mode 100644 index 00000000000..216dca014d4 --- /dev/null +++ b/src/Coordination/KeeperFeatureFlags.cpp @@ -0,0 +1,102 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +namespace +{ + +std::pair getByteAndBitIndex(size_t num) +{ + size_t byte_idx = num / 8; + auto bit_idx = (7 - num % 8); + return {byte_idx, bit_idx}; +} + +} + +IMPLEMENT_SETTING_ENUM(KeeperFeatureFlag, ErrorCodes::BAD_ARGUMENTS, + {{"filtered_list", KeeperFeatureFlag::FILTERED_LIST}, + {"multi_read", KeeperFeatureFlag::MULTI_READ}, + {"check_not_exists", KeeperFeatureFlag::CHECK_NOT_EXISTS}}); + +KeeperFeatureFlags::KeeperFeatureFlags() +{ + /// get byte idx of largest value + auto [byte_idx, _] = getByteAndBitIndex(all_keeper_feature_flags.size() - 1); + feature_flags = std::string(byte_idx + 1, 0); +} + +KeeperFeatureFlags::KeeperFeatureFlags(std::string feature_flags_) + : feature_flags(std::move(feature_flags_)) +{} + +void KeeperFeatureFlags::fromApiVersion(KeeperApiVersion keeper_api_version) +{ + if (keeper_api_version == KeeperApiVersion::ZOOKEEPER_COMPATIBLE) + return; + + if (keeper_api_version >= KeeperApiVersion::WITH_FILTERED_LIST) + enableFeatureFlag(KeeperFeatureFlag::FILTERED_LIST); + + if (keeper_api_version >= KeeperApiVersion::WITH_MULTI_READ) + enableFeatureFlag(KeeperFeatureFlag::MULTI_READ); + + if (keeper_api_version >= KeeperApiVersion::WITH_CHECK_NOT_EXISTS) + enableFeatureFlag(KeeperFeatureFlag::CHECK_NOT_EXISTS); +} + +bool KeeperFeatureFlags::isEnabled(KeeperFeatureFlag feature_flag) const +{ + auto [byte_idx, bit_idx] = getByteAndBitIndex(feature_flag); + + if (byte_idx > feature_flags.size()) + return false; + + return feature_flags[byte_idx] & (1 << bit_idx); +} + +void KeeperFeatureFlags::setFeatureFlags(std::string feature_flags_) +{ + feature_flags = std::move(feature_flags_); +} + +void KeeperFeatureFlags::enableFeatureFlag(KeeperFeatureFlag feature_flag) +{ + auto [byte_idx, bit_idx] = getByteAndBitIndex(feature_flag); + chassert(byte_idx < feature_flags.size()); + + feature_flags[byte_idx] |= (1 << bit_idx); +} + +void KeeperFeatureFlags::disableFeatureFlag(KeeperFeatureFlag feature_flag) +{ + auto [byte_idx, bit_idx] = getByteAndBitIndex(feature_flag); + chassert(byte_idx < feature_flags.size()); + + feature_flags[byte_idx] &= ~(1 << bit_idx); +} + +const std::string & KeeperFeatureFlags::getFeatureFlags() const +{ + return feature_flags; +} + +void KeeperFeatureFlags::logFlags(Poco::Logger * log) const +{ + for (const auto & feature_flag : all_keeper_feature_flags) + { + auto is_enabled = isEnabled(feature_flag); + LOG_INFO(log, "Keeper feature flag {}: {}", SettingFieldKeeperFeatureFlagTraits::toString(feature_flag), is_enabled ? "enabled" : "disabled"); + } +} + +} diff --git a/src/Coordination/KeeperFeatureFlags.h b/src/Coordination/KeeperFeatureFlags.h new file mode 100644 index 00000000000..cdd4704a7ca --- /dev/null +++ b/src/Coordination/KeeperFeatureFlags.h @@ -0,0 +1,49 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +enum KeeperFeatureFlag +{ + FILTERED_LIST = 0, + MULTI_READ, + CHECK_NOT_EXISTS, +}; + +static inline constexpr std::array all_keeper_feature_flags +{ + KeeperFeatureFlag::FILTERED_LIST, + KeeperFeatureFlag::MULTI_READ, + KeeperFeatureFlag::CHECK_NOT_EXISTS, +}; + +DECLARE_SETTING_ENUM(KeeperFeatureFlag); + +class KeeperFeatureFlags +{ +public: + KeeperFeatureFlags(); + + explicit KeeperFeatureFlags(std::string feature_flags_); + + /// backwards compatibility + void fromApiVersion(KeeperApiVersion keeper_api_version); + + bool isEnabled(KeeperFeatureFlag feature) const; + + void setFeatureFlags(std::string feature_flags_); + const std::string & getFeatureFlags() const; + + void enableFeatureFlag(KeeperFeatureFlag feature); + void disableFeatureFlag(KeeperFeatureFlag feature); + + void logFlags(Poco::Logger * log) const; +private: + std::string feature_flags; +}; + +} diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index b3150f11cfb..43719e5a69e 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -108,23 +108,19 @@ KeeperServer::KeeperServer( const Poco::Util::AbstractConfiguration & config, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, + KeeperContextPtr keeper_context_, KeeperSnapshotManagerS3 & snapshot_manager_s3, KeeperStateMachine::CommitCallback commit_callback) : server_id(configuration_and_settings_->server_id) , coordination_settings(configuration_and_settings_->coordination_settings) , log(&Poco::Logger::get("KeeperServer")) , is_recovering(config.getBool("keeper_server.force_recovery", false)) - , keeper_context{std::make_shared()} + , keeper_context{std::move(keeper_context_)} , create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true)) { if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); - keeper_context->initialize(config); - - keeper_context->digest_enabled = config.getBool("keeper_server.digest_enabled", false); - keeper_context->ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); - state_machine = nuraft::cs_new( responses_queue_, snapshots_queue_, diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index db4e9c1962e..8f416b1f48c 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -72,6 +72,7 @@ public: const Poco::Util::AbstractConfiguration & config_, ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, + KeeperContextPtr keeper_context_, KeeperSnapshotManagerS3 & snapshot_manager_s3, KeeperStateMachine::CommitCallback commit_callback); diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 8abcd062b7f..ecf795f4068 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -285,7 +285,7 @@ void KeeperStorage::initializeSystemNodes() // insert child system nodes for (const auto & [path, data] : keeper_context->system_nodes_with_data) { - assert(keeper_api_version_path.starts_with(keeper_system_path)); + assert(path.starts_with(keeper_system_path)); Node child_system_node; child_system_node.setData(data); auto [map_key, _] = container.insert(std::string{path}, child_system_node); @@ -1060,7 +1060,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce ProfileEvents::increment(ProfileEvents::KeeperGetRequest); Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); - if (request.path == Coordination::keeper_api_version_path) + if (request.path == Coordination::keeper_api_feature_flags_path) return {}; if (!storage.uncommitted_state.getNode(request.path)) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 005c67ad261..2793b23c572 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -2,7 +2,9 @@ #include #include "Common/ZooKeeper/IKeeper.h" +#include "Coordination/KeeperConstants.h" #include "Coordination/KeeperContext.h" +#include "Coordination/KeeperFeatureFlags.h" #include "Coordination/KeeperStorage.h" #include "Core/Defines.h" #include "IO/WriteHelpers.h" @@ -2346,18 +2348,19 @@ TEST_P(CoordinationTest, TestDurableState) } } -TEST_P(CoordinationTest, TestCurrentApiVersion) +TEST_P(CoordinationTest, TestFeatureFlags) { using namespace Coordination; KeeperStorage storage{500, "", keeper_context}; auto request = std::make_shared(); - request->path = DB::keeper_api_version_path; + request->path = DB::keeper_api_feature_flags_path; auto responses = storage.processRequest(request, 0, std::nullopt, true, true); const auto & get_response = getSingleResponse(responses); - uint8_t keeper_version{0}; - DB::ReadBufferFromOwnString buf(get_response.data); - DB::readIntText(keeper_version, buf); - EXPECT_EQ(keeper_version, static_cast(latest_keeper_api_version)); + DB::KeeperFeatureFlags feature_flags; + feature_flags.setFeatureFlags(get_response.data); + ASSERT_TRUE(feature_flags.isEnabled(KeeperFeatureFlag::FILTERED_LIST)); + ASSERT_TRUE(feature_flags.isEnabled(KeeperFeatureFlag::MULTI_READ)); + ASSERT_FALSE(feature_flags.isEnabled(KeeperFeatureFlag::CHECK_NOT_EXISTS)); } TEST_P(CoordinationTest, TestSystemNodeModify) diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index 559e12ad5ee..9a6a592f2c4 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -31,7 +31,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(context->getZooKeeper()->getConnectedZooKeeperIndex()); res_columns[4]->insert(context->getZooKeeperSessionUptime()); res_columns[5]->insert(context->getZooKeeper()->expired()); - res_columns[6]->insert(context->getZooKeeper()->getApiVersion()); + res_columns[6]->insert(0); res_columns[7]->insert(context->getZooKeeper()->getClientID()); for (const auto & elem : context->getAuxiliaryZooKeepers()) @@ -42,7 +42,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(elem.second->getConnectedZooKeeperIndex()); res_columns[4]->insert(elem.second->getSessionUptime()); res_columns[5]->insert(elem.second->expired()); - res_columns[6]->insert(elem.second->getApiVersion()); + res_columns[6]->insert(0); res_columns[7]->insert(elem.second->getClientID()); } diff --git a/tests/integration/test_keeper_api_version_config/test.py b/tests/integration/test_keeper_api_version_config/test.py deleted file mode 100644 index 34d3acc4f04..00000000000 --- a/tests/integration/test_keeper_api_version_config/test.py +++ /dev/null @@ -1,87 +0,0 @@ -#!/usr/bin/env python3 - -import pytest -import os -from helpers.cluster import ClickHouseCluster -import helpers.keeper_utils as keeper_utils -from kazoo.client import KazooClient, KazooState - -CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) -cluster = ClickHouseCluster(__file__) - -# clickhouse itself will use external zookeeper -node = cluster.add_instance( - "node", - main_configs=["configs/enable_keeper.xml"], - stay_alive=True, -) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - yield cluster - - finally: - cluster.shutdown() - - -def get_connection_zk(nodename, timeout=30.0): - _fake_zk_instance = KazooClient( - hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout - ) - _fake_zk_instance.start() - return _fake_zk_instance - - -def restart_clickhouse(api_version=None, expect_fail=True): - node.stop_clickhouse() - node.copy_file_to_container( - os.path.join(CURRENT_TEST_DIR, "configs/enable_keeper.xml"), - "/etc/clickhouse-server/config.d/enable_keeper.xml", - ) - - if api_version: - node.replace_in_config( - "/etc/clickhouse-server/config.d/enable_keeper.xml", - "", - f"{api_version}<\\/api_version>", - ) - - node.start_clickhouse(retry_start=not expect_fail) - keeper_utils.wait_until_connected(cluster, node) - - -def test_keeper_api_version(started_cluster): - restart_clickhouse() - - def assert_version(string_version, version_number): - node.wait_for_log_line( - f"Detected server's API version: {string_version}", look_behind_lines=1000 - ) - - try: - node_zk = get_connection_zk(node.name) - assert node_zk.get("/keeper/api_version")[0] == str(version_number).encode() - finally: - if node_zk: - node_zk.stop() - node_zk.close() - - assert_version("WITH_CHECK_NOT_EXISTS", 3) - - for i, version in enumerate( - [ - "ZOOKEEPER_COMPATIBLE", - "WITH_FILTERED_LIST", - "WITH_MULTI_READ", - "WITH_CHECK_NOT_EXISTS", - ] - ): - restart_clickhouse(version) - assert_version(version, i) - - with pytest.raises(Exception): - restart_clickhouse("INVALID_VERSION", expect_fail=True) diff --git a/tests/integration/test_keeper_api_version_config/__init__.py b/tests/integration/test_keeper_feature_flags_config/__init__.py similarity index 100% rename from tests/integration/test_keeper_api_version_config/__init__.py rename to tests/integration/test_keeper_feature_flags_config/__init__.py diff --git a/tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml b/tests/integration/test_keeper_feature_flags_config/configs/enable_keeper.xml similarity index 97% rename from tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml rename to tests/integration/test_keeper_feature_flags_config/configs/enable_keeper.xml index c153d025598..53a169c4c3c 100644 --- a/tests/integration/test_keeper_api_version_config/configs/enable_keeper.xml +++ b/tests/integration/test_keeper_feature_flags_config/configs/enable_keeper.xml @@ -18,7 +18,7 @@ 0 - + diff --git a/tests/integration/test_keeper_feature_flags_config/test.py b/tests/integration/test_keeper_feature_flags_config/test.py new file mode 100644 index 00000000000..432c413cbe8 --- /dev/null +++ b/tests/integration/test_keeper_feature_flags_config/test.py @@ -0,0 +1,92 @@ +#!/usr/bin/env python3 + +import pytest +import os +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as keeper_utils +from kazoo.client import KazooClient, KazooState + +CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) +cluster = ClickHouseCluster(__file__) + +# clickhouse itself will use external zookeeper +node = cluster.add_instance( + "node", + main_configs=["configs/enable_keeper.xml"], + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def get_connection_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient( + hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout + ) + _fake_zk_instance.start() + return _fake_zk_instance + + +def restart_clickhouse(feature_flags=[], expect_fail=True): + node.stop_clickhouse() + node.copy_file_to_container( + os.path.join(CURRENT_TEST_DIR, "configs/enable_keeper.xml"), + "/etc/clickhouse-server/config.d/enable_keeper.xml", + ) + + if len(feature_flags) > 0: + feature_flags_config = "" + + for feature, is_enabled in feature_flags: + feature_flags_config += f"<{feature}>{is_enabled}<\\/{feature}>" + + feature_flags_config += "<\\/feature_flags>" + + node.replace_in_config( + "/etc/clickhouse-server/config.d/enable_keeper.xml", + "", + feature_flags_config + ) + + node.start_clickhouse(retry_start=not expect_fail) + keeper_utils.wait_until_connected(cluster, node) + + +def test_keeper_feature_flags(started_cluster): + restart_clickhouse() + + def assert_feature_flags(feature_flags): + res = keeper_utils.send_4lw_cmd(started_cluster, node, "ftfl") + + for feature, is_enabled in feature_flags: + node.wait_for_log_line( + f"ZooKeeperClient: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", look_behind_lines=1000 + ) + + node.wait_for_log_line( + f"KeeperContext: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", look_behind_lines=1000 + ) + + assert f"{feature}\t{1 if is_enabled else 0}" in res + + assert_feature_flags([("filtered_list", 1), ("multi_read", 1), ("check_not_exists", 0)]) + + feature_flags = [("multi_read", 0), ("check_not_exists", 1)] + restart_clickhouse(feature_flags) + assert_feature_flags(feature_flags + [("filtered_list", 1)]) + + feature_flags = [("multi_read", 0), ("check_not_exists", 0), ("filtered_list", 0)] + restart_clickhouse(feature_flags) + assert_feature_flags(feature_flags) + + with pytest.raises(Exception): + restart_clickhouse([("invalid_feature", 1)], expect_fail=True) From eb16745033fd34b5c9c32124b6bb870f7c795f9c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Jun 2023 11:04:07 +0000 Subject: [PATCH 122/515] Collect sets from indexHint actions as well. --- src/Planner/Planner.cpp | 44 ++++++++++++++++++++++++++++------------- 1 file changed, 30 insertions(+), 14 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index c1f472bb5a8..4ac81e28f92 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -9,6 +9,7 @@ #include #include +#include #include #include @@ -895,6 +896,34 @@ void addOffsetStep(QueryPlan & query_plan, const QueryAnalysisResult & query_ana query_plan.addStep(std::move(offsets_step)); } +void collectSetsFromActionsDAG(const ActionsDAGPtr & dag, std::unordered_set & useful_sets) +{ + for (const auto & node : dag->getNodes()) + { + if (node.column) + { + const IColumn * column = node.column.get(); + if (const auto * column_const = typeid_cast(column)) + column = &column_const->getDataColumn(); + + if (const auto * column_set = typeid_cast(column)) + useful_sets.insert(column_set->getData().get()); + } + + if (node.type == ActionsDAG::ActionType::FUNCTION && node.function_base->getName() == "indexHint") + { + ActionsDAG::NodeRawConstPtrs children; + if (const auto * adaptor = typeid_cast(node.function_base.get())) + { + if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) + { + collectSetsFromActionsDAG(index_hint->getActions(), useful_sets); + } + } + } + } +} + void addBuildSubqueriesForSetsStepIfNeeded( QueryPlan & query_plan, const SelectQueryOptions & select_query_options, @@ -907,20 +936,7 @@ void addBuildSubqueriesForSetsStepIfNeeded( PreparedSets::SubqueriesForSets subqueries_for_sets; for (const auto & actions_to_execute : result_actions_to_execute) - { - for (const auto & node : actions_to_execute->getNodes()) - { - if (node.column) - { - const IColumn * column = node.column.get(); - if (const auto * column_const = typeid_cast(column)) - column = &column_const->getDataColumn(); - - if (const auto * column_set = typeid_cast(column)) - useful_sets.insert(column_set->getData().get()); - } - } - } + collectSetsFromActionsDAG(actions_to_execute, useful_sets); auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.set.get()); }; auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate)); From 708a99a6ea63409ce33f83d450592eaa42411ebb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 12 Jun 2023 11:19:38 +0000 Subject: [PATCH 123/515] Automatic style fix --- .../test_keeper_feature_flags_config/test.py | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_keeper_feature_flags_config/test.py b/tests/integration/test_keeper_feature_flags_config/test.py index 432c413cbe8..bb7252e9ec8 100644 --- a/tests/integration/test_keeper_feature_flags_config/test.py +++ b/tests/integration/test_keeper_feature_flags_config/test.py @@ -45,7 +45,7 @@ def restart_clickhouse(feature_flags=[], expect_fail=True): if len(feature_flags) > 0: feature_flags_config = "" - + for feature, is_enabled in feature_flags: feature_flags_config += f"<{feature}>{is_enabled}<\\/{feature}>" @@ -54,7 +54,7 @@ def restart_clickhouse(feature_flags=[], expect_fail=True): node.replace_in_config( "/etc/clickhouse-server/config.d/enable_keeper.xml", "", - feature_flags_config + feature_flags_config, ) node.start_clickhouse(retry_start=not expect_fail) @@ -69,16 +69,20 @@ def test_keeper_feature_flags(started_cluster): for feature, is_enabled in feature_flags: node.wait_for_log_line( - f"ZooKeeperClient: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", look_behind_lines=1000 + f"ZooKeeperClient: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", + look_behind_lines=1000, ) node.wait_for_log_line( - f"KeeperContext: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", look_behind_lines=1000 + f"KeeperContext: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", + look_behind_lines=1000, ) assert f"{feature}\t{1 if is_enabled else 0}" in res - assert_feature_flags([("filtered_list", 1), ("multi_read", 1), ("check_not_exists", 0)]) + assert_feature_flags( + [("filtered_list", 1), ("multi_read", 1), ("check_not_exists", 0)] + ) feature_flags = [("multi_read", 0), ("check_not_exists", 1)] restart_clickhouse(feature_flags) From a7e6264d567ffa7456d00df017675f0acf4ca90f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 12 Jun 2023 13:05:44 +0000 Subject: [PATCH 124/515] Add backward compatibility --- programs/keeper/CMakeLists.txt | 1 + src/Common/ZooKeeper/CMakeLists.txt | 2 ++ src/Coordination/FourLetterCommand.cpp | 2 +- src/Coordination/KeeperConstants.h | 1 - src/Coordination/KeeperContext.cpp | 4 ++++ src/Storages/System/StorageSystemZooKeeperConnection.cpp | 4 ++-- 6 files changed, 10 insertions(+), 4 deletions(-) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 4f74cc06801..555c6431865 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -44,6 +44,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperLogStore.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperServer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperContext.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperFeatureFlags.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManager.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManagerS3.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateMachine.cpp diff --git a/src/Common/ZooKeeper/CMakeLists.txt b/src/Common/ZooKeeper/CMakeLists.txt index a9a335d1461..3f7e87ff4a7 100644 --- a/src/Common/ZooKeeper/CMakeLists.txt +++ b/src/Common/ZooKeeper/CMakeLists.txt @@ -2,6 +2,8 @@ include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") add_headers_and_sources(clickhouse_common_zookeeper .) +list(APPEND clickhouse_common_zookeeper_sources ${CMAKE_CURRENT_SOURCE_DIR}/../../../src/Coordination/KeeperFeatureFlags.cpp) + # for clickhouse server add_library(clickhouse_common_zookeeper ${clickhouse_common_zookeeper_headers} ${clickhouse_common_zookeeper_sources}) target_compile_definitions (clickhouse_common_zookeeper PRIVATE -DZOOKEEPER_LOG) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 55120e70d99..10d13657fb0 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -490,7 +490,7 @@ String RecoveryCommand::run() String ApiVersionCommand::run() { - return toString(0); + return toString(static_cast(KeeperApiVersion::WITH_MULTI_READ)); } String CreateSnapshotCommand::run() diff --git a/src/Coordination/KeeperConstants.h b/src/Coordination/KeeperConstants.h index 6d9c03ca8a9..84cbb0ab7c5 100644 --- a/src/Coordination/KeeperConstants.h +++ b/src/Coordination/KeeperConstants.h @@ -14,7 +14,6 @@ enum class KeeperApiVersion : uint8_t WITH_CHECK_NOT_EXISTS, }; - const std::string keeper_system_path = "/keeper"; const std::string keeper_api_version_path = keeper_system_path + "/api_version"; const std::string keeper_api_feature_flags_path = keeper_system_path + "/feature_flags"; diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index dc408afc19a..e6f30c81310 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -11,6 +11,10 @@ KeeperContext::KeeperContext() feature_flags.enableFeatureFlag(KeeperFeatureFlag::FILTERED_LIST); feature_flags.enableFeatureFlag(KeeperFeatureFlag::MULTI_READ); system_nodes_with_data[keeper_api_feature_flags_path] = feature_flags.getFeatureFlags(); + + + /// for older clients, the default is equivalent to WITH_MULTI_READ version + system_nodes_with_data[keeper_api_version_path] = toString(static_cast(KeeperApiVersion::WITH_MULTI_READ)); } void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index 9a6a592f2c4..99872be6ba0 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -31,7 +31,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(context->getZooKeeper()->getConnectedZooKeeperIndex()); res_columns[4]->insert(context->getZooKeeperSessionUptime()); res_columns[5]->insert(context->getZooKeeper()->expired()); - res_columns[6]->insert(0); + res_columns[6]->insert(static_cast(KeeperApiVersion::WITH_MULTI_READ)); res_columns[7]->insert(context->getZooKeeper()->getClientID()); for (const auto & elem : context->getAuxiliaryZooKeepers()) @@ -42,7 +42,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(elem.second->getConnectedZooKeeperIndex()); res_columns[4]->insert(elem.second->getSessionUptime()); res_columns[5]->insert(elem.second->expired()); - res_columns[6]->insert(0); + res_columns[6]->insert(static_cast(KeeperApiVersion::WITH_MULTI_READ)); res_columns[7]->insert(elem.second->getClientID()); } From 25f08f8d194f77d0ee56e7c5132b9d5c4244a30f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 12 Jun 2023 15:35:44 +0200 Subject: [PATCH 125/515] Remove duplicate include --- src/Coordination/Changelog.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index cc2ea491e17..ffa22a6b888 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -14,7 +14,6 @@ #include #include #include #include From 70252321750b6a8d3ab6c41f658b76705a2e55b9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Jun 2023 18:59:07 +0000 Subject: [PATCH 126/515] Fixing cache for sets. --- src/Processors/Transforms/CreatingSetsTransform.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index a1e43525ab1..3139fa5ed98 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -86,7 +86,7 @@ void CreatingSetsTransform::startSubquery() } } - if (subquery.set) + if (subquery.set && !set_from_cache) LOG_TRACE(log, "Creating set, key: {}", subquery.key); if (subquery.table) LOG_TRACE(log, "Filling temporary table."); @@ -97,7 +97,7 @@ void CreatingSetsTransform::startSubquery() /// TODO: make via port table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), nullptr, /*async_insert=*/false)); - done_with_set = !subquery.set; + done_with_set = !subquery.set || set_from_cache; done_with_table = !subquery.table; if ((done_with_set && !set_from_cache) /*&& done_with_join*/ && done_with_table) @@ -175,10 +175,10 @@ void CreatingSetsTransform::consume(Chunk chunk) Chunk CreatingSetsTransform::generate() { - if (subquery.set) + if (subquery.set && !set_from_cache) { subquery.set->finishInsert(); - subquery.promise_to_fill_set.set_value(subquery.set); + //subquery.promise_to_fill_set.set_value(subquery.set); if (promise_to_build) promise_to_build->set_value(subquery.set); } From 6f1c4865372b408d0f773a8dd9b1db8e63b5e4bb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 13 Jun 2023 10:25:53 +0200 Subject: [PATCH 127/515] Fix build --- src/Coordination/Changelog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index ffa22a6b888..94062140bac 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -13,7 +13,7 @@ #include #include #include -#include #include From bf69755adab474fbd166209ab7675537d1a9aeeb Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Tue, 13 Jun 2023 01:18:36 -0700 Subject: [PATCH 128/515] Address some usability issues with INTO OUTFILE usage. --- src/Client/ClientBase.cpp | 20 ++++++++++++++++++++ src/Parsers/ASTQueryWithOutput.cpp | 2 ++ src/Parsers/ASTQueryWithOutput.h | 1 + src/Parsers/ParserQueryWithOutput.cpp | 6 ++++++ 4 files changed, 29 insertions(+) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 6f295c11070..fc108b8f57d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -568,6 +568,13 @@ try CompressionMethod compression_method = chooseCompressionMethod(out_file, compression_method_string); UInt64 compression_level = 3; + if (query_with_output->is_outfile_append && query_with_output->is_outfile_truncate) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot use INTO OUTFILE with APPEND and TRUNCATE simultaneously."); + } + if (query_with_output->is_outfile_append && compression_method != CompressionMethod::None) { throw Exception( @@ -589,9 +596,22 @@ try range.second); } + std::error_code ec; + if (std::filesystem::is_regular_file(out_file, ec)) + { + if (!query_with_output->is_outfile_append && !query_with_output->is_outfile_truncate) + { + throw Exception( + ErrorCodes::CANNOT_OPEN_FILE, + "File {} exists, consider using 'INTO OUTFILE ... APPEND' or 'INTO OUTFILE ... TRUNCATE' if appropriate.", + out_file); + } + } auto flags = O_WRONLY | O_EXCL; if (query_with_output->is_outfile_append) flags |= O_APPEND; + else if (query_with_output->is_outfile_truncate) + flags |= O_TRUNC; else flags |= O_CREAT; diff --git a/src/Parsers/ASTQueryWithOutput.cpp b/src/Parsers/ASTQueryWithOutput.cpp index 5f717715a69..4bf1e6cb231 100644 --- a/src/Parsers/ASTQueryWithOutput.cpp +++ b/src/Parsers/ASTQueryWithOutput.cpp @@ -39,6 +39,8 @@ void ASTQueryWithOutput::formatImpl(const FormatSettings & s, FormatState & stat s.ostr << (s.hilite ? hilite_keyword : ""); if (is_outfile_append) s.ostr << " APPEND"; + if (is_outfile_truncate) + s.ostr << " TRUNCATE"; if (is_into_outfile_with_stdout) s.ostr << " AND STDOUT"; s.ostr << (s.hilite ? hilite_none : ""); diff --git a/src/Parsers/ASTQueryWithOutput.h b/src/Parsers/ASTQueryWithOutput.h index 7db021405e7..6f9cafc89a9 100644 --- a/src/Parsers/ASTQueryWithOutput.h +++ b/src/Parsers/ASTQueryWithOutput.h @@ -17,6 +17,7 @@ public: ASTPtr out_file; bool is_into_outfile_with_stdout = false; bool is_outfile_append = false; + bool is_outfile_truncate = false; ASTPtr format; ASTPtr settings_ast; ASTPtr compression; diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 6796f4528c4..2bfe7353be4 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -109,6 +109,12 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query_with_output.is_outfile_append = true; } + ParserKeyword s_truncate("TRUNCATE"); + if (s_truncate.ignore(pos, expected)) + { + query_with_output.is_outfile_truncate = true; + } + ParserKeyword s_stdout("AND STDOUT"); if (s_stdout.ignore(pos, expected)) { From 2c018f5261553dd6106639f22c148fbdd61d8fc4 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 3 Jun 2023 20:59:04 +0200 Subject: [PATCH 129/515] do not call finalize after exception --- src/Storages/StorageS3.cpp | 11 +- tests/integration/helpers/client.py | 2 + tests/integration/helpers/cluster.py | 2 + .../integration/helpers/s3_mocks/broken_s3.py | 241 +++++++++++++++--- .../test_checking_s3_blobs_paranoid/test.py | 98 ++++++- tests/integration/test_merge_tree_s3/test.py | 222 +++++++++++++++- 6 files changed, 530 insertions(+), 46 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index f1a7bcb71a2..dfa5ea2667a 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -794,7 +794,7 @@ public: void onException() override { std::lock_guard lock(cancel_mutex); - finalize(); + release(); } void onFinish() override @@ -824,6 +824,15 @@ private: } } + void release() + { + if (!writer) + return; + + writer.reset(); + write_buf.reset(); + } + Block sample_block; std::optional format_settings; std::unique_ptr write_buf; diff --git a/tests/integration/helpers/client.py b/tests/integration/helpers/client.py index c2676ac08a6..fdeedb9a80d 100644 --- a/tests/integration/helpers/client.py +++ b/tests/integration/helpers/client.py @@ -121,6 +121,7 @@ class Client: user=None, password=None, database=None, + query_id=None, ): return self.get_query_request( sql, @@ -130,6 +131,7 @@ class Client: user=user, password=password, database=database, + query_id=query_id, ).get_error() @stacktraces_on_timeout_decorator diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index f57ebf40e54..c77e67062a1 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3376,6 +3376,7 @@ class ClickHouseInstance: user=None, password=None, database=None, + query_id=None, ): logging.debug(f"Executing query {sql} on {self.name}") return self.client.query_and_get_error( @@ -3386,6 +3387,7 @@ class ClickHouseInstance: user=user, password=password, database=database, + query_id=query_id, ) def query_and_get_error_with_retry( diff --git a/tests/integration/helpers/s3_mocks/broken_s3.py b/tests/integration/helpers/s3_mocks/broken_s3.py index 026a3c6f515..8ff4f9e9203 100644 --- a/tests/integration/helpers/s3_mocks/broken_s3.py +++ b/tests/integration/helpers/s3_mocks/broken_s3.py @@ -6,10 +6,10 @@ import time import urllib.parse import http.server import socketserver +import string -UPSTREAM_HOST = "minio1" -UPSTREAM_PORT = 9001 +INF_COUNT = 100000000 class MockControl: @@ -28,31 +28,88 @@ class MockControl: ], nothrow=True, ) - assert response == "OK" + assert response == "OK", response + + def setup_error_at_object_upload(self, count=None, after=None): + url = f"http://localhost:{self._port}/mock_settings/error_at_object_upload?nothing=1" + + if count is not None: + url += f"&count={count}" + + if after is not None: + url += f"&after={after}" - def setup_fail_upload(self, part_length): response = self._cluster.exec_in_container( self._cluster.get_container_id(self._container), [ "curl", "-s", - f"http://localhost:{self._port}/mock_settings/error_at_put?when_length_bigger={part_length}", + url, ], nothrow=True, ) - assert response == "OK" + assert response == "OK", response + + def setup_error_at_part_upload(self, count=None, after=None): + url = f"http://localhost:{self._port}/mock_settings/error_at_part_upload?nothing=1" + + if count is not None: + url += f"&count={count}" + + if after is not None: + url += f"&after={after}" - def setup_fake_upload(self, part_length): response = self._cluster.exec_in_container( self._cluster.get_container_id(self._container), [ "curl", "-s", - f"http://localhost:{self._port}/mock_settings/fake_put?when_length_bigger={part_length}", + url, ], nothrow=True, ) - assert response == "OK" + assert response == "OK", response + + def setup_error_at_create_multi_part_upload(self, count=None): + url = f"http://localhost:{self._port}/mock_settings/error_at_create_multi_part_upload" + + if count is not None: + url += f"?count={count}" + + response = self._cluster.exec_in_container( + self._cluster.get_container_id(self._container), + [ + "curl", + "-s", + url, + ], + nothrow=True, + ) + assert response == "OK", response + + def setup_fake_puts(self, part_length): + response = self._cluster.exec_in_container( + self._cluster.get_container_id(self._container), + [ + "curl", + "-s", + f"http://localhost:{self._port}/mock_settings/fake_puts?when_length_bigger={part_length}", + ], + nothrow=True, + ) + assert response == "OK", response + + def setup_fake_multpartuploads(self): + response = self._cluster.exec_in_container( + self._cluster.get_container_id(self._container), + [ + "curl", + "-s", + f"http://localhost:{self._port}/mock_settings/setup_fake_multpartuploads?", + ], + nothrow=True, + ) + assert response == "OK", response def setup_slow_answers( self, minimal_length=0, timeout=None, probability=None, count=None @@ -77,7 +134,7 @@ class MockControl: ["curl", "-s", url], nothrow=True, ) - assert response == "OK" + assert response == "OK", response class _ServerRuntime: @@ -88,7 +145,7 @@ class _ServerRuntime: self.probability = probability_ if probability_ is not None else 1 self.timeout = timeout_ if timeout_ is not None else 0.1 self.minimal_length = minimal_length_ if minimal_length_ is not None else 0 - self.count = count_ if count_ is not None else 2**32 + self.count = count_ if count_ is not None else INF_COUNT def __str__(self): return ( @@ -109,12 +166,32 @@ class _ServerRuntime: return _runtime.slow_put.timeout return None + class CountAfter: + def __init__(self, count_=None, after_=None): + self.count = count_ if count_ is not None else INF_COUNT + self.after = after_ if after_ is not None else 0 + + def __str__(self): + return f"count:{self.count} after:{self.after}" + + def has_effect(self): + if self.after: + self.after -= 1 + if self.after == 0: + if self.count: + self.count -= 1 + return True + return False + def __init__(self): self.lock = threading.Lock() - self.error_at_put_when_length_bigger = None + self.error_at_part_upload = None + self.error_at_object_upload = None self.fake_put_when_length_bigger = None self.fake_uploads = dict() self.slow_put = None + self.fake_multipart_upload = None + self.error_at_create_multi_part_upload = None def register_fake_upload(self, upload_id, key): with self.lock: @@ -127,10 +204,14 @@ class _ServerRuntime: return False def reset(self): - self.error_at_put_when_length_bigger = None - self.fake_put_when_length_bigger = None - self.fake_uploads = dict() - self.slow_put = None + with self.lock: + self.error_at_part_upload = None + self.error_at_object_upload = None + self.fake_put_when_length_bigger = None + self.fake_uploads = dict() + self.slow_put = None + self.fake_multipart_upload = None + self.error_at_create_multi_part_upload = None _runtime = _ServerRuntime() @@ -141,6 +222,13 @@ def _and_then(value, func): return None if value is None else func(value) +def get_random_string(length): + # choose from all lowercase letter + letters = string.ascii_lowercase + result_str = "".join(random.choice(letters) for i in range(length)) + return result_str + + class RequestHandler(http.server.BaseHTTPRequestHandler): def _ok(self): self.send_response(200) @@ -166,19 +254,30 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self._read_out() self.send_response(307) - url = f"http://{UPSTREAM_HOST}:{UPSTREAM_PORT}{self.path}" + url = ( + f"http://{self.server.upstream_host}:{self.server.upstream_port}{self.path}" + ) self.send_header("Location", url) self.end_headers() self.wfile.write(b"Redirected") def _error(self, data): self._read_out() - self.send_response(500) self.send_header("Content-Type", "text/xml") self.end_headers() self.wfile.write(bytes(data, "UTF-8")) + def _error_expected_500(self): + self._error( + '' + "" + "ExpectedError" + "mock s3 injected error" + "txfbd566d03042474888193-00608d7537" + "" + ) + def _fake_put_ok(self): self._read_out() @@ -188,6 +287,28 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.send_header("Content-Length", 0) self.end_headers() + def _fake_uploads(self, path, upload_id): + self._read_out() + + parts = [x for x in path.split("/") if x] + bucket = parts[0] + key = "/".join(parts[1:]) + data = ( + '\n' + "\n" + f"{bucket}" + f"{key}" + f"{upload_id}" + "" + ) + + self.send_response(200) + self.send_header("Content-Type", "text/xml") + self.send_header("Content-Length", len(data)) + self.end_headers() + + self.wfile.write(bytes(data, "UTF-8")) + def _fake_post_ok(self, path): self._read_out() @@ -219,18 +340,29 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if len(path) < 2: return self._error("_mock_settings: wrong command") - if path[1] == "error_at_put": + if path[1] == "error_at_part_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - _runtime.error_at_put_when_length_bigger = int( - params.get("when_length_bigger", [1024 * 1024])[0] + _runtime.error_at_part_upload = _ServerRuntime.CountAfter( + count_=_and_then(params.get("count", [None])[0], int), + after_=_and_then(params.get("after", [None])[0], int), ) return self._ok() - if path[1] == "fake_put": + + if path[1] == "error_at_object_upload": + params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) + _runtime.error_at_object_upload = _ServerRuntime.CountAfter( + count_=_and_then(params.get("count", [None])[0], int), + after_=_and_then(params.get("after", [None])[0], int), + ) + return self._ok() + + if path[1] == "fake_puts": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) _runtime.fake_put_when_length_bigger = int( params.get("when_length_bigger", [1024 * 1024])[0] ) return self._ok() + if path[1] == "slow_put": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) _runtime.slow_put = _ServerRuntime.SlowPut( @@ -241,6 +373,18 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): ) self.log_message("set slow put %s", _runtime.slow_put) return self._ok() + + if path[1] == "setup_fake_multpartuploads": + _runtime.fake_multipart_upload = True + return self._ok() + + if path[1] == "error_at_create_multi_part_upload": + params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) + _runtime.error_at_create_multi_part_upload = int( + params.get("count", [INF_COUNT])[0] + ) + return self._ok() + if path[1] == "reset": _runtime.reset() return self._ok() @@ -265,33 +409,42 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.log_message("slow put %s", timeout) time.sleep(timeout) - if _runtime.error_at_put_when_length_bigger is not None: - if content_length > _runtime.error_at_put_when_length_bigger: - return self._error( - '' - "" - "ExpectedError" - "mock s3 injected error" - "txfbd566d03042474888193-00608d7537" - "" - ) - parts = urllib.parse.urlsplit(self.path) params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) upload_id = params.get("uploadId", [None])[0] - if _runtime.fake_put_when_length_bigger is not None: - if content_length > _runtime.fake_put_when_length_bigger: - if upload_id is not None: - _runtime.register_fake_upload(upload_id, parts.path) - return self._fake_put_ok() + + if upload_id is not None: + if _runtime.error_at_part_upload is not None: + if _runtime.error_at_part_upload.has_effect(): + return self._error_expected_500() + if _runtime.fake_multipart_upload: + if _runtime.is_fake_upload(upload_id, parts.path): + return self._fake_put_ok() + else: + if _runtime.error_at_object_upload is not None: + if _runtime.error_at_object_upload.has_effect(): + return self._error_expected_500() + if _runtime.fake_put_when_length_bigger is not None: + if content_length > _runtime.fake_put_when_length_bigger: + return self._fake_put_ok() return self._redirect() def do_POST(self): parts = urllib.parse.urlsplit(self.path) - params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - upload_id = params.get("uploadId", [None])[0] + params = urllib.parse.parse_qs(parts.query, keep_blank_values=True) + uploads = params.get("uploads", [None])[0] + if uploads is not None: + if _runtime.error_at_create_multi_part_upload: + _runtime.error_at_create_multi_part_upload -= 1 + return self._error_expected_500() + if _runtime.fake_multipart_upload: + upload_id = get_random_string(5) + _runtime.register_fake_upload(upload_id, parts.path) + return self._fake_uploads(parts.path, upload_id) + + upload_id = params.get("uploadId", [None])[0] if _runtime.is_fake_upload(upload_id, parts.path): return self._fake_post_ok(parts.path) @@ -307,7 +460,15 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): class _ThreadedHTTPServer(socketserver.ThreadingMixIn, http.server.HTTPServer): """Handle requests in a separate thread.""" + def set_upstream(self, upstream_host, upstream_port): + self.upstream_host = upstream_host + self.upstream_port = upstream_port + if __name__ == "__main__": httpd = _ThreadedHTTPServer(("0.0.0.0", int(sys.argv[1])), RequestHandler) + if len(sys.argv) == 4: + httpd.set_upstream(sys.argv[2], sys.argv[3]) + else: + httpd.set_upstream("minio1", 9001) httpd.serve_forever() diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 042d57a0c43..c0f184815c9 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -54,7 +54,7 @@ def test_upload_after_check_works(cluster, broken_s3): """ ) - broken_s3.setup_fake_upload(1) + broken_s3.setup_fake_puts(1) error = node.query_and_get_error( "INSERT INTO s3_upload_after_check_works VALUES (1, 'Hello')" @@ -63,3 +63,99 @@ def test_upload_after_check_works(cluster, broken_s3): assert "Code: 499" in error, error assert "Immediately after upload" in error, error assert "suddenly disappeared" in error, error + + +def get_counters(node, query_id, log_type="ExceptionWhileProcessing"): + node.query("SYSTEM FLUSH LOGS") + return [ + int(x) + for x in node.query( + f""" + SELECT + ProfileEvents['S3CreateMultipartUpload'], + ProfileEvents['S3UploadPart'], + ProfileEvents['S3WriteRequestsErrors'] + FROM system.query_log + WHERE query_id='{query_id}' + AND type='{log_type}' + """ + ).split() + if x + ] + + +def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3): + node = cluster.instances["node"] + + broken_s3.setup_error_at_create_multi_part_upload() + + insert_query_id = "INSERT_INTO_TABLE_FUNCTION_FAIL_CREATE_MPU" + error = node.query_and_get_error( + """ + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_upload_s3_fail_create_multi_part_upload', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 100000000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100 + """, + query_id=insert_query_id, + ) + + assert "Code: 499" in error, error + assert "mock s3 injected error" in error, error + assert "DB::WriteBufferFromS3::createMultipartUpload()" in error, error + + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + node, insert_query_id + ) + assert count_create_multi_part_uploads == 1 + assert count_upload_parts == 0 + assert count_s3_errors == 1 + + +def test_upload_s3_fail_upload_part_when_multi_part_upload(cluster, broken_s3): + node = cluster.instances["node"] + + broken_s3.setup_fake_multpartuploads() + broken_s3.setup_error_at_part_upload(count=1, after=2) + + insert_query_id = "INSERT_INTO_TABLE_FUNCTION_FAIL_UPLOAD_PART" + error = node.query_and_get_error( + """ + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_upload_s3_fail_upload_part_when_multi_part_upload', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 100000000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100 + """, + query_id=insert_query_id, + ) + + assert "Code: 499" in error, error + assert "mock s3 injected error" in error, error + assert "DB::WriteBufferFromS3::writePart" in error, error + + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + node, insert_query_id + ) + assert count_create_multi_part_uploads == 1 + assert count_upload_parts >= 2 + assert ( + count_s3_errors == 2 + ) # the second is cancel multipart upload, s3_mock just redirects this request diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 2ccd517923a..f3f44f1452c 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -862,7 +862,9 @@ def test_merge_canceled_by_s3_errors(cluster, broken_s3, node_name, storage_poli min_key = node.query("SELECT min(key) FROM test_merge_canceled_by_s3_errors") assert int(min_key) == 0, min_key - broken_s3.setup_fail_upload(50000) + broken_s3.setup_error_at_object_upload() + broken_s3.setup_fake_multpartuploads() + broken_s3.setup_error_at_part_upload() node.query("SYSTEM START MERGES test_merge_canceled_by_s3_errors") @@ -905,7 +907,7 @@ def test_merge_canceled_by_s3_errors_when_move(cluster, broken_s3, node_name): settings={"materialize_ttl_after_modify": 0}, ) - broken_s3.setup_fail_upload(10000) + broken_s3.setup_error_at_object_upload(count=1, after=1) node.query("SYSTEM START MERGES merge_canceled_by_s3_errors_when_move") @@ -941,7 +943,7 @@ def test_s3_engine_heavy_write_check_mem( " ENGINE S3('http://resolver:8083/root/data/test-upload.csv', 'minio', 'minio123', 'CSV')", ) - broken_s3.setup_fake_upload(1000) + broken_s3.setup_fake_multpartuploads() broken_s3.setup_slow_answers(10 * 1024 * 1024, timeout=15, count=10) query_id = f"INSERT_INTO_S3_ENGINE_QUERY_ID_{in_flight}" @@ -987,7 +989,7 @@ def test_s3_disk_heavy_write_check_mem(cluster, broken_s3, node_name): ) node.query("SYSTEM STOP MERGES s3_test") - broken_s3.setup_fake_upload(1000) + broken_s3.setup_fake_multpartuploads() broken_s3.setup_slow_answers(10 * 1024 * 1024, timeout=10, count=50) query_id = f"INSERT_INTO_S3_DISK_QUERY_ID" @@ -1013,3 +1015,215 @@ def test_s3_disk_heavy_write_check_mem(cluster, broken_s3, node_name): assert int(result) > 0.8 * memory check_no_objects_after_drop(cluster, node_name=node_name) + + +def get_memory_usage(node, query_id): + node.query("SYSTEM FLUSH LOGS") + memory_usage = node.query( + "SELECT memory_usage" + " FROM system.query_log" + f" WHERE query_id='{query_id}'" + " AND type='QueryFinish'" + ) + return int(memory_usage) + + +def get_memory_usages(node, query_ids): + node.query("SYSTEM FLUSH LOGS") + result = [] + for query_id in query_ids: + memory_usage = node.query( + "SELECT memory_usage" + " FROM system.query_log" + f" WHERE query_id='{query_id}'" + " AND type='QueryFinish'" + ) + result.append(int(memory_usage)) + return result + + +@pytest.mark.parametrize("node_name", ["node"]) +def test_heavy_insert_select_check_memory(cluster, broken_s3, node_name): + node = cluster.instances[node_name] + + node.query( + """ + CREATE TABLE central_query_log + ( + control_plane_id UUID, + pod_id LowCardinality(String), + scrape_ts_microseconds DateTime64(6) CODEC(Delta(8), LZ4), + event_date Date, + event_time DateTime, + payload Array(String), + payload_01 String, + payload_02 String, + payload_03 String, + payload_04 String, + payload_05 String, + payload_06 String, + payload_07 String, + payload_08 String, + payload_09 String, + payload_10 String, + payload_11 String, + payload_12 String, + payload_13 String, + payload_14 String, + payload_15 String, + payload_16 String, + payload_17 String, + payload_18 String, + payload_19 String + ) + ENGINE=MergeTree() + PARTITION BY toYYYYMM(event_date) + ORDER BY (control_plane_id, event_date, pod_id) + SETTINGS + storage_policy='s3' + """ + ) + + node.query("SYSTEM STOP MERGES central_query_log") + + write_count = 2 + write_query_ids = [] + for x in range(write_count): + query_id = f"INSERT_INTO_TABLE_RANDOM_DATA_QUERY_ID_{x}" + write_query_ids.append(query_id) + node.query( + """ + INSERT INTO central_query_log + SELECT + control_plane_id, + pod_id, + toStartOfHour(event_time) + toIntervalSecond(randUniform(0,60)) as scrape_ts_microseconds, + toDate(event_time) as event_date, + event_time, + payload, + payload[1] as payload_01, + payload[2] as payload_02, + payload[3] as payload_03, + payload[4] as payload_04, + payload[5] as payload_05, + payload[6] as payload_06, + payload[7] as payload_07, + payload[8] as payload_08, + payload[9] as payload_09, + payload[10] as payload_10, + payload[11] as payload_11, + payload[12] as payload_12, + payload[13] as payload_13, + payload[14] as payload_14, + payload[15] as payload_15, + payload[16] as payload_16, + payload[17] as payload_17, + payload[18] as payload_18, + payload[19] as payload_19 + FROM + ( + SELECT + control_plane_id, + substring(payload[1], 1, 5) as pod_id, + toDateTime('2022-12-12 00:00:00') + + toIntervalDay(floor(randUniform(0,3))) + + toIntervalHour(floor(randUniform(0,24))) + + toIntervalSecond(floor(randUniform(0,60))) + as event_time, + payload + FROM + generateRandom( + 'control_plane_id UUID, payload Array(String)', + NULL, + 100, + 100 + ) + LIMIT 10000 + ) + SETTINGS + max_insert_block_size=256000000, + min_insert_block_size_rows=1000000, + min_insert_block_size_bytes=256000000 + """, + query_id=query_id, + ) + + memory = 845346116 + for memory_usage, query_id in zip( + get_memory_usages(node, write_query_ids), write_query_ids + ): + assert int(memory_usage) < 1.2 * memory, f"{memory_usage} : {query_id}" + assert int(memory_usage) > 0.8 * memory, f"{memory_usage} : {query_id}" + + broken_s3.setup_slow_answers(minimal_length=1000, timeout=5, count=20) + broken_s3.setup_fake_multpartuploads() + + insert_query_id = f"INSERT_INTO_S3_FUNCTION_QUERY_ID" + node.query( + """ + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test-upload_{_partition_id}.csv.gz', + 'minio', 'minio123', + 'CSV', auto, 'gzip' + ) + PARTITION BY formatDateTime(subtractHours(toDateTime('2022-12-13 00:00:00'), 1),'%Y-%m-%d_%H:00') + WITH toDateTime('2022-12-13 00:00:00') as time_point + SELECT + * + FROM central_query_log + WHERE + event_date >= subtractDays(toDate(time_point), 1) + AND scrape_ts_microseconds >= subtractHours(toStartOfHour(time_point), 12) + AND scrape_ts_microseconds < toStartOfDay(time_point) + SETTINGS + s3_max_inflight_parts_for_one_file=1 + """, + query_id=insert_query_id, + ) + + query_id = f"SELECT_QUERY_ID" + total = node.query( + """ + SELECT + count() + FROM central_query_log + """, + query_id=query_id, + ) + assert int(total) == 10000 * write_count + + query_id = f"SELECT_WHERE_QUERY_ID" + selected = node.query( + """ + WITH toDateTime('2022-12-13 00:00:00') as time_point + SELECT + count() + FROM central_query_log + WHERE + event_date >= subtractDays(toDate(time_point), 1) + AND scrape_ts_microseconds >= subtractHours(toStartOfHour(time_point), 12) + AND scrape_ts_microseconds < toStartOfDay(time_point) + """, + query_id=query_id, + ) + assert int(selected) < 4500, selected + assert int(selected) > 2500, selected + + node.query("SYSTEM FLUSH LOGS") + profile_events = node.query( + f""" + SELECT ProfileEvents + FROM system.query_log + WHERE query_id='{insert_query_id}' + AND type='QueryFinish' + """ + ) + + memory_usage = get_memory_usage(node, insert_query_id) + memory = 123507857 + assert int(memory_usage) < 1.2 * memory, f"{memory_usage} {profile_events}" + assert int(memory_usage) > 0.8 * memory, f"{memory_usage} {profile_events}" + + node.query(f"DROP TABLE IF EXISTS central_query_log SYNC") + remove_all_s3_objects(cluster) From 4ac090b12212567f7b9d30cb27132f566ae438c4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 13 Jun 2023 11:01:44 +0000 Subject: [PATCH 130/515] Fix tests --- src/Coordination/tests/gtest_coordination.cpp | 22 +++++++++---------- .../StorageSystemZooKeeperConnection.cpp | 4 ++-- tests/config/config.d/keeper_port.xml | 4 ++++ .../test_keeper_four_word_command/test.py | 8 +++---- ...2735_system_zookeeper_connection.reference | 2 +- 5 files changed, 22 insertions(+), 18 deletions(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 2793b23c572..b29d5bca43d 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1193,7 +1193,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 2); EXPECT_EQ(snapshot.session_id, 7); - EXPECT_EQ(snapshot.snapshot_container_size, 5); + EXPECT_EQ(snapshot.snapshot_container_size, 6); EXPECT_EQ(snapshot.session_and_timeout.size(), 2); auto buf = manager.serializeSnapshotToBuffer(snapshot); @@ -1205,7 +1205,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) auto [restored_storage, snapshot_meta, _] = manager.deserializeSnapshotFromBuffer(debuf); - EXPECT_EQ(restored_storage->container.size(), 5); + EXPECT_EQ(restored_storage->container.size(), 6); EXPECT_EQ(restored_storage->container.getValue("/").getChildren().size(), 2); EXPECT_EQ(restored_storage->container.getValue("/hello").getChildren().size(), 1); EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getChildren().size(), 0); @@ -1237,14 +1237,14 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) DB::KeeperStorageSnapshot snapshot(&storage, 50); EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 50); - EXPECT_EQ(snapshot.snapshot_container_size, 53); + EXPECT_EQ(snapshot.snapshot_container_size, 54); for (size_t i = 50; i < 100; ++i) { addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); } - EXPECT_EQ(storage.container.size(), 103); + EXPECT_EQ(storage.container.size(), 104); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 50); @@ -1254,7 +1254,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) auto debuf = manager.deserializeSnapshotBufferFromDisk(50); auto [restored_storage, meta, _] = manager.deserializeSnapshotFromBuffer(debuf); - EXPECT_EQ(restored_storage->container.size(), 53); + EXPECT_EQ(restored_storage->container.size(), 54); for (size_t i = 0; i < 50; ++i) { EXPECT_EQ(restored_storage->container.getValue("/hello_" + std::to_string(i)).getData(), "world_" + std::to_string(i)); @@ -1293,7 +1293,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) auto [restored_storage, meta, _] = manager.restoreFromLatestSnapshot(); - EXPECT_EQ(restored_storage->container.size(), 253); + EXPECT_EQ(restored_storage->container.size(), 254); for (size_t i = 0; i < 250; ++i) { @@ -1327,16 +1327,16 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) if (i % 2 == 0) storage.container.erase("/hello_" + std::to_string(i)); } - EXPECT_EQ(storage.container.size(), 28); - EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 104); + EXPECT_EQ(storage.container.size(), 29); + EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 105); EXPECT_EQ(storage.container.snapshotSizeWithVersion().second, 1); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 50); } EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + params.extension)); - EXPECT_EQ(storage.container.size(), 28); + EXPECT_EQ(storage.container.size(), 29); storage.clearGarbageAfterSnapshot(); - EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 28); + EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 29); for (size_t i = 0; i < 50; ++i) { if (i % 2 != 0) @@ -1865,7 +1865,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) auto [restored_storage, snapshot_meta, _] = new_manager.deserializeSnapshotFromBuffer(debuf); - EXPECT_EQ(restored_storage->container.size(), 5); + EXPECT_EQ(restored_storage->container.size(), 6); EXPECT_EQ(restored_storage->container.getValue("/").getChildren().size(), 2); EXPECT_EQ(restored_storage->container.getValue("/hello").getChildren().size(), 1); EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getChildren().size(), 0); diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index 99872be6ba0..9a6a592f2c4 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -31,7 +31,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(context->getZooKeeper()->getConnectedZooKeeperIndex()); res_columns[4]->insert(context->getZooKeeperSessionUptime()); res_columns[5]->insert(context->getZooKeeper()->expired()); - res_columns[6]->insert(static_cast(KeeperApiVersion::WITH_MULTI_READ)); + res_columns[6]->insert(0); res_columns[7]->insert(context->getZooKeeper()->getClientID()); for (const auto & elem : context->getAuxiliaryZooKeepers()) @@ -42,7 +42,7 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[3]->insert(elem.second->getConnectedZooKeeperIndex()); res_columns[4]->insert(elem.second->getSessionUptime()); res_columns[5]->insert(elem.second->expired()); - res_columns[6]->insert(static_cast(KeeperApiVersion::WITH_MULTI_READ)); + res_columns[6]->insert(0); res_columns[7]->insert(elem.second->getClientID()); } diff --git a/tests/config/config.d/keeper_port.xml b/tests/config/config.d/keeper_port.xml index cffd325e968..7db174c5419 100644 --- a/tests/config/config.d/keeper_port.xml +++ b/tests/config/config.d/keeper_port.xml @@ -28,5 +28,9 @@ 9234 + + + 1 + diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 2098daea5fe..adc29abb584 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -183,8 +183,8 @@ def test_cmd_mntr(started_cluster): # contains: # 10 nodes created by test # 3 nodes created by clickhouse "/clickhouse/task_queue/ddl" - # 1 root node, 2 keeper system nodes - assert int(result["zk_znode_count"]) == 13 + # 1 root node, 3 keeper system nodes + assert int(result["zk_znode_count"]) == 14 assert int(result["zk_watch_count"]) == 2 assert int(result["zk_ephemerals_count"]) == 2 assert int(result["zk_approximate_data_size"]) > 0 @@ -329,7 +329,7 @@ def test_cmd_srvr(started_cluster): assert int(result["Connections"]) == 1 assert int(result["Zxid"]) > 10 assert result["Mode"] == "leader" - assert result["Node count"] == "13" + assert result["Node count"] == "14" finally: destroy_zk_client(zk) @@ -369,7 +369,7 @@ def test_cmd_stat(started_cluster): assert int(result["Connections"]) == 1 assert int(result["Zxid"]) >= 10 assert result["Mode"] == "leader" - assert result["Node count"] == "13" + assert result["Node count"] == "14" # filter connection statistics cons = [n for n in data.split("\n") if "=" in n] diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference index 1deabd88b88..55b3579f0dd 100644 --- a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference @@ -1,2 +1,2 @@ -default ::1 9181 0 0 3 +default ::1 9181 0 0 0 zookeeper2 ::1 9181 0 0 0 From 7057e0e25fd55f8a9cb9708da223883aaa8fe902 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 13 Jun 2023 14:46:26 +0000 Subject: [PATCH 131/515] fix test --- .../02751_ip_types_aggregate_functions_states.sql.j2 | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 index 708eeab7724..7daff5a690f 100644 --- a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 @@ -3,8 +3,8 @@ {# this test checks backward compatibility of aggregate functions States against IPv4, IPv6 types #} -{% set ip4_generator = "select number::UInt32::IPv4 ip from numbers(999999999,50) order by ip" %} -{% set ip6_generator = "SELECT toIPv6(IPv6NumToString(toFixedString(reinterpretAsFixedString(number)||reinterpretAsFixedString(number), 16))) AS ip FROM numbers(1010011101, 50) order by ip" %} +{% set ip4_generator = "select num::UInt32::IPv4 ip from (select arrayJoin(range(999999999, number)) as num from numbers(999999999,50)) order by ip" %} +{% set ip6_generator = "SELECT toIPv6(IPv6NumToString(toFixedString(reinterpretAsFixedString(num)||reinterpretAsFixedString(num), 16))) AS ip FROM (select arrayJoin(range(1010011101, number)) as num from numbers(1010011101,50)) order by ip" %} {% set ip_generators = {'ip4': ip4_generator, 'ip6': ip6_generator} %} From 332893344d3cbca205b0d99671cd4c8ba26ec2da Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 13 Jun 2023 16:50:10 +0200 Subject: [PATCH 132/515] Updated lock for accessing azure blob storage iterator --- src/Storages/StorageAzureBlob.cpp | 6 +----- tests/integration/test_storage_azure_blob_storage/test.py | 1 - 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 3ee176a68b7..b9d59f04001 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -882,6 +882,7 @@ StorageAzureBlobSource::Iterator::Iterator( RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() { + std::lock_guard lock(next_mutex); if (is_finished) return {}; @@ -900,7 +901,6 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() { bool need_new_batch = false; { - std::lock_guard lock(next_mutex); need_new_batch = !blobs_with_metadata || index >= blobs_with_metadata->size(); } @@ -945,7 +945,6 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() VirtualColumnUtils::filterBlockWithQuery(query, block, getContext(), filter_ast); const auto & idxs = typeid_cast(*block.getByName("_idx").column); - std::lock_guard lock(next_mutex); blob_path_with_globs.reset(); blob_path_with_globs.emplace(); for (UInt64 idx : idxs.getData()) @@ -961,7 +960,6 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() if (outer_blobs) outer_blobs->insert(outer_blobs->end(), new_batch.begin(), new_batch.end()); - std::lock_guard lock(next_mutex); blobs_with_metadata = std::move(new_batch); for (const auto & [_, info] : *blobs_with_metadata) total_size.fetch_add(info.size_bytes, std::memory_order_relaxed); @@ -969,8 +967,6 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() } size_t current_index = index.fetch_add(1, std::memory_order_relaxed); - - std::lock_guard lock(next_mutex); return (*blobs_with_metadata)[current_index]; } } diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index f9d337b6d86..bb25ac4b029 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -551,7 +551,6 @@ def test_schema_inference_no_globs_tf(cluster): "499500\t2890\t332833500\ttest_schema_inference_no_globs_tf.csv\tcont/test_schema_inference_no_globs_tf.csv" ] - def test_schema_inference_from_globs_tf(cluster): node = cluster.instances["node"] unique_prefix = random.randint(1, 10000) From 3fd9911efe37500094532d07a1f6e0eaddaca6dd Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 13 Jun 2023 14:58:55 +0000 Subject: [PATCH 133/515] fix test reference --- ...es_aggregate_functions_states.reference.j2 | 164 +++++++++--------- 1 file changed, 82 insertions(+), 82 deletions(-) diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 index 481dd723b66..90f98cf63fd 100644 --- a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 @@ -2,18 +2,18 @@ Row 1: ────── minState: 12535288824949910799 -maxState: 18210943739258811465 +maxState: 15790547582231549775 first_valueState: 12535288824949910799 -last_valueState: 18210943739258811465 -topKState: 1594227852744382511 -groupArrayState: 8025417272361615478 -groupUniqArrayState: 919082878249747568 -uniqState: 14828781561416784358 -uniqExactState: 11391659146320471795 -uniqCombinedState: 9631896280254268221 -uniqCombined64State: 5156097536649078816 -uniqHLL12State: 9696624347265201099 -uniqThetaState: 10464560810701154023 +last_valueState: 15790547582231549775 +topKState: 4906125994014190470 +groupArrayState: 9875990674330641453 +groupUniqArrayState: 15788623094139372318 +uniqState: 13857072740564896081 +uniqExactState: 2764760571052675772 +uniqCombinedState: 927481020821507998 +uniqCombined64State: 11588178464185397904 +uniqHLL12State: 592773541433144605 +uniqThetaState: 12573391720108828030 ----- hash / State / ip6 ----- Row 1: ────── @@ -21,31 +21,31 @@ minState: 9428555662807296659 maxState: 18253481702148601156 first_valueState: 9428555662807296659 last_valueState: 18253481702148601156 -topKState: 8045294331733869941 -groupArrayState: 10451014709837753966 -groupUniqArrayState: 1954028114836070615 -uniqState: 14986562136250471284 -uniqExactState: 10032843621916709112 -uniqCombinedState: 6379274083567016598 -uniqCombined64State: 6379274083567016598 -uniqHLL12State: 9181286681186915812 -uniqThetaState: 2415188383468008881 +topKState: 4649040466816645853 +groupArrayState: 15334593495826890008 +groupUniqArrayState: 18179202420787216155 +uniqState: 1113473461736161202 +uniqExactState: 17291302630176581193 +uniqCombinedState: 7689865507370303115 +uniqCombined64State: 7689865507370303115 +uniqHLL12State: 12630756003012135681 +uniqThetaState: 11768246996604802350 ----- finalizeAggregation / State / ip4 ----- Row 1: ────── min: 59.154.201.255 -max: 59.154.202.48 +max: 59.154.202.47 first_value: 59.154.201.255 -last_value: 59.154.202.48 -topK: ['59.154.202.48','59.154.202.5','59.154.202.26','59.154.202.25','59.154.202.24','59.154.202.23','59.154.202.22','59.154.202.21','59.154.202.27','59.154.202.19'] -groupArray: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8','59.154.202.9','59.154.202.10','59.154.202.11','59.154.202.12','59.154.202.13','59.154.202.14','59.154.202.15','59.154.202.16','59.154.202.17','59.154.202.18','59.154.202.19','59.154.202.20','59.154.202.21','59.154.202.22','59.154.202.23','59.154.202.24','59.154.202.25','59.154.202.26','59.154.202.27','59.154.202.28','59.154.202.29','59.154.202.30','59.154.202.31','59.154.202.32','59.154.202.33','59.154.202.34','59.154.202.35','59.154.202.36','59.154.202.37','59.154.202.38','59.154.202.39','59.154.202.40','59.154.202.41','59.154.202.42','59.154.202.43','59.154.202.44','59.154.202.45','59.154.202.46','59.154.202.47','59.154.202.48'] -groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.48','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] -uniq: 50 -uniqExact: 50 -uniqCombined: 50 -uniqCombined64: 50 -uniqHLL12: 49 -uniqTheta: 50 +last_value: 59.154.202.47 +topK: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8'] +groupArray: ['59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.44','59.154.202.44','59.154.202.44','59.154.202.44','59.154.202.45','59.154.202.45','59.154.202.45','59.154.202.46','59.154.202.46','59.154.202.47'] +groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] +uniq: 49 +uniqExact: 49 +uniqCombined: 49 +uniqCombined64: 49 +uniqHLL12: 48 +uniqTheta: 49 ----- finalizeAggregation / State / ip6 ----- Row 1: ────── @@ -53,31 +53,31 @@ min: 8c:333c::8c:333c:0:0 max: ff8b:333c::ff8b:333c:0:0 first_value: 8c:333c::8c:333c:0:0 last_value: ff8b:333c::ff8b:333c:0:0 -topK: ['ff8b:333c::ff8b:333c:0:0','68c:333c::68c:333c:0:0','e98b:333c::e98b:333c:0:0','e88b:333c::e88b:333c:0:0','e78b:333c::e78b:333c:0:0','e68b:333c::e68b:333c:0:0','e58b:333c::e58b:333c:0:0','e48b:333c::e48b:333c:0:0','ea8b:333c::ea8b:333c:0:0','e28b:333c::e28b:333c:0:0'] -groupArray: ['8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','e8c:333c::e8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] -groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e8c:333c::e8c:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] -uniq: 50 -uniqExact: 50 -uniqCombined: 50 -uniqCombined64: 50 -uniqHLL12: 50 -uniqTheta: 50 +topK: ['dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0'] +groupArray: ['8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','b8c:333c::b8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +uniq: 49 +uniqExact: 49 +uniqCombined: 49 +uniqCombined64: 49 +uniqHLL12: 49 +uniqTheta: 49 ----- hash / IfState / ip4 ----- Row 1: ────── minIfState: 12535288824949910799 -maxIfState: 18210943739258811465 +maxIfState: 15790547582231549775 first_valueIfState: 12535288824949910799 -last_valueIfState: 18210943739258811465 -topKIfState: 1594227852744382511 -groupArrayIfState: 8025417272361615478 -groupUniqArrayIfState: 919082878249747568 -uniqIfState: 14828781561416784358 -uniqExactIfState: 11391659146320471795 -uniqCombinedIfState: 9631896280254268221 -uniqCombined64IfState: 5156097536649078816 -uniqHLL12IfState: 9696624347265201099 -uniqThetaIfState: 10464560810701154023 +last_valueIfState: 15790547582231549775 +topKIfState: 4906125994014190470 +groupArrayIfState: 9875990674330641453 +groupUniqArrayIfState: 15788623094139372318 +uniqIfState: 13857072740564896081 +uniqExactIfState: 2764760571052675772 +uniqCombinedIfState: 927481020821507998 +uniqCombined64IfState: 11588178464185397904 +uniqHLL12IfState: 592773541433144605 +uniqThetaIfState: 12573391720108828030 ----- hash / IfState / ip6 ----- Row 1: ────── @@ -85,31 +85,31 @@ minIfState: 9428555662807296659 maxIfState: 18253481702148601156 first_valueIfState: 9428555662807296659 last_valueIfState: 18253481702148601156 -topKIfState: 8045294331733869941 -groupArrayIfState: 10451014709837753966 -groupUniqArrayIfState: 1954028114836070615 -uniqIfState: 14986562136250471284 -uniqExactIfState: 10032843621916709112 -uniqCombinedIfState: 6379274083567016598 -uniqCombined64IfState: 6379274083567016598 -uniqHLL12IfState: 9181286681186915812 -uniqThetaIfState: 2415188383468008881 +topKIfState: 4649040466816645853 +groupArrayIfState: 15334593495826890008 +groupUniqArrayIfState: 18179202420787216155 +uniqIfState: 1113473461736161202 +uniqExactIfState: 17291302630176581193 +uniqCombinedIfState: 7689865507370303115 +uniqCombined64IfState: 7689865507370303115 +uniqHLL12IfState: 12630756003012135681 +uniqThetaIfState: 11768246996604802350 ----- finalizeAggregation / IfState / ip4 ----- Row 1: ────── min: 59.154.201.255 -max: 59.154.202.48 +max: 59.154.202.47 first_value: 59.154.201.255 -last_value: 59.154.202.48 -topK: ['59.154.202.48','59.154.202.5','59.154.202.26','59.154.202.25','59.154.202.24','59.154.202.23','59.154.202.22','59.154.202.21','59.154.202.27','59.154.202.19'] -groupArray: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8','59.154.202.9','59.154.202.10','59.154.202.11','59.154.202.12','59.154.202.13','59.154.202.14','59.154.202.15','59.154.202.16','59.154.202.17','59.154.202.18','59.154.202.19','59.154.202.20','59.154.202.21','59.154.202.22','59.154.202.23','59.154.202.24','59.154.202.25','59.154.202.26','59.154.202.27','59.154.202.28','59.154.202.29','59.154.202.30','59.154.202.31','59.154.202.32','59.154.202.33','59.154.202.34','59.154.202.35','59.154.202.36','59.154.202.37','59.154.202.38','59.154.202.39','59.154.202.40','59.154.202.41','59.154.202.42','59.154.202.43','59.154.202.44','59.154.202.45','59.154.202.46','59.154.202.47','59.154.202.48'] -groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.48','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] -uniq: 50 -uniqExact: 50 -uniqCombined: 50 -uniqCombined64: 50 -uniqHLL12: 49 -uniqTheta: 50 +last_value: 59.154.202.47 +topK: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8'] +groupArray: ['59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.44','59.154.202.44','59.154.202.44','59.154.202.44','59.154.202.45','59.154.202.45','59.154.202.45','59.154.202.46','59.154.202.46','59.154.202.47'] +groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] +uniq: 49 +uniqExact: 49 +uniqCombined: 49 +uniqCombined64: 49 +uniqHLL12: 48 +uniqTheta: 49 ----- finalizeAggregation / IfState / ip6 ----- Row 1: ────── @@ -117,20 +117,20 @@ min: 8c:333c::8c:333c:0:0 max: ff8b:333c::ff8b:333c:0:0 first_value: 8c:333c::8c:333c:0:0 last_value: ff8b:333c::ff8b:333c:0:0 -topK: ['ff8b:333c::ff8b:333c:0:0','68c:333c::68c:333c:0:0','e98b:333c::e98b:333c:0:0','e88b:333c::e88b:333c:0:0','e78b:333c::e78b:333c:0:0','e68b:333c::e68b:333c:0:0','e58b:333c::e58b:333c:0:0','e48b:333c::e48b:333c:0:0','ea8b:333c::ea8b:333c:0:0','e28b:333c::e28b:333c:0:0'] -groupArray: ['8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','e8c:333c::e8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] -groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e8c:333c::e8c:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] -uniq: 50 -uniqExact: 50 -uniqCombined: 50 -uniqCombined64: 50 -uniqHLL12: 50 -uniqTheta: 50 +topK: ['dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0'] +groupArray: ['8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','b8c:333c::b8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +uniq: 49 +uniqExact: 49 +uniqCombined: 49 +uniqCombined64: 49 +uniqHLL12: 49 +uniqTheta: 49 ----- Arg / hash / State / ip4 ----- Row 1: ────── argMinState: 13774589216353164344 -argMaxState: 9177365218111013695 +argMaxState: 1131101663917664667 ----- Arg / hash / State / ip6 ----- Row 1: ────── @@ -140,7 +140,7 @@ argMaxState: 16598449636475438091 Row 1: ────── argMinState: 59.154.201.255 -argMaxState: 59.154.202.48 +argMaxState: 59.154.202.47 ----- Arg / finalizeAggregation / State / ip6 ----- Row 1: ────── From 478bad32376ff2787e83107c8274f4b743c569ac Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 13 Jun 2023 15:04:11 +0000 Subject: [PATCH 134/515] Automatic style fix --- tests/integration/test_storage_azure_blob_storage/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index bb25ac4b029..f9d337b6d86 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -551,6 +551,7 @@ def test_schema_inference_no_globs_tf(cluster): "499500\t2890\t332833500\ttest_schema_inference_no_globs_tf.csv\tcont/test_schema_inference_no_globs_tf.csv" ] + def test_schema_inference_from_globs_tf(cluster): node = cluster.instances["node"] unique_prefix = random.randint(1, 10000) From bc4724490239ea34b4924da17ddabbb1f90e2bee Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 13 Jun 2023 17:06:40 +0200 Subject: [PATCH 135/515] Updated tests for CI checks --- tests/integration/test_storage_azure_blob_storage/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index f9d337b6d86..8ab5d416b03 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -558,6 +558,7 @@ def test_schema_inference_from_globs_tf(cluster): node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" max_path = "" + for i in range(10): for j in range(10): path = "{}/{}_{}/{}.csv".format( From 8dde50eb3aeef2bd83a1c7c0ec244dfe1c66bf32 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 13 Jun 2023 15:19:16 +0000 Subject: [PATCH 136/515] Automatic style fix --- tests/integration/test_storage_azure_blob_storage/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 8ab5d416b03..3d9c751be3c 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -558,7 +558,7 @@ def test_schema_inference_from_globs_tf(cluster): node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" max_path = "" - + for i in range(10): for j in range(10): path = "{}/{}_{}/{}.csv".format( From 9a5c5c7e931c41f406e316cb20cc7659c40fccc5 Mon Sep 17 00:00:00 2001 From: Yuriy Chernyshov Date: Tue, 13 Jun 2023 19:29:41 +0300 Subject: [PATCH 137/515] Update contrib/re2 to 2023-06-02 --- contrib/re2 | 2 +- src/Common/OptimizedRegularExpression.cpp | 12 ++++++------ src/Common/parseGlobs.cpp | 2 +- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/contrib/re2 b/contrib/re2 index 13ebb377c6a..03da4fc0857 160000 --- a/contrib/re2 +++ b/contrib/re2 @@ -1 +1 @@ -Subproject commit 13ebb377c6ad763ca61d12dd6f88b1126bd0b911 +Subproject commit 03da4fc0857c285e3a26782f6bc8931c4c950df4 diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index f2fe922ef19..5df9ce76098 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -669,16 +669,16 @@ unsigned OptimizedRegularExpressionImpl::match(const char * subject matches.resize(limit); for (size_t i = 0; i < limit; ++i) { - if (pieces[i] != nullptr) - { - matches[i].offset = pieces[i].data() - subject; - matches[i].length = pieces[i].length(); - } - else + if (pieces[i].empty()) { matches[i].offset = std::string::npos; matches[i].length = 0; } + else + { + matches[i].offset = pieces[i].data() - subject; + matches[i].length = pieces[i].length(); + } } return limit; } diff --git a/src/Common/parseGlobs.cpp b/src/Common/parseGlobs.cpp index f8d331c2b76..07cce38afff 100644 --- a/src/Common/parseGlobs.cpp +++ b/src/Common/parseGlobs.cpp @@ -40,7 +40,7 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob size_t current_index = 0; while (RE2::FindAndConsume(&input, enum_or_range, &matched)) { - std::string buffer = matched.ToString(); + std::string buffer{matched}; oss_for_replacing << escaped_with_globs.substr(current_index, matched.data() - escaped_with_globs.data() - current_index - 1) << '('; if (buffer.find(',') == std::string::npos) From e7d2cc4a1a44e9c52d16ede99612e28342735c15 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 13 Jun 2023 22:11:23 +0000 Subject: [PATCH 138/515] add map functions tests --- ...es_aggregate_functions_states.reference.j2 | 48 ++++++++++++++ ...ip_types_aggregate_functions_states.sql.j2 | 64 ++++++++++++++++--- 2 files changed, 102 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 index 90f98cf63fd..03b0e065151 100644 --- a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 @@ -170,3 +170,51 @@ Row 1: any: dd8b:333c::dd8b:333c:0:0 anyHeavy: dd8b:333c::dd8b:333c:0:0 anyLast: dd8b:333c::dd8b:333c:0:0 +----- Map/Map hash / State / ip4 ----- +Row 1: +────── +sumMapState: 9327034461443333306 +minMapState: 17403430892851901033 +maxMapState: 17403430892851901033 +----- Map/Map hash / State / ip6 ----- +Row 1: +────── +sumMapState: 5204122423200337352 +minMapState: 14534921151579960284 +maxMapState: 14534921151579960284 +----- Map/Map finalizeAggregation / State / ip4 ----- +Row 1: +────── +sumMap: {'59.154.201.255':1} +minMap: {'59.154.201.255':1} +maxMap: {'59.154.201.255':1} +----- Map/Map finalizeAggregation / State / ip6 ----- +Row 1: +────── +sumMap: {'dd8b:333c::dd8b:333c:0:0':1} +minMap: {'dd8b:333c::dd8b:333c:0:0':1} +maxMap: {'dd8b:333c::dd8b:333c:0:0':1} +----- Map/Array hash / State / ip4 ----- +Row 1: +────── +sumMapState: 9327034461443333306 +minMapState: 9327034461443333306 +maxMapState: 9327034461443333306 +----- Map/Array hash / State / ip6 ----- +Row 1: +────── +sumMapState: 15555709096566410627 +minMapState: 15555709096566410627 +maxMapState: 15555709096566410627 +----- Map/Array finalizeAggregation / State / ip4 ----- +Row 1: +────── +sumMap: (['59.154.201.255'],[1]) +minMap: (['59.154.201.255'],[1]) +maxMap: (['59.154.201.255'],[1]) +----- Map/Array finalizeAggregation / State / ip6 ----- +Row 1: +────── +sumMap: (['dd8b:333c::dd8b:333c:0:0'],[1]) +minMap: (['dd8b:333c::dd8b:333c:0:0'],[1]) +maxMap: (['dd8b:333c::dd8b:333c:0:0'],[1]) diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 index 7daff5a690f..7d030d4be2d 100644 --- a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 @@ -1,5 +1,4 @@ --- Tags: no-fasttest --- no-fasttest because uniqTheta +-- Tags: no-parallel, no-fasttest {# this test checks backward compatibility of aggregate functions States against IPv4, IPv6 types #} @@ -22,7 +21,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - {% for generator_name, ip_generator in ip_generators.items() %} select '----- finalizeAggregation / State / {{ generator_name }} -----'; @@ -35,7 +33,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - {% for generator_name, ip_generator in ip_generators.items() %} select '----- hash / IfState / {{ generator_name }} -----'; @@ -48,7 +45,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - {% for generator_name, ip_generator in ip_generators.items() %} select '----- finalizeAggregation / IfState / {{ generator_name }} -----'; @@ -61,8 +57,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - - {% set agg_func_list = [ "argMin", "argMax" ] %} {% for generator_name, ip_generator in ip_generators.items() %} @@ -77,8 +71,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - - {% for generator_name, ip_generator in ip_generators.items() %} select '----- Arg / finalizeAggregation / State / {{ generator_name }} -----'; @@ -91,6 +83,8 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} + + {# let's test functions with not deterministic result against 1 row, to make it deterministic #} {% set ip4_generator = "select number::UInt32::IPv4 ip from numbers(999999999,1) order by ip" %} {% set ip6_generator = "SELECT toIPv6(IPv6NumToString(toFixedString(reinterpretAsFixedString(number)||reinterpretAsFixedString(number), 16))) AS ip FROM numbers(1010011101, 1) order by ip" %} @@ -112,7 +106,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - {% for generator_name, ip_generator in ip_generators.items() %} select '----- finalizeAggregation / State / {{ generator_name }} -----'; @@ -124,3 +117,54 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} + + +{% set agg_func_list = [ "sumMap", "minMap", "maxMap" ] %} + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Map/Map hash / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}State(map(ip, 1::Int64)) )) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Map/Map finalizeAggregation / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}State(map(ip, 1::Int64)) ) AS {{ func }}{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Map/Array hash / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}State([ip], [1::Int64]) )) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Map/Array finalizeAggregation / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}State([ip], [1::Int64]) ) AS {{ func }}{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} From 0a1d0c4abd0f8ece5af6f3a3d5ccc5207dfff0f2 Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Tue, 13 Jun 2023 23:36:14 -0700 Subject: [PATCH 139/515] Fix one stateless test. --- src/Client/ClientBase.cpp | 5 ++--- tests/queries/0_stateless/02346_into_outfile_and_stdout.sh | 2 +- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index fc108b8f57d..b746d46148c 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -596,14 +596,13 @@ try range.second); } - std::error_code ec; - if (std::filesystem::is_regular_file(out_file, ec)) + if (fs::exists(out_file)) { if (!query_with_output->is_outfile_append && !query_with_output->is_outfile_truncate) { throw Exception( ErrorCodes::CANNOT_OPEN_FILE, - "File {} exists, consider using 'INTO OUTFILE ... APPEND' or 'INTO OUTFILE ... TRUNCATE' if appropriate.", + "File {} exists, consider using APPEND or TRUNCATE if appropriate.", out_file); } } diff --git a/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh b/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh index 021dc9125d4..8ec086c97ef 100755 --- a/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh +++ b/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh @@ -66,7 +66,7 @@ performBadQuery "bad_query_incorrect_usage" "SELECT 1, 2, 3 INTO OUTFILE AND STD performBadQuery "bad_query_no_into_outfile" "SELECT 1, 2, 3 AND STDOUT'" "SYNTAX_ERROR" -performFileExists "bad_query_file_exists" "SELECT 1, 2, 3 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists.out' AND STDOUT" "File exists. (CANNOT_OPEN_FILE)" +performFileExists "bad_query_file_exists" "SELECT 1, 2, 3 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists.out' AND STDOUT" "File ${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists exists, consider using APPEND or TRUNCATE if appropriate. (CANNOT_OPEN_FILE)" performCompression "compression" "SELECT * FROM (SELECT 'Hello, World! From clickhouse.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_compression.gz' AND STDOUT COMPRESSION 'GZ' FORMAT TabSeparated" From 1663905acdafcce577d58852a0835fdb760750e8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 14 Jun 2023 08:48:16 +0000 Subject: [PATCH 140/515] Hold context in pipeline resources. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 1 + src/QueryPipeline/QueryPipelineBuilder.h | 1 + 2 files changed, 2 insertions(+) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 8f610eb4380..bbea1d38fb1 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1884,6 +1884,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons processors.emplace_back(processor); pipeline.init(std::move(pipe)); + pipeline.addContext(context); // Attach QueryIdHolder if needed if (query_id_holder) pipeline.setQueryIdHolder(std::move(query_id_holder)); diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index da8443a7e33..134de59520d 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -185,6 +185,7 @@ public: void addResources(QueryPlanResourceHolder resources_) { resources = std::move(resources_); } void setQueryIdHolder(std::shared_ptr query_id_holder) { resources.query_id_holders.emplace_back(std::move(query_id_holder)); } + void addContext(ContextPtr context) { resources.interpreter_context.emplace_back(std::move(context)); } /// Convert query pipeline to pipe. static Pipe getPipe(QueryPipelineBuilder pipeline, QueryPlanResourceHolder & resources); From 918b8c4585025e8a357115945cce501c05d31be1 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 14 Jun 2023 10:51:59 +0200 Subject: [PATCH 141/515] Updated filename in test --- tests/integration/test_storage_azure_blob_storage/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 3d9c751be3c..e99ae72eb8b 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -583,13 +583,13 @@ def test_partition_by_tf(cluster): table_format = "column1 UInt32, column2 UInt32, column3 UInt32" partition_by = "column3" values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" - filename = "test_tf_{_partition_id}.csv" + filename = "test_partition_tf_{_partition_id}.csv" azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}", ) - assert "1,2,3\n" == get_azure_file_content("test_tf_3.csv") - assert "3,2,1\n" == get_azure_file_content("test_tf_1.csv") - assert "78,43,45\n" == get_azure_file_content("test_tf_45.csv") + assert "1,2,3\n" == get_azure_file_content("test_partition_tf_3.csv") + assert "3,2,1\n" == get_azure_file_content("test_partition_tf_1.csv") + assert "78,43,45\n" == get_azure_file_content("test_partition_tfs_45.csv") From c98a194b571e8c39504afc829fa91492f4dcbe2d Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 14 Jun 2023 11:00:11 +0200 Subject: [PATCH 142/515] Updated unique names for test to avoid same names by random numbers --- tests/integration/test_storage_azure_blob_storage/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index e99ae72eb8b..e2077f8face 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -300,10 +300,10 @@ def test_put_get_with_globs(cluster): azure_query( node, - f"CREATE TABLE test_{i}_{j} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV')", + f"CREATE TABLE test_put_{i}_{j} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV')", ) - query = f"insert into test_{i}_{j} VALUES {values}" + query = f"insert into test_put_{i}_{j} VALUES {values}" azure_query(node, query) azure_query( @@ -332,9 +332,9 @@ def test_azure_glob_scheherazade(cluster): unique_num = random.randint(1, 10000) azure_query( node, - f"CREATE TABLE test_{i}_{unique_num} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV')", + f"CREATE TABLE test_scheherazade_{i}_{unique_num} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV')", ) - query = f"insert into test_{i}_{unique_num} VALUES {values}" + query = f"insert into test_scheherazade_{i}_{unique_num} VALUES {values}" azure_query(node, query) jobs.append( From f6bad2c064efeb997755be8b9f313a3859a6f81f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 14 Jun 2023 09:13:38 +0000 Subject: [PATCH 143/515] Automatic style fix --- tests/integration/test_storage_azure_blob_storage/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index e2077f8face..0002ccbf483 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -334,7 +334,9 @@ def test_azure_glob_scheherazade(cluster): node, f"CREATE TABLE test_scheherazade_{i}_{unique_num} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV')", ) - query = f"insert into test_scheherazade_{i}_{unique_num} VALUES {values}" + query = ( + f"insert into test_scheherazade_{i}_{unique_num} VALUES {values}" + ) azure_query(node, query) jobs.append( From f9f1e870c8468b829b4aa449c8a9b3736b733056 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 14 Jun 2023 10:09:01 +0000 Subject: [PATCH 144/515] Fix build --- contrib/azure-cmake/CMakeLists.txt | 2 +- programs/keeper/CMakeLists.txt | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/contrib/azure-cmake/CMakeLists.txt b/contrib/azure-cmake/CMakeLists.txt index 887122e7653..a1dd4664e79 100644 --- a/contrib/azure-cmake/CMakeLists.txt +++ b/contrib/azure-cmake/CMakeLists.txt @@ -1,6 +1,6 @@ option (ENABLE_AZURE_BLOB_STORAGE "Enable Azure blob storage" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_AZURE_BLOB_STORAGE OR BUILD_STANDALONE_KEEPER OR OS_FREEBSD OR (NOT ARCH_AMD64)) +if (NOT ENABLE_AZURE_BLOB_STORAGE OR OS_FREEBSD OR (NOT ARCH_AMD64)) message(STATUS "Not using Azure blob storage") return() endif() diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 90f4f870df6..a3a8a769bff 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -148,7 +148,6 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteBufferFromTemporaryFile.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteBufferWithFinalizeCallback.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/AsynchronousBoundedReadBuffer.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/getThreadPoolReader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolRemoteFSReader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolReader.cpp From 4096c082b9f7bd49664e6341e4862c1aed3d7a73 Mon Sep 17 00:00:00 2001 From: Yuriy Chernyshov Date: Wed, 14 Jun 2023 15:09:17 +0300 Subject: [PATCH 145/515] Cleanup RE2_SOURCES --- contrib/re2-cmake/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/contrib/re2-cmake/CMakeLists.txt b/contrib/re2-cmake/CMakeLists.txt index 19939c11ebf..a081f92bc94 100644 --- a/contrib/re2-cmake/CMakeLists.txt +++ b/contrib/re2-cmake/CMakeLists.txt @@ -28,7 +28,6 @@ set(RE2_SOURCES ${SRC_DIR}/re2/regexp.cc ${SRC_DIR}/re2/set.cc ${SRC_DIR}/re2/simplify.cc - ${SRC_DIR}/re2/stringpiece.cc ${SRC_DIR}/re2/tostring.cc ${SRC_DIR}/re2/unicode_casefold.cc ${SRC_DIR}/re2/unicode_groups.cc From 7d59af1f776f9868f5b45c0c78e9e5a49d118dd3 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 14 Jun 2023 13:19:03 +0200 Subject: [PATCH 146/515] test compressed write to S3 --- src/IO/BrotliWriteBuffer.cpp | 5 +--- src/IO/BrotliWriteBuffer.h | 1 + src/IO/Bzip2WriteBuffer.cpp | 5 +--- src/IO/LZMADeflatingWriteBuffer.cpp | 5 ---- src/IO/LZMADeflatingWriteBuffer.h | 2 -- src/IO/Lz4DeflatingWriteBuffer.cpp | 15 +++++----- src/IO/Lz4DeflatingWriteBuffer.h | 2 -- src/IO/ZlibDeflatingWriteBuffer.cpp | 12 -------- src/IO/ZlibDeflatingWriteBuffer.h | 2 -- src/IO/ZstdDeflatingWriteBuffer.cpp | 6 ---- src/IO/ZstdDeflatingWriteBuffer.h | 2 -- .../test_checking_s3_blobs_paranoid/test.py | 28 ++++++++++++------- 12 files changed, 29 insertions(+), 56 deletions(-) diff --git a/src/IO/BrotliWriteBuffer.cpp b/src/IO/BrotliWriteBuffer.cpp index 47426d62a6e..a19c6770dad 100644 --- a/src/IO/BrotliWriteBuffer.cpp +++ b/src/IO/BrotliWriteBuffer.cpp @@ -42,10 +42,7 @@ BrotliWriteBuffer::BrotliWriteBuffer(std::unique_ptr out_, int comp BrotliEncoderSetParameter(brotli->state, BROTLI_PARAM_LGWIN, 24); } -BrotliWriteBuffer::~BrotliWriteBuffer() -{ - finalize(); -} +BrotliWriteBuffer::~BrotliWriteBuffer() = default; void BrotliWriteBuffer::nextImpl() { diff --git a/src/IO/BrotliWriteBuffer.h b/src/IO/BrotliWriteBuffer.h index e03fa1507ba..8cbc78bd9e7 100644 --- a/src/IO/BrotliWriteBuffer.h +++ b/src/IO/BrotliWriteBuffer.h @@ -27,6 +27,7 @@ private: class BrotliStateWrapper; std::unique_ptr brotli; + size_t in_available; const uint8_t * in_data; diff --git a/src/IO/Bzip2WriteBuffer.cpp b/src/IO/Bzip2WriteBuffer.cpp index 4b6bed70d35..b84cbdd1e41 100644 --- a/src/IO/Bzip2WriteBuffer.cpp +++ b/src/IO/Bzip2WriteBuffer.cpp @@ -45,10 +45,7 @@ Bzip2WriteBuffer::Bzip2WriteBuffer(std::unique_ptr out_, int compre { } -Bzip2WriteBuffer::~Bzip2WriteBuffer() -{ - finalize(); -} +Bzip2WriteBuffer::~Bzip2WriteBuffer() = default; void Bzip2WriteBuffer::nextImpl() { diff --git a/src/IO/LZMADeflatingWriteBuffer.cpp b/src/IO/LZMADeflatingWriteBuffer.cpp index 30e247b1016..c534a247bc3 100644 --- a/src/IO/LZMADeflatingWriteBuffer.cpp +++ b/src/IO/LZMADeflatingWriteBuffer.cpp @@ -44,11 +44,6 @@ LZMADeflatingWriteBuffer::LZMADeflatingWriteBuffer( LZMA_VERSION_STRING); } -LZMADeflatingWriteBuffer::~LZMADeflatingWriteBuffer() -{ - finalize(); -} - void LZMADeflatingWriteBuffer::nextImpl() { if (!offset()) diff --git a/src/IO/LZMADeflatingWriteBuffer.h b/src/IO/LZMADeflatingWriteBuffer.h index 2e135455e00..5a0864d6071 100644 --- a/src/IO/LZMADeflatingWriteBuffer.h +++ b/src/IO/LZMADeflatingWriteBuffer.h @@ -21,8 +21,6 @@ public: char * existing_memory = nullptr, size_t alignment = 0); - ~LZMADeflatingWriteBuffer() override; - private: void nextImpl() override; diff --git a/src/IO/Lz4DeflatingWriteBuffer.cpp b/src/IO/Lz4DeflatingWriteBuffer.cpp index c3a1b8282c3..32241cb3b1a 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.cpp +++ b/src/IO/Lz4DeflatingWriteBuffer.cpp @@ -40,11 +40,6 @@ Lz4DeflatingWriteBuffer::Lz4DeflatingWriteBuffer( LZ4F_VERSION); } -Lz4DeflatingWriteBuffer::~Lz4DeflatingWriteBuffer() -{ - finalize(); -} - void Lz4DeflatingWriteBuffer::nextImpl() { if (!offset()) @@ -107,8 +102,14 @@ void Lz4DeflatingWriteBuffer::nextImpl() if (LZ4F_isError(compressed_size)) throw Exception( ErrorCodes::LZ4_ENCODER_FAILED, - "LZ4 failed to encode stream. LZ4F version: {}", - LZ4F_VERSION); + "LZ4 failed to encode stream. LZ4F version: {}, CodeName: {}," + " in_capacity: {}, out_capacity: {}, cur_buffer_size: {}, min_compressed_block_size: {}", + LZ4F_VERSION, + LZ4F_getErrorName(compressed_size), + in_capacity, + out_capacity, + cur_buffer_size, + min_compressed_block_size); in_capacity -= cur_buffer_size; in_data = reinterpret_cast(working_buffer.end() - in_capacity); diff --git a/src/IO/Lz4DeflatingWriteBuffer.h b/src/IO/Lz4DeflatingWriteBuffer.h index 68873b5f8ee..a6440f8854f 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.h +++ b/src/IO/Lz4DeflatingWriteBuffer.h @@ -21,8 +21,6 @@ public: char * existing_memory = nullptr, size_t alignment = 0); - ~Lz4DeflatingWriteBuffer() override; - private: void nextImpl() override; diff --git a/src/IO/ZlibDeflatingWriteBuffer.cpp b/src/IO/ZlibDeflatingWriteBuffer.cpp index 43bb0405555..d433bdbecd2 100644 --- a/src/IO/ZlibDeflatingWriteBuffer.cpp +++ b/src/IO/ZlibDeflatingWriteBuffer.cpp @@ -72,18 +72,6 @@ void ZlibDeflatingWriteBuffer::nextImpl() } } -ZlibDeflatingWriteBuffer::~ZlibDeflatingWriteBuffer() -{ - try - { - finalize(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -} - void ZlibDeflatingWriteBuffer::finalizeBefore() { next(); diff --git a/src/IO/ZlibDeflatingWriteBuffer.h b/src/IO/ZlibDeflatingWriteBuffer.h index 58e709b54e6..05d6e528a23 100644 --- a/src/IO/ZlibDeflatingWriteBuffer.h +++ b/src/IO/ZlibDeflatingWriteBuffer.h @@ -24,8 +24,6 @@ public: char * existing_memory = nullptr, size_t alignment = 0); - ~ZlibDeflatingWriteBuffer() override; - private: void nextImpl() override; diff --git a/src/IO/ZstdDeflatingWriteBuffer.cpp b/src/IO/ZstdDeflatingWriteBuffer.cpp index c6d2ffc39f9..097647cbafc 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingWriteBuffer.cpp @@ -30,12 +30,6 @@ ZstdDeflatingWriteBuffer::ZstdDeflatingWriteBuffer( output = {nullptr, 0, 0}; } - -ZstdDeflatingWriteBuffer::~ZstdDeflatingWriteBuffer() -{ - finalize(); -} - void ZstdDeflatingWriteBuffer::nextImpl() { if (!offset()) diff --git a/src/IO/ZstdDeflatingWriteBuffer.h b/src/IO/ZstdDeflatingWriteBuffer.h index ba83c18d354..11e34e2890f 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.h +++ b/src/IO/ZstdDeflatingWriteBuffer.h @@ -21,8 +21,6 @@ public: char * existing_memory = nullptr, size_t alignment = 0); - ~ZstdDeflatingWriteBuffer() override; - void sync() override { out->sync(); diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index c0f184815c9..244ca8a2c81 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -84,19 +84,23 @@ def get_counters(node, query_id, log_type="ExceptionWhileProcessing"): ] -def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3): +# Add "lz4" compression method in the list after https://github.com/ClickHouse/ClickHouse/issues/50975 is fixed +@pytest.mark.parametrize( + "compression", ["none", "gzip", "br", "xz", "zstd", "bz2", "deflate"] +) +def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3, compression): node = cluster.instances["node"] broken_s3.setup_error_at_create_multi_part_upload() - insert_query_id = "INSERT_INTO_TABLE_FUNCTION_FAIL_CREATE_MPU" + insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_FAIL_CREATE_MPU_{compression}" error = node.query_and_get_error( - """ + f""" INSERT INTO TABLE FUNCTION s3( 'http://resolver:8083/root/data/test_upload_s3_fail_create_multi_part_upload', 'minio', 'minio123', - 'CSV', auto, 'none' + 'CSV', auto, '{compression}' ) SELECT * @@ -111,7 +115,6 @@ def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3): assert "Code: 499" in error, error assert "mock s3 injected error" in error, error - assert "DB::WriteBufferFromS3::createMultipartUpload()" in error, error count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( node, insert_query_id @@ -121,20 +124,26 @@ def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3): assert count_s3_errors == 1 -def test_upload_s3_fail_upload_part_when_multi_part_upload(cluster, broken_s3): +# Add "lz4" compression method in the list after https://github.com/ClickHouse/ClickHouse/issues/50975 is fixed +@pytest.mark.parametrize( + "compression", ["none", "gzip", "br", "xz", "zstd", "bz2", "deflate"] +) +def test_upload_s3_fail_upload_part_when_multi_part_upload( + cluster, broken_s3, compression +): node = cluster.instances["node"] broken_s3.setup_fake_multpartuploads() broken_s3.setup_error_at_part_upload(count=1, after=2) - insert_query_id = "INSERT_INTO_TABLE_FUNCTION_FAIL_UPLOAD_PART" + insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_FAIL_UPLOAD_PART_{compression}" error = node.query_and_get_error( - """ + f""" INSERT INTO TABLE FUNCTION s3( 'http://resolver:8083/root/data/test_upload_s3_fail_upload_part_when_multi_part_upload', 'minio', 'minio123', - 'CSV', auto, 'none' + 'CSV', auto, '{compression}' ) SELECT * @@ -149,7 +158,6 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload(cluster, broken_s3): assert "Code: 499" in error, error assert "mock s3 injected error" in error, error - assert "DB::WriteBufferFromS3::writePart" in error, error count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( node, insert_query_id From 011d666073968b1a8cbbd867513e4e8adec1362b Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 14 Jun 2023 14:55:34 +0200 Subject: [PATCH 147/515] Fixed typo in tests --- tests/integration/test_storage_azure_blob_storage/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 0002ccbf483..0de325ccd14 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -594,4 +594,4 @@ def test_partition_by_tf(cluster): assert "1,2,3\n" == get_azure_file_content("test_partition_tf_3.csv") assert "3,2,1\n" == get_azure_file_content("test_partition_tf_1.csv") - assert "78,43,45\n" == get_azure_file_content("test_partition_tfs_45.csv") + assert "78,43,45\n" == get_azure_file_content("test_partition_tf_45.csv") From 7f8162e3465318bb9847956edffcd67a4fdb2020 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Wed, 14 Jun 2023 10:42:20 -0400 Subject: [PATCH 148/515] add note regarding -MapState --- docs/en/sql-reference/aggregate-functions/combinators.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/combinators.md b/docs/en/sql-reference/aggregate-functions/combinators.md index fd693430064..a395b350a55 100644 --- a/docs/en/sql-reference/aggregate-functions/combinators.md +++ b/docs/en/sql-reference/aggregate-functions/combinators.md @@ -97,6 +97,10 @@ Result: If you apply this combinator, the aggregate function does not return the resulting value (such as the number of unique values for the [uniq](../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) function), but an intermediate state of the aggregation (for `uniq`, this is the hash table for calculating the number of unique values). This is an `AggregateFunction(...)` that can be used for further processing or stored in a table to finish aggregating later. +:::note +Please notice, that -MapState is not an invariant for the same data due to the fact that order of data in intermediate state can change, though it doesn't impact ingestion of this data. +::: + To work with these states, use: - [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md) table engine. From 0131ca8768efd5840f330b2ef11acc72c35a36a2 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Wed, 14 Jun 2023 10:49:20 -0400 Subject: [PATCH 149/515] add note regarding -MapState --- docs/ru/sql-reference/aggregate-functions/combinators.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/ru/sql-reference/aggregate-functions/combinators.md b/docs/ru/sql-reference/aggregate-functions/combinators.md index 3a7ff571f99..99d5f11442c 100644 --- a/docs/ru/sql-reference/aggregate-functions/combinators.md +++ b/docs/ru/sql-reference/aggregate-functions/combinators.md @@ -66,6 +66,10 @@ WITH anySimpleState(number) AS c SELECT toTypeName(c), c FROM numbers(1); В случае применения этого комбинатора, агрегатная функция возвращает не готовое значение (например, в случае функции [uniq](reference/uniq.md#agg_function-uniq) — количество уникальных значений), а промежуточное состояние агрегации (например, в случае функции `uniq` — хэш-таблицу для расчёта количества уникальных значений), которое имеет тип `AggregateFunction(...)` и может использоваться для дальнейшей обработки или может быть сохранено в таблицу для последующей доагрегации. +:::note +Промежуточное состояние для -MapState не является инвариантом для одних и тех же исходных данные т.к. порядок данных может меняться. Это не влияет, тем не менее, на загрузку таких данных. +::: + Для работы с промежуточными состояниями предназначены: - Движок таблиц [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md). From d0d39c8f6142cf2b954b3125eca4491f82718524 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 14 Jun 2023 18:10:30 +0000 Subject: [PATCH 150/515] Trying to fix a test. --- src/Interpreters/ActionsVisitor.cpp | 29 ++++++- .../InterpreterSelectQueryAnalyzer.h | 2 + src/Interpreters/interpretSubquery.cpp | 85 ++++++++++++------- src/Interpreters/interpretSubquery.h | 5 ++ src/Planner/CollectSets.cpp | 1 + 5 files changed, 87 insertions(+), 35 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 59bbc74ca3a..34c3dab8926 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -3,7 +3,8 @@ #include #include #include -#include "Parsers/queryToString.h" +#include +#include #include #include @@ -55,6 +56,7 @@ #include #include #include +#include #include #include @@ -1394,7 +1396,18 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool if (no_subqueries) return {}; //std::cerr << queryToString(right_in_operand) << std::endl; - auto set_key = PreparedSetKey::forSubquery(right_in_operand->getTreeHash()); + PreparedSetKey set_key; + if (data.getContext()->getSettingsRef().allow_experimental_analyzer) + { + InterpreterSelectQueryAnalyzer interpreter(right_in_operand, data.getContext(), SelectQueryOptions().analyze(true).subquery()); + auto query_tree = interpreter.getQueryTree(); + if (auto * query_node = query_tree->as()) + query_node->setIsSubquery(true); + // std::cerr << "============== " << interpreter.getQueryTree()->dumpTree() << std::endl; + set_key = PreparedSetKey::forSubquery(interpreter.getQueryTree()->getTreeHash()); + } + else + set_key = PreparedSetKey::forSubquery(right_in_operand->getTreeHash()); // std::cerr << set_key.toString() << std::endl; // std::cerr << data.prepared_sets->getSets().size() << std::endl; @@ -1446,8 +1459,16 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool * Also it doesn't make sense if it is GLOBAL IN or ordinary IN. */ { - auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth, {}); - subquery_for_set.createSource(*interpreter); + if (data.getContext()->getSettingsRef().allow_experimental_analyzer) + { + auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth); + subquery_for_set.source = std::make_unique(std::move(*interpreter).extractQueryPlan()); + } + else + { + auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth, {}); + subquery_for_set.createSource(*interpreter); + } } return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), data.getContext()->getSettingsRef(), std::move(external_table_set)); diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h index 1e0ac737536..4434fabe746 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.h +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -69,6 +69,8 @@ public: const Planner & getPlanner() const { return planner; } Planner & getPlanner() { return planner; } + const QueryTreeNodePtr & getQueryTree() const { return query_tree; } + private: ASTPtr query; ContextMutablePtr context; diff --git a/src/Interpreters/interpretSubquery.cpp b/src/Interpreters/interpretSubquery.cpp index 5f00be07fa5..b757127a14d 100644 --- a/src/Interpreters/interpretSubquery.cpp +++ b/src/Interpreters/interpretSubquery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -21,37 +22,10 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -std::shared_ptr interpretSubquery( - const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth, const Names & required_source_columns) +static ASTPtr buildQueryAST(const ASTPtr & table_expression, const ContextPtr & context, SelectQueryOptions & subquery_options); + +static ContextPtr getSubqueryContext(const ContextPtr & context) { - auto subquery_options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth); - return interpretSubquery(table_expression, context, required_source_columns, subquery_options); -} - -std::shared_ptr interpretSubquery( - const ASTPtr & table_expression, ContextPtr context, const Names & required_source_columns, const SelectQueryOptions & options) -{ - if (auto * expr = table_expression->as()) - { - ASTPtr table; - if (expr->subquery) - table = expr->subquery; - else if (expr->table_function) - table = expr->table_function; - else if (expr->database_and_table_name) - table = expr->database_and_table_name; - - return interpretSubquery(table, context, required_source_columns, options); - } - - /// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`. - const auto * subquery = table_expression->as(); - const auto * function = table_expression->as(); - const auto * table = table_expression->as(); - - if (!subquery && !table && !function) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Table expression is undefined, Method: ExpressionAnalyzer::interpretSubquery."); - /** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result. * Because the result of this query is not the result of the entire query. * Constraints work instead @@ -67,7 +41,56 @@ std::shared_ptr interpretSubquery( subquery_settings.extremes = false; subquery_context->setSettings(subquery_settings); + return subquery_context; +} + +std::shared_ptr interpretSubquery( + const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth, const Names & required_source_columns) +{ + auto subquery_options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth); + return interpretSubquery(table_expression, context, required_source_columns, subquery_options); +} + +std::shared_ptr interpretSubquery( + const ASTPtr & table_expression, ContextPtr context, const Names & required_source_columns, const SelectQueryOptions & options) +{ auto subquery_options = options.subquery(); + auto query = buildQueryAST(table_expression, context, subquery_options); + auto subquery_context = getSubqueryContext(context); + return std::make_shared(query, subquery_context, subquery_options, required_source_columns); +} + +std::shared_ptr interpretSubquery( + const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth) +{ + auto subquery_options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth).subquery(); + auto query = buildQueryAST(table_expression, context, subquery_options); + auto subquery_context = getSubqueryContext(context); + return std::make_shared(query, subquery_context, subquery_options); +} + +static ASTPtr buildQueryAST(const ASTPtr & table_expression, const ContextPtr & context, SelectQueryOptions & subquery_options) +{ + if (auto * expr = table_expression->as()) + { + ASTPtr table; + if (expr->subquery) + table = expr->subquery; + else if (expr->table_function) + table = expr->table_function; + else if (expr->database_and_table_name) + table = expr->database_and_table_name; + + return buildQueryAST(table, context, subquery_options); + } + + /// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`. + const auto * subquery = table_expression->as(); + const auto * function = table_expression->as(); + const auto * table = table_expression->as(); + + if (!subquery && !table && !function) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table expression is undefined, Method: ExpressionAnalyzer::interpretSubquery."); ASTPtr query; if (table || function) @@ -112,7 +135,7 @@ std::shared_ptr interpretSubquery( subquery_options.removeDuplicates(); } - return std::make_shared(query, subquery_context, subquery_options, required_source_columns); + return query; } } diff --git a/src/Interpreters/interpretSubquery.h b/src/Interpreters/interpretSubquery.h index 3836d1f7664..02d2003ea5f 100644 --- a/src/Interpreters/interpretSubquery.h +++ b/src/Interpreters/interpretSubquery.h @@ -6,6 +6,11 @@ namespace DB { +class InterpreterSelectQueryAnalyzer; + +std::shared_ptr interpretSubquery( + const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth); + std::shared_ptr interpretSubquery( const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth, const Names & required_source_columns); diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 68ad1ab78d3..90f3f074761 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -93,6 +93,7 @@ public: in_second_argument_node_type == QueryTreeNodeType::UNION || in_second_argument_node_type == QueryTreeNodeType::TABLE) { + // std::cerr << "======2======= " << in_second_argument->dumpTree() << std::endl; auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash()); if (sets.getFuture(set_key)) return; From 35b7474ccfe009c0a2612f55bbca436eba9218a7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Thu, 15 Jun 2023 10:22:58 -0400 Subject: [PATCH 151/515] add MapState to aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index d6cef1883f4..79b1c2ae4db 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -468,6 +468,7 @@ MSan MVCC MacBook MacOS +MapState MarkCacheBytes MarkCacheFiles MarksLoaderThreads From d34b6e5f35280b916627359d01cbf7a4d07f447b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 15 Jun 2023 16:27:51 +0000 Subject: [PATCH 152/515] Try to fix analyzer tests. --- src/Interpreters/ActionsVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 34c3dab8926..393a25463ca 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1397,7 +1397,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool return {}; //std::cerr << queryToString(right_in_operand) << std::endl; PreparedSetKey set_key; - if (data.getContext()->getSettingsRef().allow_experimental_analyzer) + if (data.getContext()->getSettingsRef().allow_experimental_analyzer && !identifier) { InterpreterSelectQueryAnalyzer interpreter(right_in_operand, data.getContext(), SelectQueryOptions().analyze(true).subquery()); auto query_tree = interpreter.getQueryTree(); From 1cb7ba4895e0666eeba8654a45ec8c124b904864 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 15 Jun 2023 16:42:38 +0000 Subject: [PATCH 153/515] Revert "Trying to fix a test." This reverts commit d0d39c8f6142cf2b954b3125eca4491f82718524. --- src/Interpreters/ActionsVisitor.cpp | 21 +++---- src/Interpreters/interpretSubquery.cpp | 85 ++++++++++---------------- src/Interpreters/interpretSubquery.h | 5 -- src/Planner/CollectSets.cpp | 1 - 4 files changed, 38 insertions(+), 74 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 393a25463ca..4b38f8c9434 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -3,8 +3,6 @@ #include #include #include -#include -#include #include #include @@ -56,10 +54,12 @@ #include #include #include -#include #include #include +#include +#include +#include namespace DB { @@ -1396,6 +1396,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool if (no_subqueries) return {}; //std::cerr << queryToString(right_in_operand) << std::endl; + PreparedSetKey set_key; if (data.getContext()->getSettingsRef().allow_experimental_analyzer && !identifier) { @@ -1404,7 +1405,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool if (auto * query_node = query_tree->as()) query_node->setIsSubquery(true); // std::cerr << "============== " << interpreter.getQueryTree()->dumpTree() << std::endl; - set_key = PreparedSetKey::forSubquery(interpreter.getQueryTree()->getTreeHash()); + set_key = PreparedSetKey::forSubquery(query_tree->getTreeHash()); } else set_key = PreparedSetKey::forSubquery(right_in_operand->getTreeHash()); @@ -1459,16 +1460,8 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool * Also it doesn't make sense if it is GLOBAL IN or ordinary IN. */ { - if (data.getContext()->getSettingsRef().allow_experimental_analyzer) - { - auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth); - subquery_for_set.source = std::make_unique(std::move(*interpreter).extractQueryPlan()); - } - else - { - auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth, {}); - subquery_for_set.createSource(*interpreter); - } + auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth, {}); + subquery_for_set.createSource(*interpreter); } return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), data.getContext()->getSettingsRef(), std::move(external_table_set)); diff --git a/src/Interpreters/interpretSubquery.cpp b/src/Interpreters/interpretSubquery.cpp index b757127a14d..5f00be07fa5 100644 --- a/src/Interpreters/interpretSubquery.cpp +++ b/src/Interpreters/interpretSubquery.cpp @@ -1,5 +1,4 @@ #include -#include #include #include @@ -22,10 +21,37 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static ASTPtr buildQueryAST(const ASTPtr & table_expression, const ContextPtr & context, SelectQueryOptions & subquery_options); - -static ContextPtr getSubqueryContext(const ContextPtr & context) +std::shared_ptr interpretSubquery( + const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth, const Names & required_source_columns) { + auto subquery_options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth); + return interpretSubquery(table_expression, context, required_source_columns, subquery_options); +} + +std::shared_ptr interpretSubquery( + const ASTPtr & table_expression, ContextPtr context, const Names & required_source_columns, const SelectQueryOptions & options) +{ + if (auto * expr = table_expression->as()) + { + ASTPtr table; + if (expr->subquery) + table = expr->subquery; + else if (expr->table_function) + table = expr->table_function; + else if (expr->database_and_table_name) + table = expr->database_and_table_name; + + return interpretSubquery(table, context, required_source_columns, options); + } + + /// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`. + const auto * subquery = table_expression->as(); + const auto * function = table_expression->as(); + const auto * table = table_expression->as(); + + if (!subquery && !table && !function) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table expression is undefined, Method: ExpressionAnalyzer::interpretSubquery."); + /** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result. * Because the result of this query is not the result of the entire query. * Constraints work instead @@ -41,56 +67,7 @@ static ContextPtr getSubqueryContext(const ContextPtr & context) subquery_settings.extremes = false; subquery_context->setSettings(subquery_settings); - return subquery_context; -} - -std::shared_ptr interpretSubquery( - const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth, const Names & required_source_columns) -{ - auto subquery_options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth); - return interpretSubquery(table_expression, context, required_source_columns, subquery_options); -} - -std::shared_ptr interpretSubquery( - const ASTPtr & table_expression, ContextPtr context, const Names & required_source_columns, const SelectQueryOptions & options) -{ auto subquery_options = options.subquery(); - auto query = buildQueryAST(table_expression, context, subquery_options); - auto subquery_context = getSubqueryContext(context); - return std::make_shared(query, subquery_context, subquery_options, required_source_columns); -} - -std::shared_ptr interpretSubquery( - const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth) -{ - auto subquery_options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth).subquery(); - auto query = buildQueryAST(table_expression, context, subquery_options); - auto subquery_context = getSubqueryContext(context); - return std::make_shared(query, subquery_context, subquery_options); -} - -static ASTPtr buildQueryAST(const ASTPtr & table_expression, const ContextPtr & context, SelectQueryOptions & subquery_options) -{ - if (auto * expr = table_expression->as()) - { - ASTPtr table; - if (expr->subquery) - table = expr->subquery; - else if (expr->table_function) - table = expr->table_function; - else if (expr->database_and_table_name) - table = expr->database_and_table_name; - - return buildQueryAST(table, context, subquery_options); - } - - /// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`. - const auto * subquery = table_expression->as(); - const auto * function = table_expression->as(); - const auto * table = table_expression->as(); - - if (!subquery && !table && !function) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Table expression is undefined, Method: ExpressionAnalyzer::interpretSubquery."); ASTPtr query; if (table || function) @@ -135,7 +112,7 @@ static ASTPtr buildQueryAST(const ASTPtr & table_expression, const ContextPtr & subquery_options.removeDuplicates(); } - return query; + return std::make_shared(query, subquery_context, subquery_options, required_source_columns); } } diff --git a/src/Interpreters/interpretSubquery.h b/src/Interpreters/interpretSubquery.h index 02d2003ea5f..3836d1f7664 100644 --- a/src/Interpreters/interpretSubquery.h +++ b/src/Interpreters/interpretSubquery.h @@ -6,11 +6,6 @@ namespace DB { -class InterpreterSelectQueryAnalyzer; - -std::shared_ptr interpretSubquery( - const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth); - std::shared_ptr interpretSubquery( const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth, const Names & required_source_columns); diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 90f3f074761..68ad1ab78d3 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -93,7 +93,6 @@ public: in_second_argument_node_type == QueryTreeNodeType::UNION || in_second_argument_node_type == QueryTreeNodeType::TABLE) { - // std::cerr << "======2======= " << in_second_argument->dumpTree() << std::endl; auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash()); if (sets.getFuture(set_key)) return; From e60506f41bbe699ac7572d9b5868ca49dd5288ed Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 16 Jun 2023 02:16:04 +0200 Subject: [PATCH 154/515] changes requested in review --- docs/en/operations/settings/settings.md | 28 +++++++++++-------- .../functions/date-time-functions.md | 12 ++++---- docs/ru/operations/settings/settings.md | 7 +++-- .../functions/date-time-functions.md | 2 +- programs/client/Client.cpp | 2 +- src/Common/DateLUT.cpp | 2 +- src/Common/DateLUT.h | 16 +++++------ src/Common/ThreadStatus.h | 3 +- src/Core/SettingsFields.h | 9 ------ src/Functions/serverConstants.cpp | 5 ++-- .../0_stateless/02737_timezone_setting.sql | 5 ++-- 11 files changed, 45 insertions(+), 46 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 1bd6e44154e..6aebff8101d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4205,9 +4205,18 @@ Use this setting only for backward compatibility if your use cases depend on old ## session_timezone {#session_timezone} -If specified, sets an implicit timezone (instead of [server default](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone). -All DateTime/DateTime64 values (and/or functions results) that have no explicit timezone specified are treated as having this timezone instead of default. -A value of `''` (empty string) configures the session timezone to the server default timezone. +Sets the implicit time zone of the current session or query. +The implicit time zone is the time zone applied to values of type DateTime/DateTime64 which have no explicitly specified time zone. +The setting takes precedence over the globally configured (server-level) implicit time zone. +A value of '' (empty string) means that the implicit time zone of session or query is equal to the [server default time zone](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone). + +You can use functions `timeZone()` and `serverTimezone()` to get the session default and server time zones respectively. + +Possible values: + +- Any timezone name from `system.time_zones`, e.g. `Europe/Berlin`, `UTC` or `Zulu` + +Default value: `''`. Examples: @@ -4223,6 +4232,7 @@ SELECT timeZone(), serverTimezone() SETTINGS session_timezone = 'Asia/Novosibirs Asia/Novosibirsk Europe/Berlin ``` +Assign session time zone 'America/Denver' to the inner DateTime without explicitly specified time zone: ```sql SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'America/Denver' FORMAT TSV @@ -4230,7 +4240,8 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric ``` :::warning -The way this setting affects parsing of Date or DateTime types may seem non-obvious, see example and explanation below: +Not all functions that parse DateTime/DateTime64 respect `session_timezone`. This can lead to subtle errors. +See the following example and explanation. ::: ```sql @@ -4246,14 +4257,9 @@ SELECT *, timezone() FROM test_tz WHERE d = '2000-01-01 00:00:00' SETTINGS sessi ``` This happens due to different parsing pipelines: - - `toDateTime('2000-01-01 00:00:00')` creates a new DateTime in a usual way, and thus `session_timezone` setting from query context is applied. - - `2000-01-01 00:00:00` is parsed to a DateTime inheriting type of `d` column, including its time zone, and `session_timezone` has no impact on this value. -Possible values: - -- Any timezone name from `system.time_zones`, e.g. `Europe/Berlin`, `UTC` or `Zulu` - -Default value: `''`. +- `toDateTime()` without explicitly given timezone used in the first `SELECT` query honors setting `session_timezone` and the global timezone. +- In the second query, a DateTime is parsed from a String, and inherits the type and time zone of the existing column`d`. Thus, setting `session_timezone` and the global timezone are not honored. **See also** diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 62bbb84053a..3f61e7a214d 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -139,7 +139,7 @@ makeDateTime32(year, month, day, hour, minute, second[, fraction[, precision[, t ## timeZone -Returns the default timezone of the current session, i.e. the value of setting [session_timezone](../../operations/settings/settings.md#session_timezone). +Returns the timezone of the current session, i.e. the value of setting [session_timezone](../../operations/settings/settings.md#session_timezone). If the function is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard, otherwise it produces a constant value. **Syntax** @@ -158,12 +158,12 @@ Type: [String](../../sql-reference/data-types/string.md). **See also** -- [serverTimezone](#serverTimeZone) +- [serverTimeZone](#serverTimeZone) -## serverTimezone +## serverTimeZone -Returns the default timezone of the server, i.e. the value of setting [timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). -If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise, it produces a constant value. +Returns the timezone of the server, i.e. the value of setting [timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). +If the function is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise, it produces a constant value. **Syntax** @@ -171,6 +171,8 @@ If it is executed in the context of a distributed table, then it generates a nor serverTimeZone() ``` +Alias: `serverTimezone`. + **Returned value** - Timezone. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 510a7bc837c..09e8eb25fe1 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4158,7 +4158,8 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric Значение по умолчанию: `''`. :::warning -То, как этот параметр влияет на парсинг значений типа Date или DateTime, может показаться неочевидным. Пример и пояснение см. ниже: +Иногда при формировании значений типа `DateTime` и `DateTime64` параметр `session_timezone` может быть проигнорирован. +Это может привести к путанице. Пример и пояснение см. ниже. ::: ```sql @@ -4174,8 +4175,8 @@ SELECT *, timezone() FROM test_tz WHERE d = '2000-01-01 00:00:00' SETTINGS sessi ``` Это происходит из-за различного происхождения значения, используемого для сравнения: -- `toDateTime('2000-01-01 00:00:00')` создаёт значение типа `DateTime` как и в любом другом случае, в том числе применяет параметр `session_timezone` из контекста запроса, -- `2000-01-01 00:00:00` парсится в `DateTime` того же типа, что и колонка `d` (в том числе с той же `timezone`), и параметр `session_timezone` в данном случае не учитывается. +- В первом запросе функция `toDateTime()`, создавая значение типа `DateTime`, принимает во внимание параметр `session_timezone` из контекста запроса; +- Во втором запросе `DateTime` формируется из строки неявно, наследуя тип колонки `d` (в том числе и числовой пояс), и параметр `session_timezone` игнорируется. **Смотрите также** diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 1ae368d71fc..17ab04b7799 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -59,7 +59,7 @@ timeZone() serverTimeZone() ``` -Синонимы: `servertimezone`, `serverTimezone`. +Синонимы: `serverTimezone`. **Возвращаемое значение** diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 9b670c105b6..97ade0ad28b 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -326,7 +326,7 @@ try // All that just to set DB::CurrentThread::get().getGlobalContext() // which is required for client timezone (pushed from server) to work. auto thread_group = std::make_shared(); - thread_group->global_context = global_context; + const_cast(thread_group->global_context) = global_context; thread_status.attachToGroup(thread_group, false); } diff --git a/src/Common/DateLUT.cpp b/src/Common/DateLUT.cpp index f7e7df016cb..2b261a28469 100644 --- a/src/Common/DateLUT.cpp +++ b/src/Common/DateLUT.cpp @@ -165,7 +165,7 @@ DateLUT & DateLUT::getInstance() return ret; } -std::string DateLUT::extractTimezoneFromContext(const DB::ContextPtr query_context) +std::string DateLUT::extractTimezoneFromContext(DB::ContextPtr query_context) { return query_context->getSettingsRef().session_timezone.value; } diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index 833b2291254..d17cbef80ee 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -26,15 +26,15 @@ public: if (DB::CurrentThread::isInitialized()) { - std::string context_timezone; + std::string timezone_from_context; const DB::ContextPtr query_context = DB::CurrentThread::get().getQueryContext(); if (query_context) { - context_timezone = extractTimezoneFromContext(query_context); + timezone_from_context = extractTimezoneFromContext(query_context); - if (!context_timezone.empty()) - return date_lut.getImplementation(context_timezone); + if (!timezone_from_context.empty()) + return date_lut.getImplementation(timezone_from_context); } /// Timezone is passed in query_context, but on CH-Client we have no query context, @@ -42,14 +42,14 @@ public: const DB::ContextPtr global_context = DB::CurrentThread::get().getGlobalContext(); if (global_context) { - context_timezone = extractTimezoneFromContext(global_context); + timezone_from_context = extractTimezoneFromContext(global_context); - if (!context_timezone.empty()) - return date_lut.getImplementation(context_timezone); + if (!timezone_from_context.empty()) + return date_lut.getImplementation(timezone_from_context); } } - return *date_lut.default_impl.load(std::memory_order_acquire); + return serverTimezoneInstance(); } static ALWAYS_INLINE const DateLUTImpl & instance(const std::string & time_zone) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 54849db8e95..061959d9f1f 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -71,8 +71,7 @@ public: /// Set up at creation, no race when reading const ContextWeakPtr query_context; - /// Cannot make it const -- we need to modify it in ch-client to process timezone from server - ContextWeakPtr global_context; + const ContextWeakPtr global_context; const FatalErrorCallback fatal_error_callback; diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 8bd7370c980..ac7c9740143 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -552,15 +552,12 @@ struct SettingFieldTimezone String value; bool changed = false; -// explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(std::string(str)); value = str; } explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(str); value = str; } explicit SettingFieldTimezone(const String & str) { validateTimezone(str); value = str; } -// explicit SettingFieldTimezone(String && str) { validateTimezone(std::string(str)); value = std::move(str); } explicit SettingFieldTimezone(String && str) { validateTimezone(str); value = std::move(str); } explicit SettingFieldTimezone(const char * str) { validateTimezone(str); value = str; } explicit SettingFieldTimezone(const Field & f) { const String & str = f.safeGet(); validateTimezone(str); value = str; } -// SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(std::string(str)); value = str; changed = true; return *this; } SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(str); value = str; changed = true; return *this; } SettingFieldTimezone & operator =(const String & str) { *this = std::string_view{str}; return *this; } SettingFieldTimezone & operator =(String && str) { validateTimezone(str); value = std::move(str); changed = true; return *this; } @@ -577,12 +574,6 @@ struct SettingFieldTimezone void readBinary(ReadBuffer & in); private: -// cctz::time_zone validated_tz; -// void validateTimezone(const std::string & str) -// { -// if (!str.empty() && !cctz::load_time_zone(str, &validated_tz)) -// throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); -// } static void validateTimezone(std::string_view str); }; diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index d3e1e6e10fe..0fda53414de 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -66,7 +66,7 @@ namespace public: static constexpr auto name = "timezone"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance().getTimeZone()}, context->isDistributed()) {} + explicit FunctionTimezone(ContextPtr context) : FunctionConstantBase(DateLUT::instance().getTimeZone(), context->isDistributed()) {} }; /// Returns the server time zone (timezone in which server runs). @@ -75,7 +75,7 @@ namespace public: static constexpr auto name = "serverTimezone"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionServerTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::serverTimezoneInstance().getTimeZone()}, context->isDistributed()) {} + explicit FunctionServerTimezone(ContextPtr context) : FunctionConstantBase(DateLUT::serverTimezoneInstance().getTimeZone(), context->isDistributed()) {} }; @@ -187,6 +187,7 @@ Returns the timezone name in which server operates. .examples{{"serverTimezone", "SELECT serverTimezone();", ""}}, .categories{"Constant", "Miscellaneous"} }); + factory.registerAlias("serverTimeZone", "serverTimezone"); } REGISTER_FUNCTION(Uptime) diff --git a/tests/queries/0_stateless/02737_timezone_setting.sql b/tests/queries/0_stateless/02737_timezone_setting.sql index 87eeec0779b..a345e596193 100644 --- a/tests/queries/0_stateless/02737_timezone_setting.sql +++ b/tests/queries/0_stateless/02737_timezone_setting.sql @@ -10,11 +10,10 @@ SELECT toDateTime(toDateTime('2022-12-12 23:23:23'), 'Europe/Zurich'); SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'America/Denver'; -- test proper serialization -SELECT toDateTime('2002-12-12 23:23:23'), toString(toDateTime('2002-12-12 23:23:23')) SETTINGS session_timezone = 'Asia/Phnom_Penh'; -SELECT toDateTime64('2002-12-12 23:23:23.123', 3), toString(toDateTime64('2002-12-12 23:23:23.123', 3)) SETTINGS session_timezone = 'Asia/Phnom_Penh'; +SELECT toDateTime('2002-12-12 23:23:23') AS dt, toString(dt) SETTINGS session_timezone = 'Asia/Phnom_Penh'; +SELECT toDateTime64('2002-12-12 23:23:23.123', 3)as dt64, toString(dt64) SETTINGS session_timezone = 'Asia/Phnom_Penh'; -- Create a table and test that DateTimes are processed correctly on insert -SET session_timezone='Asia/Novosibirsk'; CREATE TABLE test_tz_setting (d DateTime('UTC')) Engine=Memory AS SELECT toDateTime('2000-01-01 00:00:00'); INSERT INTO test_tz_setting VALUES ('2000-01-01 01:00:00'); -- this is parsed using timezone from `d` column INSERT INTO test_tz_setting VALUES (toDateTime('2000-01-02 02:00:00')); -- this is parsed using `session_timezone` From f610645c077b9f96257a8d9ce7ced71d3ac6cbba Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Fri, 16 Jun 2023 01:10:06 -0700 Subject: [PATCH 155/515] Move outfile checks before query starts running. --- src/Client/ClientBase.cpp | 103 ++++++++++++++++++++++++++------------ 1 file changed, 70 insertions(+), 33 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index b746d46148c..08e14f83982 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -103,6 +103,7 @@ namespace ErrorCodes extern const int UNRECOGNIZED_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int CANNOT_OPEN_FILE; + extern const int FILE_ALREADY_EXISTS; } } @@ -568,44 +569,12 @@ try CompressionMethod compression_method = chooseCompressionMethod(out_file, compression_method_string); UInt64 compression_level = 3; - if (query_with_output->is_outfile_append && query_with_output->is_outfile_truncate) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Cannot use INTO OUTFILE with APPEND and TRUNCATE simultaneously."); - } - - if (query_with_output->is_outfile_append && compression_method != CompressionMethod::None) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Cannot append to compressed file. Please use uncompressed file or remove APPEND keyword."); - } - if (query_with_output->compression_level) { const auto & compression_level_node = query_with_output->compression_level->as(); - bool res = compression_level_node.value.tryGet(compression_level); - auto range = getCompressionLevelRange(compression_method); - - if (!res || compression_level < range.first || compression_level > range.second) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Invalid compression level, must be positive integer in range {}-{}", - range.first, - range.second); + compression_level_node.value.tryGet(compression_level); } - if (fs::exists(out_file)) - { - if (!query_with_output->is_outfile_append && !query_with_output->is_outfile_truncate) - { - throw Exception( - ErrorCodes::CANNOT_OPEN_FILE, - "File {} exists, consider using APPEND or TRUNCATE if appropriate.", - out_file); - } - } auto flags = O_WRONLY | O_EXCL; if (query_with_output->is_outfile_append) flags |= O_APPEND; @@ -891,6 +860,74 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa } } + // Run some local checks to make sure queries into output file will work before sending to server. + try + { + if (const auto * query_with_output = dynamic_cast(parsed_query.get())) + { + String out_file; + if (query_with_output->out_file) + { + const auto & out_file_node = query_with_output->out_file->as(); + out_file = out_file_node.value.safeGet(); + + std::string compression_method_string; + + if (query_with_output->compression) + { + const auto & compression_method_node = query_with_output->compression->as(); + compression_method_string = compression_method_node.value.safeGet(); + } + + CompressionMethod compression_method = chooseCompressionMethod(out_file, compression_method_string); + UInt64 compression_level = 3; + + if (query_with_output->is_outfile_append && query_with_output->is_outfile_truncate) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot use INTO OUTFILE with APPEND and TRUNCATE simultaneously."); + } + + if (query_with_output->is_outfile_append && compression_method != CompressionMethod::None) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot append to compressed file. Please use uncompressed file or remove APPEND keyword."); + } + + if (query_with_output->compression_level) + { + const auto & compression_level_node = query_with_output->compression_level->as(); + bool res = compression_level_node.value.tryGet(compression_level); + auto range = getCompressionLevelRange(compression_method); + + if (!res || compression_level < range.first || compression_level > range.second) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Invalid compression level, must be positive integer in range {}-{}", + range.first, + range.second); + } + + if (fs::exists(out_file)) + { + if (!query_with_output->is_outfile_append && !query_with_output->is_outfile_truncate) + { + throw Exception( + ErrorCodes::FILE_ALREADY_EXISTS, + "File {} exists, consider using APPEND or TRUNCATE.", + out_file); + } + } + } + } + } + catch (...) + { + throw LocalFormatError(getCurrentExceptionMessageAndPattern(print_stack_trace), getCurrentExceptionCode()); + } + const auto & settings = global_context->getSettingsRef(); const Int32 signals_before_stop = settings.partial_result_on_first_cancel ? 2 : 1; From 508e4906c08bf3e3c5a80335d5573cc38974b7e1 Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Fri, 16 Jun 2023 01:47:28 -0700 Subject: [PATCH 156/515] Avoid double-catching the exceptions. --- src/Client/ClientBase.cpp | 93 ++++++++++++++++++--------------------- 1 file changed, 43 insertions(+), 50 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 08e14f83982..ea51c5fd0f8 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -861,72 +861,65 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa } // Run some local checks to make sure queries into output file will work before sending to server. - try + if (const auto * query_with_output = dynamic_cast(parsed_query.get())) { - if (const auto * query_with_output = dynamic_cast(parsed_query.get())) + String out_file; + if (query_with_output->out_file) { - String out_file; - if (query_with_output->out_file) + const auto & out_file_node = query_with_output->out_file->as(); + out_file = out_file_node.value.safeGet(); + + std::string compression_method_string; + + if (query_with_output->compression) { - const auto & out_file_node = query_with_output->out_file->as(); - out_file = out_file_node.value.safeGet(); + const auto & compression_method_node = query_with_output->compression->as(); + compression_method_string = compression_method_node.value.safeGet(); + } - std::string compression_method_string; + CompressionMethod compression_method = chooseCompressionMethod(out_file, compression_method_string); + UInt64 compression_level = 3; - if (query_with_output->compression) - { - const auto & compression_method_node = query_with_output->compression->as(); - compression_method_string = compression_method_node.value.safeGet(); - } + if (query_with_output->is_outfile_append && query_with_output->is_outfile_truncate) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot use INTO OUTFILE with APPEND and TRUNCATE simultaneously."); + } - CompressionMethod compression_method = chooseCompressionMethod(out_file, compression_method_string); - UInt64 compression_level = 3; + if (query_with_output->is_outfile_append && compression_method != CompressionMethod::None) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot append to compressed file. Please use uncompressed file or remove APPEND keyword."); + } - if (query_with_output->is_outfile_append && query_with_output->is_outfile_truncate) - { + if (query_with_output->compression_level) + { + const auto & compression_level_node = query_with_output->compression_level->as(); + bool res = compression_level_node.value.tryGet(compression_level); + auto range = getCompressionLevelRange(compression_method); + + if (!res || compression_level < range.first || compression_level > range.second) throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Cannot use INTO OUTFILE with APPEND and TRUNCATE simultaneously."); - } + "Invalid compression level, must be positive integer in range {}-{}", + range.first, + range.second); + } - if (query_with_output->is_outfile_append && compression_method != CompressionMethod::None) + if (fs::exists(out_file)) + { + if (!query_with_output->is_outfile_append && !query_with_output->is_outfile_truncate) { throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Cannot append to compressed file. Please use uncompressed file or remove APPEND keyword."); - } - - if (query_with_output->compression_level) - { - const auto & compression_level_node = query_with_output->compression_level->as(); - bool res = compression_level_node.value.tryGet(compression_level); - auto range = getCompressionLevelRange(compression_method); - - if (!res || compression_level < range.first || compression_level > range.second) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Invalid compression level, must be positive integer in range {}-{}", - range.first, - range.second); - } - - if (fs::exists(out_file)) - { - if (!query_with_output->is_outfile_append && !query_with_output->is_outfile_truncate) - { - throw Exception( - ErrorCodes::FILE_ALREADY_EXISTS, - "File {} exists, consider using APPEND or TRUNCATE.", - out_file); - } + ErrorCodes::FILE_ALREADY_EXISTS, + "File {} exists, consider using APPEND or TRUNCATE.", + out_file); } } } } - catch (...) - { - throw LocalFormatError(getCurrentExceptionMessageAndPattern(print_stack_trace), getCurrentExceptionCode()); - } const auto & settings = global_context->getSettingsRef(); const Int32 signals_before_stop = settings.partial_result_on_first_cancel ? 2 : 1; From 79222bace2bb2732f247aa8e6700cde9b2e30040 Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 16 Jun 2023 11:11:47 +0200 Subject: [PATCH 157/515] fix style --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index acc8fa76211..a94521b2a88 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2133,6 +2133,7 @@ seektable sequenceCount sequenceMatch sequenceNextNode +serverTimeZone serverTimezone serverUUID sessionCacheSize From 99607dc795c201b6993a4c50f6a412790c3a80bc Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Fri, 16 Jun 2023 12:17:07 +0300 Subject: [PATCH 158/515] Update 02737_timezone_setting.sql --- tests/queries/0_stateless/02737_timezone_setting.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02737_timezone_setting.sql b/tests/queries/0_stateless/02737_timezone_setting.sql index a345e596193..8543e2a2c82 100644 --- a/tests/queries/0_stateless/02737_timezone_setting.sql +++ b/tests/queries/0_stateless/02737_timezone_setting.sql @@ -11,7 +11,7 @@ SELECT toDateTime64(toDateTime64('2022-12-12 23:23:23.123', 3), 3, 'Europe/Zuric -- test proper serialization SELECT toDateTime('2002-12-12 23:23:23') AS dt, toString(dt) SETTINGS session_timezone = 'Asia/Phnom_Penh'; -SELECT toDateTime64('2002-12-12 23:23:23.123', 3)as dt64, toString(dt64) SETTINGS session_timezone = 'Asia/Phnom_Penh'; +SELECT toDateTime64('2002-12-12 23:23:23.123', 3) AS dt64, toString(dt64) SETTINGS session_timezone = 'Asia/Phnom_Penh'; -- Create a table and test that DateTimes are processed correctly on insert CREATE TABLE test_tz_setting (d DateTime('UTC')) Engine=Memory AS SELECT toDateTime('2000-01-01 00:00:00'); @@ -23,4 +23,4 @@ SELECT d FROM test_tz_setting WHERE d == '2000-01-01 01:00:00'; -- 1 row expecte SELECT d FROM test_tz_setting WHERE d == toDateTime('2000-01-01 02:00:00'); -- 0 rows expected -- Cleanup table -DROP TABLE test_tz_setting SYNC; \ No newline at end of file +DROP TABLE test_tz_setting SYNC; From d065fb756d6736a8551139166fb2f835d1bb0b20 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Fri, 16 Jun 2023 12:24:19 +0300 Subject: [PATCH 159/515] Update settings.md --- docs/ru/operations/settings/settings.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 754f9d071f2..5c2b564537b 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4132,6 +4132,8 @@ SELECT sum(number) FROM numbers(10000000000) SETTINGS partial_result_on_first_ca Задаёт значение часового пояса (session_timezone) по умолчанию для текущей сессии вместо [часового пояса сервера](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone). То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. При значении настройки `''` (пустая строка), будет совпадать с часовым поясом сервера. +Функции `timeZone()` and `serverTimezone()` возвращают часовой пояс текущей сессии и сервера соответственно. + Примеры: ```sql SELECT timeZone(), serverTimezone() FORMAT TSV From 66d86a9ea21ec8980eba214cbe4d40b2e0921f30 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Fri, 16 Jun 2023 12:33:37 +0300 Subject: [PATCH 160/515] Update settings.md --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 5c2b564537b..f83d05ff710 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4129,7 +4129,7 @@ SELECT sum(number) FROM numbers(10000000000) SETTINGS partial_result_on_first_ca ## session_timezone {#session_timezone} -Задаёт значение часового пояса (session_timezone) по умолчанию для текущей сессии вместо [часового пояса сервера](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone). То есть, все значения DateTime/DateTime64, для которых явно не задан параметр timezone, будут интерпретированы как относящиеся к указанной зоне. +Задаёт значение часового пояса (session_timezone) по умолчанию для текущей сессии вместо [часового пояса сервера](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone). То есть, все значения DateTime/DateTime64, для которых явно не задан часовой пояс, будут интерпретированы как относящиеся к указанной зоне. При значении настройки `''` (пустая строка), будет совпадать с часовым поясом сервера. Функции `timeZone()` and `serverTimezone()` возвращают часовой пояс текущей сессии и сервера соответственно. From d745180ef6ad0057efbf2f054783e84a1ef1ea3a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 16 Jun 2023 11:41:08 +0200 Subject: [PATCH 161/515] Update settings.md --- docs/en/operations/settings/settings.md | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 733bbf67a45..88dc54a8caa 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4210,13 +4210,13 @@ Use this setting only for backward compatibility if your use cases depend on old Sets the implicit time zone of the current session or query. The implicit time zone is the time zone applied to values of type DateTime/DateTime64 which have no explicitly specified time zone. The setting takes precedence over the globally configured (server-level) implicit time zone. -A value of '' (empty string) means that the implicit time zone of session or query is equal to the [server default time zone](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone). +A value of '' (empty string) means that the implicit time zone of the current session or query is equal to the [server default time zone](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone). -You can use functions `timeZone()` and `serverTimezone()` to get the session default and server time zones respectively. +You can use functions `timeZone()` and `serverTimezone()` to get the session time zone and server time zone. Possible values: -- Any timezone name from `system.time_zones`, e.g. `Europe/Berlin`, `UTC` or `Zulu` +- Any time zone name from `system.time_zones`, e.g. `Europe/Berlin`, `UTC` or `Zulu` Default value: `''`. @@ -4235,6 +4235,7 @@ Asia/Novosibirsk Europe/Berlin ``` Assign session time zone 'America/Denver' to the inner DateTime without explicitly specified time zone: + ```sql SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zurich') SETTINGS session_timezone = 'America/Denver' FORMAT TSV @@ -4260,8 +4261,8 @@ SELECT *, timezone() FROM test_tz WHERE d = '2000-01-01 00:00:00' SETTINGS sessi This happens due to different parsing pipelines: -- `toDateTime()` without explicitly given timezone used in the first `SELECT` query honors setting `session_timezone` and the global timezone. -- In the second query, a DateTime is parsed from a String, and inherits the type and time zone of the existing column`d`. Thus, setting `session_timezone` and the global timezone are not honored. +- `toDateTime()` without explicitly given time zone used in the first `SELECT` query honors setting `session_timezone` and the global time zone. +- In the second query, a DateTime is parsed from a String, and inherits the type and time zone of the existing column`d`. Thus, setting `session_timezone` and the global time zone are not honored. **See also** From 2cca632e83d88ca5cf359f7d3cf002c037ca7dc1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 16 Jun 2023 11:42:22 +0200 Subject: [PATCH 162/515] Update settings.md --- docs/en/operations/settings/settings.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 88dc54a8caa..c8558c6cda8 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4212,7 +4212,7 @@ The implicit time zone is the time zone applied to values of type DateTime/DateT The setting takes precedence over the globally configured (server-level) implicit time zone. A value of '' (empty string) means that the implicit time zone of the current session or query is equal to the [server default time zone](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone). -You can use functions `timeZone()` and `serverTimezone()` to get the session time zone and server time zone. +You can use functions `timeZone()` and `serverTimeZone()` to get the session time zone and server time zone. Possible values: @@ -4223,13 +4223,13 @@ Default value: `''`. Examples: ```sql -SELECT timeZone(), serverTimezone() FORMAT TSV +SELECT timeZone(), serverTimeZone() FORMAT TSV Europe/Berlin Europe/Berlin ``` ```sql -SELECT timeZone(), serverTimezone() SETTINGS session_timezone = 'Asia/Novosibirsk' FORMAT TSV +SELECT timeZone(), serverTimeZone() SETTINGS session_timezone = 'Asia/Novosibirsk' FORMAT TSV Asia/Novosibirsk Europe/Berlin ``` @@ -4250,10 +4250,10 @@ See the following example and explanation. ```sql CREATE TABLE test_tz (`d` DateTime('UTC')) ENGINE = Memory AS SELECT toDateTime('2000-01-01 00:00:00', 'UTC'); -SELECT *, timezone() FROM test_tz WHERE d = toDateTime('2000-01-01 00:00:00') SETTINGS session_timezone = 'Asia/Novosibirsk' +SELECT *, timeZone() FROM test_tz WHERE d = toDateTime('2000-01-01 00:00:00') SETTINGS session_timezone = 'Asia/Novosibirsk' 0 rows in set. -SELECT *, timezone() FROM test_tz WHERE d = '2000-01-01 00:00:00' SETTINGS session_timezone = 'Asia/Novosibirsk' +SELECT *, timeZone() FROM test_tz WHERE d = '2000-01-01 00:00:00' SETTINGS session_timezone = 'Asia/Novosibirsk' ┌───────────────────d─┬─timezone()───────┐ │ 2000-01-01 00:00:00 │ Asia/Novosibirsk │ └─────────────────────┴──────────────────┘ From a622f1b16c2e9e2053fbeea963ac66ae59c01478 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 16 Jun 2023 11:42:58 +0200 Subject: [PATCH 163/515] Update settings.md --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index c8558c6cda8..bdd9684982c 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4254,7 +4254,7 @@ SELECT *, timeZone() FROM test_tz WHERE d = toDateTime('2000-01-01 00:00:00') SE 0 rows in set. SELECT *, timeZone() FROM test_tz WHERE d = '2000-01-01 00:00:00' SETTINGS session_timezone = 'Asia/Novosibirsk' -┌───────────────────d─┬─timezone()───────┐ +┌───────────────────d─┬─timeZone()───────┐ │ 2000-01-01 00:00:00 │ Asia/Novosibirsk │ └─────────────────────┴──────────────────┘ ``` From 0e6ec3e21f08ec8255a0b5ddb11e6a6055733961 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 16 Jun 2023 11:44:47 +0200 Subject: [PATCH 164/515] Update settings.md --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index bdd9684982c..d0ecbedba71 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4210,7 +4210,7 @@ Use this setting only for backward compatibility if your use cases depend on old Sets the implicit time zone of the current session or query. The implicit time zone is the time zone applied to values of type DateTime/DateTime64 which have no explicitly specified time zone. The setting takes precedence over the globally configured (server-level) implicit time zone. -A value of '' (empty string) means that the implicit time zone of the current session or query is equal to the [server default time zone](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone). +A value of '' (empty string) means that the implicit time zone of the current session or query is equal to the [server time zone](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone). You can use functions `timeZone()` and `serverTimeZone()` to get the session time zone and server time zone. From 112a3e6aca8ad22971c27bdd3927fed4d22d1f2a Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 16 Jun 2023 12:19:01 +0200 Subject: [PATCH 165/515] revert tz validation optimization --- programs/keeper/CMakeLists.txt | 2 -- src/CMakeLists.txt | 5 ----- src/Core/SettingsFields.cpp | 16 ---------------- src/Core/SettingsFields.h | 12 +++++++++--- 4 files changed, 9 insertions(+), 26 deletions(-) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 3fb8126ed03..1f1138f49eb 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -108,8 +108,6 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/GraphiteWriter.cpp ${CMAKE_CURRENT_BINARY_DIR}/../../src/Daemon/GitHash.generated.cpp - ${CMAKE_CURRENT_BINARY_DIR}/../../src/Storages/System/StorageSystemTimeZones.generated.cpp - Keeper.cpp clickhouse-keeper.cpp ) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 18858f446a4..ca428fbff3a 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -192,11 +192,6 @@ list (APPEND dbms_headers Dictionaries/DictionaryStructure.h Dictionaries/getDictionaryConfigurationFromAST.h) -# Required for validation of Timezone in session_timezone setting. -# This way we don't need to create timezone via cctz each time, but check against pregenerated char** -list (APPEND dbms_sources - Storages/System/StorageSystemTimeZones.generated.cpp) - if (NOT ENABLE_SSL) list (REMOVE_ITEM clickhouse_common_io_sources Common/OpenSSLHelpers.cpp) list (REMOVE_ITEM clickhouse_common_io_headers Common/OpenSSLHelpers.h) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 834c677d38e..9c7dac5fef4 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -13,8 +13,6 @@ #include -extern const char * auto_time_zones[]; - namespace DB { namespace ErrorCodes @@ -28,14 +26,6 @@ namespace ErrorCodes namespace { - bool checkIsExistingTimeZone(const std::string_view timezone) - { - for (auto * it = auto_time_zones; *it; ++it) - if (timezone == *it) - return true; - return false; - } - template T stringToNumber(const String & str) { @@ -473,12 +463,6 @@ void SettingFieldTimezone::readBinary(ReadBuffer & in) *this = std::move(str); } -void SettingFieldTimezone::validateTimezone(std::string_view str) -{ - if (!str.empty() && !checkIsExistingTimeZone(str)) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", str); -} - String SettingFieldCustom::toString() const { return value.dump(); diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index ac7c9740143..17b64474c04 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -552,13 +553,13 @@ struct SettingFieldTimezone String value; bool changed = false; - explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(str); value = str; } + explicit SettingFieldTimezone(std::string_view str = {}) { validateTimezone(std::string(str)); value = str; } explicit SettingFieldTimezone(const String & str) { validateTimezone(str); value = str; } explicit SettingFieldTimezone(String && str) { validateTimezone(str); value = std::move(str); } explicit SettingFieldTimezone(const char * str) { validateTimezone(str); value = str; } explicit SettingFieldTimezone(const Field & f) { const String & str = f.safeGet(); validateTimezone(str); value = str; } - SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(str); value = str; changed = true; return *this; } + SettingFieldTimezone & operator =(std::string_view str) { validateTimezone(std::string(str)); value = str; changed = true; return *this; } SettingFieldTimezone & operator =(const String & str) { *this = std::string_view{str}; return *this; } SettingFieldTimezone & operator =(String && str) { validateTimezone(str); value = std::move(str); changed = true; return *this; } SettingFieldTimezone & operator =(const char * str) { *this = std::string_view{str}; return *this; } @@ -574,7 +575,12 @@ struct SettingFieldTimezone void readBinary(ReadBuffer & in); private: - static void validateTimezone(std::string_view str); + static cctz::time_zone validated_tz; + static void validateTimezone(const std::string & tz_str) + { + if (!tz_str.empty() && !cctz::load_time_zone(tz_str, &validated_tz)) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", tz_str); + } }; /// Can keep a value of any type. Used for user-defined settings. From b3476e53fc86d800ec274e78597aaecb6be8f687 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Fri, 16 Jun 2023 13:44:17 +0300 Subject: [PATCH 166/515] Update SettingsFields.cpp --- src/Core/SettingsFields.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 9c7dac5fef4..a0f643f8d30 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -20,7 +20,6 @@ namespace ErrorCodes extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH; extern const int CANNOT_PARSE_BOOL; extern const int CANNOT_PARSE_NUMBER; - extern const int BAD_ARGUMENTS; } From 43e572bd84d8e83a0fb54ce457bbfc2506b42d59 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Fri, 16 Jun 2023 14:55:34 +0300 Subject: [PATCH 167/515] typo in SettingsFields.h --- src/Core/SettingsFields.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 17b64474c04..45c62d0072a 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -575,9 +575,9 @@ struct SettingFieldTimezone void readBinary(ReadBuffer & in); private: - static cctz::time_zone validated_tz; - static void validateTimezone(const std::string & tz_str) + void validateTimezone(const std::string & tz_str) { + cctz::time_zone validated_tz; if (!tz_str.empty() && !cctz::load_time_zone(tz_str, &validated_tz)) throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid time zone: {}", tz_str); } From e20cea8901163752d9a524562087889c6646637a Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 29 May 2023 10:39:38 -0700 Subject: [PATCH 168/515] Implement endianness-independent serialization for quantileTiming --- src/AggregateFunctions/QuantileTiming.h | 30 ++++++++++++------------- src/IO/WriteHelpers.h | 4 ++-- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/src/AggregateFunctions/QuantileTiming.h b/src/AggregateFunctions/QuantileTiming.h index de6607b2527..1d73453bc67 100644 --- a/src/AggregateFunctions/QuantileTiming.h +++ b/src/AggregateFunctions/QuantileTiming.h @@ -78,14 +78,14 @@ namespace detail void serialize(WriteBuffer & buf) const { - writeBinary(count, buf); + writeBinaryLittleEndian(count, buf); buf.write(reinterpret_cast(elems), count * sizeof(elems[0])); } void deserialize(ReadBuffer & buf) { UInt16 new_count = 0; - readBinary(new_count, buf); + readBinaryLittleEndian(new_count, buf); if (new_count > TINY_MAX_ELEMS) throw Exception(ErrorCodes::INCORRECT_DATA, "The number of elements {} for the 'tiny' kind of quantileTiming is exceeding the maximum of {}", new_count, TINY_MAX_ELEMS); buf.readStrict(reinterpret_cast(elems), new_count * sizeof(elems[0])); @@ -164,14 +164,14 @@ namespace detail void serialize(WriteBuffer & buf) const { - writeBinary(elems.size(), buf); + writeBinaryLittleEndian(elems.size(), buf); buf.write(reinterpret_cast(elems.data()), elems.size() * sizeof(elems[0])); } void deserialize(ReadBuffer & buf) { size_t size = 0; - readBinary(size, buf); + readBinaryLittleEndian(size, buf); if (size > 10'000) throw Exception(ErrorCodes::INCORRECT_DATA, "The number of elements {} for the 'medium' kind of quantileTiming is too large", size); @@ -341,7 +341,7 @@ namespace detail void serialize(WriteBuffer & buf) const { - writeBinary(count, buf); + writeBinaryLittleEndian(count, buf); if (count * 2 > SMALL_THRESHOLD + BIG_SIZE) { @@ -356,8 +356,8 @@ namespace detail { if (count_small[i]) { - writeBinary(UInt16(i), buf); - writeBinary(count_small[i], buf); + writeBinaryLittleEndian(UInt16(i), buf); + writeBinaryLittleEndian(count_small[i], buf); } } @@ -365,19 +365,19 @@ namespace detail { if (count_big[i]) { - writeBinary(UInt16(i + SMALL_THRESHOLD), buf); - writeBinary(count_big[i], buf); + writeBinaryLittleEndian(UInt16(i + SMALL_THRESHOLD), buf); + writeBinaryLittleEndian(count_big[i], buf); } } /// Symbolizes end of data. - writeBinary(UInt16(BIG_THRESHOLD), buf); + writeBinaryLittleEndian(UInt16(BIG_THRESHOLD), buf); } } void deserialize(ReadBuffer & buf) { - readBinary(count, buf); + readBinaryLittleEndian(count, buf); if (count * 2 > SMALL_THRESHOLD + BIG_SIZE) { @@ -388,12 +388,12 @@ namespace detail while (true) { UInt16 index = 0; - readBinary(index, buf); + readBinaryLittleEndian(index, buf); if (index == BIG_THRESHOLD) break; UInt64 elem_count = 0; - readBinary(elem_count, buf); + readBinaryLittleEndian(elem_count, buf); if (index < SMALL_THRESHOLD) count_small[index] = elem_count; @@ -692,7 +692,7 @@ public: void serialize(WriteBuffer & buf) const { auto kind = which(); - DB::writePODBinary(kind, buf); + writeBinaryLittleEndian(kind, buf); if (kind == Kind::Tiny) tiny.serialize(buf); @@ -706,7 +706,7 @@ public: void deserialize(ReadBuffer & buf) { Kind kind; - DB::readPODBinary(kind, buf); + readBinaryLittleEndian(kind, buf); if (kind == Kind::Tiny) { diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 3d1a5aa49ef..2ee1e1651f6 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -1183,13 +1183,13 @@ inline void writeBinaryEndian(T x, WriteBuffer & buf) } template -inline void writeBinaryLittleEndian(T x, WriteBuffer & buf) +inline void writeBinaryLittleEndian(const T & x, WriteBuffer & buf) { writeBinaryEndian(x, buf); } template -inline void writeBinaryBigEndian(T x, WriteBuffer & buf) +inline void writeBinaryBigEndian(const T & x, WriteBuffer & buf) { writeBinaryEndian(x, buf); } From afa74f697cd56be60254082e2febf42943166e73 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 16 Jun 2023 19:38:50 +0000 Subject: [PATCH 169/515] Refactor a bit. --- src/Interpreters/ConcurrentHashJoin.cpp | 1 + src/Interpreters/PreparedSets.cpp | 116 ++++++-------- src/Interpreters/PreparedSets.h | 147 ++---------------- src/Interpreters/Set.cpp | 5 +- src/Interpreters/Set.h | 22 +-- src/Planner/CollectSets.cpp | 1 + src/Planner/PlannerActionsVisitor.cpp | 3 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 2 + .../MergeTree/MergeTreeSequentialSource.cpp | 1 + src/Storages/MergeTree/RPNBuilder.cpp | 14 +- src/Storages/StorageAzureBlob.cpp | 1 + src/Storages/StorageDistributed.cpp | 1 + src/Storages/StorageDummy.cpp | 1 + src/Storages/StorageFile.cpp | 1 + src/Storages/StorageMemory.cpp | 1 + src/Storages/StorageURL.cpp | 1 + .../System/StorageSystemZooKeeper.cpp | 2 + src/Storages/VirtualColumnUtils.cpp | 24 ++- 18 files changed, 111 insertions(+), 233 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 03c173a73d9..fc24f0ae029 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 6df7c748e60..c79b667789b 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -1,12 +1,23 @@ #include #include #include +#include #include #include #include #include #include "Common/logger_useful.h" #include "Processors/QueryPlan/CreatingSetsStep.h" +#include "Processors/Executors/CompletedPipelineExecutor.h" +#include "Processors/QueryPlan/BuildQueryPipelineSettings.h" +#include "Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h" +#include "Processors/Sinks/EmptySink.h" +#include "Processors/Sinks/NullSink.h" +#include +#include "Core/Block.h" +#include +#include +#include namespace DB { @@ -16,6 +27,35 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +class FutureSetFromTuple final : public FutureSet +{ +public: + FutureSetFromTuple(Block block, const Settings & settings); + + bool isReady() const override { return true; } + SetPtr get() const override { return set; } + + SetPtr buildOrderedSetInplace(const ContextPtr & context) override; + + DataTypes getTypes() const override { return set->getElementsTypes(); } + +private: + SetPtr set; + SetKeyColumns set_key_columns; +}; + + +FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} +bool FutureSetFromStorage::isReady() const { return set != nullptr; } +SetPtr FutureSetFromStorage::get() const { return set; } +DataTypes FutureSetFromStorage::getTypes() const { return set->getElementsTypes(); } + +SetPtr FutureSetFromStorage::buildOrderedSetInplace(const ContextPtr &) +{ + return set->hasExplicitSetElements() ? set : nullptr; +} + + PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) { /// Remove LowCardinality types from type list because Set doesn't support LowCardinality keys now, @@ -240,8 +280,6 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) return nullptr; } - // std::cerr << "... external_table_set " << reinterpret_cast(external_table_set.get()) << std::endl; - if (external_table_set) return subquery.set = external_table_set->buildOrderedSetInplace(context); @@ -261,6 +299,13 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) return subquery.set; } +bool FutureSetFromSubquery::isReady() const { return subquery.set != nullptr && subquery.set->isCreated(); } + +std::unique_ptr FutureSetFromSubquery::build(const ContextPtr & context) +{ + return buildPlan(context, false); +} + static SizeLimits getSizeLimitsForSet(const Settings & settings) { return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); @@ -271,25 +316,6 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c if (subquery.set) return nullptr; - // std::cerr << StackTrace().toString() << std::endl; - - // auto set_cache = context->getPreparedSetsCache(); - // if (set_cache) - // { - // auto from_cache = set_cache->findOrPromiseToBuild(subquery.key); - // if (from_cache.index() == 0) - // { - // subquery.promise_to_fill_set = std::move(std::get<0>(from_cache)); - // } - // else - // { - // LOG_TRACE(&Poco::Logger::get("FutureSetFromSubquery"), "Waiting for set, key: {}", subquery.key); - // set = std::get<1>(from_cache).get(); - // return nullptr; - // } - // } - - const auto & settings = context->getSettingsRef(); auto size_limits = getSizeLimitsForSet(settings); @@ -298,10 +324,6 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c auto plan = subquery.detachSource(); auto description = subquery.key; - // WriteBufferFromOwnString buf; - // plan->explainPlan(buf, {.header=true}); - // std::cerr << buf.str() << std::endl; - auto creating_set = std::make_unique( plan->getCurrentDataStream(), description, @@ -314,20 +336,6 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c return plan; } -// static SizeLimits getSizeLimitsForOrderedSet(const Settings & settings) -// { -// if (settings.use_index_for_in_with_subqueries_max_values && -// settings.use_index_for_in_with_subqueries_max_values < settings.max_rows_in_set) -// return getSizeLimitsForUnorderedSet(settings); - -// return SizeLimits(settings.use_index_for_in_with_subqueries_max_values, settings.max_bytes_in_set, OverflowMode::BREAK); -// } - -// SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordered_set) -// { -// return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); -// } - FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) { bool create_ordered_set = false; @@ -342,10 +350,8 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) set_key_columns.filter = ColumnUInt8::create(block.rows()); - //set->initSetElements(); set->insertFromColumns(columns, set_key_columns); set->finishInsert(); - //block(std::move(block_)) } FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, bool transform_null_in_) @@ -359,8 +365,6 @@ DataTypes FutureSetFromSubquery::getTypes() const return Set::getElementTypes(subquery.source->getCurrentDataStream().header.getColumnsWithTypeAndName(), transform_null_in); } -FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} - SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) { if (set->hasExplicitSetElements()) @@ -378,30 +382,4 @@ SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) return set; } -std::unique_ptr FutureSetFromTuple::build(const ContextPtr &) -{ - // const auto & settings = context->getSettingsRef(); - // auto size_limits = getSizeLimitsForSet(settings, false); - // fill(size_limits, settings.transform_null_in, false); - return nullptr; -} - -// void FutureSetFromTuple::buildForTuple(SizeLimits size_limits, bool transform_null_in) -// { -// fill(size_limits, transform_null_in, false); -// } - -// void FutureSetFromTuple::fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set) -// { -// //std::cerr << StackTrace().toString() << std::endl; - -// if (set) -// return; - -// set = std::make_shared(size_limits, create_ordered_set, transform_null_in); -// set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); -// set->insertFromBlock(block.getColumnsWithTypeAndName()); -// set->finishInsert(); -// } - }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index a119c24ad10..f034aca747f 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -2,23 +2,13 @@ #include #include -#include #include #include #include -#include +#include #include -#include -#include -#include "Core/Block.h" -#include "Interpreters/Context.h" -#include "Interpreters/Set.h" -#include "Processors/Executors/CompletedPipelineExecutor.h" -#include "Processors/QueryPlan/BuildQueryPipelineSettings.h" -#include "Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h" -#include "Processors/Sinks/EmptySink.h" -#include "Processors/Sinks/NullSink.h" -#include +#include +#include namespace DB { @@ -32,80 +22,44 @@ class InterpreterSelectWithUnionQuery; class IQueryTreeNode; using QueryTreeNodePtr = std::shared_ptr; +struct Settings; + /// Represents a set in a query that might be referenced at analysis time and built later during execution. /// Also it can represent a constant set that is ready to use. /// At analysis stage the FutureSets are created but not necessarily filled. Then for non-constant sets there /// must be an explicit step to build them before they can be used. -/// FutureSet objects can be stored in PreparedSets and are not intended to be used from multiple threads. -// class FutureSet final -// { -// public: -// FutureSet() = default; - -// /// Create FutureSet from an object that will be created in the future. -// explicit FutureSet(const std::shared_future & future_set_) : future_set(future_set_) {} - -// /// Create FutureSet from a ready set. -// explicit FutureSet(SetPtr readySet); - -// /// The set object will be ready in the future, as opposed to 'null' object when FutureSet is default constructed. -// bool isValid() const { return future_set.valid(); } - -// /// The the value of SetPtr is ready, but the set object might not have been filled yet. -// bool isReady() const; - -// /// The set object is ready and filled. -// bool isCreated() const; - -// SetPtr get() const { chassert(isReady()); return future_set.get(); } - -// private: -// std::shared_future future_set; -// }; - +/// Set may be useful for indexes, in this case special ordered set with stored elements is build inplace. class FutureSet { public: virtual ~FutureSet() = default; virtual bool isReady() const = 0; - virtual bool isFilled() const = 0; virtual SetPtr get() const = 0; virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; - virtual std::unique_ptr build(const ContextPtr & context) = 0; virtual DataTypes getTypes() const = 0; - - // static SizeLimits getSizeLimitsForSet(const Settings & settings, bool ordered_set); }; using FutureSetPtr = std::shared_ptr; -class FutureSetFromTuple final : public FutureSet + +class FutureSetFromStorage : public FutureSet { public: - FutureSetFromTuple(Block block, const Settings & settings); + FutureSetFromStorage(SetPtr set_); - bool isReady() const override { return true; } - bool isFilled() const override { return true; } - SetPtr get() const override { return set; } - - SetPtr buildOrderedSetInplace(const ContextPtr & context) override; - - std::unique_ptr build(const ContextPtr &) override; - - DataTypes getTypes() const override { return set->getElementsTypes(); } - -/// void buildForTuple(SizeLimits size_limits, bool transform_null_in); + bool isReady() const override; + SetPtr get() const override; + SetPtr buildOrderedSetInplace(const ContextPtr &) override; + DataTypes getTypes() const override; private: SetPtr set; - Set::SetKeyColumns set_key_columns; - - //void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set); }; + /// Information on how to build set for the [GLOBAL] IN section. class SubqueryForSet { @@ -122,11 +76,6 @@ public: /// Build this set from the result of the subquery. String key; SetPtr set; - /// After set_in_progress is finished it will be put into promise_to_fill_set and thus all FutureSet's - /// that are referencing this set will be filled. - - std::promise promise_to_fill_set; - // FutureSet set = FutureSet{promise_to_fill_set.get_future()}; /// If set, put the result into the table. /// This is a temporary table for transferring to remote servers for distributed query processing. @@ -142,25 +91,18 @@ class FutureSetFromSubquery : public FutureSet, public std::enable_shared_from_t public: FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, bool transform_null_in_); - bool isReady() const override { return subquery.set != nullptr && subquery.set->isCreated(); } - bool isFilled() const override { return isReady(); } + bool isReady() const override; SetPtr get() const override { return subquery.set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override; - std::unique_ptr build(const ContextPtr & context) override - { - return buildPlan(context, false); - } + std::unique_ptr build(const ContextPtr & context); DataTypes getTypes() const override; - // void addStorage(StoragePtr storage) { subquery.table = std::move(storage); } - SubqueryForSet & getSubquery() { return subquery; } private: - //SetPtr set; SubqueryForSet subquery; FutureSetPtr external_table_set; bool transform_null_in; @@ -168,52 +110,6 @@ private: std::unique_ptr buildPlan(const ContextPtr & context, bool create_ordered_set); }; -class FutureSetFromStorage : public FutureSet -{ -public: - FutureSetFromStorage(SetPtr set_); // : set(std::move(set_) {} - - bool isReady() const override { return set != nullptr; } - bool isFilled() const override { return isReady(); } - SetPtr get() const override { return set; } - - SetPtr buildOrderedSetInplace(const ContextPtr &) override - { - return set->hasExplicitSetElements() ? set : nullptr; - } - - DataTypes getTypes() const override { return set->getElementsTypes(); } - - std::unique_ptr build(const ContextPtr &) override { return nullptr; } - -private: - SetPtr set; -}; - -// class FutureSetFromFuture : public FutureSet -// { -// public: -// FutureSetFromFuture(std::shared_future future_set_); - -// bool isReady() const override { return future_set.wait_for(std::chrono::seconds(0)) == std::future_status::ready; } -// SetPtr get() const override { return future_set.get(); } - -// SetPtr buildOrderedSetInplace(const ContextPtr &) override -// { -// fill(true); -// return set; -// } - -// std::unique_ptr build(const ContextPtr &) override -// { -// fill(false); -// return nullptr; -// } - -// private: -// std::shared_future future_set; -// } - struct PreparedSetKey { using Hash = std::pair; @@ -250,28 +146,17 @@ public: }; using SubqueriesForSets = std::vector; - // SubqueryForSet & createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, - // SizeLimits set_size_limit, bool transform_null_in); - FutureSetPtr addFromStorage(const PreparedSetKey & key, SetPtr set_); FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings); FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set); - //void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage); - FutureSetPtr getFuture(const PreparedSetKey & key) const; - //SubqueryForSet & getSubquery(const String & subquery_id); - // SetPtr get(const PreparedSetKey & key) const; /// Get subqueries and clear them. /// We need to build a plan for subqueries just once. That's why we can clear them after accessing them. /// SetPtr would still be available for consumers of PreparedSets. SubqueriesForSets detachSubqueries(); - /// Returns all sets that match the given ast hash not checking types - /// Used in KeyCondition and MergeTreeIndexConditionBloomFilter to make non exact match for types in PreparedSetKey - //std::vector getByTreeHash(IAST::Hash ast_hash) const; - const std::unordered_map & getSets() const { return sets; } bool empty() const; diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 52e9a3a9bcb..83005f3d3a7 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -265,7 +265,10 @@ bool Set::insertFromColumns(const Columns & columns, SetKeyColumns & holder) void Set::appendSetElements(SetKeyColumns & holder) { - //std::cerr << "========= " << keys_size << ' ' << holder.key_columns.size() << std::endl; + if (holder.key_columns.size() != keys_size || set_elements.size() != keys_size) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid number of key columns for set. Expected {} got {} and {}", + keys_size, holder.key_columns.size(), set_elements.size()); + size_t rows = holder.key_columns.at(0)->size(); for (size_t i = 0; i < keys_size; ++i) { diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 9b76ab30a1b..a6908d2bdd3 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -21,6 +21,17 @@ using FunctionBasePtr = std::shared_ptr; class Chunk; +/// Prepared key columns for set which can be added to fill set elements. +/// Used only to upgrade set from tuple. +struct SetKeyColumns +{ + /// The constant columns to the right of IN are not supported directly. For this, they first materialize. + ColumnRawPtrs key_columns; + Columns materialized_columns; + ColumnPtr null_map_holder; + ColumnUInt8::MutablePtr filter; +}; + /** Data structure for implementation of IN expression. */ class Set @@ -48,17 +59,6 @@ public: bool insertFromColumns(const Columns & columns); bool insertFromBlock(const ColumnsWithTypeAndName & columns); - - struct SetKeyColumns - { - //ColumnRawPtrs key_columns; - /// The constant columns to the right of IN are not supported directly. For this, they first materialize. - ColumnRawPtrs key_columns; - Columns materialized_columns; - ColumnPtr null_map_holder; - ColumnUInt8::MutablePtr filter; - }; - void initSetElements(); bool insertFromColumns(const Columns & columns, SetKeyColumns & holder); void appendSetElements(SetKeyColumns & holder); diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 68ad1ab78d3..84ab453c065 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include namespace DB diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index bbac9bf5c4b..087cd3000b9 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -17,6 +17,7 @@ #include #include +#include #include #include @@ -663,7 +664,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma column.name = planner_context->createSetKey(in_second_argument); column.type = std::make_shared(); - bool set_is_created = set->isFilled(); + bool set_is_created = set->isReady(); auto column_set = ColumnSet::create(1, std::move(set)); if (set_is_created) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 0909ee9f1eb..da27e7b1293 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -1,6 +1,8 @@ #include #include #include +//#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index e27354f9d16..5a6d59bf0be 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index b82c350fa1a..1acc3ec6ce9 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -9,6 +9,7 @@ #include #include +#include #include #include @@ -301,26 +302,15 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const auto hash = ast_node->getTreeHash(); auto key = PreparedSetKey::forSubquery(hash); - // std::cerr << ".........Getting from AST \n" << ast_node->dumpTree() << std::endl - // << key.toString() << std::endl; - for (const auto & [k, v] : prepared_sets->getSets()) { - // std::cerr << "........... " << k.toString() << std::endl; if (k.ast_hash == hash) return v; } - - //return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); } else if (dag_node) { - - // std::cerr << "...........Getting from DAG\n"; const auto * node_without_alias = getNodeWithoutAlias(dag_node); - // std::cerr << ".......... node_without_alias : " << node_without_alias->result_name - // << ' ' << node_without_alias->result_type->getName() - // << ' ' << (node_without_alias->column ? node_without_alias->column->getName() : "") << std::endl; return tryGetSetFromDAGNode(node_without_alias); } @@ -369,8 +359,6 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( auto lhs = removeNullable(recursiveRemoveLowCardinality(data_types[i])); auto rhs = removeNullable(recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index])); - // std::cerr << "============ " << lhs->getName() << ' ' << rhs->getName() << std::endl; - if (!lhs->equals(*rhs)) return false; } diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 3ee176a68b7..b80f41543b5 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -13,6 +13,7 @@ #include #include +#include #include #include #include diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 042492aba32..763dbac04cf 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -91,6 +91,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/StorageDummy.cpp b/src/Storages/StorageDummy.cpp index 4f2fb3883bf..e2396a54acb 100644 --- a/src/Storages/StorageDummy.cpp +++ b/src/Storages/StorageDummy.cpp @@ -5,6 +5,7 @@ #include #include +#include namespace DB { diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 72347789790..b3359dd3427 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -23,6 +23,7 @@ #include #include +#include #include #include #include diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index b7772d00776..6ed0583bd44 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 520576d3961..2e72c957a24 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -39,6 +39,7 @@ #include #include #include +#include namespace DB diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 3f6575cb78d..d1c978d3469 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -15,12 +15,14 @@ #include #include #include +#include #include #include #include #include #include #include +#include #include #include #include diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index be2206a78e9..58d00216dc7 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -20,6 +20,13 @@ #include #include +#include +#include +#include +#include +#include +#include + #include #include #include @@ -215,15 +222,18 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex if (column_set) { auto future_set = column_set->getData(); - if (!future_set->isFilled()) + if (!future_set->isReady()) { - auto plan = future_set->build(context); - auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); - pipeline.complete(std::make_shared(Block())); + if (auto * set_from_subquery = typeid_cast(future_set.get())) + { + auto plan = set_from_subquery->build(context); + auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + pipeline.complete(std::make_shared(Block())); - CompletedPipelineExecutor executor(pipeline); - executor.execute(); + CompletedPipelineExecutor executor(pipeline); + executor.execute(); + } } } } From 83e2196a171b21840a03dad6154d9d350d7dd00a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 17 Jun 2023 12:26:53 +0000 Subject: [PATCH 170/515] Make is_shared const --- src/Interpreters/Cache/QueryCache.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index bafa78f13d5..4bc33e36441 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -49,7 +49,7 @@ public: /// If the associated entry can be read by other users. In general, sharing is a bad idea: First, it is unlikely that different /// users pose the same queries. Second, sharing potentially breaches security. E.g. User A should not be able to bypass row /// policies on some table by running the same queries as user B for whom no row policies exist. - bool is_shared; + const bool is_shared; /// When does the entry expire? const std::chrono::time_point expires_at; From 840e73eb7444400514463885de75b6d551935481 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 17 Jun 2023 12:27:44 +0000 Subject: [PATCH 171/515] Update #includes --- 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 de2e2b9ad92..fa01b1da67c 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include #include @@ -14,7 +15,6 @@ #include #include #include -#include #include #include From 0adc98865d28fbffc5d6aff7def6d8cfb16fb91f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 17 Jun 2023 12:43:14 +0000 Subject: [PATCH 172/515] Add variable for access to found key components --- src/Interpreters/Cache/QueryCache.cpp | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 3118f386111..1afb538c01d 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -385,19 +385,22 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar return; } - if (!entry->key.is_shared && entry->key.user_name != key.user_name) + const auto & entry_key = entry->key; + const auto & entry_mapped = entry->mapped; + + if (!entry_key.is_shared && entry_key.user_name != key.user_name) { LOG_TRACE(&Poco::Logger::get("QueryCache"), "Inaccessible entry found for query {}", key.queryStringFromAst()); return; } - if (IsStale()(entry->key)) + if (IsStale()(entry_key)) { LOG_TRACE(&Poco::Logger::get("QueryCache"), "Stale entry found for query {}", key.queryStringFromAst()); return; } - if (!entry->key.is_compressed) + if (!entry_key.is_compressed) { // Cloning chunks isn't exactly great. It could be avoided by another indirection, i.e. wrapping Entry's members chunks, totals and // extremes into shared_ptrs and assuming that the lifecycle of these shared_ptrs coincides with the lifecycle of the Entry @@ -406,15 +409,15 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar // optimization. Chunks cloned_chunks; - for (const auto & chunk : entry->mapped->chunks) + for (const auto & chunk : entry_mapped->chunks) cloned_chunks.push_back(chunk.clone()); - buildSourceFromChunks(entry->key.header, std::move(cloned_chunks), entry->mapped->totals, entry->mapped->extremes); + buildSourceFromChunks(entry_key.header, std::move(cloned_chunks), entry_mapped->totals, entry_mapped->extremes); } else { Chunks decompressed_chunks; - const Chunks & chunks = entry->mapped->chunks; + const Chunks & chunks = entry_mapped->chunks; for (const auto & chunk : chunks) { const Columns & columns = chunk.getColumns(); @@ -428,7 +431,7 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar decompressed_chunks.push_back(std::move(decompressed_chunk)); } - buildSourceFromChunks(entry->key.header, std::move(decompressed_chunks), entry->mapped->totals, entry->mapped->extremes); + buildSourceFromChunks(entry_key.header, std::move(decompressed_chunks), entry_mapped->totals, entry_mapped->extremes); } LOG_TRACE(&Poco::Logger::get("QueryCache"), "Entry found for query {}", key.queryStringFromAst()); From 6e8af092897162035c0add81ef2d8d0028cff810 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 17 Jun 2023 12:48:29 +0000 Subject: [PATCH 173/515] Provide query cache reader with dummy block --- 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 fa01b1da67c..5cf86412c3e 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -734,7 +734,7 @@ static std::tuple executeQueryImpl( && res.pipeline.pulling()) { QueryCache::Key key( - ast, res.pipeline.getHeader(), + ast, /*dummy for header*/ {}, context->getUserName(), /*dummy for is_shared*/ false, /*dummy value for expires_at*/ std::chrono::system_clock::from_time_t(1), /*dummy value for is_compressed*/ false); From 69e7c1cc82944305e5019630674bacab5170b319 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 17 Jun 2023 18:22:15 +0000 Subject: [PATCH 174/515] Introduce convenience ctor for Key construction during read --- src/Interpreters/Cache/QueryCache.cpp | 5 +++++ src/Interpreters/Cache/QueryCache.h | 4 ++++ src/Interpreters/executeQuery.cpp | 6 +----- 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 1afb538c01d..e2cf5a7d6dd 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -132,6 +132,11 @@ QueryCache::Key::Key( { } +QueryCache::Key::Key(ASTPtr ast_, const String & user_name_) + : QueryCache::Key(ast_, {}, user_name_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST or user name +{ +} + bool QueryCache::Key::operator==(const Key & other) const { return ast->getTreeHash() == other.ast->getTreeHash(); diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 4bc33e36441..6ef7cc60918 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -58,12 +58,16 @@ public: /// (we could theoretically apply compression also to the totals and extremes but it's an obscure use case) const bool is_compressed; + /// Ctor to construct a Key for writing into query cache. Key(ASTPtr ast_, Block header_, const String & user_name_, bool is_shared_, std::chrono::time_point expires_at_, bool is_compressed); + /// Ctor to construct a Key for reading from query cache (this operation only needs the AST + user name). + Key(ASTPtr ast_, const String & user_name_); + bool operator==(const Key & other) const; String queryStringFromAst() const; }; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 5cf86412c3e..b45ccf63be6 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -733,11 +733,7 @@ static std::tuple executeQueryImpl( && (can_use_query_cache && settings.enable_reads_from_query_cache) && res.pipeline.pulling()) { - QueryCache::Key key( - ast, /*dummy for header*/ {}, - context->getUserName(), /*dummy for is_shared*/ false, - /*dummy value for expires_at*/ std::chrono::system_clock::from_time_t(1), - /*dummy value for is_compressed*/ false); + QueryCache::Key key(ast, context->getUserName()); QueryCache::Reader reader = query_cache->createReader(key); if (reader.hasCacheEntryForKey()) { From 24717ed908688402d023b933f8b635f3439870fb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 17 Jun 2023 18:24:20 +0000 Subject: [PATCH 175/515] Factorize check if query cache exists --- src/Interpreters/executeQuery.cpp | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index b45ccf63be6..cb68cacebda 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -645,7 +645,8 @@ static std::tuple executeQueryImpl( } } - bool can_use_query_cache = settings.use_query_cache && !internal && !ast->as(); + QueryCachePtr query_cache = context->getQueryCache(); + const bool can_use_query_cache = query_cache != nullptr && settings.use_query_cache && !internal && !ast->as(); if (!async_insert) { @@ -727,10 +728,8 @@ static std::tuple executeQueryImpl( /// - passive (read) use of the query cache is enabled, and /// - the query cache knows the query result /// then replace the pipeline by a new pipeline with a single source that is populated from the query cache - auto query_cache = context->getQueryCache(); bool read_result_from_query_cache = false; /// a query must not read from *and* write to the query cache at the same time - if (query_cache != nullptr - && (can_use_query_cache && settings.enable_reads_from_query_cache) + if (can_use_query_cache && settings.enable_reads_from_query_cache && res.pipeline.pulling()) { QueryCache::Key key(ast, context->getUserName()); @@ -749,7 +748,6 @@ static std::tuple executeQueryImpl( /// - active (write) use of the query cache is enabled /// then add a processor on top of the pipeline which stores the result in the query cache. if (!read_result_from_query_cache - && query_cache != nullptr && can_use_query_cache && settings.enable_writes_to_query_cache && res.pipeline.pulling() && (!astContainsNonDeterministicFunctions(ast, context) || settings.query_cache_store_results_of_queries_with_nondeterministic_functions)) @@ -927,6 +925,7 @@ static std::tuple executeQueryImpl( context, ast, my_can_use_query_cache = can_use_query_cache, + query_cache = query_cache, enable_writes_to_query_cache = settings.enable_writes_to_query_cache, query_cache_store_results_of_queries_with_nondeterministic_functions = settings.query_cache_store_results_of_queries_with_nondeterministic_functions, log_queries, @@ -941,9 +940,7 @@ static std::tuple executeQueryImpl( { /// If active (write) use of the query cache is enabled and the query is eligible for result caching, then store the query /// result buffered in the special-purpose cache processor (added on top of the pipeline) into the cache. - auto query_cache = context->getQueryCache(); - if (query_cache != nullptr - && pulling_pipeline + if (pulling_pipeline && my_can_use_query_cache && enable_writes_to_query_cache && (!astContainsNonDeterministicFunctions(ast, context) || query_cache_store_results_of_queries_with_nondeterministic_functions)) { From 5311ebe1d295939bfd576e39cb31734e0f4f42d1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 17 Jun 2023 20:12:51 +0000 Subject: [PATCH 176/515] Make query cache work effective with nested queries. --- src/Interpreters/executeQuery.cpp | 227 +++++++++--------- .../System/StorageSystemQueryCache.cpp | 2 +- ...494_query_cache_nested_query_bug.reference | 2 + .../02494_query_cache_nested_query_bug.sh | 22 ++ 4 files changed, 135 insertions(+), 118 deletions(-) create mode 100644 tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference create mode 100755 tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index cb68cacebda..c52dab722c9 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -646,91 +646,16 @@ 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(); + const bool can_use_query_cache = query_cache != nullptr && settings.use_query_cache && !internal && (ast->as() || ast->as()); + bool write_into_query_cache = false; if (!async_insert) { - /// We need to start the (implicit) transaction before getting the interpreter as this will get links to the latest snapshots - if (!context->getCurrentTransaction() && settings.implicit_transaction && !ast->as()) + /// If it is a non-internal SELECT, and passive/read use of the query cache is enabled, and the cache knows the query, then set + /// a pipeline with a source populated by the query cache. + auto get_result_from_query_cache = [&]() { - try - { - if (context->isGlobalContext()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot create transactions"); - - execute_implicit_tcl_query(context, ASTTransactionControl::BEGIN); - } - catch (Exception & e) - { - e.addMessage("while starting a transaction with 'implicit_transaction'"); - throw; - } - } - - interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal)); - - const auto & query_settings = context->getSettingsRef(); - if (context->getCurrentTransaction() && query_settings.throw_on_unsupported_query_inside_transaction) - { - if (!interpreter->supportsTransactions()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported for this type of query ({})", ast->getID()); - - } - - if (!interpreter->ignoreQuota() && !quota_checked) - { - quota = context->getQuota(); - if (quota) - { - if (ast->as() || ast->as()) - { - quota->used(QuotaType::QUERY_SELECTS, 1); - } - else if (ast->as()) - { - quota->used(QuotaType::QUERY_INSERTS, 1); - } - quota->used(QuotaType::QUERIES, 1); - quota->checkExceeded(QuotaType::ERRORS); - } - } - - if (!interpreter->ignoreLimits()) - { - limits.mode = LimitsMode::LIMITS_CURRENT; - limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode); - } - - if (auto * insert_interpreter = typeid_cast(&*interpreter)) - { - /// Save insertion table (not table function). TODO: support remote() table function. - auto table_id = insert_interpreter->getDatabaseTable(); - if (!table_id.empty()) - context->setInsertionTable(std::move(table_id)); - - if (insert_data_buffer_holder) - insert_interpreter->addBuffer(std::move(insert_data_buffer_holder)); - } - - { - std::unique_ptr span; - if (OpenTelemetry::CurrentContext().isTraceEnabled()) - { - auto * raw_interpreter_ptr = interpreter.get(); - String class_name(demangle(typeid(*raw_interpreter_ptr).name())); - span = std::make_unique(class_name + "::execute()"); - } - - res = interpreter->execute(); - - /// If - /// - it is a SELECT query, - /// - passive (read) use of the query cache is enabled, and - /// - the query cache knows the query result - /// then replace the pipeline by a new pipeline with a single source that is populated from the query cache - bool read_result_from_query_cache = false; /// a query must not read from *and* write to the query cache at the same time - if (can_use_query_cache && settings.enable_reads_from_query_cache - && res.pipeline.pulling()) + if (can_use_query_cache && settings.enable_reads_from_query_cache) { QueryCache::Key key(ast, context->getUserName()); QueryCache::Reader reader = query_cache->createReader(key); @@ -739,39 +664,114 @@ static std::tuple executeQueryImpl( QueryPipeline pipeline; pipeline.readFromQueryCache(reader.getSource(), reader.getSourceTotals(), reader.getSourceExtremes()); res.pipeline = std::move(pipeline); - read_result_from_query_cache = true; + return true; } } + return false; + }; - /// If - /// - it is a SELECT query, and - /// - active (write) use of the query cache is enabled - /// then add a processor on top of the pipeline which stores the result in the query cache. - if (!read_result_from_query_cache - && can_use_query_cache && settings.enable_writes_to_query_cache - && res.pipeline.pulling() - && (!astContainsNonDeterministicFunctions(ast, context) || settings.query_cache_store_results_of_queries_with_nondeterministic_functions)) + if (!get_result_from_query_cache()) + { + /// We need to start the (implicit) transaction before getting the interpreter as this will get links to the latest snapshots + if (!context->getCurrentTransaction() && settings.implicit_transaction && !ast->as()) { - QueryCache::Key key( - ast, res.pipeline.getHeader(), - context->getUserName(), settings.query_cache_share_between_users, - std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl), - settings.query_cache_compress_entries); - - const size_t num_query_runs = query_cache->recordQueryRun(key); - if (num_query_runs > settings.query_cache_min_query_runs) + try { - auto query_cache_writer = std::make_shared(query_cache->createWriter( - key, - std::chrono::milliseconds(settings.query_cache_min_query_duration.totalMilliseconds()), - settings.query_cache_squash_partial_results, - settings.max_block_size, - settings.query_cache_max_size_in_bytes, - settings.query_cache_max_entries)); - res.pipeline.writeResultIntoQueryCache(query_cache_writer); + if (context->isGlobalContext()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot create transactions"); + + execute_implicit_tcl_query(context, ASTTransactionControl::BEGIN); + } + catch (Exception & e) + { + e.addMessage("while starting a transaction with 'implicit_transaction'"); + throw; } } + interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal)); + + const auto & query_settings = context->getSettingsRef(); + if (context->getCurrentTransaction() && query_settings.throw_on_unsupported_query_inside_transaction) + { + if (!interpreter->supportsTransactions()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported for this type of query ({})", ast->getID()); + + } + + if (!interpreter->ignoreQuota() && !quota_checked) + { + quota = context->getQuota(); + if (quota) + { + if (ast->as() || ast->as()) + { + quota->used(QuotaType::QUERY_SELECTS, 1); + } + else if (ast->as()) + { + quota->used(QuotaType::QUERY_INSERTS, 1); + } + quota->used(QuotaType::QUERIES, 1); + quota->checkExceeded(QuotaType::ERRORS); + } + } + + if (!interpreter->ignoreLimits()) + { + limits.mode = LimitsMode::LIMITS_CURRENT; + limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode); + } + + if (auto * insert_interpreter = typeid_cast(&*interpreter)) + { + /// Save insertion table (not table function). TODO: support remote() table function. + auto table_id = insert_interpreter->getDatabaseTable(); + if (!table_id.empty()) + context->setInsertionTable(std::move(table_id)); + + if (insert_data_buffer_holder) + insert_interpreter->addBuffer(std::move(insert_data_buffer_holder)); + } + + { + std::unique_ptr span; + if (OpenTelemetry::CurrentContext().isTraceEnabled()) + { + auto * raw_interpreter_ptr = interpreter.get(); + String class_name(demangle(typeid(*raw_interpreter_ptr).name())); + span = std::make_unique(class_name + "::execute()"); + } + + res = interpreter->execute(); + + /// If it is a non-internal SELECT query, and active/write use of the query cache is enabled, then add a processor on + /// top of the pipeline which stores the result in the query cache. + if (can_use_query_cache && settings.enable_writes_to_query_cache + && (!astContainsNonDeterministicFunctions(ast, context) || settings.query_cache_store_results_of_queries_with_nondeterministic_functions)) + { + QueryCache::Key key( + ast, res.pipeline.getHeader(), + context->getUserName(), settings.query_cache_share_between_users, + std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl), + settings.query_cache_compress_entries); + + const size_t num_query_runs = query_cache->recordQueryRun(key); + if (num_query_runs > settings.query_cache_min_query_runs) + { + auto query_cache_writer = std::make_shared(query_cache->createWriter( + key, + std::chrono::milliseconds(settings.query_cache_min_query_duration.totalMilliseconds()), + settings.query_cache_squash_partial_results, + settings.max_block_size, + settings.query_cache_max_size_in_bytes, + settings.query_cache_max_entries)); + res.pipeline.writeResultIntoQueryCache(query_cache_writer); + write_into_query_cache = true; + } + } + + } } } @@ -924,10 +924,7 @@ static std::tuple executeQueryImpl( auto finish_callback = [elem, context, ast, - my_can_use_query_cache = can_use_query_cache, - query_cache = query_cache, - enable_writes_to_query_cache = settings.enable_writes_to_query_cache, - query_cache_store_results_of_queries_with_nondeterministic_functions = settings.query_cache_store_results_of_queries_with_nondeterministic_functions, + write_into_query_cache, log_queries, log_queries_min_type = settings.log_queries_min_type, log_queries_min_query_duration_ms = settings.log_queries_min_query_duration_ms.totalMilliseconds(), @@ -938,14 +935,10 @@ static std::tuple executeQueryImpl( pulling_pipeline = pipeline.pulling(), query_span](QueryPipeline & query_pipeline) mutable { - /// If active (write) use of the query cache is enabled and the query is eligible for result caching, then store the query - /// result buffered in the special-purpose cache processor (added on top of the pipeline) into the cache. - if (pulling_pipeline - && my_can_use_query_cache && enable_writes_to_query_cache - && (!astContainsNonDeterministicFunctions(ast, context) || query_cache_store_results_of_queries_with_nondeterministic_functions)) - { + if (write_into_query_cache) + /// Trigger the actual write of the buffered query result into the query cache. This is done explicitly to prevent + /// partial/garbage results in case of exceptions during query execution. query_pipeline.finalizeWriteInQueryCache(); - } QueryStatusPtr process_list_elem = context->getProcessListElement(); diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index 3dfc5cf298a..117fb4e8a5c 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -29,7 +29,7 @@ StorageSystemQueryCache::StorageSystemQueryCache(const StorageID & table_id_) void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - auto query_cache = context->getQueryCache(); + QueryCachePtr query_cache = context->getQueryCache(); if (!query_cache) return; diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference new file mode 100644 index 00000000000..389e2621455 --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference @@ -0,0 +1,2 @@ +2 +0 diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh new file mode 100755 index 00000000000..f6fb15f76eb --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash +# Tags: no-parallel +# Tag no-parallel: Messes with internal cache + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# Start with empty query cache (QC). +${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE" + +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS tab" +${CLICKHOUSE_CLIENT} --query "CREATE TABLE tab (a UInt64) ENGINE=MergeTree() ORDER BY a" +${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (1) (2) (3)" +${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (3) (4) (5)" + +# Verify that the first query does two aggregations and the second query zero aggregations. Since query cache is currently not integrated +# with EXPLAIN PLAN, we need need to check the logs. +${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab SETTINGS use_query_cache=1, max_threads=1" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab SETTINGS use_query_cache=1, max_threads=1" 2>&1 | grep "Aggregated. " | wc -l + +${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE" From b5b67f809495d4dc71f7bfbffccc41dd828c2f7a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 17 Jun 2023 21:11:09 +0000 Subject: [PATCH 177/515] Exclude fasttest --- tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh index f6fb15f76eb..f451e9b0709 100755 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest # Tag no-parallel: Messes with internal cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From 880c9aea09d737489e274c6ff05f12c5ab072c63 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Jun 2023 10:47:43 +0000 Subject: [PATCH 178/515] Fixing style. --- src/Interpreters/PreparedSets.cpp | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index c79b667789b..3c5e930a6cf 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -2,19 +2,17 @@ #include #include #include -#include #include -#include #include -#include "Common/logger_useful.h" -#include "Processors/QueryPlan/CreatingSetsStep.h" -#include "Processors/Executors/CompletedPipelineExecutor.h" -#include "Processors/QueryPlan/BuildQueryPipelineSettings.h" -#include "Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h" -#include "Processors/Sinks/EmptySink.h" -#include "Processors/Sinks/NullSink.h" +#include +#include +#include +#include +#include +#include +#include #include -#include "Core/Block.h" +#include #include #include #include From f3398f9908214f8d5ab06c5c625f2cf9f006364b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Jun 2023 12:17:52 +0000 Subject: [PATCH 179/515] Try to make more consisten type check for sets. --- src/Storages/MergeTree/RPNBuilder.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index 1acc3ec6ce9..d7dd7ff3654 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -384,8 +384,11 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( if (node_without_alias->column) { auto future_set = tryGetSetFromDAGNode(node_without_alias); + if (const auto * from_subquery = typeid_cast(future_set.get())) + return future_set; + auto set_types = future_set->getTypes(); - if (types_match(future_set->getTypes())) + if (types_match(set_types)) return future_set; } } From 8e7d06e0a4211de261a3aa9ef48561df30802403 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Jun 2023 12:56:24 +0000 Subject: [PATCH 180/515] Remove isReady from FutureSet iface. --- src/Functions/in.cpp | 8 ++------ src/Interpreters/ExpressionActions.cpp | 11 ++++------- src/Interpreters/PreparedSets.cpp | 11 +++++++---- src/Interpreters/PreparedSets.h | 6 +----- src/Planner/PlannerActionsVisitor.cpp | 2 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 4 ++-- src/Storages/MergeTree/RPNBuilder.cpp | 6 +----- src/Storages/VirtualColumnUtils.cpp | 2 +- 8 files changed, 19 insertions(+), 31 deletions(-) diff --git a/src/Functions/in.cpp b/src/Functions/in.cpp index 0e576b92aad..c19d1e72003 100644 --- a/src/Functions/in.cpp +++ b/src/Functions/in.cpp @@ -123,14 +123,10 @@ public: } auto future_set = column_set->getData(); - if (!future_set || !future_set->isReady()) + auto set = future_set ? future_set->get() : nullptr; + if (!future_set || !set) throw Exception(ErrorCodes::LOGICAL_ERROR, "Not-ready Set passed as the second argument for function '{}'", getName()); - // if (auto * for_tuple = typeid_cast(future_set.get())) - // if (!for_tuple->isReady()) - // for_tuple->buildForTuple(size_limits, transform_null_in); - - auto set = future_set->get(); auto set_types = set->getDataTypes(); if (tuple && set_types.size() != 1 && set_types.size() == tuple->tupleSize()) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 989fa12cba0..f1c577948eb 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -936,15 +936,12 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con for (const auto & action : actions) { if (action.node->type == ActionsDAG::ActionType::COLUMN && action.node->result_name == set_to_check) - { // Constant ColumnSet cannot be empty, so we only need to check non-constant ones. if (const auto * column_set = checkAndGetColumn(action.node->column.get())) - { - auto set = column_set->getData(); - if (set && set->isReady() && set->get()->getTotalRowCount() == 0) - return true; - } - } + if (auto future_set = column_set->getData()) + if (auto set = future_set->get()) + if (set->getTotalRowCount() == 0) + return true; } } diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 3c5e930a6cf..95cfa57857b 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -30,9 +30,7 @@ class FutureSetFromTuple final : public FutureSet public: FutureSetFromTuple(Block block, const Settings & settings); - bool isReady() const override { return true; } SetPtr get() const override { return set; } - SetPtr buildOrderedSetInplace(const ContextPtr & context) override; DataTypes getTypes() const override { return set->getElementsTypes(); } @@ -44,7 +42,6 @@ private: FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} -bool FutureSetFromStorage::isReady() const { return set != nullptr; } SetPtr FutureSetFromStorage::get() const { return set; } DataTypes FutureSetFromStorage::getTypes() const { return set->getElementsTypes(); } @@ -297,7 +294,13 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) return subquery.set; } -bool FutureSetFromSubquery::isReady() const { return subquery.set != nullptr && subquery.set->isCreated(); } +SetPtr FutureSetFromSubquery::get() const +{ + if (subquery.set != nullptr && subquery.set->isCreated()) + return subquery.set; + + return nullptr; +} std::unique_ptr FutureSetFromSubquery::build(const ContextPtr & context) { diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index f034aca747f..1e04bbbe3e8 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -34,9 +34,7 @@ class FutureSet public: virtual ~FutureSet() = default; - virtual bool isReady() const = 0; virtual SetPtr get() const = 0; - virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; virtual DataTypes getTypes() const = 0; @@ -50,7 +48,6 @@ class FutureSetFromStorage : public FutureSet public: FutureSetFromStorage(SetPtr set_); - bool isReady() const override; SetPtr get() const override; SetPtr buildOrderedSetInplace(const ContextPtr &) override; DataTypes getTypes() const override; @@ -91,8 +88,7 @@ class FutureSetFromSubquery : public FutureSet, public std::enable_shared_from_t public: FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, bool transform_null_in_); - bool isReady() const override; - SetPtr get() const override { return subquery.set; } + SetPtr get() const override; SetPtr buildOrderedSetInplace(const ContextPtr & context) override; diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 087cd3000b9..49312e5c4ad 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -664,7 +664,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma column.name = planner_context->createSetKey(in_second_argument); column.type = std::make_shared(); - bool set_is_created = set->isReady(); + bool set_is_created = set->get() != nullptr; auto column_set = ColumnSet::create(1, std::move(set)); if (set_is_created) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index da27e7b1293..751a1f95b7b 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -137,7 +137,7 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets for (auto & [description, future_set] : subqueries_for_sets) { - if (future_set->isReady()) + if (future_set->get()) continue; auto plan = future_set->build(context); @@ -172,7 +172,7 @@ std::vector> DelayedCreatingSetsStep::makePlansForSet for (auto & [description, future_set] : step.subqueries_for_sets) { - if (future_set->isReady()) + if (future_set->get()) continue; auto plan = future_set->build(step.context); diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index d7dd7ff3654..ad3fb984ce7 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -384,11 +384,7 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( if (node_without_alias->column) { auto future_set = tryGetSetFromDAGNode(node_without_alias); - if (const auto * from_subquery = typeid_cast(future_set.get())) - return future_set; - - auto set_types = future_set->getTypes(); - if (types_match(set_types)) + if (types_match(future_set->getTypes())) return future_set; } } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 58d00216dc7..b83afe766aa 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -222,7 +222,7 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex if (column_set) { auto future_set = column_set->getData(); - if (!future_set->isReady()) + if (!future_set->get()) { if (auto * set_from_subquery = typeid_cast(future_set.get())) { From 7280078bc6198647e8e695b8658beadc0e864ce2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Jun 2023 19:01:56 +0000 Subject: [PATCH 181/515] Refactoring a bit more. --- src/Interpreters/ActionsVisitor.cpp | 15 +- src/Interpreters/GlobalSubqueriesVisitor.h | 4 +- src/Interpreters/PreparedSets.cpp | 208 ++++++++++++------ src/Interpreters/PreparedSets.h | 91 ++++---- src/Interpreters/Set.h | 2 + src/Planner/CollectSets.cpp | 11 +- src/Planner/Planner.cpp | 8 +- src/Planner/PlannerActionsVisitor.cpp | 7 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 10 +- src/Processors/QueryPlan/CreatingSetsStep.h | 8 +- .../Optimizations/filterPushDown.cpp | 2 +- .../Transforms/CreatingSetsTransform.cpp | 10 +- src/Storages/KVStorageUtils.cpp | 13 +- src/Storages/MergeTree/RPNBuilder.cpp | 30 +-- 14 files changed, 248 insertions(+), 171 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 4b38f8c9434..91c490f8a4c 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -449,8 +449,8 @@ FutureSetPtr makeExplicitSet( if (const auto * low_cardinality_type = typeid_cast(element_type.get())) element_type = low_cardinality_type->getDictionaryType(); - auto set_key = PreparedSetKey::forLiteral(right_arg->getTreeHash(), set_element_types); - if (auto set = prepared_sets.getFuture(set_key)) + auto set_key = right_arg->getTreeHash(); + if (auto set = prepared_sets.find(set_key, set_element_types)) return set; /// Already prepared. Block block; @@ -1397,18 +1397,17 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool return {}; //std::cerr << queryToString(right_in_operand) << std::endl; - PreparedSetKey set_key; + PreparedSets::Hash set_key; if (data.getContext()->getSettingsRef().allow_experimental_analyzer && !identifier) { InterpreterSelectQueryAnalyzer interpreter(right_in_operand, data.getContext(), SelectQueryOptions().analyze(true).subquery()); auto query_tree = interpreter.getQueryTree(); if (auto * query_node = query_tree->as()) query_node->setIsSubquery(true); - // std::cerr << "============== " << interpreter.getQueryTree()->dumpTree() << std::endl; - set_key = PreparedSetKey::forSubquery(query_tree->getTreeHash()); + set_key = query_tree->getTreeHash(); } else - set_key = PreparedSetKey::forSubquery(right_in_operand->getTreeHash()); + set_key = right_in_operand->getTreeHash(); // std::cerr << set_key.toString() << std::endl; // std::cerr << data.prepared_sets->getSets().size() << std::endl; @@ -1416,7 +1415,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool // for (const auto & [k, v] : data.prepared_sets->getSets()) // std::cerr << "... " << k.toString(); - if (auto set = data.prepared_sets->getFuture(set_key)) + if (auto set = data.prepared_sets->findSubquery(set_key)) return set; FutureSetPtr external_table_set; @@ -1449,7 +1448,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool // String set_id = right_in_operand->getColumnName(); //bool transform_null_in = data.getContext()->getSettingsRef().transform_null_in; SubqueryForSet subquery_for_set; // = data.prepared_sets->createOrGetSubquery(set_id, set_key, data.set_size_limit, transform_null_in); - subquery_for_set.key = set_key.toString(); //right_in_operand->getColumnName(); + subquery_for_set.key = PreparedSets::toString(set_key, {}); //right_in_operand->getColumnName(); /** The following happens for GLOBAL INs or INs: * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 678258489f4..5b8f0c60d35 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -189,11 +189,11 @@ public: // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); // subquery_for_set.createSource(*interpreter, external_storage); auto key = subquery_or_table_name->getColumnName(); - auto set_key = PreparedSetKey::forSubquery(database_and_table_name->getTreeHash()); + auto set_key = database_and_table_name->getTreeHash(); // std::cerr << "====== Adding key " << set_key.toString() << std::endl; - if (!prepared_sets->getFuture(set_key)) + if (!prepared_sets->findSubquery(set_key)) { SubqueryForSet subquery_for_set; subquery_for_set.key = std::move(key); diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 95cfa57857b..f74fe86acce 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -33,7 +33,7 @@ public: SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override; - DataTypes getTypes() const override { return set->getElementsTypes(); } + const DataTypes & getTypes() const override { return set->getElementsTypes(); } private: SetPtr set; @@ -43,7 +43,7 @@ private: FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} SetPtr FutureSetFromStorage::get() const { return set; } -DataTypes FutureSetFromStorage::getTypes() const { return set->getElementsTypes(); } +const DataTypes & FutureSetFromStorage::getTypes() const { return set->getElementsTypes(); } SetPtr FutureSetFromStorage::buildOrderedSetInplace(const ContextPtr &) { @@ -51,47 +51,47 @@ SetPtr FutureSetFromStorage::buildOrderedSetInplace(const ContextPtr &) } -PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) -{ - /// Remove LowCardinality types from type list because Set doesn't support LowCardinality keys now, - /// just converts LowCardinality to ordinary types. - for (auto & type : types_) - type = recursiveRemoveLowCardinality(type); +// PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) +// { +// /// Remove LowCardinality types from type list because Set doesn't support LowCardinality keys now, +// /// just converts LowCardinality to ordinary types. +// for (auto & type : types_) +// type = recursiveRemoveLowCardinality(type); - PreparedSetKey key; - key.ast_hash = hash; - key.types = std::move(types_); - return key; -} +// PreparedSetKey key; +// key.ast_hash = hash; +// key.types = std::move(types_); +// return key; +// } -PreparedSetKey PreparedSetKey::forSubquery(Hash hash) -{ - PreparedSetKey key; - key.ast_hash = hash; - return key; -} +// PreparedSetKey PreparedSetKey::forSubquery(Hash hash) +// { +// PreparedSetKey key; +// key.ast_hash = hash; +// return key; +// } -bool PreparedSetKey::operator==(const PreparedSetKey & other) const -{ - if (ast_hash != other.ast_hash) - return false; +// bool PreparedSetKey::operator==(const PreparedSetKey & other) const +// { +// if (ast_hash != other.ast_hash) +// return false; - if (types.size() != other.types.size()) - return false; +// if (types.size() != other.types.size()) +// return false; - for (size_t i = 0; i < types.size(); ++i) - { - if (!types[i]->equals(*other.types[i])) - return false; - } +// for (size_t i = 0; i < types.size(); ++i) +// { +// if (!types[i]->equals(*other.types[i])) +// return false; +// } - return true; -} +// return true; +// } -String PreparedSetKey::toString() const +String PreparedSets::toString(const PreparedSets::Hash & hash, const DataTypes & types) { WriteBufferFromOwnString buf; - buf << "__set_" << ast_hash.first << "_" << ast_hash.second; + buf << "__set_" << hash.first << "_" << hash.second; if (!types.empty()) { buf << "("; @@ -143,54 +143,104 @@ String PreparedSetKey::toString() const // it->second->addStorage(std::move(storage)); // } +static bool equals(const DataTypes & lhs, const DataTypes & rhs) +{ + size_t size = lhs.size(); + if (size != rhs.size()) + return false; -FutureSetPtr PreparedSets::addFromStorage(const PreparedSetKey & key, SetPtr set_) + for (size_t i = 0; i < size; ++i) + { + if (!lhs[i]->equals(*rhs[i])) + return false; + } + + return true; +} + +static bool tryInsertSet(std::vector> & sets, FutureSetPtr new_set) +{ + auto types = new_set->getTypes(); + for (const auto & set : sets) + if (equals(set->getTypes(), new_set->getTypes())) + return false; + + sets.push_back(std::move(new_set)); + return true; +} + +static FutureSetPtr findSet(const std::vector> & sets, const DataTypes & types) +{ + for (const auto & set : sets) + if (equals(set->getTypes(), types)) + return set; + + return nullptr; +} + +FutureSetPtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_) { auto from_storage = std::make_shared(std::move(set_)); - auto [it, inserted] = sets.emplace(key, std::move(from_storage)); + auto & sets_by_hash = sets[key]; - if (!inserted) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + if (!tryInsertSet(sets_by_hash, from_storage)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, from_storage->getTypes())); - return it->second; + return from_storage; } -FutureSetPtr PreparedSets::addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings) +FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Settings & settings) { auto from_tuple = std::make_shared(std::move(block), settings); - auto [it, inserted] = sets.emplace(key, std::move(from_tuple)); + auto & sets_by_hash = sets[key]; - if (!inserted) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + if (!tryInsertSet(sets_by_hash, from_tuple)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, from_tuple->getTypes())); - return it->second; + return from_tuple; } -FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set) +FutureSetPtr PreparedSets::addFromSubquery(const Hash & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set) { - auto id = subquery.key; - auto from_subquery = std::make_shared(std::move(subquery), std::move(external_table_set), settings.transform_null_in); - auto [it, inserted] = sets.emplace(key, from_subquery); + auto from_subquery = std::make_shared(std::move(subquery), std::move(external_table_set), settings); + auto [it, inserted] = sets_from_subqueries.emplace(key, from_subquery); if (!inserted) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, {})); // std::cerr << key.toString() << std::endl; // std::cerr << "========= PreparedSets::addFromSubquery\n"; // std::cerr << StackTrace().toString() << std::endl; - subqueries.emplace_back(SetAndName{.name = id, .set = std::move(from_subquery)}); - return it->second; + return from_subquery; } -FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const +FutureSetPtr PreparedSets::find(const Hash & key, const DataTypes & types) const { auto it = sets.find(key); if (it == sets.end()) - return {}; + return nullptr; + + return findSet(it->second, types); +} + +std::shared_ptr PreparedSets::findSubquery(const Hash & key) const +{ + auto it = sets_from_subqueries.find(key); + if (it == sets_from_subqueries.end()) + return nullptr; + return it->second; } +// FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const +// { +// auto it = sets.find(key); +// if (it == sets.end()) +// return {}; +// return it->second; +// } + // SetPtr PreparedSets::get(const PreparedSetKey & key) const // { // auto it = sets.find(key); @@ -210,10 +260,14 @@ FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const // return res; // } -PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries() +std::vector> PreparedSets::detachSubqueries() { - auto res = std::move(subqueries); - subqueries = SubqueriesForSets(); + std::vector> res; + res.reserve(sets_from_subqueries.size()); + for (auto & [_, set] : sets_from_subqueries) + res.push_back(std::move(set)); + + sets_from_subqueries.clear(); return res; } @@ -267,10 +321,10 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) if (!context->getSettingsRef().use_index_for_in_with_subqueries) return nullptr; - if (subquery.set) + if (auto set = get()) { - if (subquery.set->hasExplicitSetElements()) - return subquery.set; + if (set->hasExplicitSetElements()) + return set; return nullptr; } @@ -278,10 +332,11 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) if (external_table_set) return subquery.set = external_table_set->buildOrderedSetInplace(context); - auto plan = buildPlan(context, true); + auto plan = buildPlan(context); if (!plan) return nullptr; + subquery.set->fillSetElements(); auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); pipeline.complete(std::make_shared(Block())); @@ -304,7 +359,7 @@ SetPtr FutureSetFromSubquery::get() const std::unique_ptr FutureSetFromSubquery::build(const ContextPtr & context) { - return buildPlan(context, false); + return buildPlan(context); } static SizeLimits getSizeLimitsForSet(const Settings & settings) @@ -312,15 +367,12 @@ static SizeLimits getSizeLimitsForSet(const Settings & settings) return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); } -std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & context, bool create_ordered_set) +std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & context) { - if (subquery.set) + if (subquery.set->isCreated()) return nullptr; const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings); - - subquery.set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); auto plan = subquery.detachSource(); auto description = subquery.key; @@ -355,15 +407,25 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) set->finishInsert(); } -FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, bool transform_null_in_) - : subquery(std::move(subquery_)), external_table_set(std::move(external_table_set_)), transform_null_in(transform_null_in_) {} - -DataTypes FutureSetFromSubquery::getTypes() const +FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, const Settings & settings) + : subquery(std::move(subquery_)), external_table_set(std::move(external_table_set_)) { - if (subquery.set) - return subquery.set->getElementsTypes(); + bool create_ordered_set = false; + auto size_limits = getSizeLimitsForSet(settings); + subquery.set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); + if (subquery.source) + subquery.set->setHeader(subquery.source->getCurrentDataStream().header.getColumnsWithTypeAndName()); +} - return Set::getElementTypes(subquery.source->getCurrentDataStream().header.getColumnsWithTypeAndName(), transform_null_in); +void FutureSetFromSubquery::setQueryPlan(std::unique_ptr source) +{ + subquery.source = std::move(source); + subquery.set->setHeader(subquery.source->getCurrentDataStream().header.getColumnsWithTypeAndName()); +} + +const DataTypes & FutureSetFromSubquery::getTypes() const +{ + return subquery.set->getElementsTypes(); } SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 1e04bbbe3e8..67f8a65ca17 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -37,7 +37,7 @@ public: virtual SetPtr get() const = 0; virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; - virtual DataTypes getTypes() const = 0; + virtual const DataTypes & getTypes() const = 0; }; using FutureSetPtr = std::shared_ptr; @@ -50,7 +50,7 @@ public: SetPtr get() const override; SetPtr buildOrderedSetInplace(const ContextPtr &) override; - DataTypes getTypes() const override; + const DataTypes & getTypes() const override; private: SetPtr set; @@ -86,7 +86,7 @@ public: class FutureSetFromSubquery : public FutureSet, public std::enable_shared_from_this { public: - FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, bool transform_null_in_); + FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, const Settings & settings); SetPtr get() const override; @@ -94,74 +94,87 @@ public: std::unique_ptr build(const ContextPtr & context); - DataTypes getTypes() const override; + const DataTypes & getTypes() const override; SubqueryForSet & getSubquery() { return subquery; } + void setQueryPlan(std::unique_ptr source); private: SubqueryForSet subquery; FutureSetPtr external_table_set; - bool transform_null_in; - std::unique_ptr buildPlan(const ContextPtr & context, bool create_ordered_set); + std::unique_ptr buildPlan(const ContextPtr & context); }; -struct PreparedSetKey -{ - using Hash = std::pair; +// struct PreparedSetKey +// { +// using Hash = std::pair; - /// Prepared sets for tuple literals are indexed by the hash of the tree contents and by the desired - /// data types of set elements (two different Sets can be required for two tuples with the same contents - /// if left hand sides of the IN operators have different types). - static PreparedSetKey forLiteral(Hash hash, DataTypes types_); +// /// Prepared sets for tuple literals are indexed by the hash of the tree contents and by the desired +// /// data types of set elements (two different Sets can be required for two tuples with the same contents +// /// if left hand sides of the IN operators have different types). +// static PreparedSetKey forLiteral(Hash hash, DataTypes types_); - /// Prepared sets for subqueries are indexed only by the AST contents because the type of the resulting - /// set is fully determined by the subquery. - static PreparedSetKey forSubquery(Hash hash); +// /// Prepared sets for subqueries are indexed only by the AST contents because the type of the resulting +// /// set is fully determined by the subquery. +// static PreparedSetKey forSubquery(Hash hash); - Hash ast_hash; - DataTypes types; /// Empty for subqueries. +// Hash ast_hash; +// DataTypes types; /// Empty for subqueries. - bool operator==(const PreparedSetKey & other) const; +// bool operator==(const PreparedSetKey & other) const; - String toString() const; +// String toString() const; - struct Hashing - { - UInt64 operator()(const PreparedSetKey & key) const { return key.ast_hash.first; } - }; -}; +// struct Hashing +// { +// UInt64 operator()(const PreparedSetKey & key) const { return key.ast_hash.first; } +// }; +// }; class PreparedSets { public: - struct SetAndName + + using Hash = std::pair; + struct Hashing { - String name; - std::shared_ptr set; + UInt64 operator()(const Hash & key) const { return key.first ^ key.second; } }; - using SubqueriesForSets = std::vector; - FutureSetPtr addFromStorage(const PreparedSetKey & key, SetPtr set_); - FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings); - FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set); + // struct SetAndName + // { + // String name; + // std::shared_ptr set; + // }; - FutureSetPtr getFuture(const PreparedSetKey & key) const; + using Sets = std::unordered_map>, Hashing>; + using SetsFromSubqueries = std::unordered_map, Hashing>; + + FutureSetPtr addFromStorage(const Hash & key, SetPtr set_); + FutureSetPtr addFromTuple(const Hash & key, Block block, const Settings & settings); + FutureSetPtr addFromSubquery(const Hash & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set); + + FutureSetPtr find(const Hash & key, const DataTypes & types) const; + std::shared_ptr findSubquery(const Hash & key) const; + + //FutureSetPtr getFuture(const PreparedSetKey & key) const; /// Get subqueries and clear them. /// We need to build a plan for subqueries just once. That's why we can clear them after accessing them. /// SetPtr would still be available for consumers of PreparedSets. - SubqueriesForSets detachSubqueries(); + std::vector> detachSubqueries(); - const std::unordered_map & getSets() const { return sets; } + const Sets & getNormalSets() const { return sets; } + const SetsFromSubqueries & getSetsFromSubquery() const { return sets_from_subqueries; } bool empty() const; -private: - std::unordered_map sets; + static String toString(const Hash & key, const DataTypes & types); - /// This is the information required for building sets - SubqueriesForSets subqueries; +private: + Sets sets; + SetsFromSubqueries sets_from_subqueries; }; using PreparedSetsPtr = std::shared_ptr; diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index a6908d2bdd3..df30992dd35 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -47,6 +47,8 @@ public: { } + void fillSetElements() { fill_set_elements = true; } + /** Set can be created either from AST or from a stream of data (subquery result). */ diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 84ab453c065..3b9512436ad 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -61,9 +61,8 @@ public: if (storage_set) { /// Handle storage_set as ready set. - auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash()); + auto set_key = in_second_argument->getTreeHash(); sets.addFromStorage(set_key, storage_set->getSet()); - //planner_context.registerSet(set_key, PlannerSet(FutureSet(storage_set->getSet()))); } else if (const auto * constant_node = in_second_argument->as()) { @@ -82,8 +81,8 @@ public: if (const auto * low_cardinality_type = typeid_cast(element_type.get())) element_type = low_cardinality_type->getDictionaryType(); - auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); - if (sets.getFuture(set_key)) + auto set_key = in_second_argument->getTreeHash(); + if (sets.find(set_key, set_element_types)) return; sets.addFromTuple(set_key, std::move(set), settings); @@ -94,8 +93,8 @@ public: in_second_argument_node_type == QueryTreeNodeType::UNION || in_second_argument_node_type == QueryTreeNodeType::TABLE) { - auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash()); - if (sets.getFuture(set_key)) + auto set_key = in_second_argument->getTreeHash(); + if (sets.findSubquery(set_key)) return; auto subquery_to_execute = in_second_argument; diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 4ac81e28f92..e7c10f6ef7a 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -933,18 +933,18 @@ void addBuildSubqueriesForSetsStepIfNeeded( auto subqueries = planner_context->getPreparedSets().detachSubqueries(); std::unordered_set useful_sets; - PreparedSets::SubqueriesForSets subqueries_for_sets; + //PreparedSets::SubqueriesForSets subqueries_for_sets; for (const auto & actions_to_execute : result_actions_to_execute) collectSetsFromActionsDAG(actions_to_execute, useful_sets); - auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.set.get()); }; + auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.get()); }; auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate)); subqueries.erase(it, subqueries.end()); for (auto & subquery : subqueries) { - auto & subquery_for_set = subquery.set->getSubquery(); + auto & subquery_for_set = subquery->getSubquery(); auto subquery_options = select_query_options.subquery(); Planner subquery_planner( subquery_for_set.query_tree, @@ -952,7 +952,7 @@ void addBuildSubqueriesForSetsStepIfNeeded( planner_context->getGlobalPlannerContext()); subquery_planner.buildQueryPlanIfNeeded(); - subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); + subquery->setQueryPlan(std::make_unique(std::move(subquery_planner).extractQueryPlan())); } //addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 49312e5c4ad..0c84052578c 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -651,14 +651,13 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma element_type = low_cardinality_type->getDictionaryType(); } - auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); + auto set_key = in_second_argument->getTreeHash(); + auto set = planner_context->getPreparedSets().find(set_key, set_element_types); - - auto set = planner_context->getPreparedSets().getFuture(set_key); if (!set) throw Exception(ErrorCodes::LOGICAL_ERROR, "No set is registered for key {}", - set_key.toString()); + PreparedSets::toString(set_key, set_element_types)); ColumnWithTypeAndName column; column.name = planner_context->createSetKey(in_second_argument); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 751a1f95b7b..ab028d967d8 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -126,7 +126,7 @@ void CreatingSetsStep::describePipeline(FormatSettings & settings) const IQueryPlanStep::describePipeline(processors, settings); } -void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets subqueries_for_sets, ContextPtr context) +void addCreatingSetsStep(QueryPlan & query_plan, std::vector> sets_from_subqueries, ContextPtr context) { DataStreams input_streams; input_streams.emplace_back(query_plan.getCurrentDataStream()); @@ -135,7 +135,7 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets plans.emplace_back(std::make_unique(std::move(query_plan))); query_plan = QueryPlan(); - for (auto & [description, future_set] : subqueries_for_sets) + for (auto & future_set : sets_from_subqueries) { if (future_set->get()) continue; @@ -170,7 +170,7 @@ std::vector> DelayedCreatingSetsStep::makePlansForSet // plans.emplace_back(std::make_unique(std::move(query_plan))); // query_plan = QueryPlan(); - for (auto & [description, future_set] : step.subqueries_for_sets) + for (auto & future_set : step.sets_from_subquery) { if (future_set->get()) continue; @@ -197,8 +197,8 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, } DelayedCreatingSetsStep::DelayedCreatingSetsStep( - DataStream input_stream, PreparedSets::SubqueriesForSets subqueries_for_sets_, ContextPtr context_) - : subqueries_for_sets(std::move(subqueries_for_sets_)), context(std::move(context_)) + DataStream input_stream, std::vector> sets_from_subquery_, ContextPtr context_) + : sets_from_subquery(std::move(sets_from_subquery_)), context(std::move(context_)) { input_streams = {input_stream}; output_stream = std::move(input_stream); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index 2a6cb43c45a..c41bb782e0b 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -52,7 +52,7 @@ public: class DelayedCreatingSetsStep final : public IQueryPlanStep { public: - DelayedCreatingSetsStep(DataStream input_stream, PreparedSets::SubqueriesForSets subqueries_for_sets_, ContextPtr context_); + DelayedCreatingSetsStep(DataStream input_stream, std::vector> sets_from_subquery_, ContextPtr context_); String getName() const override { return "DelayedCreatingSets"; } @@ -61,14 +61,14 @@ public: static std::vector> makePlansForSets(DelayedCreatingSetsStep && step); ContextPtr getContext() const { return context; } - PreparedSets::SubqueriesForSets detachSubqueries() { return std::move(subqueries_for_sets); } + std::vector> detachSets() { return std::move(sets_from_subquery); } private: - PreparedSets::SubqueriesForSets subqueries_for_sets; + std::vector> sets_from_subquery; ContextPtr context; }; -void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets subqueries_for_sets, ContextPtr context); +void addCreatingSetsStep(QueryPlan & query_plan, std::vector> sets_from_subquery, ContextPtr context); void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context); diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 63ba3d5b56c..5f04bf9ec18 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -273,7 +273,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes /// CreatingSets does not change header. /// We can push down filter and update header. /// Filter - DelayedCreatingSets - Something - child = std::make_unique(filter->getOutputStream(), delayed->detachSubqueries(), delayed->getContext()); + child = std::make_unique(filter->getOutputStream(), delayed->detachSets(), delayed->getContext()); std::swap(parent, child); std::swap(parent_node->children, child_node->children); std::swap(parent_node->children.front(), child_node->children.front()); diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index 3139fa5ed98..aa374446f3e 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -135,11 +135,11 @@ void CreatingSetsTransform::init() { is_initialized = true; - if (subquery.set) - { - //std::cerr << "=========== " << getInputPort().getHeader().dumpStructure() << std::endl; - subquery.set->setHeader(getInputPort().getHeader().getColumnsWithTypeAndName()); - } + // if (subquery.set) + // { + // //std::cerr << "=========== " << getInputPort().getHeader().dumpStructure() << std::endl; + // subquery.set->setHeader(getInputPort().getHeader().getColumnsWithTypeAndName()); + // } watch.restart(); startSubquery(); diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index f4a4b6e9af1..d205086639d 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -66,13 +66,14 @@ bool traverseASTFilter( return false; value = args.children.at(1); - PreparedSetKey set_key; - if ((value->as() || value->as())) - set_key = PreparedSetKey::forSubquery(value->getTreeHash()); - else - set_key = PreparedSetKey::forLiteral(value->getTreeHash(), {primary_key_type}); + PreparedSets::Hash set_key = value->getTreeHash(); + FutureSetPtr future_set; + + if ((value->as() || value->as())) + future_set = prepared_sets->findSubquery(set_key); + else + future_set = prepared_sets->find(set_key, {primary_key_type}); - FutureSetPtr future_set = prepared_sets->getFuture(set_key); if (!future_set) return false; diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index ad3fb984ce7..1bdbbfb3d40 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -299,14 +299,13 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const if (ast_node && prepared_sets) { - auto hash = ast_node->getTreeHash(); - auto key = PreparedSetKey::forSubquery(hash); + auto key = ast_node->getTreeHash(); + const auto & sets = prepared_sets->getNormalSets(); + auto it = sets.find(key); + if (it != sets.end() && !it->second.empty()) + return it->second.at(0); - for (const auto & [k, v] : prepared_sets->getSets()) - { - if (k.ast_hash == hash) - return v; - } + return prepared_sets->findSubquery(key); } else if (dag_node) { @@ -324,9 +323,9 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) if (prepared_sets && ast_node) { if (ast_node->as() || ast_node->as()) - return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); + return prepared_sets->findSubquery(ast_node->getTreeHash()); - return prepared_sets->getFuture(PreparedSetKey::forLiteral(ast_node->getTreeHash(), data_types)); + return prepared_sets->find(ast_node->getTreeHash(), data_types); } else if (dag_node) { @@ -369,14 +368,17 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( if (prepared_sets && ast_node) { if (ast_node->as() || ast_node->as()) - return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); + return prepared_sets->findSubquery(ast_node->getTreeHash()); auto tree_hash = ast_node->getTreeHash(); - for (const auto & [key, future_set] : prepared_sets->getSets()) - { - if (key.ast_hash == tree_hash && types_match(key.types)) + const auto & sets = prepared_sets->getNormalSets(); + auto it = sets.find(tree_hash); + if (it == sets.end()) + return nullptr; + + for (const auto & future_set : it->second) + if (types_match(future_set->getTypes())) return future_set; - } } else { From 4754d7863c1a4eb8a1109a13f7e632e81be9501e Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 20 Jun 2023 03:21:03 +0200 Subject: [PATCH 182/515] small --- src/Common/DateLUT.h | 12 +++++++----- src/Core/Settings.h | 2 +- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Common/DateLUT.h b/src/Common/DateLUT.h index d17cbef80ee..2045d4895e7 100644 --- a/src/Common/DateLUT.h +++ b/src/Common/DateLUT.h @@ -17,9 +17,9 @@ class DateLUT : private boost::noncopyable { public: - /// Return singleton DateLUTImpl instance for session timezone. - /// The session timezone is configured by a session setting. - /// If not set (empty string), it is the server timezone. + /// Return DateLUTImpl instance for session timezone. + /// session_timezone is a session-level setting. + /// If setting is not set, returns the server timezone. static ALWAYS_INLINE const DateLUTImpl & instance() { const auto & date_lut = getInstance(); @@ -37,7 +37,8 @@ public: return date_lut.getImplementation(timezone_from_context); } - /// Timezone is passed in query_context, but on CH-Client we have no query context, + /// On the server side, timezone is passed in query_context, + /// but on CH-client side we have no query context, /// and each time we modify client's global context const DB::ContextPtr global_context = DB::CurrentThread::get().getGlobalContext(); if (global_context) @@ -61,7 +62,8 @@ public: return date_lut.getImplementation(time_zone); } - // Return singleton DateLUTImpl for the server time zone. + /// Return singleton DateLUTImpl for the server time zone. + /// It may be set using 'timezone' server setting. static ALWAYS_INLINE const DateLUTImpl & serverTimezoneInstance() { const auto & date_lut = getInstance(); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 344f9a5ee6d..ccfe5c057e3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -768,7 +768,7 @@ class IColumn; M(Bool, allow_experimental_undrop_table_query, false, "Allow to use undrop query to restore dropped table in a limited time", 0) \ M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ - M(Timezone, session_timezone, "", "The default timezone for the current session. The server default timezone, if empty.", 0) \ + M(Timezone, session_timezone, "", "The default timezone for current session or query. The server default timezone, if empty.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. From b4273a1963dd5f66858d4a7c0feb7bc85dedc184 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Jun 2023 14:53:13 +0000 Subject: [PATCH 183/515] Fixing some tests. --- src/Interpreters/ActionsVisitor.cpp | 11 ++++++++--- src/Interpreters/PreparedSets.cpp | 12 +++++++++--- src/Interpreters/PreparedSets.h | 2 -- src/Interpreters/Set.cpp | 16 ++++++---------- src/Interpreters/Set.h | 2 +- src/Planner/CollectSets.cpp | 8 +++++--- src/Planner/PlannerActionsVisitor.cpp | 17 +++++++++++------ src/Processors/QueryPlan/CreatingSetsStep.cpp | 8 ++++++-- 8 files changed, 46 insertions(+), 30 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 91c490f8a4c..8ddc035417b 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -445,9 +445,14 @@ FutureSetPtr makeExplicitSet( if (left_tuple_type && left_tuple_type->getElements().size() != 1) set_element_types = left_tuple_type->getElements(); - for (auto & element_type : set_element_types) - if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - element_type = low_cardinality_type->getDictionaryType(); + set_element_types = Set::getElementTypes(std::move(set_element_types), context->getSettingsRef().transform_null_in); + + // for (auto & element_type : set_element_types) + // if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + // element_type = low_cardinality_type->getDictionaryType(); + + // if (!set_element_types.empty()) + // std::cerr << "========== " << set_element_types[0]->getName() << std::endl; auto set_key = right_arg->getTreeHash(); if (auto set = prepared_sets.find(set_key, set_element_types)) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index f74fe86acce..3281e7e9bb8 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -271,8 +271,6 @@ std::vector> PreparedSets::detachSubqueri return res; } -bool PreparedSets::empty() const { return sets.empty(); } - void SubqueryForSet::createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_) { source = std::make_unique(); @@ -330,13 +328,18 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) } if (external_table_set) - return subquery.set = external_table_set->buildOrderedSetInplace(context); + { + auto set = external_table_set->buildOrderedSetInplace(context); + if (set) + return subquery.set = set; + } auto plan = buildPlan(context); if (!plan) return nullptr; subquery.set->fillSetElements(); + subquery.set->initSetElements(); auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); pipeline.complete(std::make_shared(Block())); @@ -377,6 +380,9 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c auto plan = subquery.detachSource(); auto description = subquery.key; + if (!plan) + return nullptr; + auto creating_set = std::make_unique( plan->getCurrentDataStream(), description, diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 67f8a65ca17..967cef615c5 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -168,8 +168,6 @@ public: const Sets & getNormalSets() const { return sets; } const SetsFromSubqueries & getSetsFromSubquery() const { return sets_from_subqueries; } - bool empty() const; - static String toString(const Hash & key, const DataTypes & types); private: diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 83005f3d3a7..17df5064642 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -103,22 +103,18 @@ void NO_INLINE Set::insertFromBlockImplCase( } -DataTypes Set::getElementTypes(const ColumnsWithTypeAndName & header, bool transform_null_in) +DataTypes Set::getElementTypes(DataTypes types, bool transform_null_in) { - DataTypes data_types; - data_types.reserve(header.size()); - - for (const auto & column : header) + for (auto & type : types) { - data_types.push_back(column.type); - if (const auto * low_cardinality_type = typeid_cast(data_types.back().get())) - data_types.back() = low_cardinality_type->getDictionaryType(); + if (const auto * low_cardinality_type = typeid_cast(type.get())) + type = low_cardinality_type->getDictionaryType(); if (!transform_null_in) - data_types.back() = removeNullable(data_types.back()); + type = removeNullable(type); } - return data_types; + return types; } diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index df30992dd35..d1218830969 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -92,7 +92,7 @@ public: bool areTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const; void checkTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const; - static DataTypes getElementTypes(const ColumnsWithTypeAndName & header, bool transform_null_in); + static DataTypes getElementTypes(DataTypes types, bool transform_null_in); private: size_t keys_size = 0; diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 3b9512436ad..43431bd46ae 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -77,9 +77,11 @@ public: if (left_tuple_type && left_tuple_type->getElements().size() != 1) set_element_types = left_tuple_type->getElements(); - for (auto & element_type : set_element_types) - if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - element_type = low_cardinality_type->getDictionaryType(); + set_element_types = Set::getElementTypes(std::move(set_element_types), settings.transform_null_in); + + // for (auto & element_type : set_element_types) + // if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + // element_type = low_cardinality_type->getDictionaryType(); auto set_key = in_second_argument->getTreeHash(); if (sets.find(set_key, set_element_types)) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 0c84052578c..a7a2c6edc6c 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -639,6 +639,9 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma in_second_argument_node_type == QueryTreeNodeType::UNION || in_second_argument_node_type == QueryTreeNodeType::TABLE; + FutureSetPtr set; + auto set_key = in_second_argument->getTreeHash(); + if (!subquery_or_table) { set_element_types = {in_first_argument->getResultType()}; @@ -646,13 +649,15 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma if (left_tuple_type && left_tuple_type->getElements().size() != 1) set_element_types = left_tuple_type->getElements(); - for (auto & element_type : set_element_types) - if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - element_type = low_cardinality_type->getDictionaryType(); - } + // for (auto & element_type : set_element_types) + // if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + // element_type = low_cardinality_type->getDictionaryType(); - auto set_key = in_second_argument->getTreeHash(); - auto set = planner_context->getPreparedSets().find(set_key, set_element_types); + set_element_types = Set::getElementTypes(std::move(set_element_types), planner_context->getQueryContext()->getSettingsRef().transform_null_in); + set = planner_context->getPreparedSets().find(set_key, set_element_types); + } + else + set = planner_context->getPreparedSets().findSubquery(set_key); if (!set) throw Exception(ErrorCodes::LOGICAL_ERROR, diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index ab028d967d8..54d81dd7650 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -190,10 +190,14 @@ std::vector> DelayedCreatingSetsStep::makePlansForSet void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context) { - if (!prepared_sets || prepared_sets->empty()) + if (!prepared_sets) return; - addCreatingSetsStep(query_plan, prepared_sets->detachSubqueries(), context); + auto subqueries = prepared_sets->detachSubqueries(); + if (subqueries.empty()) + return; + + addCreatingSetsStep(query_plan, std::move(subqueries), context); } DelayedCreatingSetsStep::DelayedCreatingSetsStep( From ff8c01c6672f1781fd04fbf95997bfe5ea1d32fc Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 20 Jun 2023 16:37:52 +0000 Subject: [PATCH 184/515] Split long test 02149_schema_inference_formats_with_schema into several tests to avoid timeout in debug --- ..._inference_formats_with_schema_1.reference | 170 +++++++++++++ ..._schema_inference_formats_with_schema_1.sh | 39 +++ ..._inference_formats_with_schema_2.reference | 238 ++++++++++++++++++ ..._schema_inference_formats_with_schema_2.sh | 39 +++ ..._inference_formats_with_schema_3.reference | 27 ++ ..._schema_inference_formats_with_schema_3.sh | 34 +++ 6 files changed, 547 insertions(+) create mode 100644 tests/queries/0_stateless/02149_schema_inference_formats_with_schema_1.reference create mode 100755 tests/queries/0_stateless/02149_schema_inference_formats_with_schema_1.sh create mode 100644 tests/queries/0_stateless/02149_schema_inference_formats_with_schema_2.reference create mode 100755 tests/queries/0_stateless/02149_schema_inference_formats_with_schema_2.sh create mode 100644 tests/queries/0_stateless/02149_schema_inference_formats_with_schema_3.reference create mode 100755 tests/queries/0_stateless/02149_schema_inference_formats_with_schema_3.sh diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_1.reference b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_1.reference new file mode 100644 index 00000000000..4e020427ad0 --- /dev/null +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_1.reference @@ -0,0 +1,170 @@ +Arrow +int8 Nullable(Int8) +uint8 Nullable(UInt8) +int16 Nullable(Int16) +uint16 Nullable(UInt16) +int32 Nullable(Int32) +uint32 Nullable(UInt32) +int64 Nullable(Int64) +uint64 Nullable(UInt64) +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Nullable(Float32) +float64 Nullable(Float64) +decimal32 Nullable(Decimal(9, 5)) +decimal64 Nullable(Decimal(18, 5)) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Nullable(UInt16) +date32 Nullable(Date32) +0 1970-01-01 +1 1970-01-02 +str Nullable(String) +fixed_string Nullable(FixedString(3)) +Str: 0 100 +Str: 1 200 +array Array(Nullable(UInt64)) +tuple Tuple(`1` Nullable(UInt64), `2` Nullable(String)) +map Map(String, Nullable(UInt64)) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(`1` Array(Nullable(UInt64)), `2` Map(String, Nullable(UInt64)))) +nested2 Tuple(`1` Tuple(`1` Array(Array(Nullable(UInt64))), `2` Map(UInt64, Array(Tuple(`1` Nullable(UInt64), `2` Nullable(String))))), `2` Nullable(UInt8)) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +ArrowStream +int8 Nullable(Int8) +uint8 Nullable(UInt8) +int16 Nullable(Int16) +uint16 Nullable(UInt16) +int32 Nullable(Int32) +uint32 Nullable(UInt32) +int64 Nullable(Int64) +uint64 Nullable(UInt64) +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Nullable(Float32) +float64 Nullable(Float64) +decimal32 Nullable(Decimal(9, 5)) +decimal64 Nullable(Decimal(18, 5)) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Nullable(UInt16) +date32 Nullable(Date32) +0 1970-01-01 +1 1970-01-02 +str Nullable(String) +fixed_string Nullable(FixedString(3)) +Str: 0 100 +Str: 1 200 +array Array(Nullable(UInt64)) +tuple Tuple(`1` Nullable(UInt64), `2` Nullable(String)) +map Map(String, Nullable(UInt64)) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(`1` Array(Nullable(UInt64)), `2` Map(String, Nullable(UInt64)))) +nested2 Tuple(`1` Tuple(`1` Array(Array(Nullable(UInt64))), `2` Map(UInt64, Array(Tuple(`1` Nullable(UInt64), `2` Nullable(String))))), `2` Nullable(UInt8)) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +Parquet +int8 Nullable(Int8) +uint8 Nullable(UInt8) +int16 Nullable(Int16) +uint16 Nullable(UInt16) +int32 Nullable(Int32) +uint32 Nullable(UInt32) +int64 Nullable(Int64) +uint64 Nullable(UInt64) +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Nullable(Float32) +float64 Nullable(Float64) +decimal32 Nullable(Decimal(9, 5)) +decimal64 Nullable(Decimal(18, 5)) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Nullable(UInt16) +date32 Nullable(Date32) +0 1970-01-01 +1 1970-01-02 +str Nullable(String) +fixed_string Nullable(FixedString(3)) +Str: 0 100 +Str: 1 200 +array Array(Nullable(UInt64)) +tuple Tuple(`1` Nullable(UInt64), `2` Nullable(String)) +map Map(String, Nullable(UInt64)) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(`1` Array(Nullable(UInt64)), `2` Map(String, Nullable(UInt64)))) +nested2 Tuple(`1` Tuple(`1` Array(Array(Nullable(UInt64))), `2` Map(UInt64, Array(Tuple(`1` Nullable(UInt64), `2` Nullable(String))))), `2` Nullable(UInt8)) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +ORC +int8 Nullable(Int8) +uint8 Nullable(Int8) +int16 Nullable(Int16) +uint16 Nullable(Int16) +int32 Nullable(Int32) +uint32 Nullable(Int32) +int64 Nullable(Int64) +uint64 Nullable(Int64) +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Nullable(Float32) +float64 Nullable(Float64) +decimal32 Nullable(Decimal(9, 5)) +decimal64 Nullable(Decimal(18, 5)) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Nullable(Date32) +date32 Nullable(Date32) +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 +str Nullable(String) +fixed_string Nullable(String) +Str: 0 100 +Str: 1 200 +array Array(Nullable(Int64)) +tuple Tuple(`1` Nullable(Int64), `2` Nullable(String)) +map Map(String, Nullable(Int64)) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(`1` Array(Nullable(Int64)), `2` Map(String, Nullable(Int64)))) +nested2 Tuple(`1` Tuple(`1` Array(Array(Nullable(Int64))), `2` Map(Int64, Array(Tuple(`1` Nullable(Int64), `2` Nullable(String))))), `2` Nullable(Int8)) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +Native +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Date +date32 Date32 +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(UInt64, String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) +nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_1.sh b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_1.sh new file mode 100755 index 00000000000..a74d7c138d3 --- /dev/null +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_1.sh @@ -0,0 +1,39 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +DATA_FILE=$CURDIR/test_$CLICKHOUSE_TEST_UNIQUE_NAME.data + +for format in Arrow ArrowStream Parquet ORC Native +do + echo $format + $CLICKHOUSE_LOCAL -q "select toInt8(-number) as int8, toUInt8(number) as uint8, toInt16(-number) as int16, toUInt16(number) as uint16, toInt32(-number) as int32, toUInt32(number) as uint32, toInt64(-number) as int64, toUInt64(number) as uint64 from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', '$format')" + $CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', '$format')" + + $CLICKHOUSE_LOCAL -q "select toFloat32(number * 1.2) as float32, toFloat64(number / 1.3) as float64, toDecimal32(number / 0.3, 5) as decimal32, toDecimal64(number / 0.003, 5) as decimal64 from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', '$format')" + $CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', '$format')" + + $CLICKHOUSE_LOCAL -q "select toDate(number) as date, toDate32(number) as date32 from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', '$format')" + $CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', '$format')" + + $CLICKHOUSE_LOCAL -q "select concat('Str: ', toString(number)) as str, toFixedString(toString((number + 1) * 100 % 1000), 3) as fixed_string from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', '$format')" + $CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', '$format')" + + $CLICKHOUSE_LOCAL -q "select [number, number + 1] as array, (number, toString(number)) as tuple, map(toString(number), number) as map from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', '$format')" + $CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', '$format')" + + $CLICKHOUSE_LOCAL -q "select [([number, number + 1], map('42', number)), ([], map()), ([42], map('42', 42))] as nested1, (([[number], [number + 1], []], map(number, [(number, '42'), (number + 1, '42')])), 42) as nested2 from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', '$format')" + $CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', '$format')" +done + +rm $DATA_FILE diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_2.reference b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_2.reference new file mode 100644 index 00000000000..ec6ac9539f8 --- /dev/null +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_2.reference @@ -0,0 +1,238 @@ +TSVWithNamesAndTypes +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Date +date32 Date32 +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(UInt64, String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) +nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +TSVRawWithNamesAndTypes +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Date +date32 Date32 +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(UInt64, String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) +nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +CSVWithNamesAndTypes +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Date +date32 Date32 +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(UInt64, String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) +nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +JSONCompactEachRowWithNamesAndTypes +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Date +date32 Date32 +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(UInt64, String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) +nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +JSONCompactStringsEachRowWithNamesAndTypes +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Date +date32 Date32 +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(UInt64, String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) +nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +RowBinaryWithNamesAndTypes +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Date +date32 Date32 +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(UInt64, String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) +nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) +CustomSeparatedWithNamesAndTypes +int8 Int8 +uint8 UInt8 +int16 Int16 +uint16 UInt16 +int32 Int32 +uint32 UInt32 +int64 Int64 +uint64 UInt64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +decimal32 Decimal(9, 5) +decimal64 Decimal(18, 5) +0 0 0 0 +1.2 0.7692307692307692 3.33333 333.33333 +date Date +date32 Date32 +1970-01-01 1970-01-01 +1970-01-02 1970-01-02 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(UInt64) +tuple Tuple(UInt64, String) +map Map(String, UInt64) +[0,1] (0,'0') {'0':0} +[1,2] (1,'1') {'1':1} +nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) +nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) +[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) +[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_2.sh b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_2.sh new file mode 100755 index 00000000000..f6d2d7483a1 --- /dev/null +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_2.sh @@ -0,0 +1,39 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +DATA_FILE=$CURDIR/test_$CLICKHOUSE_TEST_UNIQUE_NAME.data + +for format in TSVWithNamesAndTypes TSVRawWithNamesAndTypes CSVWithNamesAndTypes JSONCompactEachRowWithNamesAndTypes JSONCompactStringsEachRowWithNamesAndTypes RowBinaryWithNamesAndTypes CustomSeparatedWithNamesAndTypes +do + echo $format + $CLICKHOUSE_LOCAL -q "select toInt8(-number) as int8, toUInt8(number) as uint8, toInt16(-number) as int16, toUInt16(number) as uint16, toInt32(-number) as int32, toUInt32(number) as uint32, toInt64(-number) as int64, toUInt64(number) as uint64 from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', '$format')" + $CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', '$format')" + + $CLICKHOUSE_LOCAL -q "select toFloat32(number * 1.2) as float32, toFloat64(number / 1.3) as float64, toDecimal32(number / 0.3, 5) as decimal32, toDecimal64(number / 0.003, 5) as decimal64 from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', '$format')" + $CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', '$format')" + + $CLICKHOUSE_LOCAL -q "select toDate(number) as date, toDate32(number) as date32 from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', '$format')" + $CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', '$format')" + + $CLICKHOUSE_LOCAL -q "select concat('Str: ', toString(number)) as str, toFixedString(toString((number + 1) * 100 % 1000), 3) as fixed_string from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', '$format')" + $CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', '$format')" + + $CLICKHOUSE_LOCAL -q "select [number, number + 1] as array, (number, toString(number)) as tuple, map(toString(number), number) as map from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', '$format')" + $CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', '$format')" + + $CLICKHOUSE_LOCAL -q "select [([number, number + 1], map('42', number)), ([], map()), ([42], map('42', 42))] as nested1, (([[number], [number + 1], []], map(number, [(number, '42'), (number + 1, '42')])), 42) as nested2 from numbers(2) format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', '$format')" + $CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', '$format')" +done + +rm $DATA_FILE + diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_3.reference b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_3.reference new file mode 100644 index 00000000000..caa6e99c224 --- /dev/null +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_3.reference @@ -0,0 +1,27 @@ +Avro + +int8 Int32 +uint8 Int32 +int16 Int32 +uint16 Int32 +int32 Int32 +uint32 Int32 +int64 Int64 +uint64 Int64 +0 0 0 0 0 0 0 0 +-1 1 -1 1 -1 1 -1 1 +float32 Float32 +float64 Float64 +0 0 +1.2 0.7692307692307692 +date Date32 +1970-01-01 +1970-01-02 +str String +fixed_string FixedString(3) +Str: 0 100 +Str: 1 200 +array Array(Int64) +nested Array(Array(Array(Int64))) +[0,1] [[[0],[1]]] +[1,2] [[[1],[2]]] diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_3.sh b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_3.sh new file mode 100755 index 00000000000..6ac530abf99 --- /dev/null +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_3.sh @@ -0,0 +1,34 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +DATA_FILE=$CURDIR/test_$CLICKHOUSE_TEST_UNIQUE_NAME.data + +echo "Avro" + +echo $format +$CLICKHOUSE_LOCAL -q "select toInt8(-number) as int8, toUInt8(number) as uint8, toInt16(-number) as int16, toUInt16(number) as uint16, toInt32(-number) as int32, toUInt32(number) as uint32, toInt64(-number) as int64, toUInt64(number) as uint64 from numbers(2) format Avro" > $DATA_FILE +$CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', 'Avro')" +$CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', 'Avro')" + +$CLICKHOUSE_LOCAL -q "select toFloat32(number * 1.2) as float32, toFloat64(number / 1.3) as float64 from numbers(2) format Avro" > $DATA_FILE +$CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', 'Avro')" +$CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', 'Avro')" + +$CLICKHOUSE_LOCAL -q "select toDate(number) as date from numbers(2) format Avro" > $DATA_FILE +$CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', 'Avro')" +$CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', 'Avro')" + +$CLICKHOUSE_LOCAL -q "select concat('Str: ', toString(number)) as str, toFixedString(toString((number + 1) * 100 % 1000), 3) as fixed_string from numbers(2) format Avro" > $DATA_FILE +$CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', 'Avro')" +$CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', 'Avro')" + +$CLICKHOUSE_LOCAL -q "select [number, number + 1] as array, [[[number], [number + 1]]] as nested from numbers(2) format Avro" > $DATA_FILE +$CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', 'Avro')" +$CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', 'Avro')" + +rm $DATA_FILE + From c6ada111a31438aac10d8ca44366746a97073a8b Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 20 Jun 2023 16:38:47 +0000 Subject: [PATCH 185/515] Delete old test --- ...ma_inference_formats_with_schema.reference | 435 ------------------ ...49_schema_inference_formats_with_schema.sh | 65 --- 2 files changed, 500 deletions(-) delete mode 100644 tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference delete mode 100755 tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference deleted file mode 100644 index 9a7a1611a7b..00000000000 --- a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference +++ /dev/null @@ -1,435 +0,0 @@ -Arrow -int8 Nullable(Int8) -uint8 Nullable(UInt8) -int16 Nullable(Int16) -uint16 Nullable(UInt16) -int32 Nullable(Int32) -uint32 Nullable(UInt32) -int64 Nullable(Int64) -uint64 Nullable(UInt64) -0 0 0 0 0 0 0 0 --1 1 -1 1 -1 1 -1 1 -float32 Nullable(Float32) -float64 Nullable(Float64) -decimal32 Nullable(Decimal(9, 5)) -decimal64 Nullable(Decimal(18, 5)) -0 0 0 0 -1.2 0.7692307692307692 3.33333 333.33333 -date Nullable(UInt16) -date32 Nullable(Date32) -0 1970-01-01 -1 1970-01-02 -str Nullable(String) -fixed_string Nullable(FixedString(3)) -Str: 0 100 -Str: 1 200 -array Array(Nullable(UInt64)) -tuple Tuple(`1` Nullable(UInt64), `2` Nullable(String)) -map Map(String, Nullable(UInt64)) -[0,1] (0,'0') {'0':0} -[1,2] (1,'1') {'1':1} -nested1 Array(Tuple(`1` Array(Nullable(UInt64)), `2` Map(String, Nullable(UInt64)))) -nested2 Tuple(`1` Tuple(`1` Array(Array(Nullable(UInt64))), `2` Map(UInt64, Array(Tuple(`1` Nullable(UInt64), `2` Nullable(String))))), `2` Nullable(UInt8)) -[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) -[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) -ArrowStream -int8 Nullable(Int8) -uint8 Nullable(UInt8) -int16 Nullable(Int16) -uint16 Nullable(UInt16) -int32 Nullable(Int32) -uint32 Nullable(UInt32) -int64 Nullable(Int64) -uint64 Nullable(UInt64) -0 0 0 0 0 0 0 0 --1 1 -1 1 -1 1 -1 1 -float32 Nullable(Float32) -float64 Nullable(Float64) -decimal32 Nullable(Decimal(9, 5)) -decimal64 Nullable(Decimal(18, 5)) -0 0 0 0 -1.2 0.7692307692307692 3.33333 333.33333 -date Nullable(UInt16) -date32 Nullable(Date32) -0 1970-01-01 -1 1970-01-02 -str Nullable(String) -fixed_string Nullable(FixedString(3)) -Str: 0 100 -Str: 1 200 -array Array(Nullable(UInt64)) -tuple Tuple(`1` Nullable(UInt64), `2` Nullable(String)) -map Map(String, Nullable(UInt64)) -[0,1] (0,'0') {'0':0} -[1,2] (1,'1') {'1':1} -nested1 Array(Tuple(`1` Array(Nullable(UInt64)), `2` Map(String, Nullable(UInt64)))) -nested2 Tuple(`1` Tuple(`1` Array(Array(Nullable(UInt64))), `2` Map(UInt64, Array(Tuple(`1` Nullable(UInt64), `2` Nullable(String))))), `2` Nullable(UInt8)) -[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) -[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) -Parquet -int8 Nullable(Int8) -uint8 Nullable(UInt8) -int16 Nullable(Int16) -uint16 Nullable(UInt16) -int32 Nullable(Int32) -uint32 Nullable(UInt32) -int64 Nullable(Int64) -uint64 Nullable(UInt64) -0 0 0 0 0 0 0 0 --1 1 -1 1 -1 1 -1 1 -float32 Nullable(Float32) -float64 Nullable(Float64) -decimal32 Nullable(Decimal(9, 5)) -decimal64 Nullable(Decimal(18, 5)) -0 0 0 0 -1.2 0.7692307692307692 3.33333 333.33333 -date Nullable(UInt16) -date32 Nullable(Date32) -0 1970-01-01 -1 1970-01-02 -str Nullable(String) -fixed_string Nullable(FixedString(3)) -Str: 0 100 -Str: 1 200 -array Array(Nullable(UInt64)) -tuple Tuple(`1` Nullable(UInt64), `2` Nullable(String)) -map Map(String, Nullable(UInt64)) -[0,1] (0,'0') {'0':0} -[1,2] (1,'1') {'1':1} -nested1 Array(Tuple(`1` Array(Nullable(UInt64)), `2` Map(String, Nullable(UInt64)))) -nested2 Tuple(`1` Tuple(`1` Array(Array(Nullable(UInt64))), `2` Map(UInt64, Array(Tuple(`1` Nullable(UInt64), `2` Nullable(String))))), `2` Nullable(UInt8)) -[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) -[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) -ORC -int8 Nullable(Int8) -uint8 Nullable(Int8) -int16 Nullable(Int16) -uint16 Nullable(Int16) -int32 Nullable(Int32) -uint32 Nullable(Int32) -int64 Nullable(Int64) -uint64 Nullable(Int64) -0 0 0 0 0 0 0 0 --1 1 -1 1 -1 1 -1 1 -float32 Nullable(Float32) -float64 Nullable(Float64) -decimal32 Nullable(Decimal(9, 5)) -decimal64 Nullable(Decimal(18, 5)) -0 0 0 0 -1.2 0.7692307692307692 3.33333 333.33333 -date Nullable(Date32) -date32 Nullable(Date32) -1970-01-01 1970-01-01 -1970-01-02 1970-01-02 -str Nullable(String) -fixed_string Nullable(String) -Str: 0 100 -Str: 1 200 -array Array(Nullable(Int64)) -tuple Tuple(`1` Nullable(Int64), `2` Nullable(String)) -map Map(String, Nullable(Int64)) -[0,1] (0,'0') {'0':0} -[1,2] (1,'1') {'1':1} -nested1 Array(Tuple(`1` Array(Nullable(Int64)), `2` Map(String, Nullable(Int64)))) -nested2 Tuple(`1` Tuple(`1` Array(Array(Nullable(Int64))), `2` Map(Int64, Array(Tuple(`1` Nullable(Int64), `2` Nullable(String))))), `2` Nullable(Int8)) -[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) -[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) -Native -int8 Int8 -uint8 UInt8 -int16 Int16 -uint16 UInt16 -int32 Int32 -uint32 UInt32 -int64 Int64 -uint64 UInt64 -0 0 0 0 0 0 0 0 --1 1 -1 1 -1 1 -1 1 -float32 Float32 -float64 Float64 -decimal32 Decimal(9, 5) -decimal64 Decimal(18, 5) -0 0 0 0 -1.2 0.7692307692307692 3.33333 333.33333 -date Date -date32 Date32 -1970-01-01 1970-01-01 -1970-01-02 1970-01-02 -str String -fixed_string FixedString(3) -Str: 0 100 -Str: 1 200 -array Array(UInt64) -tuple Tuple(UInt64, String) -map Map(String, UInt64) -[0,1] (0,'0') {'0':0} -[1,2] (1,'1') {'1':1} -nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) -nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) -[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) -[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) -TSVWithNamesAndTypes -int8 Int8 -uint8 UInt8 -int16 Int16 -uint16 UInt16 -int32 Int32 -uint32 UInt32 -int64 Int64 -uint64 UInt64 -0 0 0 0 0 0 0 0 --1 1 -1 1 -1 1 -1 1 -float32 Float32 -float64 Float64 -decimal32 Decimal(9, 5) -decimal64 Decimal(18, 5) -0 0 0 0 -1.2 0.7692307692307692 3.33333 333.33333 -date Date -date32 Date32 -1970-01-01 1970-01-01 -1970-01-02 1970-01-02 -str String -fixed_string FixedString(3) -Str: 0 100 -Str: 1 200 -array Array(UInt64) -tuple Tuple(UInt64, String) -map Map(String, UInt64) -[0,1] (0,'0') {'0':0} -[1,2] (1,'1') {'1':1} -nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) -nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) -[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) -[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) -TSVRawWithNamesAndTypes -int8 Int8 -uint8 UInt8 -int16 Int16 -uint16 UInt16 -int32 Int32 -uint32 UInt32 -int64 Int64 -uint64 UInt64 -0 0 0 0 0 0 0 0 --1 1 -1 1 -1 1 -1 1 -float32 Float32 -float64 Float64 -decimal32 Decimal(9, 5) -decimal64 Decimal(18, 5) -0 0 0 0 -1.2 0.7692307692307692 3.33333 333.33333 -date Date -date32 Date32 -1970-01-01 1970-01-01 -1970-01-02 1970-01-02 -str String -fixed_string FixedString(3) -Str: 0 100 -Str: 1 200 -array Array(UInt64) -tuple Tuple(UInt64, String) -map Map(String, UInt64) -[0,1] (0,'0') {'0':0} -[1,2] (1,'1') {'1':1} -nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) -nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) -[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) -[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) -CSVWithNamesAndTypes -int8 Int8 -uint8 UInt8 -int16 Int16 -uint16 UInt16 -int32 Int32 -uint32 UInt32 -int64 Int64 -uint64 UInt64 -0 0 0 0 0 0 0 0 --1 1 -1 1 -1 1 -1 1 -float32 Float32 -float64 Float64 -decimal32 Decimal(9, 5) -decimal64 Decimal(18, 5) -0 0 0 0 -1.2 0.7692307692307692 3.33333 333.33333 -date Date -date32 Date32 -1970-01-01 1970-01-01 -1970-01-02 1970-01-02 -str String -fixed_string FixedString(3) -Str: 0 100 -Str: 1 200 -array Array(UInt64) -tuple Tuple(UInt64, String) -map Map(String, UInt64) -[0,1] (0,'0') {'0':0} -[1,2] (1,'1') {'1':1} -nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) -nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) -[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) -[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) -JSONCompactEachRowWithNamesAndTypes -int8 Int8 -uint8 UInt8 -int16 Int16 -uint16 UInt16 -int32 Int32 -uint32 UInt32 -int64 Int64 -uint64 UInt64 -0 0 0 0 0 0 0 0 --1 1 -1 1 -1 1 -1 1 -float32 Float32 -float64 Float64 -decimal32 Decimal(9, 5) -decimal64 Decimal(18, 5) -0 0 0 0 -1.2 0.7692307692307692 3.33333 333.33333 -date Date -date32 Date32 -1970-01-01 1970-01-01 -1970-01-02 1970-01-02 -str String -fixed_string FixedString(3) -Str: 0 100 -Str: 1 200 -array Array(UInt64) -tuple Tuple(UInt64, String) -map Map(String, UInt64) -[0,1] (0,'0') {'0':0} -[1,2] (1,'1') {'1':1} -nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) -nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) -[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) -[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) -JSONCompactStringsEachRowWithNamesAndTypes -int8 Int8 -uint8 UInt8 -int16 Int16 -uint16 UInt16 -int32 Int32 -uint32 UInt32 -int64 Int64 -uint64 UInt64 -0 0 0 0 0 0 0 0 --1 1 -1 1 -1 1 -1 1 -float32 Float32 -float64 Float64 -decimal32 Decimal(9, 5) -decimal64 Decimal(18, 5) -0 0 0 0 -1.2 0.7692307692307692 3.33333 333.33333 -date Date -date32 Date32 -1970-01-01 1970-01-01 -1970-01-02 1970-01-02 -str String -fixed_string FixedString(3) -Str: 0 100 -Str: 1 200 -array Array(UInt64) -tuple Tuple(UInt64, String) -map Map(String, UInt64) -[0,1] (0,'0') {'0':0} -[1,2] (1,'1') {'1':1} -nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) -nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) -[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) -[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) -RowBinaryWithNamesAndTypes -int8 Int8 -uint8 UInt8 -int16 Int16 -uint16 UInt16 -int32 Int32 -uint32 UInt32 -int64 Int64 -uint64 UInt64 -0 0 0 0 0 0 0 0 --1 1 -1 1 -1 1 -1 1 -float32 Float32 -float64 Float64 -decimal32 Decimal(9, 5) -decimal64 Decimal(18, 5) -0 0 0 0 -1.2 0.7692307692307692 3.33333 333.33333 -date Date -date32 Date32 -1970-01-01 1970-01-01 -1970-01-02 1970-01-02 -str String -fixed_string FixedString(3) -Str: 0 100 -Str: 1 200 -array Array(UInt64) -tuple Tuple(UInt64, String) -map Map(String, UInt64) -[0,1] (0,'0') {'0':0} -[1,2] (1,'1') {'1':1} -nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) -nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) -[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) -[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) -CustomSeparatedWithNamesAndTypes -int8 Int8 -uint8 UInt8 -int16 Int16 -uint16 UInt16 -int32 Int32 -uint32 UInt32 -int64 Int64 -uint64 UInt64 -0 0 0 0 0 0 0 0 --1 1 -1 1 -1 1 -1 1 -float32 Float32 -float64 Float64 -decimal32 Decimal(9, 5) -decimal64 Decimal(18, 5) -0 0 0 0 -1.2 0.7692307692307692 3.33333 333.33333 -date Date -date32 Date32 -1970-01-01 1970-01-01 -1970-01-02 1970-01-02 -str String -fixed_string FixedString(3) -Str: 0 100 -Str: 1 200 -array Array(UInt64) -tuple Tuple(UInt64, String) -map Map(String, UInt64) -[0,1] (0,'0') {'0':0} -[1,2] (1,'1') {'1':1} -nested1 Array(Tuple(Array(UInt64), Map(String, UInt64))) -nested2 Tuple(Tuple(Array(Array(UInt64)), Map(UInt64, Array(Tuple(UInt64, String)))), UInt8) -[([0,1],{'42':0}),([],{}),([42],{'42':42})] (([[0],[1],[]],{0:[(0,'42'),(1,'42')]}),42) -[([1,2],{'42':1}),([],{}),([42],{'42':42})] (([[1],[2],[]],{1:[(1,'42'),(2,'42')]}),42) -Avro -CustomSeparatedWithNamesAndTypes -int8 Int32 -uint8 Int32 -int16 Int32 -uint16 Int32 -int32 Int32 -uint32 Int32 -int64 Int64 -uint64 Int64 -0 0 0 0 0 0 0 0 --1 1 -1 1 -1 1 -1 1 -float32 Float32 -float64 Float64 -0 0 -1.2 0.7692307692307692 -date Date32 -1970-01-01 -1970-01-02 -str String -fixed_string FixedString(3) -Str: 0 100 -Str: 1 200 -array Array(Int64) -nested Array(Array(Array(Int64))) -[0,1] [[[0],[1]]] -[1,2] [[[1],[2]]] diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh deleted file mode 100755 index cf5a086fb5e..00000000000 --- a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.sh +++ /dev/null @@ -1,65 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-parallel, no-fasttest - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - - -USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -FILE_NAME=test_$CLICKHOUSE_TEST_UNIQUE_NAME.data -DATA_FILE=$USER_FILES_PATH/$FILE_NAME - -for format in Arrow ArrowStream Parquet ORC Native TSVWithNamesAndTypes TSVRawWithNamesAndTypes CSVWithNamesAndTypes JSONCompactEachRowWithNamesAndTypes JSONCompactStringsEachRowWithNamesAndTypes RowBinaryWithNamesAndTypes CustomSeparatedWithNamesAndTypes -do - echo $format - $CLICKHOUSE_CLIENT -q "select toInt8(-number) as int8, toUInt8(number) as uint8, toInt16(-number) as int16, toUInt16(number) as uint16, toInt32(-number) as int32, toUInt32(number) as uint32, toInt64(-number) as int64, toUInt64(number) as uint64 from numbers(2) format $format" > $DATA_FILE - $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', '$format')" - $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', '$format')" - - $CLICKHOUSE_CLIENT -q "select toFloat32(number * 1.2) as float32, toFloat64(number / 1.3) as float64, toDecimal32(number / 0.3, 5) as decimal32, toDecimal64(number / 0.003, 5) as decimal64 from numbers(2) format $format" > $DATA_FILE - $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', '$format')" - $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', '$format')" - - $CLICKHOUSE_CLIENT -q "select toDate(number) as date, toDate32(number) as date32 from numbers(2) format $format" > $DATA_FILE - $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', '$format')" - $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', '$format')" - - $CLICKHOUSE_CLIENT -q "select concat('Str: ', toString(number)) as str, toFixedString(toString((number + 1) * 100 % 1000), 3) as fixed_string from numbers(2) format $format" > $DATA_FILE - $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', '$format')" - $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', '$format')" - - $CLICKHOUSE_CLIENT -q "select [number, number + 1] as array, (number, toString(number)) as tuple, map(toString(number), number) as map from numbers(2) format $format" > $DATA_FILE - $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', '$format')" - $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', '$format')" - - $CLICKHOUSE_CLIENT -q "select [([number, number + 1], map('42', number)), ([], map()), ([42], map('42', 42))] as nested1, (([[number], [number + 1], []], map(number, [(number, '42'), (number + 1, '42')])), 42) as nested2 from numbers(2) format $format" > $DATA_FILE - $CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', '$format')" - $CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', '$format')" -done - -echo "Avro" - -echo $format -$CLICKHOUSE_CLIENT -q "select toInt8(-number) as int8, toUInt8(number) as uint8, toInt16(-number) as int16, toUInt16(number) as uint16, toInt32(-number) as int32, toUInt32(number) as uint32, toInt64(-number) as int64, toUInt64(number) as uint64 from numbers(2) format Avro" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Avro')" -$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Avro')" - -$CLICKHOUSE_CLIENT -q "select toFloat32(number * 1.2) as float32, toFloat64(number / 1.3) as float64 from numbers(2) format Avro" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Avro')" -$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Avro')" - -$CLICKHOUSE_CLIENT -q "select toDate(number) as date from numbers(2) format Avro" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Avro')" -$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Avro')" - -$CLICKHOUSE_CLIENT -q "select concat('Str: ', toString(number)) as str, toFixedString(toString((number + 1) * 100 % 1000), 3) as fixed_string from numbers(2) format Avro" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Avro')" -$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Avro')" - -$CLICKHOUSE_CLIENT -q "select [number, number + 1] as array, [[[number], [number + 1]]] as nested from numbers(2) format Avro" > $DATA_FILE -$CLICKHOUSE_CLIENT -q "desc file('$FILE_NAME', 'Avro')" -$CLICKHOUSE_CLIENT -q "select * from file('$FILE_NAME', 'Avro')" - -rm $DATA_FILE - From 518ba6cbf6ee96deec772eebfaf8417347215460 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Jun 2023 16:43:22 +0000 Subject: [PATCH 186/515] Fix more tests. --- src/Interpreters/PreparedSets.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 3281e7e9bb8..17918a46954 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -265,9 +265,8 @@ std::vector> PreparedSets::detachSubqueri std::vector> res; res.reserve(sets_from_subqueries.size()); for (auto & [_, set] : sets_from_subqueries) - res.push_back(std::move(set)); + res.push_back(set); - sets_from_subqueries.clear(); return res; } From 5a1f819e7b068f9416e0aa9db6aec8697baa2fb8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Jun 2023 17:33:09 +0000 Subject: [PATCH 187/515] Fix more tests. --- src/Interpreters/ActionsVisitor.cpp | 2 +- src/Interpreters/PreparedSets.cpp | 39 ++++++++++++++++----------- src/Interpreters/PreparedSets.h | 12 ++++++--- src/Planner/CollectSets.cpp | 2 +- src/Planner/PlannerActionsVisitor.cpp | 6 ++++- src/Storages/KVStorageUtils.cpp | 2 +- src/Storages/MergeTree/RPNBuilder.cpp | 6 ++--- 7 files changed, 43 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 8ddc035417b..a9c0bbbbbc0 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -455,7 +455,7 @@ FutureSetPtr makeExplicitSet( // std::cerr << "========== " << set_element_types[0]->getName() << std::endl; auto set_key = right_arg->getTreeHash(); - if (auto set = prepared_sets.find(set_key, set_element_types)) + if (auto set = prepared_sets.findTuple(set_key, set_element_types)) return set; /// Already prepared. Block block; diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 17918a46954..144436589a0 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -178,21 +178,10 @@ static FutureSetPtr findSet(const std::vector> & sets return nullptr; } -FutureSetPtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_) -{ - auto from_storage = std::make_shared(std::move(set_)); - auto & sets_by_hash = sets[key]; - - if (!tryInsertSet(sets_by_hash, from_storage)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, from_storage->getTypes())); - - return from_storage; -} - FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Settings & settings) { auto from_tuple = std::make_shared(std::move(block), settings); - auto & sets_by_hash = sets[key]; + auto & sets_by_hash = sets_from_tuple[key]; if (!tryInsertSet(sets_by_hash, from_tuple)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, from_tuple->getTypes())); @@ -200,6 +189,17 @@ FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Set return from_tuple; } +FutureSetPtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_) +{ + auto from_storage = std::make_shared(std::move(set_)); + auto [it, inserted] = sets_from_storage.emplace(key, from_storage); + + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, {})); + + return from_storage; +} + FutureSetPtr PreparedSets::addFromSubquery(const Hash & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set) { auto from_subquery = std::make_shared(std::move(subquery), std::move(external_table_set), settings); @@ -215,10 +215,10 @@ FutureSetPtr PreparedSets::addFromSubquery(const Hash & key, SubqueryForSet subq return from_subquery; } -FutureSetPtr PreparedSets::find(const Hash & key, const DataTypes & types) const +FutureSetPtr PreparedSets::findTuple(const Hash & key, const DataTypes & types) const { - auto it = sets.find(key); - if (it == sets.end()) + auto it = sets_from_tuple.find(key); + if (it == sets_from_tuple.end()) return nullptr; return findSet(it->second, types); @@ -233,6 +233,15 @@ std::shared_ptr PreparedSets::findSubquery(const Hash & k return it->second; } +std::shared_ptr PreparedSets::findStorage(const Hash & key) const +{ + auto it = sets_from_storage.find(key); + if (it == sets_from_storage.end()) + return nullptr; + + return it->second; +} + // FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const // { // auto it = sets.find(key); diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 967cef615c5..49545ad1e2a 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -148,14 +148,16 @@ public: // std::shared_ptr set; // }; - using Sets = std::unordered_map>, Hashing>; + using SetsFromTuple = std::unordered_map>, Hashing>; + using SetsFromStorage = std::unordered_map, Hashing>; using SetsFromSubqueries = std::unordered_map, Hashing>; FutureSetPtr addFromStorage(const Hash & key, SetPtr set_); FutureSetPtr addFromTuple(const Hash & key, Block block, const Settings & settings); FutureSetPtr addFromSubquery(const Hash & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set); - FutureSetPtr find(const Hash & key, const DataTypes & types) const; + FutureSetPtr findTuple(const Hash & key, const DataTypes & types) const; + std::shared_ptr findStorage(const Hash & key) const; std::shared_ptr findSubquery(const Hash & key) const; //FutureSetPtr getFuture(const PreparedSetKey & key) const; @@ -165,13 +167,15 @@ public: /// SetPtr would still be available for consumers of PreparedSets. std::vector> detachSubqueries(); - const Sets & getNormalSets() const { return sets; } + const SetsFromTuple & getSetsFromTuple() const { return sets_from_tuple; } + const SetsFromStorage & getSetsFromStorage() const { return sets_from_storage; } const SetsFromSubqueries & getSetsFromSubquery() const { return sets_from_subqueries; } static String toString(const Hash & key, const DataTypes & types); private: - Sets sets; + SetsFromTuple sets_from_tuple; + SetsFromStorage sets_from_storage; SetsFromSubqueries sets_from_subqueries; }; diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 43431bd46ae..b0e6b272c9f 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -84,7 +84,7 @@ public: // element_type = low_cardinality_type->getDictionaryType(); auto set_key = in_second_argument->getTreeHash(); - if (sets.find(set_key, set_element_types)) + if (sets.findTuple(set_key, set_element_types)) return; sets.addFromTuple(set_key, std::move(set), settings); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index a7a2c6edc6c..40b6ec109dc 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -654,10 +654,14 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma // element_type = low_cardinality_type->getDictionaryType(); set_element_types = Set::getElementTypes(std::move(set_element_types), planner_context->getQueryContext()->getSettingsRef().transform_null_in); - set = planner_context->getPreparedSets().find(set_key, set_element_types); + set = planner_context->getPreparedSets().findTuple(set_key, set_element_types); } else + { set = planner_context->getPreparedSets().findSubquery(set_key); + if (!set) + set = planner_context->getPreparedSets().findStorage(set_key); + } if (!set) throw Exception(ErrorCodes::LOGICAL_ERROR, diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index d205086639d..b3b60fc064c 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -72,7 +72,7 @@ bool traverseASTFilter( if ((value->as() || value->as())) future_set = prepared_sets->findSubquery(set_key); else - future_set = prepared_sets->find(set_key, {primary_key_type}); + future_set = prepared_sets->findTuple(set_key, {primary_key_type}); if (!future_set) return false; diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index 1bdbbfb3d40..0b42111be45 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -300,7 +300,7 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const if (ast_node && prepared_sets) { auto key = ast_node->getTreeHash(); - const auto & sets = prepared_sets->getNormalSets(); + const auto & sets = prepared_sets->getSetsFromTuple(); auto it = sets.find(key); if (it != sets.end() && !it->second.empty()) return it->second.at(0); @@ -325,7 +325,7 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) if (ast_node->as() || ast_node->as()) return prepared_sets->findSubquery(ast_node->getTreeHash()); - return prepared_sets->find(ast_node->getTreeHash(), data_types); + return prepared_sets->findTuple(ast_node->getTreeHash(), data_types); } else if (dag_node) { @@ -371,7 +371,7 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( return prepared_sets->findSubquery(ast_node->getTreeHash()); auto tree_hash = ast_node->getTreeHash(); - const auto & sets = prepared_sets->getNormalSets(); + const auto & sets = prepared_sets->getSetsFromTuple(); auto it = sets.find(tree_hash); if (it == sets.end()) return nullptr; From c0b1d7f4f2bc9dc92c35b751c37a5f747f0a297a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Jun 2023 19:12:22 +0000 Subject: [PATCH 188/515] Fixing 01774_tuple_null_in --- src/Interpreters/ActionsVisitor.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index a9c0bbbbbc0..9125b38b48c 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -445,17 +445,17 @@ FutureSetPtr makeExplicitSet( if (left_tuple_type && left_tuple_type->getElements().size() != 1) set_element_types = left_tuple_type->getElements(); - set_element_types = Set::getElementTypes(std::move(set_element_types), context->getSettingsRef().transform_null_in); - - // for (auto & element_type : set_element_types) - // if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - // element_type = low_cardinality_type->getDictionaryType(); + for (auto & element_type : set_element_types) + if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + element_type = low_cardinality_type->getDictionaryType(); // if (!set_element_types.empty()) // std::cerr << "========== " << set_element_types[0]->getName() << std::endl; + auto set_element_keys = Set::getElementTypes(set_element_types, context->getSettingsRef().transform_null_in); + auto set_key = right_arg->getTreeHash(); - if (auto set = prepared_sets.findTuple(set_key, set_element_types)) + if (auto set = prepared_sets.findTuple(set_key, set_element_keys)) return set; /// Already prepared. Block block; From e4f057e3040d6088d5fffec568fc167fefba56b8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Jun 2023 19:40:37 +0000 Subject: [PATCH 189/515] Fix Set index with constant nullable comparison. --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 2 +- ...2789_set_index_nullable_condition_bug.reference | 1 + .../02789_set_index_nullable_condition_bug.sql | 14 ++++++++++++++ 3 files changed, 16 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02789_set_index_nullable_condition_bug.reference create mode 100644 tests/queries/0_stateless/02789_set_index_nullable_condition_bug.sql diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 120b3e43472..78e8b460433 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -554,7 +554,7 @@ void MergeTreeIndexConditionSet::traverseAST(ASTPtr & node) const if (atomFromAST(node)) { if (node->as() || node->as()) - node = makeASTFunction("__bitWrapperFunc", node); + else } else node = std::make_shared(UNKNOWN_FIELD); diff --git a/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.reference b/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.reference new file mode 100644 index 00000000000..e6ce356cd45 --- /dev/null +++ b/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.reference @@ -0,0 +1 @@ +v1 v2 diff --git a/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.sql b/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.sql new file mode 100644 index 00000000000..84c53d35d71 --- /dev/null +++ b/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.sql @@ -0,0 +1,14 @@ +CREATE OR REPLACE TABLE test_table +( + col1 String, + col2 String, + INDEX test_table_col2_idx col2 TYPE set(0) GRANULARITY 1 +) ENGINE = MergeTree() + ORDER BY col1 +AS SELECT 'v1', 'v2'; + +SELECT * FROM test_table +WHERE 1 == 1 AND col1 == col1 OR + 0 AND col2 == NULL; + +drop table if exists test_table; From 5bda0b535d9b161acf17980ebddd252e1dd4f587 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Jun 2023 19:45:03 +0000 Subject: [PATCH 190/515] Fixing build. --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 78e8b460433..6f4c5e48ea8 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -1,3 +1,4 @@ +#include "Parsers/queryToString.h" #include #include @@ -554,7 +555,7 @@ void MergeTreeIndexConditionSet::traverseAST(ASTPtr & node) const if (atomFromAST(node)) { if (node->as() || node->as()) - else + node = makeASTFunction("__bitWrapperFunc", makeASTFunction("toUInt8OrDefault", node)); } else node = std::make_shared(UNKNOWN_FIELD); From 266e81702a3c7cf68ac3f78995ce7b1fea8686e3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Jun 2023 19:48:39 +0000 Subject: [PATCH 191/515] Fixing clang-tidy --- src/Interpreters/PreparedSets.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 144436589a0..03c18730cc5 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -88,10 +88,10 @@ SetPtr FutureSetFromStorage::buildOrderedSetInplace(const ContextPtr &) // return true; // } -String PreparedSets::toString(const PreparedSets::Hash & hash, const DataTypes & types) +String PreparedSets::toString(const PreparedSets::Hash & key, const DataTypes & types) { WriteBufferFromOwnString buf; - buf << "__set_" << hash.first << "_" << hash.second; + buf << "__set_" << key.first << "_" << key.second; if (!types.empty()) { buf << "("; From 44798971f9bd4f17e1c856b5345532cd6c96f2a7 Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Tue, 20 Jun 2023 18:25:38 -0700 Subject: [PATCH 192/515] Fix a test, add APPEND to suggest. --- src/Client/Suggest.cpp | 2 +- tests/queries/0_stateless/02346_into_outfile_and_stdout.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 4e38add0ef5..c9cc27b4337 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -43,7 +43,7 @@ Suggest::Suggest() "IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE", "USER", "ROLE", "PROFILE", "QUOTA", "POLICY", "ROW", "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE", "IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "RANDOMIZED", - "INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE", "CLEANUP" + "INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE", "CLEANUP", "APPEND" }); } diff --git a/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh b/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh index 8ec086c97ef..38145ab3e51 100755 --- a/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh +++ b/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh @@ -66,7 +66,7 @@ performBadQuery "bad_query_incorrect_usage" "SELECT 1, 2, 3 INTO OUTFILE AND STD performBadQuery "bad_query_no_into_outfile" "SELECT 1, 2, 3 AND STDOUT'" "SYNTAX_ERROR" -performFileExists "bad_query_file_exists" "SELECT 1, 2, 3 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists.out' AND STDOUT" "File ${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists exists, consider using APPEND or TRUNCATE if appropriate. (CANNOT_OPEN_FILE)" +performFileExists "bad_query_file_exists" "SELECT 1, 2, 3 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists.out' AND STDOUT" "File ${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists.out exists, consider using APPEND or TRUNCATE." performCompression "compression" "SELECT * FROM (SELECT 'Hello, World! From clickhouse.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_compression.gz' AND STDOUT COMPRESSION 'GZ' FORMAT TabSeparated" From dab42af754735f6d3e002af224a8f2e708124e14 Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Tue, 20 Jun 2023 22:43:25 -0700 Subject: [PATCH 193/515] Fix another test. --- .../0_stateless/02050_clickhouse_client_local_exception.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02050_clickhouse_client_local_exception.sh b/tests/queries/0_stateless/02050_clickhouse_client_local_exception.sh index eac085a914e..a2bb29377ac 100755 --- a/tests/queries/0_stateless/02050_clickhouse_client_local_exception.sh +++ b/tests/queries/0_stateless/02050_clickhouse_client_local_exception.sh @@ -10,4 +10,4 @@ function cleanup() rm "${CLICKHOUSE_TMP}/test_exception" } trap cleanup EXIT -$CLICKHOUSE_LOCAL --query="SELECT 1 INTO OUTFILE '${CLICKHOUSE_TMP}/test_exception' FORMAT Native" 2>&1 | grep -q "Code: 76. DB::ErrnoException:" && echo 'OK' || echo 'FAIL' ||: +$CLICKHOUSE_LOCAL --query="SELECT 1 INTO OUTFILE '${CLICKHOUSE_TMP}/test_exception' FORMAT Native" 2>&1 | grep -q "Code: 504. DB::Exception:" && echo 'OK' || echo 'FAIL' ||: From 5e4805665974ded066713ccdaacc76cfff618b5d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 21 Jun 2023 09:21:01 +0000 Subject: [PATCH 194/515] Disable analyzer --- .../0_stateless/02494_query_cache_nested_query_bug.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh index f451e9b0709..c5ca1d88b4d 100755 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-parallel # Tag no-parallel: Messes with internal cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) @@ -16,7 +16,7 @@ ${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (3) (4) (5)" # Verify that the first query does two aggregations and the second query zero aggregations. Since query cache is currently not integrated # with EXPLAIN PLAN, we need need to check the logs. -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab SETTINGS use_query_cache=1, max_threads=1" 2>&1 | grep "Aggregated. " | wc -l -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab SETTINGS use_query_cache=1, max_threads=1" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer=0" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer=0" 2>&1 | grep "Aggregated. " | wc -l ${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE" From 0149f905decbb5de715245463c82f470bd8af7c1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 21 Jun 2023 11:26:54 +0200 Subject: [PATCH 195/515] Update MergeTreeIndexSet.cpp --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 6f4c5e48ea8..7f6edfc7ec4 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -1,4 +1,3 @@ -#include "Parsers/queryToString.h" #include #include From 446320b182e7ceba9df32307e7c18be9cbbcfe34 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 21 Jun 2023 11:28:15 +0200 Subject: [PATCH 196/515] Update 02789_set_index_nullable_condition_bug.sql --- .../0_stateless/02789_set_index_nullable_condition_bug.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.sql b/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.sql index 84c53d35d71..fed030f170d 100644 --- a/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.sql +++ b/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.sql @@ -1,4 +1,5 @@ -CREATE OR REPLACE TABLE test_table +drop table if exists test_table; +CREATE TABLE test_table ( col1 String, col2 String, From 52f6a4f98a713e3e2b622c304f8d6d6de2deba92 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 21 Jun 2023 12:46:09 +0200 Subject: [PATCH 197/515] Fix style --- .../0_stateless/02149_schema_inference_formats_with_schema_3.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_3.sh b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_3.sh index 6ac530abf99..fef85f0f1ed 100755 --- a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_3.sh +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_3.sh @@ -9,7 +9,6 @@ DATA_FILE=$CURDIR/test_$CLICKHOUSE_TEST_UNIQUE_NAME.data echo "Avro" -echo $format $CLICKHOUSE_LOCAL -q "select toInt8(-number) as int8, toUInt8(number) as uint8, toInt16(-number) as int16, toUInt16(number) as uint16, toInt32(-number) as int32, toUInt32(number) as uint32, toInt64(-number) as int64, toUInt64(number) as uint64 from numbers(2) format Avro" > $DATA_FILE $CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', 'Avro')" $CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', 'Avro')" From c5f8cb644e6187a1080a00988726631e556065c7 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 21 Jun 2023 12:46:24 +0200 Subject: [PATCH 198/515] Update test reference --- .../02149_schema_inference_formats_with_schema_3.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_3.reference b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_3.reference index caa6e99c224..d076bf9ed4e 100644 --- a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_3.reference +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema_3.reference @@ -1,5 +1,4 @@ Avro - int8 Int32 uint8 Int32 int16 Int32 From e650a6094aa95a4d2b27eb66d8229452ca59ac5c Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Wed, 21 Jun 2023 20:20:37 +0800 Subject: [PATCH 199/515] fix core dump when compile expression --- src/DataTypes/Native.cpp | 6 ++++-- .../02790_fix_coredump_when_compile_expression.reference | 1 + .../02790_fix_coredump_when_compile_expression.sql | 4 ++++ 3 files changed, 9 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02790_fix_coredump_when_compile_expression.reference create mode 100644 tests/queries/0_stateless/02790_fix_coredump_when_compile_expression.sql diff --git a/src/DataTypes/Native.cpp b/src/DataTypes/Native.cpp index 6f1ea851dce..9af50a180ea 100644 --- a/src/DataTypes/Native.cpp +++ b/src/DataTypes/Native.cpp @@ -123,9 +123,11 @@ llvm::Value * nativeCast(llvm::IRBuilderBase & b, const DataTypePtr & from_type, } else if (to_type->isNullable()) { - auto * from_native_type = toNativeType(b, from_type); + auto * to_native_type = toNativeType(b, to_type); auto * inner = nativeCast(b, from_type, value, removeNullable(to_type)); - return b.CreateInsertValue(llvm::Constant::getNullValue(from_native_type), inner, {0}); + auto * res_ptr = b.CreateAlloca(to_native_type); + auto * res_value = b.CreateLoad(to_native_type, res_ptr); + return b.CreateInsertValue(res_value, inner, {0}); } else { diff --git a/tests/queries/0_stateless/02790_fix_coredump_when_compile_expression.reference b/tests/queries/0_stateless/02790_fix_coredump_when_compile_expression.reference new file mode 100644 index 00000000000..b8626c4cff2 --- /dev/null +++ b/tests/queries/0_stateless/02790_fix_coredump_when_compile_expression.reference @@ -0,0 +1 @@ +4 diff --git a/tests/queries/0_stateless/02790_fix_coredump_when_compile_expression.sql b/tests/queries/0_stateless/02790_fix_coredump_when_compile_expression.sql new file mode 100644 index 00000000000..90995da0cca --- /dev/null +++ b/tests/queries/0_stateless/02790_fix_coredump_when_compile_expression.sql @@ -0,0 +1,4 @@ +CREATE TABLE test (col1 Nullable(DOUBLE), col2 Nullable(DOUBLE), col3 DOUBLE) ENGINE=Memory; + +insert into test values(1.0 , 2.0, 3.0); +select multiIf(col1 > 2, col2/col3, 4.0) from test SETTINGS min_count_to_compile_expression=0; From 75b7121759ac4bb1fc12078042d52a445ae319a5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 21 Jun 2023 12:37:06 +0000 Subject: [PATCH 200/515] Exclude fasttest again :-( --- .../queries/0_stateless/02494_query_cache_nested_query_bug.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh index c5ca1d88b4d..394367919e9 100755 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest # Tag no-parallel: Messes with internal cache +# no-fasttest: Produces wrong results in fasttest, unclear why, didn't reproduce locally. CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 3224e49757a4d6258f1601915b8ecdffebb79a72 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Wed, 21 Jun 2023 22:46:41 +0800 Subject: [PATCH 201/515] fix --- src/DataTypes/Native.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/DataTypes/Native.cpp b/src/DataTypes/Native.cpp index 9af50a180ea..fd3716c2291 100644 --- a/src/DataTypes/Native.cpp +++ b/src/DataTypes/Native.cpp @@ -125,9 +125,7 @@ llvm::Value * nativeCast(llvm::IRBuilderBase & b, const DataTypePtr & from_type, { auto * to_native_type = toNativeType(b, to_type); auto * inner = nativeCast(b, from_type, value, removeNullable(to_type)); - auto * res_ptr = b.CreateAlloca(to_native_type); - auto * res_value = b.CreateLoad(to_native_type, res_ptr); - return b.CreateInsertValue(res_value, inner, {0}); + return b.CreateInsertValue(llvm::Constant::getNullValue(to_native_type), inner, {0}); } else { From 9aa4647f4fb2c25a74df952df2d6d815c27e9ee9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 21 Jun 2023 20:48:12 +0200 Subject: [PATCH 202/515] Delete feature --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 5 +--- src/Disks/IO/CachedOnDiskReadBufferFromFile.h | 2 -- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 5 +--- .../IO/CachedOnDiskWriteBufferFromFile.h | 2 -- .../Cached/CachedObjectStorage.cpp | 15 ---------- .../Cached/CachedObjectStorage.h | 4 --- src/IO/ReadSettings.h | 1 - src/IO/WriteSettings.h | 1 - src/Interpreters/Cache/FileCache.cpp | 14 +++------ src/Interpreters/Cache/FileCache.h | 1 - src/Interpreters/Cache/FileCacheSettings.cpp | 2 -- src/Interpreters/Cache/FileCacheSettings.h | 2 -- src/Interpreters/Cache/FileSegment.h | 7 ----- src/Interpreters/Cache/Metadata.cpp | 3 -- .../InterpreterDescribeCacheQuery.cpp | 2 -- .../System/StorageSystemFilesystemCache.cpp | 29 +++++++++---------- 16 files changed, 20 insertions(+), 75 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 76d54f9d27c..59cf8f0174c 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -69,7 +69,6 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile( , allow_seeks_after_first_read(allow_seeks_after_first_read_) , use_external_buffer(use_external_buffer_) , query_context_holder(cache_->getQueryContextHolder(query_id, settings_)) - , is_persistent(settings_.is_file_cache_persistent) , cache_log(cache_log_) { } @@ -125,7 +124,7 @@ void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size) } else { - CreateFileSegmentSettings create_settings(is_persistent ? FileSegmentKind::Persistent : FileSegmentKind::Regular); + CreateFileSegmentSettings create_settings(FileSegmentKind::Regular); file_segments = cache->getOrSet(cache_key, offset, size, file_size.value(), create_settings); } @@ -149,8 +148,6 @@ CachedOnDiskReadBufferFromFile::getCacheReadBuffer(const FileSegment & file_segm { ProfileEventTimeIncrement watch(ProfileEvents::CachedReadBufferCreateBufferMicroseconds); - /// Use is_persistent flag from in-memory state of the filesegment, - /// because it is consistent with what is written on disk. auto path = file_segment.getPathInLocalCache(); ReadSettings local_read_settings{settings}; diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h index 8faf23ad343..b4e7701de75 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h @@ -147,8 +147,6 @@ private: FileCache::QueryContextHolderPtr query_context_holder; - bool is_persistent; - std::shared_ptr cache_log; }; diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index b7727555480..33d4ed7b3d7 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -194,7 +194,6 @@ CachedOnDiskWriteBufferFromFile::CachedOnDiskWriteBufferFromFile( FileCachePtr cache_, const String & source_path_, const FileCache::Key & key_, - bool is_persistent_cache_file_, const String & query_id_, const WriteSettings & settings_) : WriteBufferFromFileDecorator(std::move(impl_)) @@ -202,7 +201,6 @@ CachedOnDiskWriteBufferFromFile::CachedOnDiskWriteBufferFromFile( , cache(cache_) , source_path(source_path_) , key(key_) - , is_persistent_cache_file(is_persistent_cache_file_) , query_id(query_id_) , enable_cache_log(!query_id_.empty() && settings_.enable_filesystem_cache_log) , throw_on_error_from_cache(settings_.throw_on_error_from_cache) @@ -255,8 +253,7 @@ void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size, bool t try { - auto segment_kind = is_persistent_cache_file ? FileSegmentKind::Persistent : FileSegmentKind::Regular; - if (!cache_writer->write(data, size, current_download_offset, segment_kind)) + if (!cache_writer->write(data, size, current_download_offset, FileSegmentKind::Regular)) { LOG_INFO(log, "Write-through cache is stopped as cache limit is reached and nothing can be evicted"); return; diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index 8642886d6de..8d39b6eed42 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -72,7 +72,6 @@ public: FileCachePtr cache_, const String & source_path_, const FileCache::Key & key_, - bool is_persistent_cache_file_, const String & query_id_, const WriteSettings & settings_); @@ -89,7 +88,6 @@ private: String source_path; FileCache::Key key; - bool is_persistent_cache_file; size_t current_download_offset = 0; const String query_id; diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index b1cf8226895..3e7c4d12c42 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -119,7 +119,6 @@ std::unique_ptr CachedObjectStorage::writeObject( /// N cache, implementation_buffer->getFileName(), key, - modified_write_settings.is_file_cache_persistent, CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() ? std::string(CurrentThread::getQueryId()) : "", modified_write_settings); } @@ -164,20 +163,6 @@ void CachedObjectStorage::removeObjectsIfExist(const StoredObjects & objects) object_storage->removeObjectsIfExist(objects); } -ReadSettings CachedObjectStorage::getAdjustedSettingsFromMetadataFile(const ReadSettings & settings, const std::string & path) const -{ - ReadSettings new_settings{settings}; - new_settings.is_file_cache_persistent = isFileWithPersistentCache(path) && cache_settings.do_not_evict_index_and_mark_files; - return new_settings; -} - -WriteSettings CachedObjectStorage::getAdjustedSettingsFromMetadataFile(const WriteSettings & settings, const std::string & path) const -{ - WriteSettings new_settings{settings}; - new_settings.is_file_cache_persistent = isFileWithPersistentCache(path) && cache_settings.do_not_evict_index_and_mark_files; - return new_settings; -} - void CachedObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT const StoredObject & object_from, const StoredObject & object_to, diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index ba9fbd02d94..76f16c9d930 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -108,10 +108,6 @@ public: bool supportParallelWrite() const override { return object_storage->supportParallelWrite(); } - ReadSettings getAdjustedSettingsFromMetadataFile(const ReadSettings & settings, const std::string & path) const override; - - WriteSettings getAdjustedSettingsFromMetadataFile(const WriteSettings & settings, const std::string & path) const override; - const FileCacheSettings & getCacheSettings() const { return cache_settings; } static bool canUseReadThroughCache(const ReadSettings & settings); diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index dae4261e92c..87f249823b2 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -98,7 +98,6 @@ struct ReadSettings bool enable_filesystem_cache = true; bool read_from_filesystem_cache_if_exists_otherwise_bypass_cache = false; bool enable_filesystem_cache_log = false; - bool is_file_cache_persistent = false; /// Some files can be made non-evictable. /// Don't populate cache when the read is not part of query execution (e.g. background thread). bool avoid_readthrough_cache_outside_query_context = true; diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index e160796d9a3..8f22e44145a 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -19,7 +19,6 @@ struct WriteSettings /// Filesystem cache settings bool enable_filesystem_cache_on_write_operations = false; bool enable_filesystem_cache_log = false; - bool is_file_cache_persistent = false; bool throw_on_error_from_cache = false; bool s3_allow_parallel_part_upload = true; diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 3d1b85dfbde..1bbc02eceaf 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -52,7 +52,6 @@ namespace ErrorCodes FileCache::FileCache(const FileCacheSettings & settings) : max_file_segment_size(settings.max_file_segment_size) - , allow_persistent_files(settings.do_not_evict_index_and_mark_files) , bypass_cache_threshold(settings.enable_bypass_cache_with_threashold ? settings.bypass_cache_threashold : 0) , delayed_cleanup_interval_ms(settings.delayed_cleanup_interval_ms) , log(&Poco::Logger::get("FileCache")) @@ -642,10 +641,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) { chassert(segment_metadata->file_segment->assertCorrectness()); - const bool is_persistent = allow_persistent_files && segment_metadata->file_segment->isPersistent(); - const bool releasable = segment_metadata->releasable() && !is_persistent; - - if (releasable) + if (segment_metadata->releasable()) { auto segment = segment_metadata->file_segment; if (segment->state() == FileSegment::State::DOWNLOADED) @@ -820,10 +816,6 @@ void FileCache::removeAllReleasable() { assertInitialized(); - /// Only releasable file segments are evicted. - /// `remove_persistent_files` defines whether non-evictable by some criteria files - /// (they do not comply with the cache eviction policy) should also be removed. - auto lock = lockCache(); main_priority->iterate([&](LockedKey & locked_key, FileSegmentMetadataPtr segment_metadata) @@ -926,7 +918,9 @@ void FileCache::loadMetadata() parsed = tryParse(offset, offset_with_suffix.substr(0, delim_pos)); if (offset_with_suffix.substr(delim_pos+1) == "persistent") { - segment_kind = FileSegmentKind::Persistent; + /// For compatibility. Persistent files are no loger supported. + fs::remove(offset_it->path()); + continue; } if (offset_with_suffix.substr(delim_pos+1) == "temporary") { diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index dc5dd67710c..c211fdfc976 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -131,7 +131,6 @@ private: using KeyAndOffset = FileCacheKeyAndOffset; const size_t max_file_segment_size; - const bool allow_persistent_files; const size_t bypass_cache_threshold = 0; const size_t delayed_cleanup_interval_ms; diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index 1fe51bf5f3e..dbe3e8ced02 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -47,8 +47,6 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & else bypass_cache_threashold = FILECACHE_BYPASS_THRESHOLD; - do_not_evict_index_and_mark_files = config.getUInt64(config_prefix + ".do_not_evict_index_and_mark_files", true); - boundary_alignment = config.getUInt64(config_prefix + ".boundary_alignment", DBMS_DEFAULT_BUFFER_SIZE); delayed_cleanup_interval_ms = config.getUInt64(config_prefix + ".delayed_cleanup_interval_ms", FILECACHE_DELAYED_CLEANUP_INTERVAL_MS); diff --git a/src/Interpreters/Cache/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h index eeb2a02c131..cddb8fd459b 100644 --- a/src/Interpreters/Cache/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -22,8 +22,6 @@ struct FileCacheSettings size_t cache_hits_threshold = FILECACHE_DEFAULT_HITS_THRESHOLD; bool enable_filesystem_query_cache_limit = false; - bool do_not_evict_index_and_mark_files = true; - bool enable_bypass_cache_with_threashold = false; size_t bypass_cache_threashold = FILECACHE_BYPASS_THRESHOLD; size_t delayed_cleanup_interval_ms = FILECACHE_DELAYED_CLEANUP_INTERVAL_MS; diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 681c0d719e4..062f168a046 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -37,11 +37,6 @@ enum class FileSegmentKind */ Regular, - /* `Persistent` file segment can't be evicted from cache, - * it should be removed manually. - */ - Persistent, - /* `Temporary` file segment is removed right after releasing. * Also corresponding files are removed during cache loading (if any). */ @@ -155,8 +150,6 @@ public: FileSegmentKind getKind() const { return segment_kind; } - bool isPersistent() const { return segment_kind == FileSegmentKind::Persistent; } - bool isUnbound() const { return is_unbound; } String getPathInLocalCache() const; diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 20af931bbc2..495e335b7ae 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -141,9 +141,6 @@ String CacheMetadata::getFileNameForFileSegment(size_t offset, FileSegmentKind s String file_suffix; switch (segment_kind) { - case FileSegmentKind::Persistent: - file_suffix = "_persistent"; - break; case FileSegmentKind::Temporary: file_suffix = "_temporary"; break; diff --git a/src/Interpreters/InterpreterDescribeCacheQuery.cpp b/src/Interpreters/InterpreterDescribeCacheQuery.cpp index ca875ee57b2..1d2c567d16f 100644 --- a/src/Interpreters/InterpreterDescribeCacheQuery.cpp +++ b/src/Interpreters/InterpreterDescribeCacheQuery.cpp @@ -24,7 +24,6 @@ static Block getSampleBlock() ColumnWithTypeAndName{std::make_shared(), "current_size"}, ColumnWithTypeAndName{std::make_shared(), "current_elements"}, ColumnWithTypeAndName{std::make_shared(), "path"}, - ColumnWithTypeAndName{std::make_shared>(), "do_not_evict_index_and_mark_files"}, }; return Block(columns); } @@ -49,7 +48,6 @@ BlockIO InterpreterDescribeCacheQuery::execute() res_columns[5]->insert(cache->getUsedCacheSize()); res_columns[6]->insert(cache->getFileSegmentsNum()); res_columns[7]->insert(cache->getBasePath()); - res_columns[8]->insert(settings.do_not_evict_index_and_mark_files); BlockIO res; size_t num_rows = res_columns[0]->size(); diff --git a/src/Storages/System/StorageSystemFilesystemCache.cpp b/src/Storages/System/StorageSystemFilesystemCache.cpp index 8e9ad2ac501..e03fd9ca081 100644 --- a/src/Storages/System/StorageSystemFilesystemCache.cpp +++ b/src/Storages/System/StorageSystemFilesystemCache.cpp @@ -26,7 +26,6 @@ NamesAndTypesList StorageSystemFilesystemCache::getNamesAndTypes() {"cache_hits", std::make_shared()}, {"references", std::make_shared()}, {"downloaded_size", std::make_shared()}, - {"persistent", std::make_shared>()}, {"kind", std::make_shared()}, {"unbound", std::make_shared>()}, }; @@ -48,25 +47,25 @@ void StorageSystemFilesystemCache::fillData(MutableColumns & res_columns, Contex for (const auto & file_segment : *file_segments) { - res_columns[0]->insert(cache_name); - res_columns[1]->insert(cache->getBasePath()); + size_t i = 0; + res_columns[i++]->insert(cache_name); + res_columns[i++]->insert(cache->getBasePath()); /// Do not use `file_segment->getPathInLocalCache` here because it will lead to nullptr dereference /// (because file_segments in getSnapshot doesn't have `cache` field set) - res_columns[2]->insert(cache->getPathInLocalCache(file_segment->key(), file_segment->offset(), file_segment->getKind())); - res_columns[3]->insert(file_segment->key().toString()); + res_columns[i++]->insert(cache->getPathInLocalCache(file_segment->key(), file_segment->offset(), file_segment->getKind())); + res_columns[i++]->insert(file_segment->key().toString()); const auto & range = file_segment->range(); - res_columns[4]->insert(range.left); - res_columns[5]->insert(range.right); - res_columns[6]->insert(range.size()); - res_columns[7]->insert(FileSegment::stateToString(file_segment->state())); - res_columns[8]->insert(file_segment->getHitsCount()); - res_columns[9]->insert(file_segment->getRefCount()); - res_columns[10]->insert(file_segment->getDownloadedSize(false)); - res_columns[11]->insert(file_segment->isPersistent()); - res_columns[12]->insert(toString(file_segment->getKind())); - res_columns[13]->insert(file_segment->isUnbound()); + res_columns[i++]->insert(range.left); + res_columns[i++]->insert(range.right); + res_columns[i++]->insert(range.size()); + res_columns[i++]->insert(FileSegment::stateToString(file_segment->state())); + res_columns[i++]->insert(file_segment->getHitsCount()); + res_columns[i++]->insert(file_segment->getRefCount()); + res_columns[i++]->insert(file_segment->getDownloadedSize(false)); + res_columns[i++]->insert(toString(file_segment->getKind())); + res_columns[i++]->insert(file_segment->isUnbound()); } } } From 5229544b559366a9a10df0ecb17485a54eee51fb Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 21 Jun 2023 22:17:39 +0200 Subject: [PATCH 203/515] Added function getCurrrentBatchAndScheduleNext to IObjectStorageIteratorAsync --- src/Disks/ObjectStorages/ObjectStorageIterator.h | 6 ++++++ .../ObjectStorageIteratorAsync.cpp | 16 ++++++++++++++++ .../ObjectStorages/ObjectStorageIteratorAsync.h | 3 ++- src/Storages/StorageAzureBlob.cpp | 11 +++++++---- 4 files changed, 31 insertions(+), 5 deletions(-) diff --git a/src/Disks/ObjectStorages/ObjectStorageIterator.h b/src/Disks/ObjectStorages/ObjectStorageIterator.h index 2ff5ce60acc..841b0ea6664 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIterator.h +++ b/src/Disks/ObjectStorages/ObjectStorageIterator.h @@ -14,6 +14,7 @@ public: virtual bool isValid() = 0; virtual RelativePathWithMetadata current() = 0; virtual RelativePathsWithMetadata currentBatch() = 0; + virtual std::optional getCurrrentBatchAndScheduleNext() = 0; virtual size_t getAccumulatedSize() const = 0; virtual ~IObjectStorageIterator() = default; @@ -53,6 +54,11 @@ public: return batch; } + virtual std::optional getCurrrentBatchAndScheduleNext() override + { + return std::nullopt; + } + size_t getAccumulatedSize() const override { return batch.size(); diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp index f91c19f2fb9..7425f629a5a 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp @@ -100,6 +100,22 @@ RelativePathsWithMetadata IObjectStorageIteratorAsync::currentBatch() return current_batch; } +std::optional IObjectStorageIteratorAsync::getCurrrentBatchAndScheduleNext() +{ + std::lock_guard lock(mutex); + if (!is_initialized) + nextBatch(); + + if (current_batch_iterator != current_batch.end()) + { + auto temp_current_batch = current_batch; + nextBatch(); + return temp_current_batch; + } + + return std::nullopt; +} + size_t IObjectStorageIteratorAsync::getAccumulatedSize() const { return accumulated_size.load(std::memory_order_relaxed); diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h index a2b06da9a91..b0dd3cef39c 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h @@ -27,6 +27,7 @@ public: RelativePathWithMetadata current() override; RelativePathsWithMetadata currentBatch() override; size_t getAccumulatedSize() const override; + std::optional getCurrrentBatchAndScheduleNext() override; ~IObjectStorageIteratorAsync() override { @@ -48,7 +49,7 @@ protected: bool is_initialized{false}; bool is_finished{false}; - mutable std::mutex mutex; + mutable std::recursive_mutex mutex; ThreadPool list_objects_pool; ThreadPoolCallbackRunner list_objects_scheduler; std::future outcome_future; diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index b9d59f04001..91dc92f09e8 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -882,7 +882,6 @@ StorageAzureBlobSource::Iterator::Iterator( RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() { - std::lock_guard lock(next_mutex); if (is_finished) return {}; @@ -901,6 +900,7 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() { bool need_new_batch = false; { + std::lock_guard lock(next_mutex); need_new_batch = !blobs_with_metadata || index >= blobs_with_metadata->size(); } @@ -909,10 +909,10 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() RelativePathsWithMetadata new_batch; while (new_batch.empty()) { - if (object_storage_iterator->isValid()) + auto result = object_storage_iterator->getCurrrentBatchAndScheduleNext(); + if (result.has_value()) { - new_batch = object_storage_iterator->currentBatch(); - object_storage_iterator->nextBatch(); + new_batch = result.value(); } else { @@ -945,6 +945,7 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() VirtualColumnUtils::filterBlockWithQuery(query, block, getContext(), filter_ast); const auto & idxs = typeid_cast(*block.getByName("_idx").column); + std::lock_guard lock(next_mutex); blob_path_with_globs.reset(); blob_path_with_globs.emplace(); for (UInt64 idx : idxs.getData()) @@ -960,6 +961,7 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() if (outer_blobs) outer_blobs->insert(outer_blobs->end(), new_batch.begin(), new_batch.end()); + std::lock_guard lock(next_mutex); blobs_with_metadata = std::move(new_batch); for (const auto & [_, info] : *blobs_with_metadata) total_size.fetch_add(info.size_bytes, std::memory_order_relaxed); @@ -967,6 +969,7 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() } size_t current_index = index.fetch_add(1, std::memory_order_relaxed); + std::lock_guard lock(next_mutex); return (*blobs_with_metadata)[current_index]; } } From 23120c44679902163832c27d2a9787a502e45d50 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 21 Jun 2023 22:32:33 +0200 Subject: [PATCH 204/515] fix build --- src/IO/LZMADeflatingWriteBuffer.h | 2 ++ src/IO/Lz4DeflatingWriteBuffer.h | 2 ++ src/IO/ZlibDeflatingWriteBuffer.h | 2 ++ src/IO/ZstdDeflatingWriteBuffer.h | 2 ++ 4 files changed, 8 insertions(+) diff --git a/src/IO/LZMADeflatingWriteBuffer.h b/src/IO/LZMADeflatingWriteBuffer.h index 5a0864d6071..2e135455e00 100644 --- a/src/IO/LZMADeflatingWriteBuffer.h +++ b/src/IO/LZMADeflatingWriteBuffer.h @@ -21,6 +21,8 @@ public: char * existing_memory = nullptr, size_t alignment = 0); + ~LZMADeflatingWriteBuffer() override; + private: void nextImpl() override; diff --git a/src/IO/Lz4DeflatingWriteBuffer.h b/src/IO/Lz4DeflatingWriteBuffer.h index a6440f8854f..68873b5f8ee 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.h +++ b/src/IO/Lz4DeflatingWriteBuffer.h @@ -21,6 +21,8 @@ public: char * existing_memory = nullptr, size_t alignment = 0); + ~Lz4DeflatingWriteBuffer() override; + private: void nextImpl() override; diff --git a/src/IO/ZlibDeflatingWriteBuffer.h b/src/IO/ZlibDeflatingWriteBuffer.h index 05d6e528a23..58e709b54e6 100644 --- a/src/IO/ZlibDeflatingWriteBuffer.h +++ b/src/IO/ZlibDeflatingWriteBuffer.h @@ -24,6 +24,8 @@ public: char * existing_memory = nullptr, size_t alignment = 0); + ~ZlibDeflatingWriteBuffer() override; + private: void nextImpl() override; diff --git a/src/IO/ZstdDeflatingWriteBuffer.h b/src/IO/ZstdDeflatingWriteBuffer.h index 11e34e2890f..ba83c18d354 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.h +++ b/src/IO/ZstdDeflatingWriteBuffer.h @@ -21,6 +21,8 @@ public: char * existing_memory = nullptr, size_t alignment = 0); + ~ZstdDeflatingWriteBuffer() override; + void sync() override { out->sync(); From 0af028ab3f19dc1dc3501d5e5b2dd0e9aa540fdc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 21 Jun 2023 20:55:27 +0000 Subject: [PATCH 205/515] Refactor a bit more. --- src/Interpreters/ActionsVisitor.cpp | 12 +- src/Interpreters/GlobalSubqueriesVisitor.h | 9 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/PreparedSets.cpp | 137 +++++++++++------ src/Interpreters/PreparedSets.h | 145 +++++++++--------- src/Planner/CollectSets.cpp | 28 +--- src/Planner/Planner.cpp | 8 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 26 ++-- src/Processors/QueryPlan/CreatingSetsStep.h | 16 +- .../Transforms/CreatingSetsTransform.cpp | 60 +++----- .../Transforms/CreatingSetsTransform.h | 8 +- src/QueryPipeline/QueryPipelineBuilder.cpp | 8 +- src/QueryPipeline/QueryPipelineBuilder.h | 8 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +- 14 files changed, 232 insertions(+), 239 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 9125b38b48c..7c3af4ce691 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1406,7 +1406,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool if (data.getContext()->getSettingsRef().allow_experimental_analyzer && !identifier) { InterpreterSelectQueryAnalyzer interpreter(right_in_operand, data.getContext(), SelectQueryOptions().analyze(true).subquery()); - auto query_tree = interpreter.getQueryTree(); + const auto & query_tree = interpreter.getQueryTree(); if (auto * query_node = query_tree->as()) query_node->setIsSubquery(true); set_key = query_tree->getTreeHash(); @@ -1449,11 +1449,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool } } - /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. - // String set_id = right_in_operand->getColumnName(); - //bool transform_null_in = data.getContext()->getSettingsRef().transform_null_in; - SubqueryForSet subquery_for_set; // = data.prepared_sets->createOrGetSubquery(set_id, set_key, data.set_size_limit, transform_null_in); - subquery_for_set.key = PreparedSets::toString(set_key, {}); //right_in_operand->getColumnName(); + std::unique_ptr source = std::make_unique(); /** The following happens for GLOBAL INs or INs: * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, @@ -1465,10 +1461,10 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool */ { auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth, {}); - subquery_for_set.createSource(*interpreter); + interpreter->buildQueryPlan(*source); } - return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), data.getContext()->getSettingsRef(), std::move(external_table_set)); + return data.prepared_sets->addFromSubquery(set_key, std::move(source), nullptr, std::move(external_table_set), data.getContext()->getSettingsRef()); } else { diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 5b8f0c60d35..8c784d3c2ff 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -188,20 +188,17 @@ public: { // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); // subquery_for_set.createSource(*interpreter, external_storage); - auto key = subquery_or_table_name->getColumnName(); auto set_key = database_and_table_name->getTreeHash(); // std::cerr << "====== Adding key " << set_key.toString() << std::endl; if (!prepared_sets->findSubquery(set_key)) { - SubqueryForSet subquery_for_set; - subquery_for_set.key = std::move(key); - subquery_for_set.table = std::move(external_storage); - subquery_for_set.createSource(*interpreter); + std::unique_ptr source = std::make_unique(); + interpreter->buildQueryPlan(*source); //std::cerr << reinterpret_cast(prepared_sets.get()) << std::endl; - auto future_set = prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), getContext()->getSettingsRef(), nullptr); + auto future_set = prepared_sets->addFromSubquery(set_key, std::move(source), std::move(external_storage), nullptr, getContext()->getSettingsRef()); // std::cerr << "... Future set " << reinterpret_cast(external_storage_holder.get()) << " " << reinterpret_cast(future_set.get()) << std::endl; external_storage_holder->future_set = std::move(future_set); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 52238ad5932..01736731f2e 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -3151,7 +3151,7 @@ void InterpreterSelectQuery::executeExtremes(QueryPlan & query_plan) void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_plan) { - auto subqueries = prepared_sets->detachSubqueries(); + auto subqueries = prepared_sets->getSubqueries(); if (!subqueries.empty()) { diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 03c18730cc5..2aa3e02c713 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -200,17 +200,42 @@ FutureSetPtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_) return from_storage; } -FutureSetPtr PreparedSets::addFromSubquery(const Hash & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set) +FutureSetPtr PreparedSets::addFromSubquery( + const Hash & key, + std::unique_ptr source, + StoragePtr external_table, + FutureSetPtr external_table_set, + const Settings & settings) { - auto from_subquery = std::make_shared(std::move(subquery), std::move(external_table_set), settings); + auto from_subquery = std::make_shared( + toString(key, {}), + std::move(source), + std::move(external_table), + std::move(external_table_set), + settings); + auto [it, inserted] = sets_from_subqueries.emplace(key, from_subquery); if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, {})); - // std::cerr << key.toString() << std::endl; - // std::cerr << "========= PreparedSets::addFromSubquery\n"; - // std::cerr << StackTrace().toString() << std::endl; + return from_subquery; +} + +FutureSetPtr PreparedSets::addFromSubquery( + const Hash & key, + QueryTreeNodePtr query_tree, + const Settings & settings) +{ + auto from_subquery = std::make_shared( + toString(key, {}), + std::move(query_tree), + settings); + + auto [it, inserted] = sets_from_subqueries.emplace(key, from_subquery); + + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, {})); return from_subquery; } @@ -269,7 +294,7 @@ std::shared_ptr PreparedSets::findStorage(const Hash & key // return res; // } -std::vector> PreparedSets::detachSubqueries() +std::vector> PreparedSets::getSubqueries() { std::vector> res; res.reserve(sets_from_subqueries.size()); @@ -279,25 +304,25 @@ std::vector> PreparedSets::detachSubqueri return res; } -void SubqueryForSet::createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_) -{ - source = std::make_unique(); - interpreter.buildQueryPlan(*source); - if (table_) - table = table_; -} +// void SubqueryForSet::createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_) +// { +// source = std::make_unique(); +// interpreter.buildQueryPlan(*source); +// if (table_) +// table = table_; +// } -bool SubqueryForSet::hasSource() const -{ - return source != nullptr; -} +// bool SubqueryForSet::hasSource() const +// { +// return source != nullptr; +// } -QueryPlanPtr SubqueryForSet::detachSource() -{ - auto res = std::move(source); - source = nullptr; - return res; -} +// QueryPlanPtr SubqueryForSet::detachSource() +// { +// auto res = std::move(source); +// source = nullptr; +// return res; +// } std::variant, SharedSet> PreparedSetsCache::findOrPromiseToBuild(const String & key) @@ -339,15 +364,15 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) { auto set = external_table_set->buildOrderedSetInplace(context); if (set) - return subquery.set = set; + return set_and_key->set = set; } auto plan = buildPlan(context); if (!plan) return nullptr; - subquery.set->fillSetElements(); - subquery.set->initSetElements(); + set_and_key->set->fillSetElements(); + set_and_key->set->initSetElements(); auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); pipeline.complete(std::make_shared(Block())); @@ -355,15 +380,15 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) CompletedPipelineExecutor executor(pipeline); executor.execute(); - subquery.set->checkIsCreated(); + set_and_key->set->checkIsCreated(); - return subquery.set; + return set_and_key->set; } SetPtr FutureSetFromSubquery::get() const { - if (subquery.set != nullptr && subquery.set->isCreated()) - return subquery.set; + if (set_and_key->set != nullptr && set_and_key->set->isCreated()) + return set_and_key->set; return nullptr; } @@ -380,22 +405,20 @@ static SizeLimits getSizeLimitsForSet(const Settings & settings) std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & context) { - if (subquery.set->isCreated()) + if (set_and_key->set->isCreated()) return nullptr; const auto & settings = context->getSettingsRef(); - auto plan = subquery.detachSource(); - auto description = subquery.key; + auto plan = std::move(source); if (!plan) return nullptr; auto creating_set = std::make_unique( plan->getCurrentDataStream(), - description, - subquery, - shared_from_this(), + set_and_key, + external_table, SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), context); creating_set->setStepDescription("Create set for subquery"); @@ -421,25 +444,49 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) set->finishInsert(); } -FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, const Settings & settings) - : subquery(std::move(subquery_)), external_table_set(std::move(external_table_set_)) +FutureSetFromSubquery::FutureSetFromSubquery( + String key, + std::unique_ptr source_, + StoragePtr external_table_, + FutureSetPtr external_table_set_, + const Settings & settings) + : external_table(std::move(external_table_)) + , external_table_set(std::move(external_table_set_)) + , source(std::move(source_)) { + set_and_key = std::make_shared(); + set_and_key->key = std::move(key); + bool create_ordered_set = false; auto size_limits = getSizeLimitsForSet(settings); - subquery.set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); - if (subquery.source) - subquery.set->setHeader(subquery.source->getCurrentDataStream().header.getColumnsWithTypeAndName()); + set_and_key->set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); + set_and_key->set->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName()); } -void FutureSetFromSubquery::setQueryPlan(std::unique_ptr source) +FutureSetFromSubquery::FutureSetFromSubquery( + String key, + QueryTreeNodePtr query_tree_, + //FutureSetPtr external_table_set_, + const Settings & settings) + : query_tree(std::move(query_tree_)) { - subquery.source = std::move(source); - subquery.set->setHeader(subquery.source->getCurrentDataStream().header.getColumnsWithTypeAndName()); + set_and_key = std::make_shared(); + set_and_key->key = std::move(key); + + bool create_ordered_set = false; + auto size_limits = getSizeLimitsForSet(settings); + set_and_key->set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); +} + +void FutureSetFromSubquery::setQueryPlan(std::unique_ptr source_) +{ + source = std::move(source_); + set_and_key->set->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName()); } const DataTypes & FutureSetFromSubquery::getTypes() const { - return subquery.set->getElementsTypes(); + return set_and_key->set->getElementsTypes(); } SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 49545ad1e2a..339f4a6b435 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -24,6 +24,16 @@ using QueryTreeNodePtr = std::shared_ptr; struct Settings; +/// This is a structure for prepared sets cache. +/// SetPtr can be taken from cache, so we should pass holder for it. +struct SetAndKey +{ + String key; + SetPtr set; +}; + +using SetAndKeyPtr = std::shared_ptr; + /// Represents a set in a query that might be referenced at analysis time and built later during execution. /// Also it can represent a constant set that is ready to use. /// At analysis stage the FutureSets are created but not necessarily filled. Then for non-constant sets there @@ -35,22 +45,20 @@ public: virtual ~FutureSet() = default; virtual SetPtr get() const = 0; - virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; - virtual const DataTypes & getTypes() const = 0; + virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; }; using FutureSetPtr = std::shared_ptr; - class FutureSetFromStorage : public FutureSet { public: FutureSetFromStorage(SetPtr set_); SetPtr get() const override; - SetPtr buildOrderedSetInplace(const ContextPtr &) override; const DataTypes & getTypes() const override; + SetPtr buildOrderedSetInplace(const ContextPtr &) override; private: SetPtr set; @@ -58,80 +66,67 @@ private: /// Information on how to build set for the [GLOBAL] IN section. -class SubqueryForSet +// class SubqueryForSet +// { +// public: + +// void createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_ = nullptr); + +// bool hasSource() const; + +// /// Returns query plan for the set's source +// /// and removes it from SubqueryForSet because we need to build it only once. +// std::unique_ptr detachSource(); + +// /// Build this set from the result of the subquery. +// String key; +// SetPtr set; + +// /// If set, put the result into the table. +// /// This is a temporary table for transferring to remote servers for distributed query processing. +// StoragePtr table; + +// /// The source is obtained using the InterpreterSelectQuery subquery. +// std::unique_ptr source; +// QueryTreeNodePtr query_tree; +// }; + +class FutureSetFromSubquery : public FutureSet { public: + FutureSetFromSubquery( + String key, + std::unique_ptr source_, + StoragePtr external_table_, + FutureSetPtr external_table_set_, + const Settings & settings); - void createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_ = nullptr); - - bool hasSource() const; - - /// Returns query plan for the set's source - /// and removes it from SubqueryForSet because we need to build it only once. - std::unique_ptr detachSource(); - - /// Build this set from the result of the subquery. - String key; - SetPtr set; - - /// If set, put the result into the table. - /// This is a temporary table for transferring to remote servers for distributed query processing. - StoragePtr table; - - /// The source is obtained using the InterpreterSelectQuery subquery. - std::unique_ptr source; - QueryTreeNodePtr query_tree; -}; - -class FutureSetFromSubquery : public FutureSet, public std::enable_shared_from_this -{ -public: - FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, const Settings & settings); + FutureSetFromSubquery( + String key, + QueryTreeNodePtr query_tree_, + //FutureSetPtr external_table_set_, + const Settings & settings); SetPtr get() const override; - + const DataTypes & getTypes() const override; SetPtr buildOrderedSetInplace(const ContextPtr & context) override; std::unique_ptr build(const ContextPtr & context); - const DataTypes & getTypes() const override; - - SubqueryForSet & getSubquery() { return subquery; } - void setQueryPlan(std::unique_ptr source); + QueryTreeNodePtr detachQueryTree() { return std::move(query_tree); } + void setQueryPlan(std::unique_ptr source_); private: - SubqueryForSet subquery; + SetAndKeyPtr set_and_key; + StoragePtr external_table; FutureSetPtr external_table_set; + std::unique_ptr source; + QueryTreeNodePtr query_tree; + std::unique_ptr buildPlan(const ContextPtr & context); }; -// struct PreparedSetKey -// { -// using Hash = std::pair; - -// /// Prepared sets for tuple literals are indexed by the hash of the tree contents and by the desired -// /// data types of set elements (two different Sets can be required for two tuples with the same contents -// /// if left hand sides of the IN operators have different types). -// static PreparedSetKey forLiteral(Hash hash, DataTypes types_); - -// /// Prepared sets for subqueries are indexed only by the AST contents because the type of the resulting -// /// set is fully determined by the subquery. -// static PreparedSetKey forSubquery(Hash hash); - -// Hash ast_hash; -// DataTypes types; /// Empty for subqueries. - -// bool operator==(const PreparedSetKey & other) const; - -// String toString() const; - -// struct Hashing -// { -// UInt64 operator()(const PreparedSetKey & key) const { return key.ast_hash.first; } -// }; -// }; - class PreparedSets { public: @@ -142,30 +137,30 @@ public: UInt64 operator()(const Hash & key) const { return key.first ^ key.second; } }; - // struct SetAndName - // { - // String name; - // std::shared_ptr set; - // }; - using SetsFromTuple = std::unordered_map>, Hashing>; using SetsFromStorage = std::unordered_map, Hashing>; using SetsFromSubqueries = std::unordered_map, Hashing>; FutureSetPtr addFromStorage(const Hash & key, SetPtr set_); FutureSetPtr addFromTuple(const Hash & key, Block block, const Settings & settings); - FutureSetPtr addFromSubquery(const Hash & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set); + + FutureSetPtr addFromSubquery( + const Hash & key, + std::unique_ptr source, + StoragePtr external_table, + FutureSetPtr external_table_set, + const Settings & settings); + + FutureSetPtr addFromSubquery( + const Hash & key, + QueryTreeNodePtr query_tree, + const Settings & settings); FutureSetPtr findTuple(const Hash & key, const DataTypes & types) const; std::shared_ptr findStorage(const Hash & key) const; std::shared_ptr findSubquery(const Hash & key) const; - //FutureSetPtr getFuture(const PreparedSetKey & key) const; - - /// Get subqueries and clear them. - /// We need to build a plan for subqueries just once. That's why we can clear them after accessing them. - /// SetPtr would still be available for consumers of PreparedSets. - std::vector> detachSubqueries(); + std::vector> getSubqueries(); const SetsFromTuple & getSetsFromTuple() const { return sets_from_tuple; } const SetsFromStorage & getSetsFromStorage() const { return sets_from_storage; } diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index b0e6b272c9f..df3558745fb 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -46,12 +46,6 @@ public: auto in_second_argument_node_type = in_second_argument->getNodeType(); const auto & settings = planner_context.getQueryContext()->getSettingsRef(); - - // String set_key = planner_context.createSetKey(in_second_argument); - - // if (planner_context.hasSet(set_key)) - // return; - auto & sets = planner_context.getPreparedSets(); /// Tables and table functions are replaced with subquery at Analysis stage, except special Set table. @@ -129,27 +123,7 @@ public: subquery_to_execute = std::move(subquery_for_table); } - // auto subquery_options = select_query_options.subquery(); - // Planner subquery_planner( - // in_second_argument, - // subquery_options, - // planner_context.getGlobalPlannerContext()); - // subquery_planner.buildQueryPlanIfNeeded(); - - // const auto & settings = planner_context.getQueryContext()->getSettingsRef(); - // SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; - // bool tranform_null_in = settings.transform_null_in; - // auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); - - SubqueryForSet subquery_for_set; - subquery_for_set.key = planner_context.createSetKey(in_second_argument); - subquery_for_set.query_tree = std::move(subquery_to_execute); - //subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); - - /// TODO - sets.addFromSubquery(set_key, std::move(subquery_for_set), settings, nullptr); - - //planner_context.registerSet(set_key, PlannerSet(in_second_argument)); + sets.addFromSubquery(set_key, std::move(subquery_to_execute), settings); } else { diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index e7c10f6ef7a..d524f2bed98 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -930,7 +930,7 @@ void addBuildSubqueriesForSetsStepIfNeeded( const PlannerContextPtr & planner_context, const std::vector & result_actions_to_execute) { - auto subqueries = planner_context->getPreparedSets().detachSubqueries(); + auto subqueries = planner_context->getPreparedSets().getSubqueries(); std::unordered_set useful_sets; //PreparedSets::SubqueriesForSets subqueries_for_sets; @@ -944,10 +944,10 @@ void addBuildSubqueriesForSetsStepIfNeeded( for (auto & subquery : subqueries) { - auto & subquery_for_set = subquery->getSubquery(); + auto query_tree = subquery->detachQueryTree(); auto subquery_options = select_query_options.subquery(); Planner subquery_planner( - subquery_for_set.query_tree, + query_tree, subquery_options, planner_context->getGlobalPlannerContext()); subquery_planner.buildQueryPlanIfNeeded(); @@ -955,8 +955,6 @@ void addBuildSubqueriesForSetsStepIfNeeded( subquery->setQueryPlan(std::make_unique(std::move(subquery_planner).extractQueryPlan())); } - //addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); - if (!subqueries.empty()) { auto step = std::make_unique( diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 54d81dd7650..06099eef872 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -36,15 +36,13 @@ static ITransformingStep::Traits getTraits() CreatingSetStep::CreatingSetStep( const DataStream & input_stream_, - String description_, - SubqueryForSet & subquery_for_set_, - FutureSetPtr set_, + SetAndKeyPtr set_and_key_, + StoragePtr external_table_, SizeLimits network_transfer_limits_, ContextPtr context_) : ITransformingStep(input_stream_, Block{}, getTraits()) - , description(std::move(description_)) - , subquery_for_set(subquery_for_set_) - , set(std::move(set_)) + , set_and_key(std::move(set_and_key_)) + , external_table(std::move(external_table_)) , network_transfer_limits(std::move(network_transfer_limits_)) , context(std::move(context_)) { @@ -52,7 +50,7 @@ CreatingSetStep::CreatingSetStep( void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - pipeline.addCreatingSetsTransform(getOutputStream().header, subquery_for_set, std::move(set), network_transfer_limits, context->getPreparedSetsCache()); + pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(set_and_key), std::move(external_table), network_transfer_limits, context->getPreparedSetsCache()); } void CreatingSetStep::updateOutputStream() @@ -65,16 +63,16 @@ void CreatingSetStep::describeActions(FormatSettings & settings) const String prefix(settings.offset, ' '); settings.out << prefix; - if (subquery_for_set.set) + if (set_and_key->set) settings.out << "Set: "; - settings.out << description << '\n'; + settings.out << set_and_key->key << '\n'; } void CreatingSetStep::describeActions(JSONBuilder::JSONMap & map) const { - if (subquery_for_set.set) - map.add("Set", description); + if (set_and_key->set) + map.add("Set", set_and_key->key); } @@ -126,7 +124,7 @@ void CreatingSetsStep::describePipeline(FormatSettings & settings) const IQueryPlanStep::describePipeline(processors, settings); } -void addCreatingSetsStep(QueryPlan & query_plan, std::vector> sets_from_subqueries, ContextPtr context) +void addCreatingSetsStep(QueryPlan & query_plan, std::vector> sets_from_subquery, ContextPtr context) { DataStreams input_streams; input_streams.emplace_back(query_plan.getCurrentDataStream()); @@ -135,7 +133,7 @@ void addCreatingSetsStep(QueryPlan & query_plan, std::vector(std::move(query_plan))); query_plan = QueryPlan(); - for (auto & future_set : sets_from_subqueries) + for (auto & future_set : sets_from_subquery) { if (future_set->get()) continue; @@ -193,7 +191,7 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, if (!prepared_sets) return; - auto subqueries = prepared_sets->detachSubqueries(); + auto subqueries = prepared_sets->getSubqueries(); if (subqueries.empty()) return; diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index c41bb782e0b..7110775da79 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -13,12 +13,11 @@ class CreatingSetStep : public ITransformingStep { public: CreatingSetStep( - const DataStream & input_stream_, - String description_, - SubqueryForSet & subquery_for_set_, - FutureSetPtr set_, - SizeLimits network_transfer_limits_, - ContextPtr context_); + const DataStream & input_stream_, + SetAndKeyPtr set_and_key_, + StoragePtr external_table_, + SizeLimits network_transfer_limits_, + ContextPtr context_); String getName() const override { return "CreatingSet"; } @@ -30,9 +29,8 @@ public: private: void updateOutputStream() override; - String description; - SubqueryForSet & subquery_for_set; - FutureSetPtr set; + SetAndKeyPtr set_and_key; + StoragePtr external_table; SizeLimits network_transfer_limits; ContextPtr context; }; diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index aa374446f3e..afce1355f7a 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -25,13 +25,13 @@ CreatingSetsTransform::~CreatingSetsTransform() = default; CreatingSetsTransform::CreatingSetsTransform( Block in_header_, Block out_header_, - SubqueryForSet & subquery_for_set_, - FutureSetPtr set_, + SetAndKeyPtr set_and_key_, + StoragePtr external_table_, SizeLimits network_transfer_limits_, PreparedSetsCachePtr prepared_sets_cache_) : IAccumulatingTransform(std::move(in_header_), std::move(out_header_)) - , subquery(subquery_for_set_) - , set(std::move(set_)) + , set_and_key(std::move(set_and_key_)) + , external_table(std::move(external_table_)) , network_transfer_limits(std::move(network_transfer_limits_)) , prepared_sets_cache(std::move(prepared_sets_cache_)) { @@ -54,31 +54,30 @@ void CreatingSetsTransform::work() void CreatingSetsTransform::startSubquery() { /// Lookup the set in the cache if we don't need to build table. - if (prepared_sets_cache && !subquery.table) + if (prepared_sets_cache && !external_table) { /// Try to find the set in the cache and wait for it to be built. /// Retry if the set from cache fails to be built. while (true) { - auto from_cache = prepared_sets_cache->findOrPromiseToBuild(subquery.key); + auto from_cache = prepared_sets_cache->findOrPromiseToBuild(set_and_key->key); if (from_cache.index() == 0) { - LOG_TRACE(log, "Building set, key: {}", subquery.key); + LOG_TRACE(log, "Building set, key: {}", set_and_key->key); promise_to_build = std::move(std::get<0>(from_cache)); } else { - LOG_TRACE(log, "Waiting for set to be build by another thread, key: {}", subquery.key); + LOG_TRACE(log, "Waiting for set to be build by another thread, key: {}", set_and_key->key); SharedSet set_built_by_another_thread = std::move(std::get<1>(from_cache)); const SetPtr & ready_set = set_built_by_another_thread.get(); if (!ready_set) { - LOG_TRACE(log, "Failed to use set from cache, key: {}", subquery.key); + LOG_TRACE(log, "Failed to use set from cache, key: {}", set_and_key->key); continue; } - //subquery.promise_to_fill_set.set_value(ready_set); - subquery.set = ready_set; //.reset(); + set_and_key->set = ready_set; done_with_set = true; set_from_cache = true; } @@ -86,21 +85,19 @@ void CreatingSetsTransform::startSubquery() } } - if (subquery.set && !set_from_cache) - LOG_TRACE(log, "Creating set, key: {}", subquery.key); - if (subquery.table) + if (set_and_key->set && !set_from_cache) + LOG_TRACE(log, "Creating set, key: {}", set_and_key->key); + if (external_table) LOG_TRACE(log, "Filling temporary table."); - // std::cerr << StackTrace().toString() << std::endl; - - if (subquery.table) + if (external_table) /// TODO: make via port - table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), nullptr, /*async_insert=*/false)); + table_out = QueryPipeline(external_table->write({}, external_table->getInMemoryMetadataPtr(), nullptr, /*async_insert=*/false)); - done_with_set = !subquery.set || set_from_cache; - done_with_table = !subquery.table; + done_with_set = !set_and_key->set || set_from_cache; + done_with_table = !external_table; - if ((done_with_set && !set_from_cache) /*&& done_with_join*/ && done_with_table) + if ((done_with_set && !set_from_cache) && done_with_table) throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: nothing to do with subquery"); if (table_out.initialized()) @@ -120,9 +117,9 @@ void CreatingSetsTransform::finishSubquery() } else if (read_rows != 0) { - if (subquery.set) - LOG_DEBUG(log, "Created Set with {} entries from {} rows in {} sec.", subquery.set->getTotalRowCount(), read_rows, seconds); - if (subquery.table) + if (set_and_key->set) + LOG_DEBUG(log, "Created Set with {} entries from {} rows in {} sec.", set_and_key->set->getTotalRowCount(), read_rows, seconds); + if (external_table) LOG_DEBUG(log, "Created Table with {} rows in {} sec.", read_rows, seconds); } else @@ -135,12 +132,6 @@ void CreatingSetsTransform::init() { is_initialized = true; - // if (subquery.set) - // { - // //std::cerr << "=========== " << getInputPort().getHeader().dumpStructure() << std::endl; - // subquery.set->setHeader(getInputPort().getHeader().getColumnsWithTypeAndName()); - // } - watch.restart(); startSubquery(); } @@ -152,7 +143,7 @@ void CreatingSetsTransform::consume(Chunk chunk) if (!done_with_set) { - if (!subquery.set->insertFromBlock(block.getColumnsWithTypeAndName())) + if (!set_and_key->set->insertFromBlock(block.getColumnsWithTypeAndName())) done_with_set = true; } @@ -175,12 +166,11 @@ void CreatingSetsTransform::consume(Chunk chunk) Chunk CreatingSetsTransform::generate() { - if (subquery.set && !set_from_cache) + if (set_and_key->set && !set_from_cache) { - subquery.set->finishInsert(); - //subquery.promise_to_fill_set.set_value(subquery.set); + set_and_key->set->finishInsert(); if (promise_to_build) - promise_to_build->set_value(subquery.set); + promise_to_build->set_value(set_and_key->set); } if (table_out.initialized()) diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index 27c330bdbc3..d1ec7dcbca7 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -29,8 +29,8 @@ public: CreatingSetsTransform( Block in_header_, Block out_header_, - SubqueryForSet & subquery_for_set_, - FutureSetPtr set_, + SetAndKeyPtr set_and_key_, + StoragePtr external_table_, SizeLimits network_transfer_limits_, PreparedSetsCachePtr prepared_sets_cache_); @@ -43,8 +43,8 @@ public: Chunk generate() override; private: - SubqueryForSet & subquery; - FutureSetPtr set; + SetAndKeyPtr set_and_key; + StoragePtr external_table; std::optional> promise_to_build; QueryPipeline table_out; diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index dca185cec8b..dedf85e409c 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -571,8 +571,8 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe void QueryPipelineBuilder::addCreatingSetsTransform( const Block & res_header, - SubqueryForSet & subquery_for_set, - FutureSetPtr set, + SetAndKeyPtr set_and_key, + StoragePtr external_table, const SizeLimits & limits, PreparedSetsCachePtr prepared_sets_cache) { @@ -581,8 +581,8 @@ void QueryPipelineBuilder::addCreatingSetsTransform( auto transform = std::make_shared( getHeader(), res_header, - subquery_for_set, - std::move(set), + std::move(set_and_key), + std::move(external_table), limits, std::move(prepared_sets_cache)); diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 134de59520d..e744e3612ce 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -33,8 +33,8 @@ class TableJoin; class QueryPipelineBuilder; using QueryPipelineBuilderPtr = std::unique_ptr; -class FutureSet; -using FutureSetPtr = std::shared_ptr; +struct SetAndKey; +using SetAndKeyPtr = std::shared_ptr; class PreparedSetsCache; using PreparedSetsCachePtr = std::shared_ptr; @@ -146,8 +146,8 @@ public: void addCreatingSetsTransform( const Block & res_header, - SubqueryForSet & subquery_for_set, - FutureSetPtr set, + SetAndKeyPtr set_and_key, + StoragePtr external_table, const SizeLimits & limits, PreparedSetsCachePtr prepared_sets_cache); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c3cf146e88e..125fe98203f 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -978,7 +978,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd if (ranges.ranges.empty()) break; - auto & index_and_condition = skip_indexes.useful_indices[idx]; + const auto & index_and_condition = skip_indexes.useful_indices[idx]; auto & stat = useful_indices_stat[idx]; stat.total_parts.fetch_add(1, std::memory_order_relaxed); stat.total_granules.fetch_add(ranges.ranges.getNumberOfMarks(), std::memory_order_relaxed); @@ -1006,7 +1006,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd if (ranges.ranges.empty()) break; - auto & indices_and_condition = skip_indexes.merged_indices[idx]; + const auto & indices_and_condition = skip_indexes.merged_indices[idx]; auto & stat = merged_indices_stat[idx]; stat.total_parts.fetch_add(1, std::memory_order_relaxed); From a050b9e9d98fb25270e5cf41b98135fe44f6d8f2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Jun 2023 01:28:52 +0300 Subject: [PATCH 206/515] Update FileCache.cpp --- src/Interpreters/Cache/FileCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index c1256e43afa..f42fc1f8dfc 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -923,7 +923,7 @@ void FileCache::loadMetadata() parsed = tryParse(offset, offset_with_suffix.substr(0, delim_pos)); if (offset_with_suffix.substr(delim_pos+1) == "persistent") { - /// For compatibility. Persistent files are no loger supported. + /// For compatibility. Persistent files are no longer supported. fs::remove(offset_it->path()); continue; } From 0c440b9d6f98bf1630e1b23ed058450086488697 Mon Sep 17 00:00:00 2001 From: Anton Kozlov Date: Wed, 3 May 2023 12:59:19 +0000 Subject: [PATCH 207/515] Report loading status for executable dictionaries correctly --- .../ExecutableDictionarySource.cpp | 17 +++++-- .../ExecutablePoolDictionarySource.cpp | 1 + src/Processors/Sources/ShellCommandSource.cpp | 4 ++ src/Processors/Sources/ShellCommandSource.h | 3 ++ ...utable_missing_strict_check_dictionary.xml | 29 +++++++++++ .../test_executable_dictionary/test.py | 48 +++++++++++-------- 6 files changed, 79 insertions(+), 23 deletions(-) create mode 100644 tests/integration/test_executable_dictionary/dictionaries/executable_missing_strict_check_dictionary.xml diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 91f914fb7ad..e40ef07de9e 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -114,7 +114,10 @@ QueryPipeline ExecutableDictionarySource::loadAll() auto command = configuration.command; updateCommandIfNeeded(command, coordinator_configuration.execute_direct, context); - return QueryPipeline(coordinator->createPipe(command, configuration.command_arguments, sample_block, context)); + ShellCommandSourceConfiguration command_configuration { + .check_exit_code = true, + }; + return QueryPipeline(coordinator->createPipe(command, configuration.command_arguments, {}, sample_block, context, command_configuration)); } QueryPipeline ExecutableDictionarySource::loadUpdatedAll() @@ -148,7 +151,11 @@ QueryPipeline ExecutableDictionarySource::loadUpdatedAll() update_time = new_update_time; LOG_TRACE(log, "loadUpdatedAll {}", command); - return QueryPipeline(coordinator->createPipe(command, command_arguments, sample_block, context)); + + ShellCommandSourceConfiguration command_configuration { + .check_exit_code = true, + }; + return QueryPipeline(coordinator->createPipe(command, command_arguments, {}, sample_block, context, command_configuration)); } QueryPipeline ExecutableDictionarySource::loadIds(const std::vector & ids) @@ -179,7 +186,11 @@ QueryPipeline ExecutableDictionarySource::getStreamForBlock(const Block & block) Pipes shell_input_pipes; shell_input_pipes.emplace_back(std::move(shell_input_pipe)); - auto pipe = coordinator->createPipe(command, configuration.command_arguments, std::move(shell_input_pipes), sample_block, context); + ShellCommandSourceConfiguration command_configuration { + .check_exit_code = true, + }; + + auto pipe = coordinator->createPipe(command, configuration.command_arguments, std::move(shell_input_pipes), sample_block, context, command_configuration); if (configuration.implicit_key) pipe.addTransform(std::make_shared(block, pipe.getHeader())); diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 0cc7696585f..94685060a46 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -132,6 +132,7 @@ QueryPipeline ExecutablePoolDictionarySource::getStreamForBlock(const Block & bl ShellCommandSourceConfiguration command_configuration; command_configuration.read_fixed_number_of_rows = true; command_configuration.number_of_rows_to_read = block.rows(); + command_configuration.check_exit_code = true; Pipes shell_input_pipes; shell_input_pipes.emplace_back(std::move(shell_input_pipe)); diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 8a7bbcf3261..ace175f251c 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -352,7 +352,11 @@ namespace } if (!executor->pull(chunk)) + { + if (configuration.check_exit_code) + command->wait(); return {}; + } current_read_rows += chunk.getNumRows(); } diff --git a/src/Processors/Sources/ShellCommandSource.h b/src/Processors/Sources/ShellCommandSource.h index b9afaa345cf..6dc6781cc4c 100644 --- a/src/Processors/Sources/ShellCommandSource.h +++ b/src/Processors/Sources/ShellCommandSource.h @@ -33,6 +33,9 @@ struct ShellCommandSourceConfiguration size_t number_of_rows_to_read = 0; /// Max block size size_t max_block_size = DEFAULT_BLOCK_SIZE; + /// Will throw if the command exited with + /// non-zero status code + size_t check_exit_code = false; }; class ShellCommandSourceCoordinator diff --git a/tests/integration/test_executable_dictionary/dictionaries/executable_missing_strict_check_dictionary.xml b/tests/integration/test_executable_dictionary/dictionaries/executable_missing_strict_check_dictionary.xml new file mode 100644 index 00000000000..8562851e69c --- /dev/null +++ b/tests/integration/test_executable_dictionary/dictionaries/executable_missing_strict_check_dictionary.xml @@ -0,0 +1,29 @@ + + + executable_input_missing_executable + + + + executable_not_found + CSV + + + + 86400 + 90000 + + + + + + + input + + + result + String + + + + + diff --git a/tests/integration/test_executable_dictionary/test.py b/tests/integration/test_executable_dictionary/test.py index 43e6ec0a800..22f3442bb95 100644 --- a/tests/integration/test_executable_dictionary/test.py +++ b/tests/integration/test_executable_dictionary/test.py @@ -217,33 +217,21 @@ def test_executable_implicit_input_argument_python(started_cluster): def test_executable_input_signalled_python(started_cluster): skip_test_msan(node) - assert ( - node.query( - "SELECT dictGet('executable_input_signalled_python', 'result', toUInt64(1))" - ) - == "Default result\n" + assert node.query_and_get_error( + "SELECT dictGet('executable_input_signalled_python', 'result', toUInt64(1))" ) - assert ( - node.query( - "SELECT dictGet('executable_input_signalled_pool_python', 'result', toUInt64(1))" - ) - == "Default result\n" + assert node.query_and_get_error( + "SELECT dictGet('executable_input_signalled_pool_python', 'result', toUInt64(1))" ) def test_executable_implicit_input_signalled_python(started_cluster): skip_test_msan(node) - assert ( - node.query( - "SELECT dictGet('executable_implicit_input_signalled_python', 'result', toUInt64(1))" - ) - == "Default result\n" + assert node.query_and_get_error( + "SELECT dictGet('executable_implicit_input_signalled_python', 'result', toUInt64(1))" ) - assert ( - node.query( - "SELECT dictGet('executable_implicit_input_signalled_pool_python', 'result', toUInt64(1))" - ) - == "Default result\n" + assert node.query_and_get_error( + "SELECT dictGet('executable_implicit_input_signalled_pool_python', 'result', toUInt64(1))" ) @@ -482,3 +470,23 @@ def test_executable_source_updated_python(started_cluster): ) == "Value 1 1\n" ) + + +def test_executable_source_exit_code_check(started_cluster): + skip_test_msan(node) + assert "DB::Exception" in node.query_and_get_error( + "SELECT * FROM dictionary(executable_input_missing_executable) ORDER BY input" + ) + assert "DB::Exception" in node.query_and_get_error( + "SELECT dictGet('executable_input_missing_executable', 'result', toUInt64(1))" + ) + + assert ( + node.query( + "SELECT status FROM system.dictionaries WHERE name='executable_input_missing_executable'" + ) + == "FAILED\n" + ) + assert "DB::Exception" in node.query( + "SELECT last_exception FROM system.dictionaries WHERE name='executable_input_missing_executable'" + ) From b3ce64caa5d548edbfa93d15f55062f572b60fd7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 22 Jun 2023 12:55:05 +0200 Subject: [PATCH 208/515] Fix tests --- tests/config/config.d/storage_conf.xml | 13 --- .../02344_describe_cache.reference | 4 +- ...ilesystem_cache_persistent_files.reference | 79 ------------------- ...2382_filesystem_cache_persistent_files.sql | 76 ------------------ 4 files changed, 2 insertions(+), 170 deletions(-) delete mode 100644 tests/queries/0_stateless/02382_filesystem_cache_persistent_files.reference delete mode 100644 tests/queries/0_stateless/02382_filesystem_cache_persistent_files.sql diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index dee03307177..deee71bd812 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -57,7 +57,6 @@ s3_cache/ 128Mi 1 - 0 100 @@ -65,7 +64,6 @@ s3_disk_2 s3_cache_2/ 128Mi - 0 100Mi 100 @@ -77,7 +75,6 @@ 22548578304 1 1 - 0 100 @@ -87,7 +84,6 @@ 128Mi 1 1 - 0 100 @@ -95,7 +91,6 @@ s3_disk_5 s3_cache_5/ 128Mi - 0 100 @@ -103,7 +98,6 @@ s3_disk_6 s3_cache_6/ 128Mi - 0 1 100 100 @@ -113,7 +107,6 @@ s3_disk_6 s3_cache_small/ 1000 - 1 100 @@ -122,7 +115,6 @@ s3_cache_small_segment_size/ 128Mi 10Ki - 0 1 100 @@ -146,7 +138,6 @@ local_cache/ 22548578304 1 - 0 100 @@ -154,7 +145,6 @@ local_disk local_cache_2/ 22548578304 - 0 100 @@ -164,7 +154,6 @@ 22548578304 1 1 - 0 100 @@ -173,7 +162,6 @@ s3_cache_5 s3_cache_multi/ 22548578304 - 0 100 @@ -181,7 +169,6 @@ s3_cache_multi s3_cache_multi_2/ 22548578304 - 0 100 diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index d8a2ffab1fa..2e6df4a6a0b 100644 --- a/tests/queries/0_stateless/02344_describe_cache.reference +++ b/tests/queries/0_stateless/02344_describe_cache.reference @@ -1,2 +1,2 @@ -134217728 10000000 8388608 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 -134217728 10000000 104857600 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 +134217728 10000000 8388608 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ +134217728 10000000 104857600 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ diff --git a/tests/queries/0_stateless/02382_filesystem_cache_persistent_files.reference b/tests/queries/0_stateless/02382_filesystem_cache_persistent_files.reference deleted file mode 100644 index e77afc98007..00000000000 --- a/tests/queries/0_stateless/02382_filesystem_cache_persistent_files.reference +++ /dev/null @@ -1,79 +0,0 @@ --- { echo } - -SET enable_filesystem_cache_on_write_operations=0; -SYSTEM DROP FILESYSTEM CACHE; -DROP TABLE IF EXISTS nopers; -CREATE TABLE nopers (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false; -SYSTEM STOP MERGES nopers; -INSERT INTO nopers SELECT number, toString(number) FROM numbers(10); -SELECT * FROM nopers FORMAT Null; -SELECT sum(size) FROM system.filesystem_cache; -195 -SELECT extract(local_path, '.*/([\w.]+)') as file, extract(cache_path, '.*/([\w.]+)') as cache, size -FROM -( - SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path - FROM system.remote_data_paths -) AS data_paths -INNER JOIN - system.filesystem_cache AS caches -ON data_paths.cache_path = caches.cache_path -ORDER BY file, cache, size; -data.bin 0 114 -data.mrk3 0 80 -format_version.txt 0 1 -DROP TABLE IF EXISTS test; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_small', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false; -SYSTEM STOP MERGES test; -INSERT INTO test SELECT number, toString(number) FROM numbers(100); -SELECT * FROM test FORMAT Null; -SELECT sum(size) FROM system.filesystem_cache; -1021 -SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; -5 -SELECT count() FROM system.filesystem_cache; -5 -SELECT extract(local_path, '.*/([\w.]+)') as file, extract(cache_path, '.*/([\w.]+)') as cache, size -FROM -( - SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path - FROM system.remote_data_paths -) AS data_paths -INNER JOIN - system.filesystem_cache AS caches -ON data_paths.cache_path = caches.cache_path -ORDER BY file, cache, size; -data.bin 0 114 -data.bin 0 746 -data.mrk3 0 80 -data.mrk3 0_persistent 80 -format_version.txt 0 1 -DROP TABLE IF EXISTS test2; -CREATE TABLE test2 (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_small', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false; -SYSTEM STOP MERGES test2; -INSERT INTO test2 SELECT number, toString(number) FROM numbers(100000); -SELECT * FROM test2 FORMAT Null; -SELECT sum(size) FROM system.filesystem_cache; -795 -SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; -5 -SELECT count() FROM system.filesystem_cache; -5 -SELECT extract(local_path, '.*/([\w.]+)') as file, extract(cache_path, '.*/([\w.]+)') as cache, size -FROM -( - SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path - FROM system.remote_data_paths -) AS data_paths -INNER JOIN - system.filesystem_cache AS caches -ON data_paths.cache_path = caches.cache_path -ORDER BY file, cache, size; -data.bin 0 114 -data.mrk3 0 80 -data.mrk3 0_persistent 80 -data.mrk3 0_persistent 520 -format_version.txt 0 1 -DROP TABLE test; -DROP TABLE test2; -DROP TABLE nopers; diff --git a/tests/queries/0_stateless/02382_filesystem_cache_persistent_files.sql b/tests/queries/0_stateless/02382_filesystem_cache_persistent_files.sql deleted file mode 100644 index 6486840602e..00000000000 --- a/tests/queries/0_stateless/02382_filesystem_cache_persistent_files.sql +++ /dev/null @@ -1,76 +0,0 @@ --- Tags: no-fasttest, no-parallel, no-s3-storage, no-random-settings - --- { echo } - -SET enable_filesystem_cache_on_write_operations=0; - -SYSTEM DROP FILESYSTEM CACHE; - -DROP TABLE IF EXISTS nopers; -CREATE TABLE nopers (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false; -SYSTEM STOP MERGES nopers; - -INSERT INTO nopers SELECT number, toString(number) FROM numbers(10); -SELECT * FROM nopers FORMAT Null; -SELECT sum(size) FROM system.filesystem_cache; - -SELECT extract(local_path, '.*/([\w.]+)') as file, extract(cache_path, '.*/([\w.]+)') as cache, size -FROM -( - SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path - FROM system.remote_data_paths -) AS data_paths -INNER JOIN - system.filesystem_cache AS caches -ON data_paths.cache_path = caches.cache_path -ORDER BY file, cache, size; - -DROP TABLE IF EXISTS test; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_small', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false; -SYSTEM STOP MERGES test; - -INSERT INTO test SELECT number, toString(number) FROM numbers(100); -SELECT * FROM test FORMAT Null; - -SELECT sum(size) FROM system.filesystem_cache; - -SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; -SELECT count() FROM system.filesystem_cache; - -SELECT extract(local_path, '.*/([\w.]+)') as file, extract(cache_path, '.*/([\w.]+)') as cache, size -FROM -( - SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path - FROM system.remote_data_paths -) AS data_paths -INNER JOIN - system.filesystem_cache AS caches -ON data_paths.cache_path = caches.cache_path -ORDER BY file, cache, size; - -DROP TABLE IF EXISTS test2; -CREATE TABLE test2 (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_small', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false; -SYSTEM STOP MERGES test2; - -INSERT INTO test2 SELECT number, toString(number) FROM numbers(100000); -SELECT * FROM test2 FORMAT Null; - -SELECT sum(size) FROM system.filesystem_cache; - -SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path; -SELECT count() FROM system.filesystem_cache; - -SELECT extract(local_path, '.*/([\w.]+)') as file, extract(cache_path, '.*/([\w.]+)') as cache, size -FROM -( - SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path - FROM system.remote_data_paths -) AS data_paths -INNER JOIN - system.filesystem_cache AS caches -ON data_paths.cache_path = caches.cache_path -ORDER BY file, cache, size; - -DROP TABLE test; -DROP TABLE test2; -DROP TABLE nopers; From 95349a405b46c00fb6024a2e90defe0ee28b0b89 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 22 Jun 2023 11:09:26 +0200 Subject: [PATCH 209/515] release buffers with exception context --- src/Daemon/BaseDaemon.cpp | 4 ++++ .../Transforms/ExceptionKeepingTransform.cpp | 10 +++++----- .../Transforms/ExceptionKeepingTransform.h | 2 +- src/Storages/HDFS/StorageHDFS.cpp | 2 +- src/Storages/MessageQueueSink.h | 2 +- src/Storages/PartitionedSink.cpp | 4 ++-- src/Storages/PartitionedSink.h | 2 +- src/Storages/StorageAzureBlob.cpp | 2 +- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageS3.cpp | 18 +++++++++++------- src/Storages/StorageURL.cpp | 2 +- src/Storages/StorageURL.h | 2 +- 12 files changed, 30 insertions(+), 22 deletions(-) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index bfd5568b71d..732d5b1d2f4 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -417,6 +417,8 @@ private: { SentryWriter::onFault(sig, error_message, stack_trace); +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wunreachable-code" /// Advice the user to send it manually. if constexpr (std::string_view(VERSION_OFFICIAL).contains("official build")) { @@ -436,6 +438,8 @@ private: { LOG_FATAL(log, "This ClickHouse version is not official and should be upgraded to the official build."); } +#pragma clang diagnostic pop + } /// ClickHouse Keeper does not link to some part of Settings. diff --git a/src/Processors/Transforms/ExceptionKeepingTransform.cpp b/src/Processors/Transforms/ExceptionKeepingTransform.cpp index 266407f21a5..3c40c078225 100644 --- a/src/Processors/Transforms/ExceptionKeepingTransform.cpp +++ b/src/Processors/Transforms/ExceptionKeepingTransform.cpp @@ -76,7 +76,7 @@ IProcessor::Status ExceptionKeepingTransform::prepare() if (data.exception) { stage = Stage::Exception; - onException(); + onException(data.exception); output.pushData(std::move(data)); return Status::PortFull; } @@ -139,7 +139,7 @@ void ExceptionKeepingTransform::work() stage = Stage::Exception; ready_output = true; data.exception = exception; - onException(); + onException(data.exception); } } else if (stage == Stage::Consume || stage == Stage::Generate) @@ -153,7 +153,7 @@ void ExceptionKeepingTransform::work() stage = Stage::Exception; ready_output = true; data.exception = exception; - onException(); + onException(data.exception); } else stage = Stage::Generate; @@ -167,7 +167,7 @@ void ExceptionKeepingTransform::work() stage = Stage::Exception; ready_output = true; data.exception = exception; - onException(); + onException(data.exception); } else { @@ -189,7 +189,7 @@ void ExceptionKeepingTransform::work() stage = Stage::Exception; ready_output = true; data.exception = exception; - onException(); + onException(data.exception); } } } diff --git a/src/Processors/Transforms/ExceptionKeepingTransform.h b/src/Processors/Transforms/ExceptionKeepingTransform.h index e2bc161971e..cec0e0eea31 100644 --- a/src/Processors/Transforms/ExceptionKeepingTransform.h +++ b/src/Processors/Transforms/ExceptionKeepingTransform.h @@ -52,7 +52,7 @@ protected: virtual void onConsume(Chunk chunk) = 0; virtual GenerateResult onGenerate() = 0; virtual void onFinish() {} - virtual void onException() {} + virtual void onException(std::exception_ptr /* exception */) {} public: ExceptionKeepingTransform(const Block & in_header, const Block & out_header, bool ignore_on_start_and_finish_ = true); diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 827d89e26a4..7a9d996e3c2 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -512,7 +512,7 @@ public: cancelled = true; } - void onException() override + void onException(std::exception_ptr /* exception */) override { std::lock_guard lock(cancel_mutex); finalize(); diff --git a/src/Storages/MessageQueueSink.h b/src/Storages/MessageQueueSink.h index 590bee7ee4f..b3c1e61734f 100644 --- a/src/Storages/MessageQueueSink.h +++ b/src/Storages/MessageQueueSink.h @@ -40,7 +40,7 @@ public: void onStart() override; void onFinish() override; void onCancel() override { onFinish(); } - void onException() override { onFinish(); } + void onException(std::exception_ptr /* exception */) override { onFinish(); } protected: /// Do some specific initialization before consuming data. diff --git a/src/Storages/PartitionedSink.cpp b/src/Storages/PartitionedSink.cpp index 363b4557290..18442a8691f 100644 --- a/src/Storages/PartitionedSink.cpp +++ b/src/Storages/PartitionedSink.cpp @@ -111,11 +111,11 @@ void PartitionedSink::consume(Chunk chunk) } } -void PartitionedSink::onException() +void PartitionedSink::onException(std::exception_ptr exception) { for (auto & [_, sink] : partition_id_to_sink) { - sink->onException(); + sink->onException(exception); } } diff --git a/src/Storages/PartitionedSink.h b/src/Storages/PartitionedSink.h index c4bf9c0622c..68edeb6fd73 100644 --- a/src/Storages/PartitionedSink.h +++ b/src/Storages/PartitionedSink.h @@ -22,7 +22,7 @@ public: void consume(Chunk chunk) override; - void onException() override; + void onException(std::exception_ptr exception) override; void onFinish() override; diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 336c4eaed9b..65af2ed6d6e 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -489,7 +489,7 @@ public: cancelled = true; } - void onException() override + void onException(std::exception_ptr /* exception */) override { std::lock_guard lock(cancel_mutex); finalize(); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 7c04de1a28a..50aa47ffca9 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1031,7 +1031,7 @@ public: cancelled = true; } - void onException() override + void onException(std::exception_ptr /* exception */) override { std::lock_guard cancel_lock(cancel_mutex); finalize(); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 3c0dbdcd766..292ae4813dd 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -813,10 +813,18 @@ public: cancelled = true; } - void onException() override + void onException(std::exception_ptr exception) override { std::lock_guard lock(cancel_mutex); - release(); + try + { + std::rethrow_exception(exception); + } + catch (...) + { + /// An exception context is needed to proper delete write buffers without finalization + release(); + } } void onFinish() override @@ -840,17 +848,13 @@ private: catch (...) { /// Stop ParallelFormattingOutputFormat correctly. - writer.reset(); - write_buf->finalize(); + release(); throw; } } void release() { - if (!writer) - return; - writer.reset(); write_buf.reset(); } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 1d6aed204cb..651b50518d2 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -480,7 +480,7 @@ void StorageURLSink::onCancel() cancelled = true; } -void StorageURLSink::onException() +void StorageURLSink::onException(std::exception_ptr /* exception */) { std::lock_guard lock(cancel_mutex); finalize(); diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index e3305cda89e..c4a5ce1aa7b 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -236,7 +236,7 @@ public: std::string getName() const override { return "StorageURLSink"; } void consume(Chunk chunk) override; void onCancel() override; - void onException() override; + void onException(std::exception_ptr exception) override; void onFinish() override; private: From 6929a9b852a6dc25e4bbcb85ba384f416c90c89e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 22 Jun 2023 14:01:28 +0300 Subject: [PATCH 210/515] Update WriteBufferFromOStream.cpp --- src/IO/WriteBufferFromOStream.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromOStream.cpp b/src/IO/WriteBufferFromOStream.cpp index 2d0d5976f85..5074c226dd3 100644 --- a/src/IO/WriteBufferFromOStream.cpp +++ b/src/IO/WriteBufferFromOStream.cpp @@ -18,7 +18,14 @@ void WriteBufferFromOStream::nextImpl() ostr->flush(); if (!ostr->good()) - throw Exception(ErrorCodes::CANNOT_WRITE_TO_OSTREAM, "Cannot write to ostream at offset {}", count()); + { + /// FIXME do not call finalize in dtors (and remove iostreams) + bool avoid_throwing_exceptions = std::uncaught_exceptions(); + if (avoid_throwing_exceptions) + LOG_ERROR(&Poco::Logger::get("WriteBufferFromOStream"), "Cannot write to ostream at offset {}. Stack trace: {}", count(), StackTrace().toString()); + else + throw Exception(ErrorCodes::CANNOT_WRITE_TO_OSTREAM, "Cannot write to ostream at offset {}", count()); + } } WriteBufferFromOStream::WriteBufferFromOStream( From 4fbf52f2de95983b40921d65c68df6a59d6cd570 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 22 Jun 2023 13:08:38 +0200 Subject: [PATCH 211/515] Less loggers --- src/Interpreters/Cache/Metadata.cpp | 16 ++++++++-------- src/Interpreters/Cache/Metadata.h | 2 +- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 20af931bbc2..af8520fab8a 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -58,6 +58,11 @@ KeyMetadata::KeyMetadata( , key_path(key_path_) , cleanup_queue(cleanup_queue_) , created_base_directory(created_base_directory_) +#ifdef ABORT_ON_LOGICAL_ERROR + , log(&Poco::Logger::get("LockedKey(" + key.toString() + ")")) +#else + , log(&Poco::Logger::get("LockedKey")) +#endif { if (created_base_directory) chassert(fs::exists(key_path)); @@ -324,11 +329,6 @@ void CacheMetadata::doCleanup() LockedKey::LockedKey(std::shared_ptr key_metadata_) : key_metadata(key_metadata_) , lock(key_metadata->guard.lock()) -#ifdef ABORT_ON_LOGICAL_ERROR - , log(&Poco::Logger::get("LockedKey(" + key_metadata_->key.toString() + ")")) -#else - , log(&Poco::Logger::get("LockedKey")) -#endif { } @@ -338,7 +338,7 @@ LockedKey::~LockedKey() return; key_metadata->key_state = KeyMetadata::KeyState::REMOVING; - LOG_DEBUG(log, "Submitting key {} for removal", getKey()); + LOG_DEBUG(key_metadata->log, "Submitting key {} for removal", getKey()); key_metadata->cleanup_queue.add(getKey()); } @@ -396,7 +396,7 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegm auto file_segment = it->second->file_segment; LOG_DEBUG( - log, "Remove from cache. Key: {}, offset: {}, size: {}", + key_metadata->log, "Remove from cache. Key: {}, offset: {}, size: {}", getKey(), offset, file_segment->reserved_size); chassert(file_segment->assertCorrectnessUnlocked(segment_lock)); @@ -409,7 +409,7 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegm if (exists) { fs::remove(path); - LOG_TEST(log, "Removed file segment at path: {}", path); + LOG_TEST(key_metadata->log, "Removed file segment at path: {}", path); } else if (file_segment->downloaded_size) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist", path); diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index 6172808bcf5..c9a8e97beee 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -70,6 +70,7 @@ private: KeyGuard guard; CleanupQueue & cleanup_queue; std::atomic created_base_directory = false; + Poco::Logger * log; }; using KeyMetadataPtr = std::shared_ptr; @@ -171,7 +172,6 @@ struct LockedKey : private boost::noncopyable private: const std::shared_ptr key_metadata; KeyGuard::Lock lock; /// `lock` must be destructed before `key_metadata`. - Poco::Logger * log; }; } From 8ff8125ebda3352e802ecc46b3f74b79caad6e05 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 22 Jun 2023 13:24:50 +0200 Subject: [PATCH 212/515] Better --- src/Interpreters/Cache/Metadata.cpp | 7 ++++--- src/Interpreters/Cache/Metadata.h | 1 + 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index af8520fab8a..b19d095692d 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -53,15 +53,16 @@ KeyMetadata::KeyMetadata( const Key & key_, const std::string & key_path_, CleanupQueue & cleanup_queue_, + [[maybe_unused]] Poco::Logger * log_, bool created_base_directory_) : key(key_) , key_path(key_path_) , cleanup_queue(cleanup_queue_) , created_base_directory(created_base_directory_) #ifdef ABORT_ON_LOGICAL_ERROR - , log(&Poco::Logger::get("LockedKey(" + key.toString() + ")")) + , log(&Poco::Logger::get("Key(" + key.toString() + ")")) #else - , log(&Poco::Logger::get("LockedKey")) + , log(log_) #endif { if (created_base_directory) @@ -195,7 +196,7 @@ LockedKeyPtr CacheMetadata::lockKeyMetadata( it = emplace( key, std::make_shared( - key, getPathForKey(key), *cleanup_queue, is_initial_load)).first; + key, getPathForKey(key), *cleanup_queue, log, is_initial_load)).first; } key_metadata = it->second; diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index c9a8e97beee..635594a289a 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -44,6 +44,7 @@ struct KeyMetadata : public std::map, const Key & key_, const std::string & key_path_, CleanupQueue & cleanup_queue_, + Poco::Logger * log_, bool created_base_directory_ = false); enum class KeyState From a1c131c0da77e16b9cf63e8a081ac6e01f08ad7f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Jun 2023 10:29:51 +0000 Subject: [PATCH 213/515] Query Cache: Try to fix bad cast from ColumnConst to ColumnVector Resolves #49445 The query cache buffers query result chunks and eventually squashes them before insertion into the cache. Here, squashing failed because not all chunks were of the same type. Looks like chunks of the same underlying type (e.g. UInt8) in a query result be of mixed const, sparse or low-cardinality type. Fix this by always materializing the data regardless of the compression. Strange thing is that the failing query in the stress test (*) isn't able to reproduce the bug, and I haven't managed to trigger the issue otherwise, so no test case is added. (*) SELECT 1 UNION ALL SELECT 1 INTERSECT SELECT 1 E.g. here: https://s3.amazonaws.com/clickhouse-test-reports/0/18817517ed6f8849e3d979e10fbb273e0edf0eaa/stress_test__debug_/fatal_messages.txt --- src/Interpreters/Cache/QueryCache.cpp | 10 +++++----- src/Processors/Chunk.cpp | 26 ++++++++++++++++++++++++++ src/Processors/Chunk.h | 3 +++ 3 files changed, 34 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 3118f386111..364a740abe0 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -263,23 +263,23 @@ void QueryCache::Writer::finalizeWrite() if (auto entry = cache.getWithKey(key); entry.has_value() && !IsStale()(entry->key)) { - /// same check as in ctor because a parallel Writer could have inserted the current key in the meantime + /// Same check as in ctor because a parallel Writer could have inserted the current key in the meantime LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (non-stale entry found), query: {}", key.queryStringFromAst()); return; } if (squash_partial_results) { - // Squash partial result chunks to chunks of size 'max_block_size' each. This costs some performance but provides a more natural - // compression of neither too small nor big blocks. Also, it will look like 'max_block_size' is respected when the query result is - // served later on from the query cache. + /// Squash partial result chunks to chunks of size 'max_block_size' each. This costs some performance but provides a more natural + /// compression of neither too small nor big blocks. Also, it will look like 'max_block_size' is respected when the query result is + /// served later on from the query cache. Chunks squashed_chunks; size_t rows_remaining_in_squashed = 0; /// how many further rows can the last squashed chunk consume until it reaches max_block_size for (auto & chunk : query_result->chunks) { - convertToFullIfSparse(chunk); + convertToFullIfNeeded(chunk); const size_t rows_chunk = chunk.getNumRows(); if (rows_chunk == 0) diff --git a/src/Processors/Chunk.cpp b/src/Processors/Chunk.cpp index 0a4b2413e4c..24181866087 100644 --- a/src/Processors/Chunk.cpp +++ b/src/Processors/Chunk.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -202,6 +203,24 @@ const ChunkMissingValues::RowsBitMask & ChunkMissingValues::getDefaultsBitmask(s return none; } +void convertToFullIfConst(Chunk & chunk) +{ + size_t num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + for (auto & column : columns) + column = column->convertToFullColumnIfConst(); + chunk.setColumns(std::move(columns), num_rows); +} + +void convertToFullIfLowCardinality(Chunk & chunk) +{ + size_t num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + for (auto & column : columns) + column = recursiveRemoveLowCardinality(column); + chunk.setColumns(std::move(columns), num_rows); +} + void convertToFullIfSparse(Chunk & chunk) { size_t num_rows = chunk.getNumRows(); @@ -212,4 +231,11 @@ void convertToFullIfSparse(Chunk & chunk) chunk.setColumns(std::move(columns), num_rows); } +void convertToFullIfNeeded(Chunk & chunk) +{ + convertToFullIfSparse(chunk); + convertToFullIfConst(chunk); + convertToFullIfLowCardinality(chunk); +} + } diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index 6f2097b71f1..45610ae3276 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -149,6 +149,9 @@ private: /// It's needed, when you have to access to the internals of the column, /// or when you need to perform operation with two columns /// and their structure must be equal (e.g. compareAt). +void convertToFullIfConst(Chunk & chunk); +void convertToFullIfLowCardinality(Chunk & chunk); void convertToFullIfSparse(Chunk & chunk); +void convertToFullIfNeeded(Chunk & chunk); } From a9f1393862c628f0cd46bdf7ef3e963b3f516846 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 22 Jun 2023 11:43:11 +0000 Subject: [PATCH 214/515] Make the fix more minimal --- src/Interpreters/Cache/QueryCache.cpp | 4 +++- src/Processors/Chunk.cpp | 17 ----------------- src/Processors/Chunk.h | 2 -- 3 files changed, 3 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 364a740abe0..74a1c3c9b18 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -233,6 +233,7 @@ void QueryCache::Writer::buffer(Chunk && chunk, ChunkType chunk_type) auto & buffered_chunk = (chunk_type == ChunkType::Totals) ? query_result->totals : query_result->extremes; convertToFullIfSparse(chunk); + convertToFullIfConst(chunk); if (!buffered_chunk.has_value()) buffered_chunk = std::move(chunk); @@ -279,7 +280,8 @@ void QueryCache::Writer::finalizeWrite() for (auto & chunk : query_result->chunks) { - convertToFullIfNeeded(chunk); + convertToFullIfSparse(chunk); + convertToFullIfConst(chunk); const size_t rows_chunk = chunk.getNumRows(); if (rows_chunk == 0) diff --git a/src/Processors/Chunk.cpp b/src/Processors/Chunk.cpp index 24181866087..9ec5bb7adde 100644 --- a/src/Processors/Chunk.cpp +++ b/src/Processors/Chunk.cpp @@ -212,30 +212,13 @@ void convertToFullIfConst(Chunk & chunk) chunk.setColumns(std::move(columns), num_rows); } -void convertToFullIfLowCardinality(Chunk & chunk) -{ - size_t num_rows = chunk.getNumRows(); - auto columns = chunk.detachColumns(); - for (auto & column : columns) - column = recursiveRemoveLowCardinality(column); - chunk.setColumns(std::move(columns), num_rows); -} - void convertToFullIfSparse(Chunk & chunk) { size_t num_rows = chunk.getNumRows(); auto columns = chunk.detachColumns(); for (auto & column : columns) column = recursiveRemoveSparse(column); - chunk.setColumns(std::move(columns), num_rows); } -void convertToFullIfNeeded(Chunk & chunk) -{ - convertToFullIfSparse(chunk); - convertToFullIfConst(chunk); - convertToFullIfLowCardinality(chunk); -} - } diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index 45610ae3276..413872d512d 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -150,8 +150,6 @@ private: /// or when you need to perform operation with two columns /// and their structure must be equal (e.g. compareAt). void convertToFullIfConst(Chunk & chunk); -void convertToFullIfLowCardinality(Chunk & chunk); void convertToFullIfSparse(Chunk & chunk); -void convertToFullIfNeeded(Chunk & chunk); } From 35c5483ad25d7c34adb45adcff8c16cd73a7ad09 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 22 Jun 2023 14:56:20 +0300 Subject: [PATCH 215/515] Update WriteBufferFromOStream.cpp --- src/IO/WriteBufferFromOStream.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/IO/WriteBufferFromOStream.cpp b/src/IO/WriteBufferFromOStream.cpp index 5074c226dd3..e0ec0b770e2 100644 --- a/src/IO/WriteBufferFromOStream.cpp +++ b/src/IO/WriteBufferFromOStream.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB From ee68e85d310a6843ed833fe9c985a540620f9940 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 22 Jun 2023 12:23:51 +0000 Subject: [PATCH 216/515] better reporting of broken parts --- .../IMergeTreeDataPartInfoForReader.h | 2 ++ src/Storages/MergeTree/IMergeTreeReader.cpp | 9 +++++++ src/Storages/MergeTree/IMergeTreeReader.h | 2 ++ .../LoadedMergeTreeDataPartInfoForReader.h | 2 ++ src/Storages/MergeTree/MergeTreeData.cpp | 21 --------------- .../MergeTree/MergeTreeReaderCompact.cpp | 24 ++++++++++++++--- .../MergeTree/MergeTreeReaderWide.cpp | 26 ++++++++----------- .../ReplicatedMergeTreePartCheckThread.cpp | 2 +- src/Storages/MergeTree/checkDataPart.cpp | 24 ++++++++++++++++- src/Storages/MergeTree/checkDataPart.h | 1 + 10 files changed, 72 insertions(+), 41 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h b/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h index dbed58ba91c..2cc73556f04 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h @@ -66,6 +66,8 @@ public: virtual const SerializationInfoByName & getSerializationInfos() const = 0; + virtual String getTableName() const = 0; + virtual void reportBroken() = 0; }; diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index be9aedaf4d5..d93e88fdeb2 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -278,4 +278,13 @@ void IMergeTreeReader::checkNumberOfColumns(size_t num_columns_to_read) const "Expected {}, got {}", requested_columns.size(), num_columns_to_read); } +String IMergeTreeReader::getMessageForDiagnosticOfBrokenPart(size_t from_mark, size_t max_rows_to_read) const +{ + return fmt::format( + "(while reading from part {} in table {} from mark {} with max_rows_to_read = {})", + data_part_info_for_read->getDataPartStorage()->getFullPath(), + data_part_info_for_read->getTableName(), + from_mark, max_rows_to_read); +} + } diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index a72d83a55e4..fcab35fb4c2 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -74,6 +74,8 @@ protected: void checkNumberOfColumns(size_t num_columns_to_read) const; + String getMessageForDiagnosticOfBrokenPart(size_t from_mark, size_t max_rows_to_read) const; + /// avg_value_size_hints are used to reduce the number of reallocations when creating columns of variable size. ValueSizeMap avg_value_size_hints; /// Stores states for IDataType::deserializeBinaryBulk diff --git a/src/Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h b/src/Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h index aec102f3f7d..f1cb8b34ecf 100644 --- a/src/Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h +++ b/src/Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h @@ -56,6 +56,8 @@ public: SerializationPtr getSerialization(const NameAndTypePair & column) const override { return data_part->getSerialization(column.name); } + String getTableName() const override { return data_part->storage.getStorageID().getNameForLogs(); } + MergeTreeData::DataPartPtr getDataPart() const { return data_part; } private: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c3ffb892871..33d65ceb5da 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -179,8 +179,6 @@ namespace ErrorCodes extern const int ZERO_COPY_REPLICATION_ERROR; extern const int NOT_INITIALIZED; extern const int SERIALIZATION_ERROR; - extern const int NETWORK_ERROR; - extern const int SOCKET_TIMEOUT; extern const int TOO_MANY_MUTATIONS; } @@ -1174,25 +1172,6 @@ static void preparePartForRemoval(const MergeTreeMutableDataPartPtr & part) } } -static bool isRetryableException(const Exception & e) -{ - if (isNotEnoughMemoryErrorCode(e.code())) - return true; - - if (e.code() == ErrorCodes::NETWORK_ERROR || e.code() == ErrorCodes::SOCKET_TIMEOUT) - return true; - -#if USE_AWS_S3 - const auto * s3_exception = dynamic_cast(&e); - if (s3_exception && s3_exception->isRetryableError()) - return true; -#endif - - /// In fact, there can be other similar situations. - /// But it is OK, because there is a safety guard against deleting too many parts. - return false; -} - static constexpr size_t loading_parts_initial_backoff_ms = 100; static constexpr size_t loading_parts_max_backoff_ms = 5000; static constexpr size_t loading_parts_max_tries = 3; diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index d22684eaa9d..f65e66ff52d 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -10,7 +11,6 @@ namespace ErrorCodes { extern const int CANNOT_READ_ALL_DATA; extern const int ARGUMENT_OUT_OF_BOUND; - extern const int MEMORY_LIMIT_EXCEEDED; } @@ -112,6 +112,12 @@ void MergeTreeReaderCompact::initialize() compressed_data_buffer = non_cached_buffer.get(); } } + catch (const Exception & e) + { + if (!isRetryableException(e)) + data_part_info_for_read->reportBroken(); + throw; + } catch (...) { data_part_info_for_read->reportBroken(); @@ -207,11 +213,11 @@ size_t MergeTreeReaderCompact::readRows( } catch (Exception & e) { - if (e.code() != ErrorCodes::MEMORY_LIMIT_EXCEEDED) + if (!isRetryableException(e)) data_part_info_for_read->reportBroken(); /// Better diagnostics. - e.addMessage("(while reading column " + columns_to_read[pos].name + ")"); + e.addMessage(getMessageForDiagnosticOfBrokenPart(from_mark, max_rows_to_read)); throw; } catch (...) @@ -315,6 +321,7 @@ void MergeTreeReaderCompact::readData( } void MergeTreeReaderCompact::prefetchBeginOfRange(Priority priority) +try { if (!initialized) { @@ -326,6 +333,17 @@ void MergeTreeReaderCompact::prefetchBeginOfRange(Priority priority) seekToMark(all_mark_ranges.front().begin, 0); data_buffer->prefetch(priority); } +catch (const Exception & e) +{ + if (!isRetryableException(e)) + data_part_info_for_read->reportBroken(); + throw; +} +catch (...) +{ + data_part_info_for_read->reportBroken(); + throw; +} void MergeTreeReaderCompact::seekToMark(size_t row_index, size_t column_index) { diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 38032fae826..140fb6da5df 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -20,11 +21,6 @@ namespace constexpr auto DATA_FILE_EXTENSION = ".bin"; } -namespace ErrorCodes -{ - extern const int MEMORY_LIMIT_EXCEEDED; -} - MergeTreeReaderWide::MergeTreeReaderWide( MergeTreeDataPartInfoForReaderPtr data_part_info_, NamesAndTypesList columns_, @@ -51,6 +47,12 @@ MergeTreeReaderWide::MergeTreeReaderWide( for (size_t i = 0; i < columns_to_read.size(); ++i) addStreams(columns_to_read[i], serializations[i], profile_callback_, clock_type_); } + catch (const Exception & e) + { + if (!isRetryableException(e)) + data_part_info_for_read->reportBroken(); + throw; + } catch (...) { data_part_info_for_read->reportBroken(); @@ -76,9 +78,9 @@ void MergeTreeReaderWide::prefetchBeginOfRange(Priority priority) /// of range only once so there is no such problem. /// 4. continue_reading == false, as we haven't read anything yet. } - catch (Exception & e) + catch (const Exception & e) { - if (e.code() != ErrorCodes::MEMORY_LIMIT_EXCEEDED) + if (!isRetryableException(e)) data_part_info_for_read->reportBroken(); throw; } @@ -184,22 +186,16 @@ size_t MergeTreeReaderWide::readRows( } catch (Exception & e) { - if (e.code() != ErrorCodes::MEMORY_LIMIT_EXCEEDED) + if (!isRetryableException(e)) data_part_info_for_read->reportBroken(); /// Better diagnostics. - const auto & part_storage = data_part_info_for_read->getDataPartStorage(); - e.addMessage( - fmt::format( - "(while reading from part {} located on disk {} of type {}, from mark {} with max_rows_to_read = {})", - part_storage->getFullPath(), part_storage->getDiskName(), part_storage->getDiskType(), - toString(from_mark), toString(max_rows_to_read))); + e.addMessage(getMessageForDiagnosticOfBrokenPart(from_mark, max_rows_to_read)); throw; } catch (...) { data_part_info_for_read->reportBroken(); - throw; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 7bb8d9d758e..43256505135 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -422,7 +422,7 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na /// Don't count the part as broken if there is not enough memory to load it. /// In fact, there can be many similar situations. /// But it is OK, because there is a safety guard against deleting too many parts. - if (isNotEnoughMemoryErrorCode(e.code())) + if (isRetryableException(e)) throw; tryLogCurrentException(log, __PRETTY_FUNCTION__); diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 5c090cdd739..1967357a840 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include @@ -33,6 +34,8 @@ namespace ErrorCodes extern const int CANNOT_MUNMAP; extern const int CANNOT_MREMAP; extern const int UNEXPECTED_FILE_IN_DATA_PART; + extern const int NETWORK_ERROR; + extern const int SOCKET_TIMEOUT; } @@ -47,6 +50,25 @@ bool isNotEnoughMemoryErrorCode(int code) || code == ErrorCodes::CANNOT_MREMAP; } +bool isRetryableException(const Exception & e) +{ + if (isNotEnoughMemoryErrorCode(e.code())) + return true; + + if (e.code() == ErrorCodes::NETWORK_ERROR || e.code() == ErrorCodes::SOCKET_TIMEOUT) + return true; + +#if USE_AWS_S3 + const auto * s3_exception = dynamic_cast(&e); + if (s3_exception && s3_exception->isRetryableError()) + return true; +#endif + + /// In fact, there can be other similar situations. + /// But it is OK, because there is a safety guard against deleting too many parts. + return false; +} + static IMergeTreeDataPart::Checksums checkDataPart( MergeTreeData::DataPartPtr data_part, @@ -302,7 +324,7 @@ IMergeTreeDataPart::Checksums checkDataPart( } catch (const Exception & e) { - if (isNotEnoughMemoryErrorCode(e.code())) + if (isRetryableException(e)) throw; return drop_cache_and_check(); diff --git a/src/Storages/MergeTree/checkDataPart.h b/src/Storages/MergeTree/checkDataPart.h index d0cf25b87be..20ddecad3ed 100644 --- a/src/Storages/MergeTree/checkDataPart.h +++ b/src/Storages/MergeTree/checkDataPart.h @@ -13,5 +13,6 @@ IMergeTreeDataPart::Checksums checkDataPart( std::function is_cancelled = []{ return false; }); bool isNotEnoughMemoryErrorCode(int code); +bool isRetryableException(const Exception & e); } From 94a194d153488c7d1afac6aa212e9f3fea706f7f Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 22 Jun 2023 08:29:32 -0400 Subject: [PATCH 217/515] move redirects to docusaurus config --- docs/redirects.txt | 459 +-------------------------------------------- 1 file changed, 6 insertions(+), 453 deletions(-) diff --git a/docs/redirects.txt b/docs/redirects.txt index cea138f7237..ebeda125e01 100644 --- a/docs/redirects.txt +++ b/docs/redirects.txt @@ -1,453 +1,6 @@ -agg_functions/combinators.md query-language/agg-functions/combinators.md -agg_functions/index.md query-language/agg-functions/index.md -agg_functions/parametric_functions.md query-language/agg-functions/parametric-functions.md -agg_functions/reference.md query-language/agg-functions/reference.md -changelog/2017.md whats-new/changelog/2017.md -changelog/2018.md whats-new/changelog/2018.md -changelog/2019.md whats-new/changelog/2019.md -changelog/index.md whats-new/changelog/index.md -commercial/cloud.md https://clickhouse.com/cloud/ -data_types/array.md sql-reference/data-types/array.md -data_types/boolean.md sql-reference/data-types/boolean.md -data_types/date.md sql-reference/data-types/date.md -data_types/datetime.md sql-reference/data-types/datetime.md -data_types/datetime64.md sql-reference/data-types/datetime64.md -data_types/decimal.md sql-reference/data-types/decimal.md -data_types/domains/ipv4.md sql-reference/data-types/domains/ipv4.md -data_types/domains/ipv6.md sql-reference/data-types/domains/ipv6.md -data_types/domains/overview.md sql-reference/data-types/domains/overview.md -data_types/enum.md sql-reference/data-types/enum.md -data_types/fixedstring.md sql-reference/data-types/fixedstring.md -data_types/float.md sql-reference/data-types/float.md -data_types/index.md sql-reference/data-types/index.md -data_types/int_uint.md sql-reference/data-types/int-uint.md -data_types/nested_data_structures/aggregatefunction.md sql-reference/data-types/aggregatefunction.md -data_types/nested_data_structures/index.md sql-reference/data-types/nested-data-structures/index.md -data_types/nested_data_structures/nested.md sql-reference/data-types/nested-data-structures/nested.md -data_types/nullable.md sql-reference/data-types/nullable.md -data_types/special_data_types/expression.md sql-reference/data-types/special-data-types/expression.md -data_types/special_data_types/index.md sql-reference/data-types/special-data-types/index.md -data_types/special_data_types/interval.md sql-reference/data-types/special-data-types/interval.md -data_types/special_data_types/nothing.md sql-reference/data-types/special-data-types/nothing.md -data_types/special_data_types/set.md sql-reference/data-types/special-data-types/set.md -data_types/string.md sql-reference/data-types/string.md -data_types/tuple.md sql-reference/data-types/tuple.md -data_types/uuid.md sql-reference/data-types/uuid.md -database_engines/index.md engines/database-engines/index.md -database_engines/lazy.md engines/database-engines/lazy.md -database_engines/mysql.md engines/database-engines/mysql.md -development/browse_code.md development/browse-code.md -development/build_cross_arm.md development/build-cross-arm.md -development/build_cross_osx.md development/build-cross-osx.md -development/build_osx.md development/build-osx.md -development/developer_instruction.md development/developer-instruction.md -dicts/external_dicts.md query-language/dicts/external-dicts.md -dicts/external_dicts_dict.md query-language/dicts/external-dicts-dict.md -dicts/external_dicts_dict_layout.md query-language/dicts/external-dicts-dict-layout.md -dicts/external_dicts_dict_lifetime.md query-language/dicts/external-dicts-dict-lifetime.md -dicts/external_dicts_dict_sources.md query-language/dicts/external-dicts-dict-sources.md -dicts/external_dicts_dict_structure.md query-language/dicts/external-dicts-dict-structure.md -dicts/index.md query-language/dicts/index.md -dicts/internal_dicts.md query-language/dicts/internal-dicts.md -engines/database_engines/index.md engines/database-engines/index.md -engines/database_engines/lazy.md engines/database-engines/lazy.md -engines/database_engines/mysql.md engines/database-engines/mysql.md -engines/table-engines/log-family/log-family.md engines/table-engines/log-family/index.md -engines/table_engines/index.md engines/table-engines/index.md -engines/table_engines/integrations/hdfs.md engines/table-engines/integrations/hdfs.md -engines/table_engines/integrations/index.md engines/table-engines/integrations/index.md -engines/table_engines/integrations/jdbc.md engines/table-engines/integrations/jdbc.md -engines/table_engines/integrations/kafka.md engines/table-engines/integrations/kafka.md -engines/table_engines/integrations/mysql.md engines/table-engines/integrations/mysql.md -engines/table_engines/integrations/odbc.md engines/table-engines/integrations/odbc.md -engines/table_engines/log_family/index.md engines/table-engines/log-family/index.md -engines/table_engines/log_family/log.md engines/table-engines/log-family/log.md -engines/table_engines/log_family/log_family.md engines/table-engines/log-family/log-family.md -engines/table_engines/log_family/stripelog.md engines/table-engines/log-family/stripelog.md -engines/table_engines/log_family/tinylog.md engines/table-engines/log-family/tinylog.md -engines/table_engines/mergetree_family/aggregatingmergetree.md engines/table-engines/mergetree-family/aggregatingmergetree.md -engines/table_engines/mergetree_family/collapsingmergetree.md engines/table-engines/mergetree-family/collapsingmergetree.md -engines/table_engines/mergetree_family/custom_partitioning_key.md engines/table-engines/mergetree-family/custom-partitioning-key.md -engines/table_engines/mergetree_family/graphitemergetree.md engines/table-engines/mergetree-family/graphitemergetree.md -engines/table_engines/mergetree_family/index.md engines/table-engines/mergetree-family/index.md -engines/table_engines/mergetree_family/mergetree.md engines/table-engines/mergetree-family/mergetree.md -engines/table_engines/mergetree_family/replacingmergetree.md engines/table-engines/mergetree-family/replacingmergetree.md -engines/table_engines/mergetree_family/replication.md engines/table-engines/mergetree-family/replication.md -engines/table_engines/mergetree_family/summingmergetree.md engines/table-engines/mergetree-family/summingmergetree.md -engines/table_engines/mergetree_family/versionedcollapsingmergetree.md engines/table-engines/mergetree-family/versionedcollapsingmergetree.md -engines/table_engines/special/buffer.md engines/table-engines/special/buffer.md -engines/table_engines/special/dictionary.md engines/table-engines/special/dictionary.md -engines/table_engines/special/distributed.md engines/table-engines/special/distributed.md -engines/table_engines/special/external_data.md engines/table-engines/special/external-data.md -engines/table_engines/special/file.md engines/table-engines/special/file.md -engines/table_engines/special/generate.md engines/table-engines/special/generate.md -engines/table_engines/special/index.md engines/table-engines/special/index.md -engines/table_engines/special/join.md engines/table-engines/special/join.md -engines/table_engines/special/materializedview.md engines/table-engines/special/materializedview.md -engines/table_engines/special/memory.md engines/table-engines/special/memory.md -engines/table_engines/special/merge.md engines/table-engines/special/merge.md -engines/table_engines/special/null.md engines/table-engines/special/null.md -engines/table_engines/special/set.md engines/table-engines/special/set.md -engines/table_engines/special/url.md engines/table-engines/special/url.md -engines/table_engines/special/view.md engines/table-engines/special/view.md -extended_roadmap.md whats-new/extended-roadmap.md -formats.md interfaces/formats.md -formats/capnproto.md interfaces/formats.md -formats/csv.md interfaces/formats.md -formats/csvwithnames.md interfaces/formats.md -formats/json.md interfaces/formats.md -formats/jsoncompact.md interfaces/formats.md -formats/jsoneachrow.md interfaces/formats.md -formats/native.md interfaces/formats.md -formats/null.md interfaces/formats.md -formats/pretty.md interfaces/formats.md -formats/prettycompact.md interfaces/formats.md -formats/prettycompactmonoblock.md interfaces/formats.md -formats/prettynoescapes.md interfaces/formats.md -formats/prettyspace.md interfaces/formats.md -formats/rowbinary.md interfaces/formats.md -formats/tabseparated.md interfaces/formats.md -formats/tabseparatedraw.md interfaces/formats.md -formats/tabseparatedwithnames.md interfaces/formats.md -formats/tabseparatedwithnamesandtypes.md interfaces/formats.md -formats/tskv.md interfaces/formats.md -formats/values.md interfaces/formats.md -formats/vertical.md interfaces/formats.md -formats/verticalraw.md interfaces/formats.md -formats/xml.md interfaces/formats.md -functions/arithmetic_functions.md query-language/functions/arithmetic-functions.md -functions/array_functions.md query-language/functions/array-functions.md -functions/array_join.md query-language/functions/array-join.md -functions/bit_functions.md query-language/functions/bit-functions.md -functions/bitmap_functions.md query-language/functions/bitmap-functions.md -functions/comparison_functions.md query-language/functions/comparison-functions.md -functions/conditional_functions.md query-language/functions/conditional-functions.md -functions/date_time_functions.md query-language/functions/date-time-functions.md -functions/encoding_functions.md query-language/functions/encoding-functions.md -functions/ext_dict_functions.md query-language/functions/ext-dict-functions.md -functions/hash_functions.md query-language/functions/hash-functions.md -functions/higher_order_functions.md query-language/functions/higher-order-functions.md -functions/in_functions.md query-language/functions/in-functions.md -functions/index.md query-language/functions/index.md -functions/ip_address_functions.md query-language/functions/ip-address-functions.md -functions/json_functions.md query-language/functions/json-functions.md -functions/logical_functions.md query-language/functions/logical-functions.md -functions/math_functions.md query-language/functions/math-functions.md -functions/other_functions.md query-language/functions/other-functions.md -functions/random_functions.md query-language/functions/random-functions.md -functions/rounding_functions.md query-language/functions/rounding-functions.md -functions/splitting_merging_functions.md query-language/functions/splitting-merging-functions.md -functions/string_functions.md query-language/functions/string-functions.md -functions/string_replace_functions.md query-language/functions/string-replace-functions.md -functions/string_search_functions.md query-language/functions/string-search-functions.md -functions/type_conversion_functions.md query-language/functions/type-conversion-functions.md -functions/url_functions.md query-language/functions/url-functions.md -functions/ym_dict_functions.md query-language/functions/ym-dict-functions.md -getting_started/example_datasets/amplab_benchmark.md getting-started/example-datasets/amplab-benchmark.md -getting_started/example_datasets/criteo.md getting-started/example-datasets/criteo.md -getting_started/example_datasets/index.md getting-started/example-datasets/index.md -getting_started/example_datasets/metrica.md getting-started/example-datasets/metrica.md -getting_started/example_datasets/nyc_taxi.md getting-started/example-datasets/nyc-taxi.md -getting_started/example_datasets/ontime.md getting-started/example-datasets/ontime.md -getting_started/example_datasets/star_schema.md getting-started/example-datasets/star-schema.md -getting_started/example_datasets/wikistat.md getting-started/example-datasets/wikistat.md -getting_started/index.md getting-started/index.md -getting_started/install.md getting-started/install.md -getting_started/playground.md getting-started/playground.md -getting_started/tutorial.md getting-started/tutorial.md -images/column_oriented.gif images/column-oriented.gif -images/row_oriented.gif images/row-oriented.gif -interfaces/http_interface.md interfaces/http.md -interfaces/third-party/client_libraries.md interfaces/third-party/client-libraries.md -interfaces/third-party_client_libraries.md interfaces/third-party/client-libraries.md -interfaces/third-party_gui.md interfaces/third-party/gui.md -interfaces/third_party/index.md interfaces/third-party/index.md -introduction/index.md -introduction/distinctive_features.md introduction/distinctive-features.md -introduction/features_considered_disadvantages.md introduction/distinctive-features.md -introduction/possible_silly_questions.md faq/general.md -introduction/ya_metrika_task.md introduction/history.md -operations/access_rights.md operations/access-rights.md -operations/configuration_files.md operations/configuration-files.md -operations/optimizing_performance/index.md operations/optimizing-performance/index.md -operations/optimizing_performance/sampling_query_profiler.md operations/optimizing-performance/sampling-query-profiler.md -operations/performance/sampling_query_profiler.md operations/optimizing-performance/sampling-query-profiler.md -operations/performance_test.md operations/performance-test.md -operations/server_configuration_parameters/index.md operations/server-configuration-parameters/index.md -operations/server_configuration_parameters/settings.md operations/server-configuration-parameters/settings.md -operations/server_settings/index.md operations/server-configuration-parameters/index.md -operations/server_settings/settings.md operations/server-configuration-parameters/settings.md -operations/settings/constraints_on_settings.md operations/settings/constraints-on-settings.md -operations/settings/permissions_for_queries.md operations/settings/permissions-for-queries.md -operations/settings/query_complexity.md operations/settings/query-complexity.md -operations/settings/settings_profiles.md operations/settings/settings-profiles.md -operations/settings/settings_users.md operations/settings/settings-users.md -operations/system_tables.md operations/system-tables.md -operations/table_engines/aggregatingmergetree.md engines/table-engines/mergetree-family/aggregatingmergetree.md -operations/table_engines/buffer.md engines/table-engines/special/buffer.md -operations/table_engines/collapsingmergetree.md engines/table-engines/mergetree-family/collapsingmergetree.md -operations/table_engines/custom_partitioning_key.md engines/table-engines/mergetree-family/custom-partitioning-key.md -operations/table_engines/dictionary.md engines/table-engines/special/dictionary.md -operations/table_engines/distributed.md engines/table-engines/special/distributed.md -operations/table_engines/external_data.md engines/table-engines/special/external-data.md -operations/table_engines/file.md engines/table-engines/special/file.md -operations/table_engines/generate.md engines/table-engines/special/generate.md -operations/table_engines/graphitemergetree.md engines/table-engines/mergetree-family/graphitemergetree.md -operations/table_engines/hdfs.md engines/table-engines/integrations/hdfs.md -operations/table_engines/index.md engines/table-engines/index.md -operations/table_engines/jdbc.md engines/table-engines/integrations/jdbc.md -operations/table_engines/join.md engines/table-engines/special/join.md -operations/table_engines/kafka.md engines/table-engines/integrations/kafka.md -operations/table_engines/log.md engines/table-engines/log-family/log.md -operations/table_engines/log_family.md engines/table-engines/log-family/log-family.md -operations/table_engines/materializedview.md engines/table-engines/special/materializedview.md -operations/table_engines/memory.md engines/table-engines/special/memory.md -operations/table_engines/merge.md engines/table-engines/special/merge.md -operations/table_engines/mergetree.md engines/table-engines/mergetree-family/mergetree.md -operations/table_engines/mysql.md engines/table-engines/integrations/mysql.md -operations/table_engines/null.md engines/table-engines/special/null.md -operations/table_engines/odbc.md engines/table-engines/integrations/odbc.md -operations/table_engines/replacingmergetree.md engines/table-engines/mergetree-family/replacingmergetree.md -operations/table_engines/replication.md engines/table-engines/mergetree-family/replication.md -operations/table_engines/set.md engines/table-engines/special/set.md -operations/table_engines/stripelog.md engines/table-engines/log-family/stripelog.md -operations/table_engines/summingmergetree.md engines/table-engines/mergetree-family/summingmergetree.md -operations/table_engines/tinylog.md engines/table-engines/log-family/tinylog.md -operations/table_engines/url.md engines/table-engines/special/url.md -operations/table_engines/versionedcollapsingmergetree.md engines/table-engines/mergetree-family/versionedcollapsingmergetree.md -operations/table_engines/view.md engines/table-engines/special/view.md -operations/utils/clickhouse-benchmark.md operations/utilities/clickhouse-benchmark.md -operations/utils/clickhouse-copier.md operations/utilities/clickhouse-copier.md -operations/utils/clickhouse-local.md operations/utilities/clickhouse-local.md -operations/utils/index.md operations/utilities/index.md -query_language/agg_functions/combinators.md sql-reference/aggregate-functions/combinators.md -query_language/agg_functions/index.md sql-reference/aggregate-functions/index.md -query_language/agg_functions/parametric_functions.md sql-reference/aggregate-functions/parametric-functions.md -query_language/agg_functions/reference.md sql-reference/aggregate-functions/reference.md -query_language/alter.md sql-reference/statements/alter.md -query_language/create.md sql-reference/statements/create.md -query_language/dicts/external_dicts.md sql-reference/dictionaries/external-dictionaries/external-dicts.md -query_language/dicts/external_dicts_dict.md sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md -query_language/dicts/external_dicts_dict_hierarchical.md sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md -query_language/dicts/external_dicts_dict_layout.md sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md -query_language/dicts/external_dicts_dict_lifetime.md sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md -query_language/dicts/external_dicts_dict_sources.md sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md -query_language/dicts/external_dicts_dict_structure.md sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md -query_language/dicts/index.md sql-reference/dictionaries/index.md -query_language/dicts/internal_dicts.md sql-reference/dictionaries/internal-dicts.md -query_language/functions/arithmetic_functions.md sql-reference/functions/arithmetic-functions.md -query_language/functions/array_functions.md sql-reference/functions/array-functions.md -query_language/functions/array_join.md sql-reference/functions/array-join.md -query_language/functions/bit_functions.md sql-reference/functions/bit-functions.md -query_language/functions/bitmap_functions.md sql-reference/functions/bitmap-functions.md -query_language/functions/comparison_functions.md sql-reference/functions/comparison-functions.md -query_language/functions/conditional_functions.md sql-reference/functions/conditional-functions.md -query_language/functions/date_time_functions.md sql-reference/functions/date-time-functions.md -query_language/functions/encoding_functions.md sql-reference/functions/encoding-functions.md -query_language/functions/ext_dict_functions.md sql-reference/functions/ext-dict-functions.md -query_language/functions/functions_for_nulls.md sql-reference/functions/functions-for-nulls.md -query_language/functions/geo.md sql-reference/functions/geo.md -query_language/functions/hash_functions.md sql-reference/functions/hash-functions.md -query_language/functions/higher_order_functions.md sql-reference/functions/higher-order-functions.md -query_language/functions/in_functions.md sql-reference/functions/in-functions.md -query_language/functions/index.md sql-reference/functions/index.md -query_language/functions/introspection.md sql-reference/functions/introspection.md -query_language/functions/ip_address_functions.md sql-reference/functions/ip-address-functions.md -query_language/functions/json_functions.md sql-reference/functions/json-functions.md -query_language/functions/logical_functions.md sql-reference/functions/logical-functions.md -query_language/functions/machine_learning_functions.md sql-reference/functions/machine-learning-functions.md -query_language/functions/math_functions.md sql-reference/functions/math-functions.md -query_language/functions/other_functions.md sql-reference/functions/other-functions.md -query_language/functions/random_functions.md sql-reference/functions/random-functions.md -query_language/functions/rounding_functions.md sql-reference/functions/rounding-functions.md -query_language/functions/splitting_merging_functions.md sql-reference/functions/splitting-merging-functions.md -query_language/functions/string_functions.md sql-reference/functions/string-functions.md -query_language/functions/string_replace_functions.md sql-reference/functions/string-replace-functions.md -query_language/functions/string_search_functions.md sql-reference/functions/string-search-functions.md -query_language/functions/type_conversion_functions.md sql-reference/functions/type-conversion-functions.md -query_language/functions/url_functions.md sql-reference/functions/url-functions.md -query_language/functions/uuid_functions.md sql-reference/functions/uuid-functions.md -query_language/functions/ym_dict_functions.md sql-reference/functions/ym-dict-functions.md -query_language/index.md sql-reference/index.md -query_language/insert_into.md sql-reference/statements/insert-into.md -query_language/misc.md sql-reference/statements/misc.md -query_language/operators.md sql-reference/operators.md -query_language/queries.md query-language.md -query_language/select.md sql-reference/statements/select.md -query_language/show.md sql-reference/statements/show.md -query_language/syntax.md sql-reference/syntax.md -query_language/system.md sql-reference/statements/system.md -query_language/table_functions/file.md sql-reference/table-functions/file.md -query_language/table_functions/generate.md sql-reference/table-functions/generate.md -query_language/table_functions/hdfs.md sql-reference/table-functions/hdfs.md -query_language/table_functions/index.md sql-reference/table-functions/index.md -query_language/table_functions/input.md sql-reference/table-functions/input.md -query_language/table_functions/jdbc.md sql-reference/table-functions/jdbc.md -query_language/table_functions/merge.md sql-reference/table-functions/merge.md -query_language/table_functions/mysql.md sql-reference/table-functions/mysql.md -query_language/table_functions/numbers.md sql-reference/table-functions/numbers.md -query_language/table_functions/odbc.md sql-reference/table-functions/odbc.md -query_language/table_functions/remote.md sql-reference/table-functions/remote.md -query_language/table_functions/url.md sql-reference/table-functions/url.md -roadmap.md whats-new/roadmap.md -security_changelog.md whats-new/security-changelog.md -sql-reference/data-types/domains/overview.md sql-reference/data-types/domains/index.md -sql_reference/aggregate_functions/combinators.md sql-reference/aggregate-functions/combinators.md -sql_reference/aggregate_functions/index.md sql-reference/aggregate-functions/index.md -sql_reference/aggregate_functions/parametric_functions.md sql-reference/aggregate-functions/parametric-functions.md -sql_reference/aggregate_functions/reference.md sql-reference/aggregate-functions/reference.md -sql_reference/ansi.md sql-reference/ansi.md -sql_reference/data_types/aggregatefunction.md sql-reference/data-types/aggregatefunction.md -sql_reference/data_types/array.md sql-reference/data-types/array.md -sql_reference/data_types/boolean.md sql-reference/data-types/boolean.md -sql_reference/data_types/date.md sql-reference/data-types/date.md -sql_reference/data_types/datetime.md sql-reference/data-types/datetime.md -sql_reference/data_types/datetime64.md sql-reference/data-types/datetime64.md -sql_reference/data_types/decimal.md sql-reference/data-types/decimal.md -sql_reference/data_types/domains/index.md sql-reference/data-types/domains/index.md -sql_reference/data_types/domains/ipv4.md sql-reference/data-types/domains/ipv4.md -sql_reference/data_types/domains/ipv6.md sql-reference/data-types/domains/ipv6.md -sql_reference/data_types/domains/overview.md sql-reference/data-types/domains/overview.md -sql_reference/data_types/enum.md sql-reference/data-types/enum.md -sql_reference/data_types/fixedstring.md sql-reference/data-types/fixedstring.md -sql_reference/data_types/float.md sql-reference/data-types/float.md -sql_reference/data_types/index.md sql-reference/data-types/index.md -sql_reference/data_types/int_uint.md sql-reference/data-types/int-uint.md -sql_reference/data_types/nested_data_structures/index.md sql-reference/data-types/nested-data-structures/index.md -sql_reference/data_types/nested_data_structures/nested.md sql-reference/data-types/nested-data-structures/nested.md -sql_reference/data_types/nullable.md sql-reference/data-types/nullable.md -sql_reference/data_types/simpleaggregatefunction.md sql-reference/data-types/simpleaggregatefunction.md -sql_reference/data_types/special_data_types/expression.md sql-reference/data-types/special-data-types/expression.md -sql_reference/data_types/special_data_types/index.md sql-reference/data-types/special-data-types/index.md -sql_reference/data_types/special_data_types/interval.md sql-reference/data-types/special-data-types/interval.md -sql_reference/data_types/special_data_types/nothing.md sql-reference/data-types/special-data-types/nothing.md -sql_reference/data_types/special_data_types/set.md sql-reference/data-types/special-data-types/set.md -sql_reference/data_types/string.md sql-reference/data-types/string.md -sql_reference/data_types/tuple.md sql-reference/data-types/tuple.md -sql_reference/data_types/uuid.md sql-reference/data-types/uuid.md -sql_reference/dictionaries/external_dictionaries/external_dicts.md sql-reference/dictionaries/external-dictionaries/external-dicts.md -sql_reference/dictionaries/external_dictionaries/external_dicts_dict.md sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md -sql_reference/dictionaries/external_dictionaries/external_dicts_dict_hierarchical.md sql-reference/dictionaries/external-dictionaries/external-dicts-dict-hierarchical.md -sql_reference/dictionaries/external_dictionaries/external_dicts_dict_layout.md sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md -sql_reference/dictionaries/external_dictionaries/external_dicts_dict_lifetime.md sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md -sql_reference/dictionaries/external_dictionaries/external_dicts_dict_sources.md sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md -sql_reference/dictionaries/external_dictionaries/external_dicts_dict_structure.md sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md -sql_reference/dictionaries/external_dictionaries/index.md sql-reference/dictionaries/external-dictionaries/index.md -sql_reference/dictionaries/index.md sql-reference/dictionaries/index.md -sql_reference/dictionaries/internal_dicts.md sql-reference/dictionaries/internal-dicts.md -sql_reference/functions/arithmetic_functions.md sql-reference/functions/arithmetic-functions.md -sql_reference/functions/array_functions.md sql-reference/functions/array-functions.md -sql_reference/functions/array_join.md sql-reference/functions/array-join.md -sql_reference/functions/bit_functions.md sql-reference/functions/bit-functions.md -sql_reference/functions/bitmap_functions.md sql-reference/functions/bitmap-functions.md -sql_reference/functions/comparison_functions.md sql-reference/functions/comparison-functions.md -sql_reference/functions/conditional_functions.md sql-reference/functions/conditional-functions.md -sql_reference/functions/date_time_functions.md sql-reference/functions/date-time-functions.md -sql_reference/functions/encoding_functions.md sql-reference/functions/encoding-functions.md -sql_reference/functions/ext_dict_functions.md sql-reference/functions/ext-dict-functions.md -sql_reference/functions/functions_for_nulls.md sql-reference/functions/functions-for-nulls.md -sql_reference/functions/geo.md sql-reference/functions/geo.md -sql_reference/functions/hash_functions.md sql-reference/functions/hash-functions.md -sql_reference/functions/higher_order_functions.md sql-reference/functions/higher-order-functions.md -sql_reference/functions/in_functions.md sql-reference/functions/in-functions.md -sql_reference/functions/index.md sql-reference/functions/index.md -sql_reference/functions/introspection.md sql-reference/functions/introspection.md -sql_reference/functions/ip_address_functions.md sql-reference/functions/ip-address-functions.md -sql_reference/functions/json_functions.md sql-reference/functions/json-functions.md -sql_reference/functions/logical_functions.md sql-reference/functions/logical-functions.md -sql_reference/functions/machine_learning_functions.md sql-reference/functions/machine-learning-functions.md -sql_reference/functions/math_functions.md sql-reference/functions/math-functions.md -sql_reference/functions/other_functions.md sql-reference/functions/other-functions.md -sql_reference/functions/random_functions.md sql-reference/functions/random-functions.md -sql_reference/functions/rounding_functions.md sql-reference/functions/rounding-functions.md -sql_reference/functions/splitting_merging_functions.md sql-reference/functions/splitting-merging-functions.md -sql_reference/functions/string_functions.md sql-reference/functions/string-functions.md -sql_reference/functions/string_replace_functions.md sql-reference/functions/string-replace-functions.md -sql_reference/functions/string_search_functions.md sql-reference/functions/string-search-functions.md -sql_reference/functions/type_conversion_functions.md sql-reference/functions/type-conversion-functions.md -sql_reference/functions/url_functions.md sql-reference/functions/url-functions.md -sql_reference/functions/uuid_functions.md sql-reference/functions/uuid-functions.md -sql_reference/functions/ym_dict_functions.md sql-reference/functions/ym-dict-functions.md -sql_reference/index.md sql-reference/index.md -sql_reference/operators.md sql-reference/operators.md -sql_reference/statements/alter.md sql-reference/statements/alter.md -sql_reference/statements/create.md sql-reference/statements/create.md -sql_reference/statements/index.md sql-reference/statements/index.md -sql_reference/statements/insert_into.md sql-reference/statements/insert-into.md -sql_reference/statements/misc.md sql-reference/statements/misc.md -sql_reference/statements/select.md sql-reference/statements/select.md -sql_reference/statements/show.md sql-reference/statements/show.md -sql_reference/statements/system.md sql-reference/statements/system.md -sql_reference/syntax.md sql-reference/syntax.md -sql_reference/table_functions/file.md sql-reference/table-functions/file.md -sql_reference/table_functions/generate.md sql-reference/table-functions/generate.md -sql_reference/table_functions/hdfs.md sql-reference/table-functions/hdfs.md -sql_reference/table_functions/index.md sql-reference/table-functions/index.md -sql_reference/table_functions/input.md sql-reference/table-functions/input.md -sql_reference/table_functions/jdbc.md sql-reference/table-functions/jdbc.md -sql_reference/table_functions/merge.md sql-reference/table-functions/merge.md -sql_reference/table_functions/mysql.md sql-reference/table-functions/mysql.md -sql_reference/table_functions/numbers.md sql-reference/table-functions/numbers.md -sql_reference/table_functions/odbc.md sql-reference/table-functions/odbc.md -sql_reference/table_functions/remote.md sql-reference/table-functions/remote.md -sql_reference/table_functions/url.md sql-reference/table-functions/url.md -system_tables.md operations/system-tables.md -system_tables/system.asynchronous_metrics.md operations/system-tables.md -system_tables/system.clusters.md operations/system-tables.md -system_tables/system.columns.md operations/system-tables.md -system_tables/system.databases.md operations/system-tables.md -system_tables/system.dictionaries.md operations/system-tables.md -system_tables/system.events.md operations/system-tables.md -system_tables/system.functions.md operations/system-tables.md -system_tables/system.merges.md operations/system-tables.md -system_tables/system.metrics.md operations/system-tables.md -system_tables/system.numbers.md operations/system-tables.md -system_tables/system.numbers_mt.md operations/system-tables.md -system_tables/system.one.md operations/system-tables.md -system_tables/system.parts.md operations/system-tables.md -system_tables/system.processes.md operations/system-tables.md -system_tables/system.replicas.md operations/system-tables.md -system_tables/system.settings.md operations/system-tables.md -system_tables/system.tables.md operations/system-tables.md -system_tables/system.zookeeper.md operations/system-tables.md -table_engines.md operations/table-engines.md -table_engines/aggregatingmergetree.md operations/table-engines/aggregatingmergetree.md -table_engines/buffer.md operations/table-engines/buffer.md -table_engines/collapsingmergetree.md operations/table-engines/collapsingmergetree.md -table_engines/custom_partitioning_key.md operations/table-engines/custom-partitioning-key.md -table_engines/dictionary.md operations/table-engines/dictionary.md -table_engines/distributed.md operations/table-engines/distributed.md -table_engines/external_data.md operations/table-engines/external-data.md -table_engines/file.md operations/table-engines/file.md -table_engines/graphitemergetree.md operations/table-engines/graphitemergetree.md -table_engines/index.md operations/table-engines/index.md -table_engines/join.md operations/table-engines/join.md -table_engines/kafka.md operations/table-engines/kafka.md -table_engines/log.md operations/table-engines/log.md -table_engines/materializedview.md operations/table-engines/materializedview.md -table_engines/memory.md operations/table-engines/memory.md -table_engines/merge.md operations/table-engines/merge.md -table_engines/mergetree.md operations/table-engines/mergetree.md -table_engines/mysql.md operations/table-engines/mysql.md -table_engines/null.md operations/table-engines/null.md -table_engines/replacingmergetree.md operations/table-engines/replacingmergetree.md -table_engines/replication.md operations/table-engines/replication.md -table_engines/set.md operations/table-engines/set.md -table_engines/summingmergetree.md operations/table-engines/summingmergetree.md -table_engines/tinylog.md operations/table-engines/tinylog.md -table_engines/view.md operations/table-engines/view.md -table_functions/file.md query-language/table-functions/file.md -table_functions/index.md query-language/table-functions/index.md -table_functions/merge.md query-language/table-functions/merge.md -table_functions/numbers.md query-language/table-functions/numbers.md -table_functions/remote.md query-language/table-functions/remote.md -utils.md operations/utils.md -utils/clickhouse-copier.md operations/utils/clickhouse-copier.md -utils/clickhouse-local.md operations/utils/clickhouse-local.md -whats_new/changelog/2017.md whats-new/changelog/2017.md -whats_new/changelog/2018.md whats-new/changelog/2018.md -whats_new/changelog/2019.md whats-new/changelog/2019.md -whats_new/changelog/index.md whats-new/changelog/index.md -whats_new/index.md whats-new/index.md -whats_new/roadmap.md whats-new/roadmap.md -whats_new/security_changelog.md whats-new/security-changelog.md +The redirects from this file were moved to the Docusaurus configuration file. +If you need to add a redirect please either open a PR in +https://github.com/clickhouse/clickhouse-docs adding the redirect to +https://github.com/ClickHouse/clickhouse-docs/blob/main/docusaurus.config.js +or open an issue in the same repo and provide the old URL and new URL to have +the redirect added. From d0bb9850614e9be96c489ecf3b8c3ad788aab5d3 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 22 Jun 2023 14:33:25 +0200 Subject: [PATCH 218/515] fix other classes based on SinkToStorage --- src/Storages/HDFS/StorageHDFS.cpp | 21 +++++++++++++++++---- src/Storages/StorageAzureBlob.cpp | 21 +++++++++++++++++---- src/Storages/StorageFile.cpp | 21 +++++++++++++++++---- src/Storages/StorageURL.cpp | 21 +++++++++++++++++---- src/Storages/StorageURL.h | 1 + 5 files changed, 69 insertions(+), 16 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 7a9d996e3c2..a41c65cdb2e 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -512,10 +512,18 @@ public: cancelled = true; } - void onException(std::exception_ptr /* exception */) override + void onException(std::exception_ptr exception) override { std::lock_guard lock(cancel_mutex); - finalize(); + try + { + std::rethrow_exception(exception); + } + catch (...) + { + /// An exception context is needed to proper delete write buffers without finalization + release(); + } } void onFinish() override @@ -540,12 +548,17 @@ private: catch (...) { /// Stop ParallelFormattingOutputFormat correctly. - writer.reset(); - write_buf->finalize(); + release(); throw; } } + void release() + { + writer.reset(); + write_buf->finalize(); + } + std::unique_ptr write_buf; OutputFormatPtr writer; std::mutex cancel_mutex; diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 65af2ed6d6e..133dbb6740f 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -489,10 +489,18 @@ public: cancelled = true; } - void onException(std::exception_ptr /* exception */) override + void onException(std::exception_ptr exception) override { std::lock_guard lock(cancel_mutex); - finalize(); + try + { + std::rethrow_exception(exception); + } + catch (...) + { + /// An exception context is needed to proper delete write buffers without finalization + release(); + } } void onFinish() override @@ -516,12 +524,17 @@ private: catch (...) { /// Stop ParallelFormattingOutputFormat correctly. - writer.reset(); - write_buf->finalize(); + release(); throw; } } + void release() + { + writer.reset(); + write_buf->finalize(); + } + Block sample_block; std::optional format_settings; std::unique_ptr write_buf; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 50aa47ffca9..ff67272e542 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1031,10 +1031,18 @@ public: cancelled = true; } - void onException(std::exception_ptr /* exception */) override + void onException(std::exception_ptr exception) override { std::lock_guard cancel_lock(cancel_mutex); - finalize(); + try + { + std::rethrow_exception(exception); + } + catch (...) + { + /// An exception context is needed to proper delete write buffers without finalization + release(); + } } void onFinish() override @@ -1058,12 +1066,17 @@ private: catch (...) { /// Stop ParallelFormattingOutputFormat correctly. - writer.reset(); - write_buf->finalize(); + release(); throw; } } + void release() + { + writer.reset(); + write_buf->finalize(); + } + StorageMetadataPtr metadata_snapshot; String table_name_for_log; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 651b50518d2..20071afb793 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -480,10 +480,18 @@ void StorageURLSink::onCancel() cancelled = true; } -void StorageURLSink::onException(std::exception_ptr /* exception */) +void StorageURLSink::onException(std::exception_ptr exception) { std::lock_guard lock(cancel_mutex); - finalize(); + try + { + std::rethrow_exception(exception); + } + catch (...) + { + /// An exception context is needed to proper delete write buffers without finalization + release(); + } } void StorageURLSink::onFinish() @@ -506,12 +514,17 @@ void StorageURLSink::finalize() catch (...) { /// Stop ParallelFormattingOutputFormat correctly. - writer.reset(); - write_buf->finalize(); + release(); throw; } } +void StorageURLSink::release() +{ + writer.reset(); + write_buf->finalize(); +} + class PartitionedStorageURLSink : public PartitionedSink { public: diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index c4a5ce1aa7b..345f813dd7c 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -241,6 +241,7 @@ public: private: void finalize(); + void release(); std::unique_ptr write_buf; OutputFormatPtr writer; std::mutex cancel_mutex; From 3c7d749d2ce80373fe7839a08fc3174e999b6a69 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 22 Jun 2023 14:47:04 +0200 Subject: [PATCH 219/515] Added check of StorageView before checking parameterized view to fix seg fault when materialized view is used with brackets --- src/Interpreters/Context.cpp | 2 +- tests/queries/0_stateless/02428_parameterized_view.reference | 1 + tests/queries/0_stateless/02428_parameterized_view.sh | 3 +++ 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 995e78d8f0b..106264320b2 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1476,7 +1476,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const StoragePtr table = DatabaseCatalog::instance().tryGetTable({database_name, table_name}, getQueryContext()); if (table) { - if (table.get()->isView() && table->as()->isParameterizedView()) + if (table.get()->isView() && table->as() && table->as()->isParameterizedView()) { function->prefer_subquery_to_function_formatting = true; return table; diff --git a/tests/queries/0_stateless/02428_parameterized_view.reference b/tests/queries/0_stateless/02428_parameterized_view.reference index b73c52c478f..422fdaa4983 100644 --- a/tests/queries/0_stateless/02428_parameterized_view.reference +++ b/tests/queries/0_stateless/02428_parameterized_view.reference @@ -40,3 +40,4 @@ ERROR 10 10 1 +ERROR diff --git a/tests/queries/0_stateless/02428_parameterized_view.sh b/tests/queries/0_stateless/02428_parameterized_view.sh index 3abfbfc22fc..ad9c672f4c5 100755 --- a/tests/queries/0_stateless/02428_parameterized_view.sh +++ b/tests/queries/0_stateless/02428_parameterized_view.sh @@ -112,7 +112,10 @@ $CLICKHOUSE_CLIENT -q "CREATE VIEW 02428_trace_view AS WITH {trace_id:String} A FROM 02428_otel_traces" $CLICKHOUSE_CLIENT -q "SELECT * FROM 02428_trace_view(trace_id='1')" +$CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW test_02428_mv1 ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/materialized_view', 'r1') ORDER BY Name AS SELECT * FROM test_02428_Catalog;" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02428_mv1(test)" 2>&1 | grep -Fq "UNKNOWN_FUNCTION" && echo 'ERROR' || echo 'OK' +$CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_mv1" $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv1" $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv2" $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv3" From 22e49748b5ff129422fca4ff2da92b26e38d3906 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Jun 2023 14:23:04 +0000 Subject: [PATCH 220/515] Cleanup. --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 4 +- src/Analyzer/SetUtils.cpp | 13 +- src/Analyzer/SetUtils.h | 10 +- src/Functions/in.cpp | 9 +- src/Interpreters/ActionsVisitor.cpp | 37 +- .../ClusterProxy/SelectStreamFactory.cpp | 8 +- .../ClusterProxy/SelectStreamFactory.h | 4 +- .../ClusterProxy/executeQuery.cpp | 1 - src/Interpreters/ExpressionAnalyzer.cpp | 119 ----- src/Interpreters/ExpressionAnalyzer.h | 8 - src/Interpreters/GlobalSubqueriesVisitor.h | 48 +- src/Interpreters/InterpreterSelectQuery.cpp | 3 - src/Interpreters/PreparedSets.cpp | 493 ++++++------------ src/Interpreters/PreparedSets.h | 65 ++- src/Interpreters/Set.cpp | 13 +- src/Interpreters/Set.h | 22 +- src/Interpreters/SetKeys.h | 18 + src/Planner/CollectSets.cpp | 18 +- src/Planner/CollectSets.h | 2 +- src/Planner/Planner.cpp | 9 +- src/Planner/PlannerActionsVisitor.cpp | 6 +- src/Planner/PlannerContext.cpp | 46 -- src/Planner/PlannerContext.h | 72 +-- src/Planner/PlannerJoinTree.cpp | 21 +- src/Planner/Utils.cpp | 3 +- src/Planner/Utils.h | 1 - .../CreateSetAndFilterOnTheFlyStep.cpp | 2 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 18 +- src/Processors/QueryPlan/CreatingSetsStep.h | 10 +- .../QueryPlan/DistributedCreateLocalPlan.cpp | 3 +- .../QueryPlan/DistributedCreateLocalPlan.h | 1 - .../QueryPlan/ReadFromMergeTree.cpp | 27 - src/Processors/QueryPlan/ReadFromRemote.cpp | 6 +- src/Processors/QueryPlan/ReadFromRemote.h | 2 - src/Storages/KVStorageUtils.cpp | 1 - src/Storages/MergeTree/KeyCondition.cpp | 33 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +- .../MergeTreeIndexConditionBloomFilter.cpp | 44 +- .../MergeTree/MergeTreeIndexFullText.cpp | 9 +- .../MergeTree/MergeTreeIndexInverted.cpp | 14 +- src/Storages/MergeTree/PartitionPruner.cpp | 9 - src/Storages/StorageDistributed.cpp | 5 +- src/Storages/StorageMergeTree.cpp | 3 +- src/Storages/StorageReplicatedMergeTree.cpp | 3 +- src/Storages/StorageSet.cpp | 4 +- .../System/StorageSystemZooKeeper.cpp | 10 +- src/Storages/VirtualColumnUtils.cpp | 20 - src/Storages/buildQueryTreeForShard.cpp | 17 +- 48 files changed, 343 insertions(+), 955 deletions(-) create mode 100644 src/Interpreters/SetKeys.h diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 98f40dc4c15..163092f1b7f 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5211,14 +5211,14 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi const auto & settings = scope.context->getSettingsRef(); - auto result_block = makeSetForConstantValue(first_argument_constant_type, + auto result_block = getSetElementsForConstantValue(first_argument_constant_type, second_argument_constant_literal, second_argument_constant_type, settings.transform_null_in); SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; - auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, 0, settings.transform_null_in); + auto set = std::make_shared(size_limits_for_set, 0, settings.transform_null_in); set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName()); set->insertFromBlock(result_block.getColumnsWithTypeAndName()); diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index 2fb05449714..15eec16e899 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -118,7 +118,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes & } -Block makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in) +Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in) { DataTypes set_element_types = {expression_type}; const auto * lhs_tuple_type = typeid_cast(expression_type.get()); @@ -135,9 +135,6 @@ Block makeSetForConstantValue(const DataTypePtr & expression_type, const Field & size_t lhs_type_depth = getCompoundTypeDepth(*expression_type); size_t rhs_type_depth = getCompoundTypeDepth(*value_type); - // SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; - // bool transform_null_in = settings.transform_null_in; - Block result_block; if (lhs_type_depth == rhs_type_depth) @@ -171,14 +168,6 @@ Block makeSetForConstantValue(const DataTypePtr & expression_type, const Field & } return result_block; - - // auto set = std::make_shared(size_limits_for_set, true /*fill_set_elements*/, tranform_null_in); - - // set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName()); - // set->insertFromBlock(result_block.getColumnsWithTypeAndName()); - // set->finishInsert(); - - // return set; } } diff --git a/src/Analyzer/SetUtils.h b/src/Analyzer/SetUtils.h index fdeaa4a3c48..c35b45dce59 100644 --- a/src/Analyzer/SetUtils.h +++ b/src/Analyzer/SetUtils.h @@ -12,18 +12,12 @@ namespace DB class Set; using SetPtr = std::shared_ptr; -/** Make set for constant part of IN subquery. +/** Get set elements for constant part of IN subquery. * Throws exception if parameters are not valid for IN function. * * Example: SELECT id FROM test_table WHERE id IN (1, 2, 3, 4); * Example: SELECT id FROM test_table WHERE id IN ((1, 2), (3, 4)); - * - * @param expression_type - type of first argument of function IN. - * @param value - constant value of second argument of function IN. - * @param value_type - type of second argument of function IN. - * - * @return SetPtr for constant value. */ -Block makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in); +Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in); } diff --git a/src/Functions/in.cpp b/src/Functions/in.cpp index c19d1e72003..9045ba677f2 100644 --- a/src/Functions/in.cpp +++ b/src/Functions/in.cpp @@ -123,9 +123,12 @@ public: } auto future_set = column_set->getData(); - auto set = future_set ? future_set->get() : nullptr; - if (!future_set || !set) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Not-ready Set passed as the second argument for function '{}'", getName()); + if (!future_set) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No Set is passed as the second argument for function '{}'", getName()); + + auto set = future_set->get(); + if (!set) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Not-ready Set is passed as the second argument for function '{}'", getName()); auto set_types = set->getDataTypes(); diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 7c3af4ce691..bf78868463a 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -445,19 +445,16 @@ FutureSetPtr makeExplicitSet( if (left_tuple_type && left_tuple_type->getElements().size() != 1) set_element_types = left_tuple_type->getElements(); - for (auto & element_type : set_element_types) - if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - element_type = low_cardinality_type->getDictionaryType(); - - // if (!set_element_types.empty()) - // std::cerr << "========== " << set_element_types[0]->getName() << std::endl; - auto set_element_keys = Set::getElementTypes(set_element_types, context->getSettingsRef().transform_null_in); auto set_key = right_arg->getTreeHash(); if (auto set = prepared_sets.findTuple(set_key, set_element_keys)) return set; /// Already prepared. + for (auto & element_type : set_element_types) + if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + element_type = low_cardinality_type->getDictionaryType(); + Block block; const auto & right_arg_func = std::dynamic_pointer_cast(right_arg); if (right_arg_func && (right_arg_func->name == "tuple" || right_arg_func->name == "array")) @@ -1400,11 +1397,21 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool { if (no_subqueries) return {}; - //std::cerr << queryToString(right_in_operand) << std::endl; PreparedSets::Hash set_key; if (data.getContext()->getSettingsRef().allow_experimental_analyzer && !identifier) { + /// Here we can be only from mutation interpreter. Normal selects with analyzed use other interpreter. + /// This is a hacky way to allow reusing cache for prepared sets. + /// + /// Mutation is executed in two stages: + /// * first, query 'SELECT count() FROM table WHERE ...' is executed to get the set of affected parts (using analyzer) + /// * second, every part is mutated separately, where plan is build "manually", usign this code as well + /// To share the Set in between first and second stage, we should use the same hash. + /// New analyzer is uses a hash from query tree, so here we also build a query tree. + /// + /// Note : this code can be safely removed, but the test 02581_share_big_sets will be too slow (and fail by timeout). + /// Note : we should use new analyzer for mutations and remove this hack. InterpreterSelectQueryAnalyzer interpreter(right_in_operand, data.getContext(), SelectQueryOptions().analyze(true).subquery()); const auto & query_tree = interpreter.getQueryTree(); if (auto * query_node = query_tree->as()) @@ -1414,12 +1421,6 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool else set_key = right_in_operand->getTreeHash(); - // std::cerr << set_key.toString() << std::endl; - // std::cerr << data.prepared_sets->getSets().size() << std::endl; - // std::cerr << reinterpret_cast(data.prepared_sets.get()) << std::endl; - // for (const auto & [k, v] : data.prepared_sets->getSets()) - // std::cerr << "... " << k.toString(); - if (auto set = data.prepared_sets->findSubquery(set_key)) return set; @@ -1440,12 +1441,12 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool if (!data.getContext()->isGlobalContext()) { - // std::cerr << ".... checking for " << identifier->getColumnName() << std::endl; + /// If we are reading from storage, it can be an external table which is used for GLOBAL IN. + /// Here, we take FutureSet which is used to build external table. + /// It will be used if set is useful for primary key. During PK analysis + /// temporary table is not filled yet, so we need to fill it first. if (auto tmp_table = data.getContext()->findExternalTable(identifier->getColumnName())) - { external_table_set = tmp_table->future_set; - // std::cerr << "Found " << reinterpret_cast(tmp_table.get()) << " " << reinterpret_cast(external_table_set.get()) << std::endl; - } } } diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 0f8a725e144..0cf3f360994 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -92,13 +92,11 @@ SelectStreamFactory::SelectStreamFactory( const Block & header_, const ColumnsDescriptionByShardNum & objects_by_shard_, const StorageSnapshotPtr & storage_snapshot_, - QueryProcessingStage::Enum processed_stage_, - PreparedSetsPtr prepared_sets_) + QueryProcessingStage::Enum processed_stage_) : header(header_), objects_by_shard(objects_by_shard_), storage_snapshot(storage_snapshot_), - processed_stage(processed_stage_), - prepared_sets(std::move(prepared_sets_)) + processed_stage(processed_stage_) { } @@ -119,7 +117,7 @@ void SelectStreamFactory::createForShard( auto emplace_local_stream = [&]() { local_plans.emplace_back(createLocalPlan( - query_ast, header, context, processed_stage, prepared_sets, shard_info.shard_num, shard_count, /*replica_num=*/0, /*replica_count=*/0, /*coordinator=*/nullptr)); + query_ast, header, context, processed_stage, shard_info.shard_num, shard_count, /*replica_num=*/0, /*replica_count=*/0, /*coordinator=*/nullptr)); }; auto emplace_remote_stream = [&](bool lazy = false, time_t local_delay = 0) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index b19012ddba6..030c0b77dd5 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -69,8 +69,7 @@ public: const Block & header_, const ColumnsDescriptionByShardNum & objects_by_shard_, const StorageSnapshotPtr & storage_snapshot_, - QueryProcessingStage::Enum processed_stage_, - PreparedSetsPtr prepared_sets_); + QueryProcessingStage::Enum processed_stage_); void createForShard( const Cluster::ShardInfo & shard_info, @@ -95,7 +94,6 @@ public: const ColumnsDescriptionByShardNum objects_by_shard; const StorageSnapshotPtr storage_snapshot; QueryProcessingStage::Enum processed_stage; - PreparedSetsPtr prepared_sets; }; } diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index a9cf3d55392..e2f1dfe8ba7 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -226,7 +226,6 @@ void executeQuery( std::move(remote_shards), header, processed_stage, - stream_factory.prepared_sets, main_table, table_func_ptr, new_context, diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index b05563cf29f..f27d23e8e94 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -450,77 +450,6 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global, b } -// void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options) -// { -// if (!prepared_sets) -// return; - -// auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name); - -// if (prepared_sets->getFuture(set_key).isValid()) -// return; /// Already prepared. - -// if (auto set_ptr_from_storage_set = isPlainStorageSetInSubquery(subquery_or_table_name)) -// { -// prepared_sets->set(set_key, set_ptr_from_storage_set); -// return; -// } - -// auto build_set = [&] () -> SetPtr -// { -// LOG_TRACE(getLogger(), "Building set, key: {}", set_key.toString()); - -// auto interpreter_subquery = interpretSubquery(subquery_or_table_name, getContext(), {}, query_options); -// auto io = interpreter_subquery->execute(); -// PullingAsyncPipelineExecutor executor(io.pipeline); - -// SetPtr set = std::make_shared(settings.size_limits_for_set_used_with_index, true, getContext()->getSettingsRef().transform_null_in); -// set->setHeader(executor.getHeader().getColumnsWithTypeAndName()); - -// Block block; -// while (executor.pull(block)) -// { -// if (block.rows() == 0) -// continue; - -// /// If the limits have been exceeded, give up and let the default subquery processing actions take place. -// if (!set->insertFromBlock(block.getColumnsWithTypeAndName())) -// return nullptr; -// } - -// set->finishInsert(); - -// return set; -// }; - -// SetPtr set; - -// auto set_cache = getContext()->getPreparedSetsCache(); -// if (set_cache) -// { -// auto from_cache = set_cache->findOrPromiseToBuild(set_key.toString()); -// if (from_cache.index() == 0) -// { -// set = build_set(); -// std::get<0>(from_cache).set_value(set); -// } -// else -// { -// LOG_TRACE(getLogger(), "Waiting for set, key: {}", set_key.toString()); -// set = std::get<1>(from_cache).get(); -// } -// } -// else -// { -// set = build_set(); -// } - -// if (!set) -// return; - -// prepared_sets->set(set_key, std::move(set)); -// } - SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name) { const auto * table = subquery_or_table_name->as(); @@ -534,54 +463,6 @@ SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_o return storage_set->getSet(); } - -/// Performance optimization for IN() if storage supports it. -// void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) -// { -// if (!node || !storage() || !storage()->supportsIndexForIn()) -// return; - -// for (auto & child : node->children) -// { -// /// Don't descend into subqueries. -// if (child->as()) -// continue; - -// /// Don't descend into lambda functions -// const auto * func = child->as(); -// if (func && func->name == "lambda") -// continue; - -// makeSetsForIndex(child); -// } - -// const auto * func = node->as(); -// if (func && functionIsInOrGlobalInOperator(func->name)) -// { -// const IAST & args = *func->arguments; -// const ASTPtr & left_in_operand = args.children.at(0); - -// if (storage()->mayBenefitFromIndexForIn(left_in_operand, getContext(), metadata_snapshot)) -// { -// const ASTPtr & arg = args.children.at(1); -// if (arg->as() || arg->as()) -// { -// if (settings.use_index_for_in_with_subqueries) -// tryMakeSetForIndexFromSubquery(arg, query_options); -// } -// else -// { -// auto temp_actions = std::make_shared(columns_after_join); -// getRootActions(left_in_operand, true, temp_actions); - -// if (prepared_sets && temp_actions->tryFindInOutputs(left_in_operand->getColumnName())) -// makeExplicitSet(func, *temp_actions, true, getContext(), settings.size_limits_for_set, *prepared_sets); -// } -// } -// } -// } - - void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts) { LogAST log; diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index e4a9e96c517..271c3943afc 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -141,11 +141,6 @@ public: void makeWindowDescriptionFromAST(const Context & context, const WindowDescriptions & existing_descriptions, WindowDescription & desc, const IAST * ast); void makeWindowDescriptions(ActionsDAGPtr actions); - /** Create Set from a subquery or a table expression in the query. The created set is suitable for using the index. - * The set will not be created if its size hits the limit. - */ - // void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options = {}); - /** Checks if subquery is not a plain StorageSet. * Because while making set we will read data from StorageSet which is not allowed. * Returns valid SetPtr from StorageSet if the latter is used after IN or nullptr otherwise. @@ -363,9 +358,6 @@ public: /// Deletes all columns except mentioned by SELECT, arranges the remaining columns and renames them to aliases. ActionsDAGPtr appendProjectResult(ExpressionActionsChain & chain) const; - /// Create Set-s that we make from IN section to use index on them. - // void makeSetsForIndex(const ASTPtr & node); - private: StorageMetadataPtr metadata_snapshot; /// If non-empty, ignore all expressions not from this list. diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 8c784d3c2ff..5b633fee9b6 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -69,7 +69,6 @@ public: void addExternalStorage(ASTPtr & ast, const Names & required_columns, bool set_alias = false) { - // std::cerr << "=============== addExternalStorage is remote " << is_remote << std::endl; /// With nondistributed queries, creating temporary tables does not make sense. if (!is_remote) return; @@ -163,49 +162,20 @@ public: nullptr, /*create_for_global_subquery*/ true); StoragePtr external_storage = external_storage_holder->getTable(); - - // std::cerr << "......... adding external table " << external_table_name << std::endl; - external_tables.emplace(external_table_name, external_storage_holder); - /// We need to materialize external tables immediately because reading from distributed - /// tables might generate local plans which can refer to external tables during index - /// analysis. It's too late to populate the external table via CreatingSetsTransform. - // if (is_explain) - // { - // /// Do not materialize external tables if it's explain statement. - // } - // else if (getContext()->getSettingsRef().use_index_for_in_with_subqueries) - // { - // auto external_table = external_storage_holder->getTable(); - // auto table_out = external_table->write({}, external_table->getInMemoryMetadataPtr(), getContext()); - // auto io = interpreter->execute(); - // io.pipeline.complete(std::move(table_out)); - // CompletedPipelineExecutor executor(io.pipeline); - // executor.execute(); - // } - // else + auto set_key = database_and_table_name->getTreeHash(); + + if (!prepared_sets->findSubquery(set_key)) { - // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); - // subquery_for_set.createSource(*interpreter, external_storage); - auto set_key = database_and_table_name->getTreeHash(); + std::unique_ptr source = std::make_unique(); + interpreter->buildQueryPlan(*source); - // std::cerr << "====== Adding key " << set_key.toString() << std::endl; - - if (!prepared_sets->findSubquery(set_key)) - { - std::unique_ptr source = std::make_unique(); - interpreter->buildQueryPlan(*source); - - //std::cerr << reinterpret_cast(prepared_sets.get()) << std::endl; - auto future_set = prepared_sets->addFromSubquery(set_key, std::move(source), std::move(external_storage), nullptr, getContext()->getSettingsRef()); - // std::cerr << "... Future set " << reinterpret_cast(external_storage_holder.get()) << " " << reinterpret_cast(future_set.get()) << std::endl; - external_storage_holder->future_set = std::move(future_set); - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Set is already created for GLOBAL IN"); - //prepared_sets->addStorageToSubquery(key, std::move(external_storage)); + auto future_set = prepared_sets->addFromSubquery(set_key, std::move(source), std::move(external_storage), nullptr, getContext()->getSettingsRef()); + external_storage_holder->future_set = std::move(future_set); } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Set is already created for GLOBAL IN"); /** NOTE If it was written IN tmp_table - the existing temporary (but not external) table, * then a new temporary table will be created (for example, _data1), diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 85c3457c6b9..6ea15312ec4 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -953,10 +953,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (storage && !options.only_analyze) { - // query_analyzer->makeSetsForIndex(select_query.where()); - // query_analyzer->makeSetsForIndex(select_query.prewhere()); query_info.prepared_sets = query_analyzer->getPreparedSets(); - from_stage = storage->getQueryProcessingStage(context, options.to_stage, storage_snapshot, query_info); } diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 2aa3e02c713..428ef873bc5 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -25,20 +25,25 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -class FutureSetFromTuple final : public FutureSet +static SizeLimits getSizeLimitsForSet(const Settings & settings) { -public: - FutureSetFromTuple(Block block, const Settings & settings); + return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); +} - SetPtr get() const override { return set; } - SetPtr buildOrderedSetInplace(const ContextPtr & context) override; +static bool equals(const DataTypes & lhs, const DataTypes & rhs) +{ + size_t size = lhs.size(); + if (size != rhs.size()) + return false; - const DataTypes & getTypes() const override { return set->getElementsTypes(); } + for (size_t i = 0; i < size; ++i) + { + if (!lhs[i]->equals(*rhs[i])) + return false; + } -private: - SetPtr set; - SetKeyColumns set_key_columns; -}; + return true; +} FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} @@ -51,42 +56,153 @@ SetPtr FutureSetFromStorage::buildOrderedSetInplace(const ContextPtr &) } -// PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) -// { -// /// Remove LowCardinality types from type list because Set doesn't support LowCardinality keys now, -// /// just converts LowCardinality to ordinary types. -// for (auto & type : types_) -// type = recursiveRemoveLowCardinality(type); +FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) +{ + auto size_limits = getSizeLimitsForSet(settings); + set = std::make_shared(size_limits, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); + set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); -// PreparedSetKey key; -// key.ast_hash = hash; -// key.types = std::move(types_); -// return key; -// } + Columns columns; + columns.reserve(block.columns()); + for (const auto & column : block) + columns.emplace_back(column.column); -// PreparedSetKey PreparedSetKey::forSubquery(Hash hash) -// { -// PreparedSetKey key; -// key.ast_hash = hash; -// return key; -// } + set_key_columns.filter = ColumnUInt8::create(block.rows()); -// bool PreparedSetKey::operator==(const PreparedSetKey & other) const -// { -// if (ast_hash != other.ast_hash) -// return false; + set->insertFromColumns(columns, set_key_columns); + set->finishInsert(); +} -// if (types.size() != other.types.size()) -// return false; +const DataTypes & FutureSetFromTuple::getTypes() const { return set->getElementsTypes(); } -// for (size_t i = 0; i < types.size(); ++i) -// { -// if (!types[i]->equals(*other.types[i])) -// return false; -// } +SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) +{ + if (set->hasExplicitSetElements()) + return set; + + const auto & settings = context->getSettingsRef(); + size_t max_values = settings.use_index_for_in_with_subqueries_max_values; + bool too_many_values = max_values && max_values < set->getTotalRowCount(); + if (!too_many_values) + { + set->fillSetElements(); + set->appendSetElements(set_key_columns); + } + + return set; +} + + +FutureSetFromSubquery::FutureSetFromSubquery( + String key, + std::unique_ptr source_, + StoragePtr external_table_, + FutureSetPtr external_table_set_, + const Settings & settings) + : external_table(std::move(external_table_)) + , external_table_set(std::move(external_table_set_)) + , source(std::move(source_)) +{ + set_and_key = std::make_shared(); + set_and_key->key = std::move(key); + + auto size_limits = getSizeLimitsForSet(settings); + set_and_key->set = std::make_shared(size_limits, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); + set_and_key->set->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName()); +} + +FutureSetFromSubquery::FutureSetFromSubquery( + String key, + QueryTreeNodePtr query_tree_, + const Settings & settings) + : query_tree(std::move(query_tree_)) +{ + set_and_key = std::make_shared(); + set_and_key->key = std::move(key); + + auto size_limits = getSizeLimitsForSet(settings); + set_and_key->set = std::make_shared(size_limits, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); +} + +SetPtr FutureSetFromSubquery::get() const +{ + if (set_and_key->set != nullptr && set_and_key->set->isCreated()) + return set_and_key->set; + + return nullptr; +} + +void FutureSetFromSubquery::setQueryPlan(std::unique_ptr source_) +{ + source = std::move(source_); + set_and_key->set->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName()); +} + +const DataTypes & FutureSetFromSubquery::getTypes() const +{ + return set_and_key->set->getElementsTypes(); +} + +std::unique_ptr FutureSetFromSubquery::build(const ContextPtr & context) +{ + if (set_and_key->set->isCreated()) + return nullptr; + + const auto & settings = context->getSettingsRef(); + + auto plan = std::move(source); + + if (!plan) + return nullptr; + + auto creating_set = std::make_unique( + plan->getCurrentDataStream(), + set_and_key, + external_table, + SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), + context); + creating_set->setStepDescription("Create set for subquery"); + plan->addStep(std::move(creating_set)); + return plan; +} + +SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) +{ + if (!context->getSettingsRef().use_index_for_in_with_subqueries) + return nullptr; + + if (auto set = get()) + { + if (set->hasExplicitSetElements()) + return set; + + return nullptr; + } + + if (external_table_set) + { + auto set = external_table_set->buildOrderedSetInplace(context); + if (set) + return set_and_key->set = set; + } + + auto plan = build(context); + if (!plan) + return nullptr; + + set_and_key->set->fillSetElements(); + auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + pipeline.complete(std::make_shared(Block())); + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); + + set_and_key->set->checkIsCreated(); + + return set_and_key->set; +} -// return true; -// } String PreparedSets::toString(const PreparedSets::Hash & key, const DataTypes & types) { @@ -108,84 +224,18 @@ String PreparedSets::toString(const PreparedSets::Hash & key, const DataTypes & return buf.str(); } -// SubqueryForSet & PreparedSets::createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, -// SizeLimits set_size_limit, bool transform_null_in) -// { -// SubqueryForSet & subquery = subqueries[subquery_id]; - -// /// If you already created a Set with the same subquery / table for another ast -// /// In that case several PreparedSetKey would share same subquery and set -// /// Not sure if it's really possible case (maybe for distributed query when set was filled by external table?) -// if (subquery.set.isValid()) -// sets[key] = subquery.set; -// else -// { -// subquery.set_in_progress = std::make_shared(set_size_limit, false, transform_null_in); -// sets[key] = FutureSet(subquery.promise_to_fill_set.get_future()); -// } - -// if (!subquery.set_in_progress) -// { -// subquery.key = key.toString(); -// subquery.set_in_progress = std::make_shared(set_size_limit, false, transform_null_in); -// } - -// return subquery; -// } - -/// If the subquery is not associated with any set, create default-constructed SubqueryForSet. -/// It's aimed to fill external table passed to SubqueryForSet::createSource. -// void PreparedSets::addStorageToSubquery(const String & subquery_id, StoragePtr storage) -// { -// auto it = subqueries.find(subquery_id); -// if (it == subqueries.end()) -// throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find subquery {}", subquery_id); - -// it->second->addStorage(std::move(storage)); -// } -static bool equals(const DataTypes & lhs, const DataTypes & rhs) -{ - size_t size = lhs.size(); - if (size != rhs.size()) - return false; - - for (size_t i = 0; i < size; ++i) - { - if (!lhs[i]->equals(*rhs[i])) - return false; - } - - return true; -} - -static bool tryInsertSet(std::vector> & sets, FutureSetPtr new_set) -{ - auto types = new_set->getTypes(); - for (const auto & set : sets) - if (equals(set->getTypes(), new_set->getTypes())) - return false; - - sets.push_back(std::move(new_set)); - return true; -} - -static FutureSetPtr findSet(const std::vector> & sets, const DataTypes & types) -{ - for (const auto & set : sets) - if (equals(set->getTypes(), types)) - return set; - - return nullptr; -} - FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Settings & settings) { auto from_tuple = std::make_shared(std::move(block), settings); + const auto & set_types = from_tuple->getTypes(); auto & sets_by_hash = sets_from_tuple[key]; - if (!tryInsertSet(sets_by_hash, from_tuple)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, from_tuple->getTypes())); + auto types = from_tuple->getTypes(); + for (const auto & set : sets_by_hash) + if (equals(set->getTypes(), set_types)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, from_tuple->getTypes())); + sets_by_hash.push_back(std::move(from_tuple)); return from_tuple; } @@ -246,7 +296,11 @@ FutureSetPtr PreparedSets::findTuple(const Hash & key, const DataTypes & types) if (it == sets_from_tuple.end()) return nullptr; - return findSet(it->second, types); + for (const auto & set : it->second) + if (equals(set->getTypes(), types)) + return set; + + return nullptr; } std::shared_ptr PreparedSets::findSubquery(const Hash & key) const @@ -267,36 +321,9 @@ std::shared_ptr PreparedSets::findStorage(const Hash & key return it->second; } -// FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const -// { -// auto it = sets.find(key); -// if (it == sets.end()) -// return {}; -// return it->second; -// } - -// SetPtr PreparedSets::get(const PreparedSetKey & key) const -// { -// auto it = sets.find(key); -// if (it == sets.end() || !it->second.isReady()) -// return nullptr; -// return it->second.get(); -// } - -// std::vector PreparedSets::getByTreeHash(IAST::Hash ast_hash) const -// { -// std::vector res; -// for (const auto & it : this->sets) -// { -// if (it.first.ast_hash == ast_hash) -// res.push_back(it.second); -// } -// return res; -// } - -std::vector> PreparedSets::getSubqueries() +PreparedSets::Subqueries PreparedSets::getSubqueries() { - std::vector> res; + PreparedSets::Subqueries res; res.reserve(sets_from_subqueries.size()); for (auto & [_, set] : sets_from_subqueries) res.push_back(set); @@ -304,33 +331,10 @@ std::vector> PreparedSets::getSubqueries( return res; } -// void SubqueryForSet::createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_) -// { -// source = std::make_unique(); -// interpreter.buildQueryPlan(*source); -// if (table_) -// table = table_; -// } - -// bool SubqueryForSet::hasSource() const -// { -// return source != nullptr; -// } - -// QueryPlanPtr SubqueryForSet::detachSource() -// { -// auto res = std::move(source); -// source = nullptr; -// return res; -// } - - std::variant, SharedSet> PreparedSetsCache::findOrPromiseToBuild(const String & key) { std::lock_guard lock(cache_mutex); - // std::cerr << "PreparedSetsCache::findOrPromiseToBuild " << key << "\n" << StackTrace().toString() << std::endl; - auto it = cache.find(key); if (it != cache.end()) { @@ -347,163 +351,4 @@ std::variant, SharedSet> PreparedSetsCache::findOrPromiseTo return promise_to_fill_set; } -SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) -{ - if (!context->getSettingsRef().use_index_for_in_with_subqueries) - return nullptr; - - if (auto set = get()) - { - if (set->hasExplicitSetElements()) - return set; - - return nullptr; - } - - if (external_table_set) - { - auto set = external_table_set->buildOrderedSetInplace(context); - if (set) - return set_and_key->set = set; - } - - auto plan = buildPlan(context); - if (!plan) - return nullptr; - - set_and_key->set->fillSetElements(); - set_and_key->set->initSetElements(); - auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); - pipeline.complete(std::make_shared(Block())); - - CompletedPipelineExecutor executor(pipeline); - executor.execute(); - - set_and_key->set->checkIsCreated(); - - return set_and_key->set; -} - -SetPtr FutureSetFromSubquery::get() const -{ - if (set_and_key->set != nullptr && set_and_key->set->isCreated()) - return set_and_key->set; - - return nullptr; -} - -std::unique_ptr FutureSetFromSubquery::build(const ContextPtr & context) -{ - return buildPlan(context); -} - -static SizeLimits getSizeLimitsForSet(const Settings & settings) -{ - return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); -} - -std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & context) -{ - if (set_and_key->set->isCreated()) - return nullptr; - - const auto & settings = context->getSettingsRef(); - - auto plan = std::move(source); - - if (!plan) - return nullptr; - - auto creating_set = std::make_unique( - plan->getCurrentDataStream(), - set_and_key, - external_table, - SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), - context); - creating_set->setStepDescription("Create set for subquery"); - plan->addStep(std::move(creating_set)); - return plan; -} - -FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) -{ - bool create_ordered_set = false; - auto size_limits = getSizeLimitsForSet(settings); - set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); - set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); - - Columns columns; - columns.reserve(block.columns()); - for (const auto & column : block) - columns.emplace_back(column.column); - - set_key_columns.filter = ColumnUInt8::create(block.rows()); - - set->insertFromColumns(columns, set_key_columns); - set->finishInsert(); -} - -FutureSetFromSubquery::FutureSetFromSubquery( - String key, - std::unique_ptr source_, - StoragePtr external_table_, - FutureSetPtr external_table_set_, - const Settings & settings) - : external_table(std::move(external_table_)) - , external_table_set(std::move(external_table_set_)) - , source(std::move(source_)) -{ - set_and_key = std::make_shared(); - set_and_key->key = std::move(key); - - bool create_ordered_set = false; - auto size_limits = getSizeLimitsForSet(settings); - set_and_key->set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); - set_and_key->set->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName()); -} - -FutureSetFromSubquery::FutureSetFromSubquery( - String key, - QueryTreeNodePtr query_tree_, - //FutureSetPtr external_table_set_, - const Settings & settings) - : query_tree(std::move(query_tree_)) -{ - set_and_key = std::make_shared(); - set_and_key->key = std::move(key); - - bool create_ordered_set = false; - auto size_limits = getSizeLimitsForSet(settings); - set_and_key->set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); -} - -void FutureSetFromSubquery::setQueryPlan(std::unique_ptr source_) -{ - source = std::move(source_); - set_and_key->set->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName()); -} - -const DataTypes & FutureSetFromSubquery::getTypes() const -{ - return set_and_key->set->getElementsTypes(); -} - -SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) -{ - if (set->hasExplicitSetElements()) - return set; - - const auto & settings = context->getSettingsRef(); - size_t max_values = settings.use_index_for_in_with_subqueries_max_values; - bool too_many_values = max_values && max_values < set->getTotalRowCount(); - if (!too_many_values) - { - set->initSetElements(); - set->appendSetElements(set_key_columns); - } - - return set; -} - }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 339f4a6b435..cb240f5260a 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -8,7 +8,7 @@ #include #include #include -#include +#include namespace DB { @@ -17,7 +17,7 @@ class QueryPlan; class Set; using SetPtr = std::shared_ptr; -class InterpreterSelectWithUnionQuery; +struct SetKeyColumns; class IQueryTreeNode; using QueryTreeNodePtr = std::shared_ptr; @@ -44,14 +44,19 @@ class FutureSet public: virtual ~FutureSet() = default; + /// Returns set if set is ready (created and filled) or nullptr if not. virtual SetPtr get() const = 0; + /// Returns set->getElementsTypes(), even if set is not created yet. virtual const DataTypes & getTypes() const = 0; + /// If possible, return set with stored elements useful for PK analysis. virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; }; using FutureSetPtr = std::shared_ptr; -class FutureSetFromStorage : public FutureSet +/// Future set from already filled set. +/// Usually it is from StorageSet. +class FutureSetFromStorage final : public FutureSet { public: FutureSetFromStorage(SetPtr set_); @@ -64,34 +69,27 @@ private: SetPtr set; }; +/// Set from tuple is filled as well as set from storage. +/// Additionally, it can be converted to set useful for PK. +class FutureSetFromTuple final : public FutureSet +{ +public: + FutureSetFromTuple(Block block, const Settings & settings); -/// Information on how to build set for the [GLOBAL] IN section. -// class SubqueryForSet -// { -// public: + SetPtr get() const override { return set; } + SetPtr buildOrderedSetInplace(const ContextPtr & context) override; -// void createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_ = nullptr); + const DataTypes & getTypes() const override; -// bool hasSource() const; +private: + SetPtr set; + SetKeyColumns set_key_columns; +}; -// /// Returns query plan for the set's source -// /// and removes it from SubqueryForSet because we need to build it only once. -// std::unique_ptr detachSource(); - -// /// Build this set from the result of the subquery. -// String key; -// SetPtr set; - -// /// If set, put the result into the table. -// /// This is a temporary table for transferring to remote servers for distributed query processing. -// StoragePtr table; - -// /// The source is obtained using the InterpreterSelectQuery subquery. -// std::unique_ptr source; -// QueryTreeNodePtr query_tree; -// }; - -class FutureSetFromSubquery : public FutureSet +/// Set from subquery can be built inplace for PK or in CreatingSet step. +/// If use_index_for_in_with_subqueries_max_values is reached, set for PK won't be created, +/// but ordinary set would be created instead. +class FutureSetFromSubquery final : public FutureSet { public: FutureSetFromSubquery( @@ -104,7 +102,6 @@ public: FutureSetFromSubquery( String key, QueryTreeNodePtr query_tree_, - //FutureSetPtr external_table_set_, const Settings & settings); SetPtr get() const override; @@ -123,10 +120,9 @@ private: std::unique_ptr source; QueryTreeNodePtr query_tree; - - std::unique_ptr buildPlan(const ContextPtr & context); }; +/// Container for all the sets used in query. class PreparedSets { public: @@ -137,7 +133,7 @@ public: UInt64 operator()(const Hash & key) const { return key.first ^ key.second; } }; - using SetsFromTuple = std::unordered_map>, Hashing>; + using SetsFromTuple = std::unordered_map>, Hashing>; using SetsFromStorage = std::unordered_map, Hashing>; using SetsFromSubqueries = std::unordered_map, Hashing>; @@ -160,11 +156,12 @@ public: std::shared_ptr findStorage(const Hash & key) const; std::shared_ptr findSubquery(const Hash & key) const; - std::vector> getSubqueries(); + using Subqueries = std::vector>; + Subqueries getSubqueries(); const SetsFromTuple & getSetsFromTuple() const { return sets_from_tuple; } - const SetsFromStorage & getSetsFromStorage() const { return sets_from_storage; } - const SetsFromSubqueries & getSetsFromSubquery() const { return sets_from_subqueries; } + // const SetsFromStorage & getSetsFromStorage() const { return sets_from_storage; } + // const SetsFromSubqueries & getSetsFromSubquery() const { return sets_from_subqueries; } static String toString(const Hash & key, const DataTypes & types); diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 17df5064642..b42ff102f72 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -167,19 +167,16 @@ void Set::setHeader(const ColumnsWithTypeAndName & header) extractNestedColumnsAndNullMap(key_columns, null_map); } - if (fill_set_elements) - { - /// Create empty columns with set values in advance. - /// It is needed because set may be empty, so method 'insertFromBlock' will be never called. - initSetElements(); - } - /// Choose data structure to use for the set. data.init(data.chooseMethod(key_columns, key_sizes)); } -void Set::initSetElements() +void Set::fillSetElements() { + if (data.getTotalRowCount()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert set to ordered because it is not empty"); + + fill_set_elements = true; set_elements.reserve(keys_size); for (const auto & type : set_elements_types) set_elements.emplace_back(type->createColumn()); diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index d1218830969..9ea46e117ef 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -21,17 +22,6 @@ using FunctionBasePtr = std::shared_ptr; class Chunk; -/// Prepared key columns for set which can be added to fill set elements. -/// Used only to upgrade set from tuple. -struct SetKeyColumns -{ - /// The constant columns to the right of IN are not supported directly. For this, they first materialize. - ColumnRawPtrs key_columns; - Columns materialized_columns; - ColumnPtr null_map_holder; - ColumnUInt8::MutablePtr filter; -}; - /** Data structure for implementation of IN expression. */ class Set @@ -41,14 +31,12 @@ public: /// (that is useful only for checking that some value is in the set and may not store the original values), /// store all set elements in explicit form. /// This is needed for subsequent use for index. - Set(const SizeLimits & limits_, bool fill_set_elements_, size_t max_elements_to_fill_, bool transform_null_in_) + Set(const SizeLimits & limits_, size_t max_elements_to_fill_, bool transform_null_in_) : log(&Poco::Logger::get("Set")), - limits(limits_), fill_set_elements(fill_set_elements_), max_elements_to_fill(max_elements_to_fill_), transform_null_in(transform_null_in_) + limits(limits_), max_elements_to_fill(max_elements_to_fill_), transform_null_in(transform_null_in_) { } - void fillSetElements() { fill_set_elements = true; } - /** Set can be created either from AST or from a stream of data (subquery result). */ @@ -61,7 +49,7 @@ public: bool insertFromColumns(const Columns & columns); bool insertFromBlock(const ColumnsWithTypeAndName & columns); - void initSetElements(); + void fillSetElements(); bool insertFromColumns(const Columns & columns, SetKeyColumns & holder); void appendSetElements(SetKeyColumns & holder); @@ -129,7 +117,7 @@ private: SizeLimits limits; /// Do we need to additionally store all elements of the set in explicit form for subsequent use for index. - bool fill_set_elements; + bool fill_set_elements = false; size_t max_elements_to_fill; /// If true, insert NULL values to set. diff --git a/src/Interpreters/SetKeys.h b/src/Interpreters/SetKeys.h new file mode 100644 index 00000000000..2cab9849c9b --- /dev/null +++ b/src/Interpreters/SetKeys.h @@ -0,0 +1,18 @@ +#pragma once +#include + +namespace DB +{ + +/// Prepared key columns for set which can be added to fill set elements. +/// Used only to upgrade set from tuple. +struct SetKeyColumns +{ + /// The constant columns to the right of IN are not supported directly. For this, they first materialize. + ColumnRawPtrs key_columns; + Columns materialized_columns; + ColumnPtr null_map_holder; + ColumnUInt8::MutablePtr filter; +}; + +} diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index df3558745fb..8dd7c6637bf 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -30,9 +30,8 @@ namespace class CollectSetsVisitor : public ConstInDepthQueryTreeVisitor { public: - explicit CollectSetsVisitor(PlannerContext & planner_context_) //, const SelectQueryOptions & select_query_options_) + explicit CollectSetsVisitor(PlannerContext & planner_context_) : planner_context(planner_context_) - //, select_query_options(select_query_options_) {} void visitImpl(const QueryTreeNodePtr & node) @@ -60,7 +59,7 @@ public: } else if (const auto * constant_node = in_second_argument->as()) { - auto set = makeSetForConstantValue( + auto set = getSetElementsForConstantValue( in_first_argument->getResultType(), constant_node->getValue(), constant_node->getResultType(), @@ -72,18 +71,12 @@ public: set_element_types = left_tuple_type->getElements(); set_element_types = Set::getElementTypes(std::move(set_element_types), settings.transform_null_in); - - // for (auto & element_type : set_element_types) - // if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - // element_type = low_cardinality_type->getDictionaryType(); - auto set_key = in_second_argument->getTreeHash(); + if (sets.findTuple(set_key, set_element_types)) return; sets.addFromTuple(set_key, std::move(set), settings); - - //planner_context.registerSet(set_key, PlannerSet(FutureSet(std::move(set)))); } else if (in_second_argument_node_type == QueryTreeNodeType::QUERY || in_second_argument_node_type == QueryTreeNodeType::UNION || @@ -141,14 +134,13 @@ public: private: PlannerContext & planner_context; - //const SelectQueryOptions & select_query_options; }; } -void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context) //, const SelectQueryOptions & select_query_options) +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context) { - CollectSetsVisitor visitor(planner_context); //, select_query_options); + CollectSetsVisitor visitor(planner_context); visitor.visit(node); } diff --git a/src/Planner/CollectSets.h b/src/Planner/CollectSets.h index e0db802d3b4..5f9f7a5a466 100644 --- a/src/Planner/CollectSets.h +++ b/src/Planner/CollectSets.h @@ -12,6 +12,6 @@ struct SelectQueryOptions; /** Collect prepared sets and sets for subqueries that are necessary to execute IN function and its variations. * Collected sets are registered in planner context. */ -void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context); //, const SelectQueryOptions & select_query_options); +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index d524f2bed98..9f6c22f90f3 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -933,8 +933,6 @@ void addBuildSubqueriesForSetsStepIfNeeded( auto subqueries = planner_context->getPreparedSets().getSubqueries(); std::unordered_set useful_sets; - //PreparedSets::SubqueriesForSets subqueries_for_sets; - for (const auto & actions_to_execute : result_actions_to_execute) collectSetsFromActionsDAG(actions_to_execute, useful_sets); @@ -994,7 +992,7 @@ void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan, auto storage = std::make_shared(StorageID{"dummy", "dummy"}, fake_column_descriptions); auto fake_table_expression = std::make_shared(std::move(storage), query_context); - auto filter_info = buildFilterInfo(additional_result_filter_ast, fake_table_expression, planner_context, select_query_options, std::move(fake_name_set)); + auto filter_info = buildFilterInfo(additional_result_filter_ast, fake_table_expression, planner_context, std::move(fake_name_set)); if (!filter_info.actions || !query_plan.isInitialized()) return; @@ -1225,7 +1223,7 @@ void Planner::buildPlanForQueryNode() } checkStoragesSupportTransactions(planner_context); - collectSets(query_tree, *planner_context); //, select_query_options); + collectSets(query_tree, *planner_context); collectTableExpressionData(query_tree, planner_context); const auto & settings = query_context->getSettingsRef(); @@ -1524,10 +1522,7 @@ void Planner::buildPlanForQueryNode() } if (!select_query_options.only_analyze) - { - //addCreatingSetsStep(query_plan, planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()), planner_context->getQueryContext()); addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); - } } SelectQueryInfo Planner::buildSelectQueryInfo() const diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 40b6ec109dc..7575828e64d 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -633,7 +633,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma DataTypes set_element_types; auto in_second_argument_node_type = in_second_argument->getNodeType(); - // std::cerr << "=========== " << in_second_argument->getNodeTypeName() << std::endl; + bool subquery_or_table = in_second_argument_node_type == QueryTreeNodeType::QUERY || in_second_argument_node_type == QueryTreeNodeType::UNION || @@ -649,10 +649,6 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma if (left_tuple_type && left_tuple_type->getElements().size() != 1) set_element_types = left_tuple_type->getElements(); - // for (auto & element_type : set_element_types) - // if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - // element_type = low_cardinality_type->getDictionaryType(); - set_element_types = Set::getElementTypes(std::move(set_element_types), planner_context->getQueryContext()->getSettingsRef().transform_null_in); set = planner_context->getPreparedSets().findTuple(set_key, set_element_types); } diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index c8aeebc17c1..3c75d4fbea8 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -118,50 +118,4 @@ PlannerContext::SetKey PlannerContext::createSetKey(const QueryTreeNodePtr & set return "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second); } -// void PlannerContext::registerSet(const SetKey & key, PlannerSet planner_set) -// { -// if (!planner_set.getSet().isValid()) -// throw Exception(ErrorCodes::LOGICAL_ERROR, "Set must be initialized"); - -// const auto & subquery_node = planner_set.getSubqueryNode(); -// if (subquery_node) -// { -// auto node_type = subquery_node->getNodeType(); - -// if (node_type != QueryTreeNodeType::QUERY && -// node_type != QueryTreeNodeType::UNION && -// node_type != QueryTreeNodeType::TABLE) -// throw Exception(ErrorCodes::LOGICAL_ERROR, -// "Invalid node for set table expression. Expected query or union. Actual {}", -// subquery_node->formatASTForErrorMessage()); -// } - -// set_key_to_set.emplace(key, std::move(planner_set)); -// } - -// bool PlannerContext::hasSet(const SetKey & key) const -// { -// return set_key_to_set.contains(key); -// } - -// const PlannerSet & PlannerContext::getSetOrThrow(const SetKey & key) const -// { -// auto it = set_key_to_set.find(key); -// if (it == set_key_to_set.end()) -// throw Exception(ErrorCodes::LOGICAL_ERROR, -// "No set is registered for key {}", -// key); - -// return it->second; -// } - -// PlannerSet * PlannerContext::getSetOrNull(const SetKey & key) -// { -// auto it = set_key_to_set.find(key); -// if (it == set_key_to_set.end()) -// return nullptr; - -// return &it->second; -// } - } diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index 78c7f8feab5..aceb313d4b5 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -44,56 +44,6 @@ private: using GlobalPlannerContextPtr = std::shared_ptr; -/** PlannerSet is wrapper around Set that is used during query planning. - * - * If subquery node is null, such set is already prepared for execution. - * - * If subquery node is not null, then set must be build from the result of the subquery. - * If subquery node is not null, it must have QUERY or UNION type. - */ -class PlannerSet -{ - -}; - -// { -// public: -// /// Construct planner set that is ready for execution -// explicit PlannerSet(FutureSetPtr set_) -// : set(std::move(set_)) -// {} - -// /// Construct planner set with set and subquery node -// explicit PlannerSet(QueryTreeNodePtr subquery_node_) -// //: set(promise_to_build_set.get_future()) -// : subquery_node(std::move(subquery_node_)) -// {} - -// /// Get a reference to a set that might be not built yet -// const FutureSetPtr & getSet() const -// { -// return set; -// } - -// /// Get subquery node -// const QueryTreeNodePtr & getSubqueryNode() const -// { -// return subquery_node; -// } - -// /// This promise will be fulfilled when set is built and all FutureSet objects will become ready -// // std::promise extractPromiseToBuildSet() -// // { -// // return std::move(promise_to_build_set); -// // } - -// private: -// //std::promise promise_to_build_set; -// FutureSetPtr set; - -// QueryTreeNodePtr subquery_node; -// }; - class PlannerContext { public: @@ -181,29 +131,9 @@ public: using SetKey = std::string; - // using SetKeyToSet = std::unordered_map; - - // /// Create set key for set source node + /// Create set key for set source node static SetKey createSetKey(const QueryTreeNodePtr & set_source_node); - // /// Register set for set key - // void registerSet(const SetKey & key, PlannerSet planner_set); - - // /// Returns true if set is registered for key, false otherwise - // bool hasSet(const SetKey & key) const; - - // /// Get set for key, if no set is registered logical exception is thrown - // const PlannerSet & getSetOrThrow(const SetKey & key) const; - - // /// Get set for key, if no set is registered null is returned - // PlannerSet * getSetOrNull(const SetKey & key); - - // /// Get registered sets - // const SetKeyToSet & getRegisteredSets() const - // { - // return set_key_to_set; - // } - PreparedSets & getPreparedSets() { return prepared_sets; } private: diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index c27ee72437c..5d8f8ca8741 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -403,8 +403,7 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, SelectQueryInfo & table_expression_query_info, - PlannerContextPtr & planner_context, - const SelectQueryOptions & select_query_options) + PlannerContextPtr & planner_context) { auto storage_id = storage->getStorageID(); const auto & query_context = planner_context->getQueryContext(); @@ -413,13 +412,12 @@ FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, if (!row_policy_filter) return {}; - return buildFilterInfo(row_policy_filter->expression, table_expression_query_info.table_expression, planner_context, select_query_options); + return buildFilterInfo(row_policy_filter->expression, table_expression_query_info.table_expression, planner_context); } FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, SelectQueryInfo & table_expression_query_info, - PlannerContextPtr & planner_context, - const SelectQueryOptions & select_query_options) + PlannerContextPtr & planner_context) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); @@ -445,15 +443,14 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, *storage, query_context); - return buildFilterInfo(parallel_replicas_custom_filter_ast, table_expression_query_info.table_expression, planner_context, select_query_options); + return buildFilterInfo(parallel_replicas_custom_filter_ast, table_expression_query_info.table_expression, planner_context); } /// Apply filters from additional_table_filters setting FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, const String & table_expression_alias, SelectQueryInfo & table_expression_query_info, - PlannerContextPtr & planner_context, - const SelectQueryOptions & select_query_options) + PlannerContextPtr & planner_context) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); @@ -487,7 +484,7 @@ FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, return {}; table_expression_query_info.additional_filter_ast = additional_filter_ast; - return buildFilterInfo(additional_filter_ast, table_expression_query_info.table_expression, planner_context, select_query_options); + return buildFilterInfo(additional_filter_ast, table_expression_query_info.table_expression, planner_context); } JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, @@ -697,14 +694,14 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } }; - auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, select_query_options); + auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context); add_filter(row_policy_filter_info, "Row-level security filter"); if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY) { if (settings.parallel_replicas_count > 1) { - auto parallel_replicas_custom_key_filter_info = buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context, select_query_options); + auto parallel_replicas_custom_key_filter_info = buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context); add_filter(parallel_replicas_custom_key_filter_info, "Parallel replicas custom key filter"); } else @@ -719,7 +716,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } const auto & table_expression_alias = table_expression->getAlias(); - auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context, select_query_options); + auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context); add_filter(additional_filters_info, "additional filter"); from_stage = storage->getQueryProcessingStage(query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 403e3f03997..733db0f00bc 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -429,7 +429,6 @@ SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, const QueryTreeNodePtr & table_expression, PlannerContextPtr & planner_context, - [[maybe_unused]] const SelectQueryOptions & select_query_options, NameSet table_expression_required_names_without_filter) { const auto & query_context = planner_context->getQueryContext(); @@ -447,7 +446,7 @@ FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, } collectSourceColumns(filter_query_tree, planner_context); - collectSets(filter_query_tree, *planner_context); //, select_query_options); + collectSets(filter_query_tree, *planner_context); auto filter_actions_dag = std::make_shared(); diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index 8071e201f88..d9412800e61 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -82,7 +82,6 @@ SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, const QueryTreeNodePtr & table_expression, PlannerContextPtr & planner_context, - const SelectQueryOptions & select_query_options, NameSet table_expression_required_names_without_filter = {}); ASTPtr parseAdditionalResultFilter(const Settings & settings); diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index ebd58c3dc95..c54d32c1385 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -105,7 +105,7 @@ CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( : ITransformingStep(input_stream_, input_stream_.header, getTraits()) , column_names(column_names_) , max_rows_in_set(max_rows_in_set_) - , own_set(std::make_shared(SizeLimits(max_rows_in_set, 0, OverflowMode::BREAK), false, 0, true)) + , own_set(std::make_shared(SizeLimits(max_rows_in_set, 0, OverflowMode::BREAK), 0, true)) , filtering_set(nullptr) , crosswise_connection(crosswise_connection_) , position(position_) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index aae632787b9..3e4dfb0c7d1 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -124,7 +124,7 @@ void CreatingSetsStep::describePipeline(FormatSettings & settings) const IQueryPlanStep::describePipeline(processors, settings); } -void addCreatingSetsStep(QueryPlan & query_plan, std::vector> sets_from_subquery, ContextPtr context) +void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::Subqueries subqueries, ContextPtr context) { DataStreams input_streams; input_streams.emplace_back(query_plan.getCurrentDataStream()); @@ -133,7 +133,7 @@ void addCreatingSetsStep(QueryPlan & query_plan, std::vector(std::move(query_plan))); query_plan = QueryPlan(); - for (auto & future_set : sets_from_subquery) + for (auto & future_set : subqueries) { if (future_set->get()) continue; @@ -157,18 +157,11 @@ void addCreatingSetsStep(QueryPlan & query_plan, std::vector> DelayedCreatingSetsStep::makePlansForSets(DelayedCreatingSetsStep && step) { - // DataStreams input_streams; - // input_streams.emplace_back(query_plan.getCurrentDataStream()); - std::vector> plans; - // plans.emplace_back(std::make_unique(std::move(query_plan))); - // query_plan = QueryPlan(); - for (auto & future_set : step.sets_from_subquery) + for (auto & future_set : step.subqueries) { if (future_set->get()) continue; @@ -179,7 +172,6 @@ std::vector> DelayedCreatingSetsStep::makePlansForSet plan->optimize(QueryPlanOptimizationSettings::fromContext(step.context)); - //input_streams.emplace_back(plan->getCurrentDataStream()); plans.emplace_back(std::move(plan)); } @@ -199,8 +191,8 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, } DelayedCreatingSetsStep::DelayedCreatingSetsStep( - DataStream input_stream, std::vector> sets_from_subquery_, ContextPtr context_) - : sets_from_subquery(std::move(sets_from_subquery_)), context(std::move(context_)) + DataStream input_stream, PreparedSets::Subqueries subqueries_, ContextPtr context_) + : subqueries(std::move(subqueries_)), context(std::move(context_)) { input_streams = {input_stream}; output_stream = std::move(input_stream); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index 7110775da79..a90b70a2fa4 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -47,10 +47,12 @@ public: void describePipeline(FormatSettings & settings) const override; }; +/// This is a temporary step which is converted to CreatingSetStep after plan optimization. +/// Can't be used by itself. class DelayedCreatingSetsStep final : public IQueryPlanStep { public: - DelayedCreatingSetsStep(DataStream input_stream, std::vector> sets_from_subquery_, ContextPtr context_); + DelayedCreatingSetsStep(DataStream input_stream, PreparedSets::Subqueries subqueries_, ContextPtr context_); String getName() const override { return "DelayedCreatingSets"; } @@ -59,14 +61,14 @@ public: static std::vector> makePlansForSets(DelayedCreatingSetsStep && step); ContextPtr getContext() const { return context; } - std::vector> detachSets() { return std::move(sets_from_subquery); } + PreparedSets::Subqueries detachSets() { return std::move(subqueries); } private: - std::vector> sets_from_subquery; + PreparedSets::Subqueries subqueries; ContextPtr context; }; -void addCreatingSetsStep(QueryPlan & query_plan, std::vector> sets_from_subquery, ContextPtr context); +void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::Subqueries subqueries, ContextPtr context); void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context); diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index 8e2221d564d..9b9cc221ca8 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -43,7 +43,6 @@ std::unique_ptr createLocalPlan( const Block & header, ContextPtr context, QueryProcessingStage::Enum processed_stage, - [[maybe_unused]] PreparedSetsPtr prepared_sets, size_t shard_num, size_t shard_count, size_t replica_num, @@ -99,7 +98,7 @@ std::unique_ptr createLocalPlan( } else { - auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options); //, prepared_sets); + auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options); interpreter.buildQueryPlan(*query_plan); } diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h index cf59027a33f..c08b9bdf67e 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h @@ -18,7 +18,6 @@ std::unique_ptr createLocalPlan( const Block & header, ContextPtr context, QueryProcessingStage::Enum processed_stage, - PreparedSetsPtr prepared_sets, size_t shard_num, size_t shard_count, size_t replica_num, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index e549a06877e..1e21d13e2b1 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1220,13 +1220,8 @@ static void buildIndexes( const auto & partition_key = metadata_snapshot->getPartitionKey(); auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); auto minmax_expression_actions = data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context)); - // minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); - // if (context->getSettingsRef().allow_experimental_analyzer) indexes->minmax_idx_condition.emplace(filter_actions_dag, context, minmax_columns_names, minmax_expression_actions, NameSet()); - // else - // indexes->minmax_idx_condition.emplace(query_info, context, minmax_columns_names, minmax_expression_actions); - indexes->partition_pruner.emplace(metadata_snapshot, filter_actions_dag, context, false /* strict */); } @@ -1402,32 +1397,10 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( result.column_names_to_read.push_back(ExpressionActions::getSmallestColumn(available_real_columns).name); } - // storage_snapshot->check(result.column_names_to_read); - // Build and check if primary key is used when necessary const auto & primary_key = metadata_snapshot->getPrimaryKey(); const Names & primary_key_column_names = primary_key.column_names; - // if (!key_condition) - // { - // if (settings.query_plan_optimize_primary_key) - // { - // NameSet array_join_name_set; - // if (query_info.syntax_analyzer_result) - // array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(); - - // key_condition.emplace(query_info.filter_actions_dag, - // context, - // primary_key_column_names, - // primary_key.expression, - // array_join_name_set); - // } - // else - // { - // key_condition.emplace(query_info, context, primary_key_column_names, primary_key.expression); - // } - // } - if (!indexes) buildIndexes(indexes, query_info.filter_actions_dag, data, context, query_info, metadata_snapshot); diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 16cb06a94d6..ed740e3e242 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -97,7 +97,6 @@ ReadFromRemote::ReadFromRemote( ClusterProxy::SelectStreamFactory::Shards shards_, Block header_, QueryProcessingStage::Enum stage_, - PreparedSetsPtr prepared_sets_, StorageID main_table_, ASTPtr table_func_ptr_, ContextMutablePtr context_, @@ -110,7 +109,6 @@ ReadFromRemote::ReadFromRemote( : ISourceStep(DataStream{.header = std::move(header_)}) , shards(std::move(shards_)) , stage(stage_) - , prepared_sets(std::move(prepared_sets_)) , main_table(std::move(main_table_)) , table_func_ptr(std::move(table_func_ptr_)) , context(std::move(context_)) @@ -152,7 +150,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream my_context = context, my_throttler = throttler, my_main_table = main_table, my_table_func_ptr = table_func_ptr, my_scalars = scalars, my_external_tables = external_tables, - my_stage = stage, my_prepared_sets = prepared_sets, local_delay = shard.local_delay, + my_stage = stage, local_delay = shard.local_delay, add_agg_info, add_totals, add_extremes, async_read, async_query_sending]() mutable -> QueryPipelineBuilder { @@ -187,7 +185,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream if (try_results.empty() || local_delay < max_remote_delay) { auto plan = createLocalPlan( - query, header, my_context, my_stage, my_prepared_sets, my_shard.shard_info.shard_num, my_shard_count, 0, 0, /*coordinator=*/nullptr); + query, header, my_context, my_stage, my_shard.shard_info.shard_num, my_shard_count, 0, 0, /*coordinator=*/nullptr); return std::move(*plan->buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(my_context), diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index 405b5727ff2..d4005d81f1b 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -26,7 +26,6 @@ public: ClusterProxy::SelectStreamFactory::Shards shards_, Block header_, QueryProcessingStage::Enum stage_, - PreparedSetsPtr prepared_sets_, StorageID main_table_, ASTPtr table_func_ptr_, ContextMutablePtr context_, @@ -47,7 +46,6 @@ public: private: ClusterProxy::SelectStreamFactory::Shards shards; QueryProcessingStage::Enum stage; - PreparedSetsPtr prepared_sets; StorageID main_table; ASTPtr table_func_ptr; ContextMutablePtr context; diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index 4c1fe4fc559..16ab99d03b4 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -77,7 +77,6 @@ bool traverseASTFilter( if (!future_set) return false; - //if (!future_set->isReady()) future_set->buildOrderedSetInplace(context); auto set = future_set->get(); diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index d41dd30098b..f909d854cf6 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1194,8 +1194,6 @@ bool KeyCondition::tryPrepareSetIndex( RPNElement & out, size_t & out_key_column_num) { - // std::cerr << "::: tryPrepareSetIndex for " << func.getColumnName() << std::endl; - // std::cerr << StackTrace().toString() << std::endl; const auto & left_arg = func.getArgumentAt(0); out_key_column_num = 0; @@ -1239,56 +1237,27 @@ bool KeyCondition::tryPrepareSetIndex( } if (indexes_mapping.empty()) - { - // std::cerr << ".. index mapping is empty\n"; return false; - } const auto right_arg = func.getArgumentAt(1); - // LOG_TRACE(&Poco::Logger::get("KK"), "Trying to get set for {}", right_arg.getColumnName()); - auto future_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types); if (!future_set) - { - // std::cerr << ".. no future set\n"; return false; - } - // LOG_TRACE(&Poco::Logger::get("KK"), "Found set for {}", right_arg.getColumnName()); - - //if (!future_set->isReady()) - //{ - // LOG_TRACE(&Poco::Logger::get("KK"), "Building set inplace for {}", right_arg.getColumnName()); - future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); - //} - - auto prepared_set = future_set->get(); + auto prepared_set = future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); if (!prepared_set) - { - - // std::cerr << ".. no prepared set\n"; return false; - } - - // LOG_TRACE(&Poco::Logger::get("KK"), "Set if ready for {}", right_arg.getColumnName()); /// The index can be prepared if the elements of the set were saved in advance. if (!prepared_set->hasExplicitSetElements()) - { - - // std::cerr << ".. no explicit elements\n"; return false; - } - - // LOG_TRACE(&Poco::Logger::get("KK"), "Has explicit elements for {}", right_arg.getColumnName()); prepared_set->checkColumnsNumber(left_args_count); for (size_t i = 0; i < indexes_mapping.size(); ++i) prepared_set->checkTypesEqual(indexes_mapping[i].tuple_index, data_types[i]); out.set_index = std::make_shared(prepared_set->getSetElements(), std::move(indexes_mapping)); - // std::cerr << ".. can use\n"; return true; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 125fe98203f..ee515106591 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1839,9 +1839,7 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( if (partition_pruner) { - auto val = partition_pruner->canBePruned(*part); - // std::cerr << "... part " << part->getNameWithState() << " cbp ? " << val << std::endl; - if (val) + if (partition_pruner->canBePruned(*part)) continue; } diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index e2bf9bde674..398a85e92ac 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -31,21 +31,13 @@ namespace ColumnWithTypeAndName getPreparedSetInfo(const ConstSetPtr & prepared_set) { - // std::cerr << "====== " << prepared_set->getDataTypes().size() << std::endl; if (prepared_set->getDataTypes().size() == 1) return {prepared_set->getSetElements()[0], prepared_set->getElementsTypes()[0], "dummy"}; Columns set_elements; for (auto & set_element : prepared_set->getSetElements()) - { - // std::cerr << set_element->dumpStructure() << std::endl; - set_elements.emplace_back(set_element->convertToFullColumnIfConst()); - } - // for (auto & set_element : prepared_set->getElementsTypes()) - // { - // // std::cerr << set_element->getName() << std::endl; - // } + set_elements.emplace_back(set_element->convertToFullColumnIfConst()); return {ColumnTuple::create(set_elements), std::make_shared(prepared_set->getElementsTypes()), "dummy"}; } @@ -319,30 +311,17 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNo if (functionIsInOrGlobalInOperator(function_name)) { - //std::cerr << StackTrace().toString() << std::endl; - - auto future_set = rhs_argument.tryGetPreparedSet(); - - //std::cerr << "==== Finding set for MergeTreeBF " << bool(future_set) << std::endl; - - if (future_set) // && !future_set->isReady()) + if (auto future_set = rhs_argument.tryGetPreparedSet(); future_set) { - //std::cerr << "==== not ready, building " << std::endl; - future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); - } - - ConstSetPtr prepared_set; - if (future_set) - prepared_set = future_set->get(); - - //std::cerr << "==== Prep set for MergeTreeBF " << bool(prepared_set) << ' ' << (prepared_set ? prepared_set->hasExplicitSetElements() : false) << std::endl; - - if (prepared_set && prepared_set->hasExplicitSetElements()) - { - const auto prepared_info = getPreparedSetInfo(prepared_set); - // std::cerr << "...... " << prepared_info.dumpStructure() << std::endl; - if (traverseTreeIn(function_name, lhs_argument, prepared_set, prepared_info.type, prepared_info.column, out)) - maybe_useful = true; + if (auto prepared_set = future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); prepared_set) + { + if (prepared_set->hasExplicitSetElements()) + { + const auto prepared_info = getPreparedSetInfo(prepared_set); + if (traverseTreeIn(function_name, lhs_argument, prepared_set, prepared_info.type, prepared_info.column, out)) + maybe_useful = true; + } + } } } else if (function_name == "equals" || @@ -387,7 +366,6 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeIn( size_t row_size = column->size(); size_t position = header.getPositionByName(key_node_column_name); const DataTypePtr & index_type = header.getByPosition(position).type; - // std::cerr << "::::: " << ColumnWithTypeAndName{column, type, ""}.dumpStructure() << " -> " << index_type->getName() << std::endl; const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, index_type); out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(index_type, converted_column, 0, row_size))); diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 422f504d046..b6a2cafe245 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -625,13 +625,10 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( return false; auto future_set = right_argument.tryGetPreparedSet(data_types); - if (future_set) // && !future_set->isReady()) - future_set->buildOrderedSetInplace(right_argument.getTreeContext().getQueryContext()); - - ConstSetPtr prepared_set; - if (future_set) - prepared_set = future_set->get(); + if (!future_set) + return false; + auto prepared_set = future_set->buildOrderedSetInplace(right_argument.getTreeContext().getQueryContext()); if (!prepared_set || !prepared_set->hasExplicitSetElements()) return false; diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index 2814b8521f6..6b4919c545d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -655,19 +655,11 @@ bool MergeTreeConditionInverted::tryPrepareSetGinFilter( if (key_tuple_mapping.empty()) return false; - //std::cerr << "==== Finding set for MergeTreeConditionInverted\n"; - auto future_set = rhs.tryGetPreparedSet(); + if (!future_set) + return false; - //std::cerr << "==== Set for MergeTreeConditionInverted" << bool(future_set) << std::endl; - - if (future_set) // && !future_set->isReady()) - future_set->buildOrderedSetInplace(rhs.getTreeContext().getQueryContext()); - - ConstSetPtr prepared_set; - if (future_set) - prepared_set = future_set->get(); - + auto prepared_set = future_set->buildOrderedSetInplace(rhs.getTreeContext().getQueryContext()); if (!prepared_set || !prepared_set->hasExplicitSetElements()) return false; diff --git a/src/Storages/MergeTree/PartitionPruner.cpp b/src/Storages/MergeTree/PartitionPruner.cpp index a397a1475d1..97bb9f3b4d4 100644 --- a/src/Storages/MergeTree/PartitionPruner.cpp +++ b/src/Storages/MergeTree/PartitionPruner.cpp @@ -29,12 +29,6 @@ PartitionPruner::PartitionPruner(const StorageMetadataPtr & metadata, ActionsDAG , partition_condition(filter_actions_dag, context, partition_key.column_names, partition_key.expression, {}, true /* single_point */, strict) , useless(strict ? partition_condition.anyUnknownOrAlwaysTrue() : partition_condition.alwaysUnknownOrTrue()) { - // auto description = getKeyCondition().getDescription(); - // std::cerr << ".... " << description.condition << std::endl; - // std::cerr << filter_actions_dag->dumpDAG() << std::endl; - // for (const auto & name : partition_key.column_names) - // std::cerr << ". " << name << std::endl; - // std::cerr << partition_key.expression->dumpActions() << std::endl; } bool PartitionPruner::canBePruned(const IMergeTreeDataPart & part) @@ -52,8 +46,6 @@ bool PartitionPruner::canBePruned(const IMergeTreeDataPart & part) else { const auto & partition_value = part.partition.value; - // for (const auto & val : partition_value) - // std::cerr << val.dump() << std::endl; std::vector index_value(partition_value.begin(), partition_value.end()); for (auto & field : index_value) { @@ -64,7 +56,6 @@ bool PartitionPruner::canBePruned(const IMergeTreeDataPart & part) is_valid = partition_condition.mayBeTrueInRange( partition_value.size(), index_value.data(), index_value.data(), partition_key.data_types); - // std::cerr << "may be true " << is_valid << std::endl; partition_filter_map.emplace(partition_id, is_valid); if (!is_valid) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index a69db7d80c0..c46192ab43b 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -88,10 +88,8 @@ #include #include #include -#include #include #include -#include #include #include #include @@ -769,8 +767,7 @@ void StorageDistributed::read( header, snapshot_data.objects_by_shard, storage_snapshot, - processed_stage, - query_info.prepared_sets); + processed_stage); auto settings = local_context->getSettingsRef(); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 655b3d5a8f4..c02c96f62be 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -231,8 +231,7 @@ void StorageMergeTree::read( header, {}, storage_snapshot, - processed_stage, - query_info.prepared_sets); + processed_stage); ClusterProxy::executeQueryWithParallelReplicas( query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 01f663aeee4..bb99e21e4ab 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4866,8 +4866,7 @@ void StorageReplicatedMergeTree::read( header, {}, storage_snapshot, - processed_stage, - query_info.prepared_sets); + processed_stage); ClusterProxy::executeQueryWithParallelReplicas( query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr, diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index f9fdcdf02a6..79369ab4bcb 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -147,7 +147,7 @@ StorageSet::StorageSet( const String & comment, bool persistent_) : StorageSetOrJoinBase{disk_, relative_path_, table_id_, columns_, constraints_, comment, persistent_} - , set(std::make_shared(SizeLimits(), false, 0, true)) + , set(std::make_shared(SizeLimits(), 0, true)) { Block header = getInMemoryMetadataPtr()->getSampleBlock(); set->setHeader(header.getColumnsWithTypeAndName()); @@ -176,7 +176,7 @@ void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_sn Block header = metadata_snapshot->getSampleBlock(); increment = 0; - set = std::make_shared(SizeLimits(), false, 0, true); + set = std::make_shared(SizeLimits(), 0, true); set->setHeader(header.getColumnsWithTypeAndName()); } diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 4cb08b90c2a..d361a4173c1 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -323,14 +323,8 @@ static void extractPathImpl(const ActionsDAG::Node & node, Paths & res, ContextP if (!future_set) return; - //if (!future_set->isReady()) - future_set->buildOrderedSetInplace(context); - - auto set = future_set->get(); - if (!set) - return; - - if (!set->hasExplicitSetElements()) + auto set = future_set->buildOrderedSetInplace(context); + if (!set || !set->hasExplicitSetElements()) return; set->checkColumnsNumber(1); diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index b83afe766aa..907fc0cd22c 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -89,25 +89,6 @@ ASTPtr buildWhereExpression(const ASTs & functions) return makeASTFunction("and", functions); } -// void buildSets(const ASTPtr & expression, ExpressionAnalyzer & analyzer) -// { -// const auto * func = expression->as(); -// if (func && functionIsInOrGlobalInOperator(func->name)) -// { -// const IAST & args = *func->arguments; -// const ASTPtr & arg = args.children.at(1); -// if (arg->as() || arg->as()) -// { -// analyzer.tryMakeSetForIndexFromSubquery(arg); -// } -// } -// else -// { -// for (const auto & child : expression->children) -// buildSets(child, analyzer); -// } -// } - } namespace VirtualColumnUtils @@ -208,7 +189,6 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex /// Let's analyze and calculate the prepared expression. auto syntax_result = TreeRewriter(context).analyze(expression_ast, block.getNamesAndTypesList()); ExpressionAnalyzer analyzer(expression_ast, syntax_result, context); - //buildSets(expression_ast, analyzer); ExpressionActionsPtr actions = analyzer.getActions(false /* add alises */, true /* project result */, CompileExpressions::yes); for (const auto & node : actions->getNodes()) diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index fa4730cbe84..1ee7d747fcc 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -1,3 +1,4 @@ + #include #include @@ -15,6 +16,8 @@ #include #include #include +#include +#include namespace DB { @@ -276,11 +279,14 @@ TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, temporary_table_expression_node->setTemporaryTableName(temporary_table_name); auto table_out = external_storage->write({}, external_storage->getInMemoryMetadataPtr(), mutable_context, /*async_insert=*/false); - auto io = interpreter.execute(); - io.pipeline.complete(std::move(table_out)); - CompletedPipelineExecutor executor(io.pipeline); - executor.execute(); + auto optimization_settings = QueryPlanOptimizationSettings::fromContext(mutable_context); + auto build_pipeline_settings = BuildQueryPipelineSettings::fromContext(mutable_context); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*query_plan.buildQueryPipeline(optimization_settings, build_pipeline_settings))); + + pipeline.complete(std::move(table_out)); + CompletedPipelineExecutor executor(pipeline); + executor.execute(); mutable_context->addExternalTable(temporary_table_name, std::move(external_storage_holder)); return temporary_table_expression_node; @@ -291,14 +297,13 @@ TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeNodePtr query_tree_to_modify) { auto & planner_context = query_info.planner_context; - const auto & query_context = planner_context->getQueryContext(); CollectColumnSourceToColumnsVisitor collect_column_source_to_columns_visitor; collect_column_source_to_columns_visitor.visit(query_tree_to_modify); const auto & column_source_to_columns = collect_column_source_to_columns_visitor.getColumnSourceToColumns(); - DistributedProductModeRewriteInJoinVisitor visitor(query_info.planner_context->getQueryContext()); + DistributedProductModeRewriteInJoinVisitor visitor(planner_context->getQueryContext()); visitor.visit(query_tree_to_modify); auto replacement_map = visitor.getReplacementMap(); From dd49f4a5651076136ed8deda33ac2f1f71ba8635 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 22 Jun 2023 17:07:52 +0200 Subject: [PATCH 221/515] Remove key --- src/Interpreters/Cache/Metadata.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index b19d095692d..75d37072711 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -53,17 +53,13 @@ KeyMetadata::KeyMetadata( const Key & key_, const std::string & key_path_, CleanupQueue & cleanup_queue_, - [[maybe_unused]] Poco::Logger * log_, + Poco::Logger * log_, bool created_base_directory_) : key(key_) , key_path(key_path_) , cleanup_queue(cleanup_queue_) , created_base_directory(created_base_directory_) -#ifdef ABORT_ON_LOGICAL_ERROR - , log(&Poco::Logger::get("Key(" + key.toString() + ")")) -#else , log(log_) -#endif { if (created_base_directory) chassert(fs::exists(key_path)); From 4b0d5a8a96c744912ddb514e779c40a560abb27c Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Thu, 22 Jun 2023 09:30:23 -0600 Subject: [PATCH 222/515] Update remote.md Simplifying this example for remoteSecure(): * no reason to be using a ReplicatedMergeTree example here * no reason to be setting index_granularity to a default value --- docs/en/sql-reference/table-functions/remote.md | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/table-functions/remote.md b/docs/en/sql-reference/table-functions/remote.md index bf0abd49fc6..8dcae6a0a79 100644 --- a/docs/en/sql-reference/table-functions/remote.md +++ b/docs/en/sql-reference/table-functions/remote.md @@ -114,9 +114,8 @@ This example uses one table from a sample dataset. The database is `imdb`, and `first_name` String, `last_name` String, `gender` FixedString(1)) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}') - ORDER BY (id, first_name, last_name, gender) - SETTINGS index_granularity = 8192 + ENGINE = MergeTree + ORDER BY (id, first_name, last_name, gender); ``` #### On the destination ClickHouse system: @@ -132,9 +131,8 @@ This example uses one table from a sample dataset. The database is `imdb`, and `first_name` String, `last_name` String, `gender` FixedString(1)) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}') - ORDER BY (id, first_name, last_name, gender) - SETTINGS index_granularity = 8192 + ENGINE = MergeTree + ORDER BY (id, first_name, last_name, gender); ``` #### Back on the source deployment: @@ -142,7 +140,7 @@ This example uses one table from a sample dataset. The database is `imdb`, and Insert into the new database and table created on the remote system. You will need the host, port, username, password, destination database, and destination table. ```sql INSERT INTO FUNCTION -remoteSecure('remote.clickhouse.cloud:9440', 'imdb.actors', 'USER', 'PASSWORD', rand()) +remoteSecure('remote.clickhouse.cloud:9440', 'imdb.actors', 'USER', 'PASSWORD') SELECT * from imdb.actors ``` From a89bfb9ec507c08c732cb1088915914a42485e41 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 22 Jun 2023 08:59:44 -0700 Subject: [PATCH 223/515] Revert back to value type as requested --- src/IO/WriteHelpers.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 2ee1e1651f6..3d1a5aa49ef 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -1183,13 +1183,13 @@ inline void writeBinaryEndian(T x, WriteBuffer & buf) } template -inline void writeBinaryLittleEndian(const T & x, WriteBuffer & buf) +inline void writeBinaryLittleEndian(T x, WriteBuffer & buf) { writeBinaryEndian(x, buf); } template -inline void writeBinaryBigEndian(const T & x, WriteBuffer & buf) +inline void writeBinaryBigEndian(T x, WriteBuffer & buf) { writeBinaryEndian(x, buf); } From 2498170253f91f7406452b99ef1dd168c8a9ec2d Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 21 Jun 2023 23:46:12 +0000 Subject: [PATCH 224/515] Fix use-after-free in StorageURL when switching URLs --- src/Formats/FormatFactory.h | 1 + src/Storages/StorageURL.cpp | 2 ++ 2 files changed, 3 insertions(+) diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 1d258beca8d..489db944ee6 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -153,6 +153,7 @@ public: /// * Parallel reading. /// To enable it, make sure `buf` is a SeekableReadBuffer implementing readBigAt(). /// * Parallel parsing. + /// `buf` must outlive the returned IInputFormat. InputFormatPtr getInput( const String & name, ReadBuffer & buf, diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 1d6aed204cb..aa574ef11be 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -361,6 +361,8 @@ Chunk StorageURLSource::generate() pipeline->reset(); reader.reset(); + input_format.reset(); + read_buf.reset(); } return {}; } From 0823d610a398ffe435e3b3d163e691f72eca89f5 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 22 Jun 2023 00:57:50 +0000 Subject: [PATCH 225/515] A test --- .../0_stateless/02790_url_multiple_tsv_files.reference | 5 +++++ tests/queries/0_stateless/02790_url_multiple_tsv_files.sql | 5 +++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/02790_url_multiple_tsv_files.reference create mode 100644 tests/queries/0_stateless/02790_url_multiple_tsv_files.sql diff --git a/tests/queries/0_stateless/02790_url_multiple_tsv_files.reference b/tests/queries/0_stateless/02790_url_multiple_tsv_files.reference new file mode 100644 index 00000000000..927a33afecf --- /dev/null +++ b/tests/queries/0_stateless/02790_url_multiple_tsv_files.reference @@ -0,0 +1,5 @@ +136 +136 +136 +136 +136 diff --git a/tests/queries/0_stateless/02790_url_multiple_tsv_files.sql b/tests/queries/0_stateless/02790_url_multiple_tsv_files.sql new file mode 100644 index 00000000000..b71bd7c737a --- /dev/null +++ b/tests/queries/0_stateless/02790_url_multiple_tsv_files.sql @@ -0,0 +1,5 @@ +select sum(*) from (select * from url('http://127.0.0.1:8123?query=select+{1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16}+as+x+format+TSV', 'TSV') settings max_threads=1, max_download_threads=1); +select sum(*) from (select * from url('http://127.0.0.1:8123?query=select+{1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16}+as+x+format+CSV', 'CSV') settings max_threads=1, max_download_threads=1); +select sum(*) from (select * from url('http://127.0.0.1:8123?query=select+{1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16}+as+x+format+JSONEachRow', 'JSONEachRow') settings max_threads=1, max_download_threads=1); +select sum(*) from (select * from url('http://127.0.0.1:8123?query=select+{1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16}+as+x+format+TSKV', 'TSKV') settings max_threads=1, max_download_threads=1); +select sum(*) from (select * from url('http://127.0.0.1:8123?query=select+{1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16}+as+x+format+Native', 'Native') settings max_threads=1, max_download_threads=1); From 456709488ead67d890d965fb04f10e5fcf4fa307 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Thu, 22 Jun 2023 20:03:36 +0300 Subject: [PATCH 226/515] Update ext-dict-functions.md --- docs/ru/sql-reference/functions/ext-dict-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/ext-dict-functions.md b/docs/ru/sql-reference/functions/ext-dict-functions.md index e6cb878d1c7..d14f0ddf027 100644 --- a/docs/ru/sql-reference/functions/ext-dict-functions.md +++ b/docs/ru/sql-reference/functions/ext-dict-functions.md @@ -5,7 +5,7 @@ sidebar_label: "Функции для работы с внешними слов --- :::note "Внимание" - Для словарей, созданных с помощью [DDL-запросов](../../sql-reference/statements/create/dictionary.md), в параметре `dict_name` указывается полное имя словаря вместе с базой данных, например: `.`. Если база данных не указана, используется текущая. +Для словарей, созданных с помощью [DDL-запросов](../../sql-reference/statements/create/dictionary.md), в параметре `dict_name` указывается полное имя словаря вместе с базой данных, например: `.`. Если база данных не указана, используется текущая. ::: # Функции для работы с внешними словарями {#ext_dict_functions} From 8afb8bf13afd654a6706b69f58da83b6096770db Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 22 Jun 2023 19:06:28 +0200 Subject: [PATCH 227/515] disable table structure check for secondary queries from Replicated db --- src/Databases/DatabaseReplicated.cpp | 4 +- src/Databases/DatabaseReplicatedWorker.cpp | 1 + .../MergeTree/registerStorageMergeTree.cpp | 8 ++- src/Storages/StorageReplicatedMergeTree.cpp | 39 ++++++++++----- src/Storages/StorageReplicatedMergeTree.h | 5 +- .../test_replicated_database/test.py | 49 +++++++++++++++++++ 6 files changed, 90 insertions(+), 16 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index e468e533818..de40ee4d82d 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -985,7 +985,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep const auto & create_query_string = metadata_it->second; if (isTableExist(table_name, getContext())) { - assert(create_query_string == readMetadataFile(table_name)); + assert(create_query_string == readMetadataFile(table_name) || getTableUUIDIfReplicated(create_query_string, getContext()) != UUIDHelpers::Nil); continue; } @@ -1274,7 +1274,7 @@ void DatabaseReplicated::commitAlterTable(const StorageID & table_id, const String & statement, ContextPtr query_context) { auto txn = query_context->getZooKeeperMetadataTransaction(); - assert(!ddl_worker->isCurrentlyActive() || txn); + assert(!ddl_worker || !ddl_worker->isCurrentlyActive() || txn); if (txn && txn->isInitialQuery()) { String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(table_id.table_name); diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 66ae5cd250c..ff2675dfd6b 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -91,6 +91,7 @@ void DatabaseReplicatedDDLWorker::initializeReplication() if (zookeeper->tryGet(database->replica_path + "/digest", digest_str)) { digest = parse(digest_str); + LOG_TRACE(log, "Metadata digest in ZooKeeper: {}", digest); std::lock_guard lock{database->metadata_mutex}; local_digest = database->tables_metadata_digest; } diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 027cd1af7c9..75f1542e30e 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -23,6 +23,7 @@ #include #include #include +#include namespace DB @@ -684,6 +685,10 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (replicated) { + bool need_check_table_structure = true; + if (auto txn = args.getLocalContext()->getZooKeeperMetadataTransaction()) + need_check_table_structure = txn->isInitialQuery(); + return std::make_shared( zookeeper_path, replica_name, @@ -696,7 +701,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) merging_params, std::move(storage_settings), args.has_force_restore_data_flag, - renaming_restrictions); + renaming_restrictions, + need_check_table_structure); } else return std::make_shared( diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index bb99e21e4ab..8f34fca85c5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -292,7 +292,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( const MergingParams & merging_params_, std::unique_ptr settings_, bool has_force_restore_data_flag, - RenamingRestrictions renaming_restrictions_) + RenamingRestrictions renaming_restrictions_, + bool need_check_structure) : MergeTreeData(table_id_, metadata_, context_, @@ -492,11 +493,17 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( /// information in /replica/metadata. other_replicas_fixed_granularity = checkFixedGranularityInZookeeper(); - checkTableStructure(zookeeper_path, metadata_snapshot); + /// Allow structure mismatch for secondary queries from Replicated database. + /// It may happen if the table was altered just after creation. + /// Metadata will be updated in cloneMetadataIfNeeded(...), metadata_version will be 0 for a while. + bool same_structure = checkTableStructure(zookeeper_path, metadata_snapshot, need_check_structure); - Coordination::Stat metadata_stat; - current_zookeeper->get(zookeeper_path + "/metadata", &metadata_stat); - setInMemoryMetadata(metadata_snapshot->withMetadataVersion(metadata_stat.version)); + if (same_structure) + { + Coordination::Stat metadata_stat; + current_zookeeper->get(zookeeper_path + "/metadata", &metadata_stat); + setInMemoryMetadata(metadata_snapshot->withMetadataVersion(metadata_stat.version)); + } } catch (Coordination::Exception & e) { @@ -1186,7 +1193,7 @@ bool StorageReplicatedMergeTree::removeTableNodesFromZooKeeper(zkutil::ZooKeeper /** Verify that list of columns and table storage_settings_ptr match those specified in ZK (/metadata). * If not, throw an exception. */ -void StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot) +bool StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot, bool strict_check) { auto zookeeper = getZooKeeper(); @@ -1201,12 +1208,20 @@ void StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_pr auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_prefix) / "columns", &columns_stat)); const ColumnsDescription & old_columns = metadata_snapshot->getColumns(); - if (columns_from_zk != old_columns) + if (columns_from_zk == old_columns) + return true; + + if (!strict_check && metadata_stat.version != 0) { - throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, - "Table columns structure in ZooKeeper is different from local table structure. Local columns:\n" - "{}\nZookeeper columns:\n{}", old_columns.toString(), columns_from_zk.toString()); + LOG_WARNING(log, "Table columns structure in ZooKeeper is different from local table structure. " + "Assuming it's because the table was altered concurrently. Metadata version: {}. Local columns:\n" + "{}\nZookeeper columns:\n{}", metadata_stat.version, old_columns.toString(), columns_from_zk.toString()); + return false; } + + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, + "Table columns structure in ZooKeeper is different from local table structure. Local columns:\n" + "{}\nZookeeper columns:\n{}", old_columns.toString(), columns_from_zk.toString()); } void StorageReplicatedMergeTree::setTableStructure(const StorageID & table_id, const ContextPtr & local_context, @@ -2993,7 +3008,9 @@ void StorageReplicatedMergeTree::cloneMetadataIfNeeded(const String & source_rep dummy_alter.alter_version = source_metadata_version; dummy_alter.create_time = time(nullptr); - zookeeper->create(replica_path + "/queue/queue-", dummy_alter.toString(), zkutil::CreateMode::PersistentSequential); + String path_created = zookeeper->create(replica_path + "/queue/queue-", dummy_alter.toString(), zkutil::CreateMode::PersistentSequential); + LOG_INFO(log, "Created an ALTER_METADATA entry {} to force metadata update after cloning replica from {}. Entry: {}", + path_created, source_replica, dummy_alter.toString()); /// We don't need to do anything with mutation_pointer, because mutation log cleanup process is different from /// replication log cleanup. A mutation is removed from ZooKeeper only if all replicas had executed the mutation, diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index c08e05090b1..bdd3f0da5bf 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -108,7 +108,8 @@ public: const MergingParams & merging_params_, std::unique_ptr settings_, bool has_force_restore_data_flag, - RenamingRestrictions renaming_restrictions_); + RenamingRestrictions renaming_restrictions_, + bool need_check_structure); void startup() override; void shutdown() override; @@ -529,7 +530,7 @@ private: */ void createNewZooKeeperNodes(); - void checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot); + bool checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot, bool strict_check = true); /// A part of ALTER: apply metadata changes only (data parts are altered separately). /// Must be called under IStorage::lockForAlter() lock. diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 2ab2fe499ff..8d4244b69b5 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -34,6 +34,7 @@ competing_node = cluster.add_instance( main_configs=["configs/config.xml"], user_configs=["configs/settings.xml"], with_zookeeper=True, + stay_alive=True, macros={"shard": 1, "replica": 3}, ) snapshotting_node = cluster.add_instance( @@ -1272,3 +1273,51 @@ def test_recover_digest_mismatch(started_cluster): dummy_node.query("DROP DATABASE IF EXISTS recover_digest_mismatch") print("Everything Okay") + +def test_replicated_table_structure_alter(started_cluster): + main_node.query("DROP DATABASE IF EXISTS table_structure") + dummy_node.query("DROP DATABASE IF EXISTS table_structure") + + main_node.query( + "CREATE DATABASE table_structure ENGINE = Replicated('/clickhouse/databases/table_structure', 'shard1', 'replica1');" + ) + dummy_node.query( + "CREATE DATABASE table_structure ENGINE = Replicated('/clickhouse/databases/table_structure', 'shard1', 'replica2');" + ) + competing_node.query( + "CREATE DATABASE table_structure ENGINE = Replicated('/clickhouse/databases/table_structure', 'shard1', 'replica3');" + ) + + competing_node.query("CREATE TABLE table_structure.mem (n int) ENGINE=Memory") + dummy_node.query("DETACH DATABASE table_structure") + + settings = {"distributed_ddl_task_timeout": 0} + main_node.query("CREATE TABLE table_structure.rmt (n int, v UInt64) ENGINE=ReplicatedReplacingMergeTree(v) ORDER BY n", settings=settings) + + competing_node.query("SYSTEM SYNC DATABASE REPLICA table_structure") + competing_node.query("DETACH DATABASE table_structure") + + main_node.query("ALTER TABLE table_structure.rmt ADD COLUMN m int", settings=settings) + main_node.query("ALTER TABLE table_structure.rmt COMMENT COLUMN v 'version'", settings=settings) + main_node.query("INSERT INTO table_structure.rmt VALUES (1, 2, 3)") + + command = "rm -f /var/lib/clickhouse/metadata/table_structure/mem.sql" + competing_node.exec_in_container(["bash", "-c", command]) + competing_node.restart_clickhouse(kill=True) + + dummy_node.query("ATTACH DATABASE table_structure") + dummy_node.query("SYSTEM SYNC DATABASE REPLICA table_structure") + dummy_node.query("SYSTEM SYNC REPLICA table_structure.rmt") + assert "1\t2\t3\n" == dummy_node.query("SELECT * FROM table_structure.rmt") + + competing_node.query("SYSTEM SYNC DATABASE REPLICA table_structure") + competing_node.query("SYSTEM SYNC REPLICA table_structure.rmt") + #time.sleep(600) + assert "mem" in competing_node.query("SHOW TABLES FROM table_structure") + assert "1\t2\t3\n" == competing_node.query("SELECT * FROM table_structure.rmt") + + main_node.query("ALTER TABLE table_structure.rmt ADD COLUMN k int") + main_node.query("INSERT INTO table_structure.rmt VALUES (1, 2, 3, 4)") + dummy_node.query("SYSTEM SYNC DATABASE REPLICA table_structure") + dummy_node.query("SYSTEM SYNC REPLICA table_structure.rmt") + assert "1\t2\t3\t0\n1\t2\t3\t4\n" == dummy_node.query("SELECT * FROM table_structure.rmt ORDER BY k") From 376b65ee1efeeec438d52750e57182aef7379cad Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 22 Jun 2023 19:06:49 +0200 Subject: [PATCH 228/515] Define Thrift version for parquet --- contrib/arrow-cmake/CMakeLists.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 5fe942d1cd0..836452c53ed 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -514,6 +514,10 @@ if (SANITIZE STREQUAL "undefined") target_compile_options(_arrow PRIVATE -fno-sanitize=undefined) endif () +# Define Thrift version for parquet (we use 0.16.0) +add_definitions(-DPARQUET_THRIFT_VERSION_MAJOR=0) +add_definitions(-DPARQUET_THRIFT_VERSION_MINOR=16) + # === tools set(TOOLS_DIR "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/tools/parquet") From 1d379108a314abc9744069c4e697b9e87818fa7b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 22 Jun 2023 19:23:14 +0200 Subject: [PATCH 229/515] Update arrow version in cmake --- contrib/arrow-cmake/CMakeLists.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 836452c53ed..46b86cb4ddb 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -31,12 +31,12 @@ endif() set (CMAKE_CXX_STANDARD 17) -set(ARROW_VERSION "6.0.1") +set(ARROW_VERSION "11.0.0") string(REGEX MATCH "^[0-9]+\\.[0-9]+\\.[0-9]+" ARROW_BASE_VERSION "${ARROW_VERSION}") -set(ARROW_VERSION_MAJOR "6") +set(ARROW_VERSION_MAJOR "11") set(ARROW_VERSION_MINOR "0") -set(ARROW_VERSION_PATCH "1") +set(ARROW_VERSION_PATCH "0") if(ARROW_VERSION_MAJOR STREQUAL "0") # Arrow 0.x.y => SO version is "x", full SO version is "x.y.0" From ccb42d0afa202aba6b4a8459bae971afa87a67dd Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 22 Jun 2023 17:26:42 +0000 Subject: [PATCH 230/515] Automatic style fix --- .../test_replicated_database/test.py | 20 ++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 8d4244b69b5..b3ba8d4737f 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1274,6 +1274,7 @@ def test_recover_digest_mismatch(started_cluster): print("Everything Okay") + def test_replicated_table_structure_alter(started_cluster): main_node.query("DROP DATABASE IF EXISTS table_structure") dummy_node.query("DROP DATABASE IF EXISTS table_structure") @@ -1292,13 +1293,20 @@ def test_replicated_table_structure_alter(started_cluster): dummy_node.query("DETACH DATABASE table_structure") settings = {"distributed_ddl_task_timeout": 0} - main_node.query("CREATE TABLE table_structure.rmt (n int, v UInt64) ENGINE=ReplicatedReplacingMergeTree(v) ORDER BY n", settings=settings) + main_node.query( + "CREATE TABLE table_structure.rmt (n int, v UInt64) ENGINE=ReplicatedReplacingMergeTree(v) ORDER BY n", + settings=settings, + ) competing_node.query("SYSTEM SYNC DATABASE REPLICA table_structure") competing_node.query("DETACH DATABASE table_structure") - main_node.query("ALTER TABLE table_structure.rmt ADD COLUMN m int", settings=settings) - main_node.query("ALTER TABLE table_structure.rmt COMMENT COLUMN v 'version'", settings=settings) + main_node.query( + "ALTER TABLE table_structure.rmt ADD COLUMN m int", settings=settings + ) + main_node.query( + "ALTER TABLE table_structure.rmt COMMENT COLUMN v 'version'", settings=settings + ) main_node.query("INSERT INTO table_structure.rmt VALUES (1, 2, 3)") command = "rm -f /var/lib/clickhouse/metadata/table_structure/mem.sql" @@ -1312,7 +1320,7 @@ def test_replicated_table_structure_alter(started_cluster): competing_node.query("SYSTEM SYNC DATABASE REPLICA table_structure") competing_node.query("SYSTEM SYNC REPLICA table_structure.rmt") - #time.sleep(600) + # time.sleep(600) assert "mem" in competing_node.query("SHOW TABLES FROM table_structure") assert "1\t2\t3\n" == competing_node.query("SELECT * FROM table_structure.rmt") @@ -1320,4 +1328,6 @@ def test_replicated_table_structure_alter(started_cluster): main_node.query("INSERT INTO table_structure.rmt VALUES (1, 2, 3, 4)") dummy_node.query("SYSTEM SYNC DATABASE REPLICA table_structure") dummy_node.query("SYSTEM SYNC REPLICA table_structure.rmt") - assert "1\t2\t3\t0\n1\t2\t3\t4\n" == dummy_node.query("SELECT * FROM table_structure.rmt ORDER BY k") + assert "1\t2\t3\t0\n1\t2\t3\t4\n" == dummy_node.query( + "SELECT * FROM table_structure.rmt ORDER BY k" + ) From 3d385be9cb9236cd59baead6e4ad83d34316e357 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Thu, 22 Jun 2023 20:56:50 +0300 Subject: [PATCH 231/515] Update Settings.h --- 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 d01caeda344..3d42bd582ed 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -770,7 +770,7 @@ class IColumn; M(Bool, allow_experimental_undrop_table_query, false, "Allow to use undrop query to restore dropped table in a limited time", 0) \ M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ - M(Timezone, session_timezone, "", "The default timezone for current session or query. The server default timezone, if empty.", 0) \ + M(Timezone, session_timezone, "", "The default timezone for current session or query. The server default timezone if empty.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. From ba4b5c335047f72bbed063434b5f4b7a38459fe6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Jun 2023 18:15:16 +0000 Subject: [PATCH 232/515] Fix stupid bug. --- src/Interpreters/PreparedSets.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 428ef873bc5..67822ecf440 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -230,12 +230,11 @@ FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Set const auto & set_types = from_tuple->getTypes(); auto & sets_by_hash = sets_from_tuple[key]; - auto types = from_tuple->getTypes(); for (const auto & set : sets_by_hash) if (equals(set->getTypes(), set_types)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, from_tuple->getTypes())); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, set_types)); - sets_by_hash.push_back(std::move(from_tuple)); + sets_by_hash.push_back(from_tuple); return from_tuple; } From 75391afbd82b3c2aa5caae8fed82324334a49bbb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Jun 2023 18:16:48 +0000 Subject: [PATCH 233/515] Fix typo. --- src/Interpreters/ActionsVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index bf78868463a..cfbe53b5e4d 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1406,7 +1406,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool /// /// Mutation is executed in two stages: /// * first, query 'SELECT count() FROM table WHERE ...' is executed to get the set of affected parts (using analyzer) - /// * second, every part is mutated separately, where plan is build "manually", usign this code as well + /// * second, every part is mutated separately, where plan is build "manually", using this code as well /// To share the Set in between first and second stage, we should use the same hash. /// New analyzer is uses a hash from query tree, so here we also build a query tree. /// From 369ad0aea397612a823f28b7aeb3ac72583066f8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Jun 2023 20:32:29 +0200 Subject: [PATCH 234/515] Remove ALTER of LIVE VIEW --- src/Interpreters/InterpreterAlterQuery.cpp | 24 +------- src/Storages/LiveView/LiveViewCommands.h | 65 ---------------------- 2 files changed, 1 insertion(+), 88 deletions(-) delete mode 100644 src/Storages/LiveView/LiveViewCommands.h diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index ec2145b38bf..e82415f1aca 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -18,8 +18,6 @@ #include #include #include -#include -#include #include #include #include @@ -117,7 +115,6 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) AlterCommands alter_commands; PartitionCommands partition_commands; MutationCommands mutation_commands; - LiveViewCommands live_view_commands; for (const auto & child : alter.command_list->children) { auto * command_ast = child->as(); @@ -137,17 +134,13 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) mutation_commands.emplace_back(std::move(*mut_command)); } - else if (auto live_view_command = LiveViewCommand::parse(command_ast)) - { - live_view_commands.emplace_back(std::move(*live_view_command)); - } else throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong parameter type in ALTER query"); } if (typeid_cast(database.get())) { - int command_types_count = !mutation_commands.empty() + !partition_commands.empty() + !live_view_commands.empty() + !alter_commands.empty(); + int command_types_count = !mutation_commands.empty() + !partition_commands.empty() + !alter_commands.empty(); bool mixed_settings_amd_metadata_alter = alter_commands.hasSettingsAlterCommand() && !alter_commands.isSettingsAlter(); if (1 < command_types_count || mixed_settings_amd_metadata_alter) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "For Replicated databases it's not allowed " @@ -170,21 +163,6 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) res.pipeline = QueryPipeline(std::move(partition_commands_pipe)); } - if (!live_view_commands.empty()) - { - live_view_commands.validate(*table); - for (const LiveViewCommand & command : live_view_commands) - { - auto live_view = std::dynamic_pointer_cast(table); - switch (command.type) - { - case LiveViewCommand::REFRESH: - live_view->refresh(); - break; - } - } - } - if (!alter_commands.empty()) { auto alter_lock = table->lockForAlter(getContext()->getSettingsRef().lock_acquire_timeout); diff --git a/src/Storages/LiveView/LiveViewCommands.h b/src/Storages/LiveView/LiveViewCommands.h deleted file mode 100644 index 2bb2dfb2752..00000000000 --- a/src/Storages/LiveView/LiveViewCommands.h +++ /dev/null @@ -1,65 +0,0 @@ -#pragma once -/* Copyright (c) 2018 BlackBerry Limited - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at -http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. */ - -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int UNKNOWN_STORAGE; -} - -struct LiveViewCommand -{ - enum Type - { - REFRESH - }; - - Type type; - - ASTPtr values; - - static LiveViewCommand refresh(const ASTPtr & values) - { - LiveViewCommand res; - res.type = REFRESH; - res.values = values; - return res; - } - - static std::optional parse(ASTAlterCommand * command) - { - if (command->type == ASTAlterCommand::LIVE_VIEW_REFRESH) - return refresh(command->values); - return {}; - } -}; - - -class LiveViewCommands : public std::vector -{ -public: - void validate(const IStorage & table) - { - if (!empty() && !dynamic_cast(&table)) - throw Exception(DB::ErrorCodes::UNKNOWN_STORAGE, "Wrong storage type. Must be StorageLiveView"); - } -}; - -} From 2b01711565e919baa39cd6e68ca42481cea6fdb6 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 22 Jun 2023 19:59:33 +0200 Subject: [PATCH 235/515] fix assert in test, revert debug message --- src/IO/Lz4DeflatingWriteBuffer.cpp | 10 ++-------- .../test_checking_s3_blobs_paranoid/test.py | 4 ++-- 2 files changed, 4 insertions(+), 10 deletions(-) diff --git a/src/IO/Lz4DeflatingWriteBuffer.cpp b/src/IO/Lz4DeflatingWriteBuffer.cpp index 35feafbc736..27c945f92cf 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.cpp +++ b/src/IO/Lz4DeflatingWriteBuffer.cpp @@ -104,14 +104,8 @@ void Lz4DeflatingWriteBuffer::nextImpl() if (LZ4F_isError(compressed_size)) throw Exception( ErrorCodes::LZ4_ENCODER_FAILED, - "LZ4 failed to encode stream. LZ4F version: {}, CodeName: {}," - " in_capacity: {}, out_capacity: {}, cur_buffer_size: {}, min_compressed_block_size: {}", - LZ4F_VERSION, - LZ4F_getErrorName(compressed_size), - in_capacity, - out_capacity, - cur_buffer_size, - min_compressed_block_size); + "LZ4 failed to encode stream. LZ4F version: {}", + LZ4F_VERSION); in_capacity -= cur_buffer_size; in_data = reinterpret_cast(working_buffer.end() - in_capacity); diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 244ca8a2c81..43a313f07dc 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -165,5 +165,5 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( assert count_create_multi_part_uploads == 1 assert count_upload_parts >= 2 assert ( - count_s3_errors == 2 - ) # the second is cancel multipart upload, s3_mock just redirects this request + count_s3_errors >= 2 + ) From f226397fa35bc7637a1a2a078018e760ecb09d6b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Jun 2023 19:21:08 +0000 Subject: [PATCH 236/515] Fix check. --- src/Interpreters/Set.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index b42ff102f72..b8b61c7c11f 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -173,9 +173,6 @@ void Set::setHeader(const ColumnsWithTypeAndName & header) void Set::fillSetElements() { - if (data.getTotalRowCount()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert set to ordered because it is not empty"); - fill_set_elements = true; set_elements.reserve(keys_size); for (const auto & type : set_elements_types) From 506d52358a1262932df6d41f4dc97503c697038f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 22 Jun 2023 20:52:50 +0000 Subject: [PATCH 237/515] Query Cache: Remove confusing defaults in server cfg --- programs/server/config.xml | 8 -------- 1 file changed, 8 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index d18b4cb2ac9..50db5fc4af6 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1542,14 +1542,6 @@ --> - - - - - - - - + + + 1073741824 + 1024 + 1048576 + 30000000 + + + RejectCertificateHandler + @@ -25,12 +29,9 @@ true sslv2,sslv3 true - - RejectCertificateHandler - \ No newline at end of file From c4ea7ab5b15add3d3b69412224f9c53e30be1661 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 09:31:09 +0200 Subject: [PATCH 313/515] Attempt to fix test_ssl_cert_authentication --- programs/server/config.xml | 7 +++++++ .../test_ssl_cert_authentication/configs/ssl_config.xml | 4 ---- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 2f69f23a718..acd6d92a896 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -282,6 +282,13 @@ true sslv2,sslv3 true + + + + RejectCertificateHandler + diff --git a/tests/integration/test_ssl_cert_authentication/configs/ssl_config.xml b/tests/integration/test_ssl_cert_authentication/configs/ssl_config.xml index d0b58d984bf..24c9eb8891f 100644 --- a/tests/integration/test_ssl_cert_authentication/configs/ssl_config.xml +++ b/tests/integration/test_ssl_cert_authentication/configs/ssl_config.xml @@ -18,10 +18,6 @@ /etc/clickhouse-server/config.d/server-key.pem /etc/clickhouse-server/config.d/ca-cert.pem relaxed - - - RejectCertificateHandler - From 5871ca0836e9a21790f6bf3cff2d3134523f3a08 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 24 Jun 2023 20:38:03 +0200 Subject: [PATCH 314/515] Fix performance tests due to warnings from jemalloc about Per-CPU arena disabled jemalloc can show the following warning: Number of CPUs detected is not deterministic. Per-CPU arena disabled It will be shown if one of the following returns different number of CPUs: - _SC_NPROCESSORS_ONLN - _SC_NPROCESSORS_CONF - sched_getaffinity() And actually for my CPU linux returns different numbers, because there are more possible CPUs then online, from dmesg: smpboot: Allowing 128 CPUs, 64 hotplug CPUs And from sysfs: # grep . /sys/devices/system/cpu/{possible,online,offline} /sys/devices/system/cpu/possible:0-127 /sys/devices/system/cpu/online:0-63 /sys/devices/system/cpu/offline:64-127 From ACPI: # acpidump -o acpi # acpixtract -a acpi # iasl -d *.dat # grep -e 'Processor Enabled' apic.dsl | sort | uniq -c 64 Processor Enabled : 0 64 Processor Enabled : 1 So I guess this is the same as what happened in this perf run [1]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/51360/5d43a64112711b339b82b1c0e8df7882546a1a3c/performance_comparison_[4_4]/report.html P.S. personally I, just use cmdline=possible_cpus=64 to fix this for my setup. Signed-off-by: Azat Khuzhin --- docker/test/performance-comparison/compare.sh | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 293ad9ac411..798d2a40b12 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -14,6 +14,13 @@ LEFT_SERVER_PORT=9001 # patched version RIGHT_SERVER_PORT=9002 +# abort_conf -- abort if some options is not recognized +# abort -- abort if something is not right in the env (i.e. per-cpu arenas does not work) +# narenas -- set them explicitly to avoid disabling per-cpu arena in env +# that returns different number of CPUs for some of the following +# _SC_NPROCESSORS_ONLN/_SC_NPROCESSORS_CONF/sched_getaffinity +export MALLOC_CONF="abort_conf:true,abort:true,narenas:$(nproc --all)" + function wait_for_server # port, pid { for _ in {1..60} @@ -109,10 +116,6 @@ function restart while pkill -f clickhouse-serv ; do echo . ; sleep 1 ; done echo all killed - # Change the jemalloc settings here. - # https://github.com/jemalloc/jemalloc/wiki/Getting-Started - export MALLOC_CONF="confirm_conf:true" - set -m # Spawn servers in their own process groups local left_server_opts=( @@ -147,8 +150,6 @@ function restart set +m - unset MALLOC_CONF - wait_for_server $LEFT_SERVER_PORT $left_pid echo left ok From cc3d27c0a30e40532541324b100f2584327b5ba3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 25 Jun 2023 15:14:29 +0000 Subject: [PATCH 315/515] clang-tidy fix --- src/Common/Exception.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index a43335ed8de..af48ce8fd99 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -164,7 +164,7 @@ std::string Exception::getStackTraceString() const { thread_stack_trace += "\nJob's origin stack trace:\n" + - StackTrace::toString(&frame_pointers[0], 0, std::ranges::find(frame_pointers, nullptr) - frame_pointers.begin()); + StackTrace::toString(frame_pointers.data(), 0, std::ranges::find(frame_pointers, nullptr) - frame_pointers.begin()); } ); From f1f0daa654755b2d12ec9548262adfe4e87fe9b6 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 15 Jun 2023 17:59:37 +0200 Subject: [PATCH 316/515] Show halves of checksums in "system.parts", "system.projection_parts" and error messages in the correct order. --- src/Compression/CompressedReadBufferBase.cpp | 4 ++-- src/Storages/Distributed/DistributedAsyncInsertHeader.cpp | 4 ++-- src/Storages/MergeTree/PartMetadataManagerWithCache.cpp | 8 ++++---- src/Storages/System/StorageSystemParts.cpp | 6 +++--- src/Storages/System/StorageSystemProjectionParts.cpp | 6 +++--- utils/checksum-for-compressed-block/main.cpp | 2 +- 6 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 662cd6bf337..bae52c8bece 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -49,8 +49,8 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c /// TODO mess up of endianness in error message. message << "Checksum doesn't match: corrupted data." - " Reference: " + getHexUIntLowercase(expected_checksum.low64) + getHexUIntLowercase(expected_checksum.high64) - + ". Actual: " + getHexUIntLowercase(calculated_checksum.low64) + getHexUIntLowercase(calculated_checksum.high64) + " Reference: " + getHexUIntLowercase(expected_checksum.high64) + getHexUIntLowercase(expected_checksum.low64) + + ". Actual: " + getHexUIntLowercase(calculated_checksum.high64) + getHexUIntLowercase(calculated_checksum.low64) + ". Size of compressed block: " + toString(size); const char * message_hardware_failure = "This is most likely due to hardware failure. " diff --git a/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp b/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp index d815f671652..e1b54304f23 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp @@ -40,8 +40,8 @@ DistributedAsyncInsertHeader DistributedAsyncInsertHeader::read(ReadBufferFromFi { throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, "Checksum of extra info doesn't match: corrupted data. Reference: {}{}. Actual: {}{}.", - getHexUIntLowercase(expected_checksum.low64), getHexUIntLowercase(expected_checksum.high64), - getHexUIntLowercase(calculated_checksum.low64), getHexUIntLowercase(calculated_checksum.high64)); + getHexUIntLowercase(expected_checksum.high64), getHexUIntLowercase(expected_checksum.low64), + getHexUIntLowercase(calculated_checksum.high64), getHexUIntLowercase(calculated_checksum.low64)); } /// Read the parts of the header. diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp index 7deae69750f..324bd4bbaee 100644 --- a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp @@ -250,8 +250,8 @@ std::unordered_map PartMetadataManagerWit ErrorCodes::CORRUPTED_DATA, "Checksums doesn't match in part {} for {}. Expected: {}. Found {}.", part->name, file_path, - getHexUIntUppercase(disk_checksum.low64) + getHexUIntUppercase(disk_checksum.high64), - getHexUIntUppercase(cache_checksums[i].low64) + getHexUIntUppercase(cache_checksums[i].high64)); + getHexUIntUppercase(disk_checksum.high64) + getHexUIntUppercase(disk_checksum.low64), + getHexUIntUppercase(cache_checksums[i].high64) + getHexUIntUppercase(cache_checksums[i].low64)); disk_checksums.push_back(disk_checksum); continue; @@ -287,8 +287,8 @@ std::unordered_map PartMetadataManagerWit ErrorCodes::CORRUPTED_DATA, "Checksums doesn't match in projection part {} {}. Expected: {}. Found {}.", part->name, proj_name, - getHexUIntUppercase(disk_checksum.low64) + getHexUIntUppercase(disk_checksum.high64), - getHexUIntUppercase(cache_checksums[i].low64) + getHexUIntUppercase(cache_checksums[i].high64)); + getHexUIntUppercase(disk_checksum.high64) + getHexUIntUppercase(disk_checksum.low64), + getHexUIntUppercase(cache_checksums[i].high64) + getHexUIntUppercase(cache_checksums[i].low64)); disk_checksums.push_back(disk_checksum); } return results; diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 95bad0a20fe..b642f4b5088 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -252,17 +252,17 @@ void StorageSystemParts::processNextStorage( if (columns_mask[src_index++]) { auto checksum = helper.hash_of_all_files; - columns[res_index++]->insert(getHexUIntLowercase(checksum.low64) + getHexUIntLowercase(checksum.high64)); + columns[res_index++]->insert(getHexUIntLowercase(checksum.high64) + getHexUIntLowercase(checksum.low64)); } if (columns_mask[src_index++]) { auto checksum = helper.hash_of_uncompressed_files; - columns[res_index++]->insert(getHexUIntLowercase(checksum.low64) + getHexUIntLowercase(checksum.high64)); + columns[res_index++]->insert(getHexUIntLowercase(checksum.high64) + getHexUIntLowercase(checksum.low64)); } if (columns_mask[src_index++]) { auto checksum = helper.uncompressed_hash_of_compressed_files; - columns[res_index++]->insert(getHexUIntLowercase(checksum.low64) + getHexUIntLowercase(checksum.high64)); + columns[res_index++]->insert(getHexUIntLowercase(checksum.high64) + getHexUIntLowercase(checksum.low64)); } } diff --git a/src/Storages/System/StorageSystemProjectionParts.cpp b/src/Storages/System/StorageSystemProjectionParts.cpp index 6508d062d37..05c83747c4d 100644 --- a/src/Storages/System/StorageSystemProjectionParts.cpp +++ b/src/Storages/System/StorageSystemProjectionParts.cpp @@ -221,17 +221,17 @@ void StorageSystemProjectionParts::processNextStorage( if (columns_mask[src_index++]) { auto checksum = helper.hash_of_all_files; - columns[res_index++]->insert(getHexUIntLowercase(checksum.low64) + getHexUIntLowercase(checksum.high64)); + columns[res_index++]->insert(getHexUIntLowercase(checksum.high64) + getHexUIntLowercase(checksum.low64)); } if (columns_mask[src_index++]) { auto checksum = helper.hash_of_uncompressed_files; - columns[res_index++]->insert(getHexUIntLowercase(checksum.low64) + getHexUIntLowercase(checksum.high64)); + columns[res_index++]->insert(getHexUIntLowercase(checksum.high64) + getHexUIntLowercase(checksum.low64)); } if (columns_mask[src_index++]) { auto checksum = helper.uncompressed_hash_of_compressed_files; - columns[res_index++]->insert(getHexUIntLowercase(checksum.low64) + getHexUIntLowercase(checksum.high64)); + columns[res_index++]->insert(getHexUIntLowercase(checksum.high64) + getHexUIntLowercase(checksum.low64)); } } diff --git a/utils/checksum-for-compressed-block/main.cpp b/utils/checksum-for-compressed-block/main.cpp index 4f9923e7638..d30a3798820 100644 --- a/utils/checksum-for-compressed-block/main.cpp +++ b/utils/checksum-for-compressed-block/main.cpp @@ -45,7 +45,7 @@ int main(int, char **) { auto flipped = flipBit(str, pos); auto checksum = CityHash_v1_0_2::CityHash128(flipped.data(), flipped.size()); - std::cout << getHexUIntLowercase(checksum.first) << getHexUIntLowercase(checksum.second) << "\t" << pos / 8 << ", " << pos % 8 << "\n"; + std::cout << getHexUIntLowercase(checksum.high64) << getHexUIntLowercase(checksum.low64) << "\t" << pos / 8 << ", " << pos % 8 << "\n"; } return 0; From 5eeda0a0d24ae14a78da79273870aec9fa6bd8a0 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 24 Jun 2023 22:17:53 +0200 Subject: [PATCH 317/515] Fix test 00961_checksums_in_system_parts_columns_table --- .../00961_checksums_in_system_parts_columns_table.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference index 186f2feab79..4bf3cfe65a2 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference @@ -1 +1 @@ -20000101_1_1_0 test_00961 b5fce9c4ef1ca42ce4ed027389c208d2 fc3b062b646cd23d4c23d7f5920f89ae da96ff1e527a8a1f908ddf2b1d0af239 +20000101_1_1_0 test_00961 e4ed027389c208d2b5fce9c4ef1ca42c 4c23d7f5920f89aefc3b062b646cd23d 908ddf2b1d0af239da96ff1e527a8a1f From 71cded08ff2813f4c4757e71a773ca8cc0a293bf Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 25 Jun 2023 14:51:29 +0200 Subject: [PATCH 318/515] Remove unnecessary include from wide_integer_impl.h --- base/base/wide_integer_impl.h | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/base/base/wide_integer_impl.h b/base/base/wide_integer_impl.h index dc6a49694ae..411841e6d9f 100644 --- a/base/base/wide_integer_impl.h +++ b/base/base/wide_integer_impl.h @@ -15,8 +15,6 @@ #include #include -#include - // NOLINTBEGIN(*) /// Use same extended double for all platforms @@ -29,6 +27,8 @@ using FromDoubleIntermediateType = long double; using FromDoubleIntermediateType = boost::multiprecision::cpp_bin_float_double_extended; #endif +namespace CityHash_v1_0_2 { struct uint128; } + namespace wide { @@ -283,8 +283,11 @@ struct integer::_impl } } - constexpr static void wide_integer_from_cityhash_uint128(integer & self, const CityHash_v1_0_2::uint128 & value) noexcept + template + constexpr static void wide_integer_from_cityhash_uint128(integer & self, const CityHashUInt128 & value) noexcept { + static_assert(sizeof(item_count) >= 2); + if constexpr (std::endian::native == std::endian::little) wide_integer_from_tuple_like(self, std::make_pair(value.low64, value.high64)); else From 477b707ff1765d8a2d62ad21b869d544b212de96 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 02:02:24 +0200 Subject: [PATCH 319/515] Revert "Merge pull request #50951 from ZhiguoZh/20230607-toyear-fix" This reverts commit 6bbd0d144df01b07a28d3d9927ce2a6c1dc2ee56, reversing changes made to 74cb79769bbaa0c4619ca7cb382e6e37c8c7d7b5. --- src/Functions/DateTimeTransforms.h | 72 ------- .../FunctionDateOrDateTimeToSomething.h | 13 -- src/Functions/IFunction.h | 24 +-- src/Functions/IFunctionAdaptors.h | 7 - ...OrDateTimeConverterWithPreimageVisitor.cpp | 199 ------------------ ...teOrDateTimeConverterWithPreimageVisitor.h | 37 ---- src/Interpreters/TreeOptimizer.cpp | 19 -- ...783_date_predicate_optimizations.reference | 52 ----- .../02783_date_predicate_optimizations.sql | 76 ------- ...dicate_optimizations_ast_rewrite.reference | 87 -------- ...te_predicate_optimizations_ast_rewrite.sql | 47 ----- 11 files changed, 1 insertion(+), 632 deletions(-) delete mode 100644 src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp delete mode 100644 src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.h delete mode 100644 tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.reference delete mode 100644 tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.sql diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 84c71c89b11..019e0c42cde 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -322,7 +322,6 @@ struct ToTimeImpl { throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToDateImpl; }; @@ -394,7 +393,6 @@ struct ToStartOfSecondImpl { throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -442,7 +440,6 @@ struct ToStartOfMillisecondImpl { throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -486,7 +483,6 @@ struct ToStartOfMicrosecondImpl { throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -524,7 +520,6 @@ struct ToStartOfNanosecondImpl { throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -723,28 +718,6 @@ struct ToYearImpl return time_zone.toYear(DayNum(d)); } - static inline constexpr bool hasPreimage() { return true; } - - static inline RangeOrNull getPreimage(const IDataType & type, const Field & point) - { - if (point.getType() != Field::Types::UInt64) return std::nullopt; - - auto year = point.get(); - if (year < DATE_LUT_MIN_YEAR || year >= DATE_LUT_MAX_YEAR) return std::nullopt; - - const DateLUTImpl & date_lut = DateLUT::instance(); - - auto start_time = date_lut.makeDateTime(year, 1, 1, 0, 0, 0); - auto end_time = date_lut.addYears(start_time, 1); - - if (isDateOrDate32(type) || isDateTime(type) || isDateTime64(type)) - return {std::make_pair(Field(start_time), Field(end_time))}; - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of argument of function {}. Should be Date, Date32, DateTime or DateTime64", - type.getName(), name); - } - using FactorTransform = ZeroTransform; }; @@ -818,7 +791,6 @@ struct ToQuarterImpl { return time_zone.toQuarter(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfYearImpl; }; @@ -843,7 +815,6 @@ struct ToMonthImpl { return time_zone.toMonth(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfYearImpl; }; @@ -869,7 +840,6 @@ struct ToDayOfMonthImpl return time_zone.toDayOfMonth(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfMonthImpl; }; @@ -917,7 +887,6 @@ struct ToDayOfYearImpl { return time_zone.toDayOfYear(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfYearImpl; }; @@ -942,7 +911,6 @@ struct ToHourImpl { throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToDateImpl; }; @@ -971,7 +939,6 @@ struct TimezoneOffsetImpl throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToTimeImpl; }; @@ -995,7 +962,6 @@ struct ToMinuteImpl { throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfHourImpl; }; @@ -1020,7 +986,6 @@ struct ToSecondImpl { throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfMinuteImpl; }; @@ -1045,7 +1010,6 @@ struct ToISOYearImpl { return time_zone.toISOYear(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1102,7 +1066,6 @@ struct ToISOWeekImpl { return time_zone.toISOWeek(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToISOYearImpl; }; @@ -1145,7 +1108,6 @@ struct ToRelativeYearNumImpl { return time_zone.toYear(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1177,7 +1139,6 @@ struct ToRelativeQuarterNumImpl { return time_zone.toRelativeQuarterNum(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1209,7 +1170,6 @@ struct ToRelativeMonthNumImpl { return time_zone.toRelativeMonthNum(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1241,7 +1201,6 @@ struct ToRelativeWeekNumImpl { return time_zone.toRelativeWeekNum(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1273,7 +1232,6 @@ struct ToRelativeDayNumImpl { return static_cast(d); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1311,7 +1269,6 @@ struct ToRelativeHourNumImpl else return static_cast(time_zone.toRelativeHourNum(DayNum(d))); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1343,7 +1300,6 @@ struct ToRelativeMinuteNumImpl { return static_cast(time_zone.toRelativeMinuteNum(DayNum(d))); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1372,7 +1328,6 @@ struct ToRelativeSecondNumImpl { return static_cast(time_zone.fromDayNum(DayNum(d))); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1397,31 +1352,6 @@ struct ToYYYYMMImpl { return time_zone.toNumYYYYMM(DayNum(d)); } - static inline constexpr bool hasPreimage() { return true; } - - static inline RangeOrNull getPreimage(const IDataType & type, const Field & point) - { - if (point.getType() != Field::Types::UInt64) return std::nullopt; - - auto year_month = point.get(); - auto year = year_month / 100; - auto month = year_month % 100; - - if (year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || (year == DATE_LUT_MAX_YEAR && month == 12)) - return std::nullopt; - - const DateLUTImpl & date_lut = DateLUT::instance(); - - auto start_time = date_lut.makeDateTime(year, month, 1, 0, 0, 0); - auto end_time = date_lut.addMonths(start_time, 1); - - if (isDateOrDate32(type) || isDateTime(type) || isDateTime64(type)) - return {std::make_pair(Field(start_time), Field(end_time))}; - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of argument of function {}. Should be Date, Date32, DateTime or DateTime64", - type.getName(), name); - } using FactorTransform = ZeroTransform; }; @@ -1446,7 +1376,6 @@ struct ToYYYYMMDDImpl { return time_zone.toNumYYYYMMDD(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1471,7 +1400,6 @@ struct ToYYYYMMDDhhmmssImpl { return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(DayNum(d))); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; diff --git a/src/Functions/FunctionDateOrDateTimeToSomething.h b/src/Functions/FunctionDateOrDateTimeToSomething.h index d98b788c7d7..82818cc3d2b 100644 --- a/src/Functions/FunctionDateOrDateTimeToSomething.h +++ b/src/Functions/FunctionDateOrDateTimeToSomething.h @@ -7,7 +7,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NOT_IMPLEMENTED; } /// See DateTimeTransforms.h @@ -84,18 +83,6 @@ public: arguments[0].type->getName(), this->getName()); } - bool hasInformationAboutPreimage() const override { return Transform::hasPreimage(); } - - RangeOrNull getPreimage(const IDataType & type, const Field & point) const override - { - if constexpr (Transform::hasPreimage()) - return Transform::getPreimage(type, point); - else - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Function {} has no information about its preimage", - Transform::name); - } - }; } diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index 433cb61d04e..1e4f8bf1102 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -2,8 +2,6 @@ #include #include -#include -#include #include #include #include @@ -32,8 +30,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } -/// A left-closed and right-open interval representing the preimage of a function. -using RangeOrNull = std::optional>; +class Field; /// The simplest executable object. /// Motivation: @@ -231,12 +228,6 @@ public: */ virtual bool hasInformationAboutMonotonicity() const { return false; } - /** Lets you know if the function has its definition of preimage. - * This is used to work with predicate optimizations, where the comparison between - * f(x) and a constant c could be converted to the comparison between x and f's preimage [b, e). - */ - virtual bool hasInformationAboutPreimage() const { return false; } - struct ShortCircuitSettings { /// Should we enable lazy execution for the first argument of short-circuit function? @@ -290,14 +281,6 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its monotonicity", getName()); } - /** Get the preimage of a function in the form of a left-closed and right-open interval. Call only if hasInformationAboutPreimage. - * std::nullopt might be returned if the point (a single value) is invalid for this function. - */ - virtual RangeOrNull getPreimage(const IDataType & /*type*/, const Field & /*point*/) const - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its preimage", getName()); - } - }; using FunctionBasePtr = std::shared_ptr; @@ -487,17 +470,12 @@ public: virtual bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const = 0; virtual bool hasInformationAboutMonotonicity() const { return false; } - virtual bool hasInformationAboutPreimage() const { return false; } using Monotonicity = IFunctionBase::Monotonicity; virtual Monotonicity getMonotonicityForRange(const IDataType & /*type*/, const Field & /*left*/, const Field & /*right*/) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its monotonicity", getName()); } - virtual RangeOrNull getPreimage(const IDataType & /*type*/, const Field & /*point*/) const - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its preimage", getName()); - } /// For non-variadic functions, return number of arguments; otherwise return zero (that should be ignored). virtual size_t getNumberOfArguments() const = 0; diff --git a/src/Functions/IFunctionAdaptors.h b/src/Functions/IFunctionAdaptors.h index 123fdbc2f50..23725b1a8b1 100644 --- a/src/Functions/IFunctionAdaptors.h +++ b/src/Functions/IFunctionAdaptors.h @@ -90,17 +90,10 @@ public: bool hasInformationAboutMonotonicity() const override { return function->hasInformationAboutMonotonicity(); } - bool hasInformationAboutPreimage() const override { return function->hasInformationAboutPreimage(); } - Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override { return function->getMonotonicityForRange(type, left, right); } - - RangeOrNull getPreimage(const IDataType & type, const Field & point) const override - { - return function->getPreimage(type, point); - } private: std::shared_ptr function; DataTypes arguments; diff --git a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp deleted file mode 100644 index a377bb4bba6..00000000000 --- a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp +++ /dev/null @@ -1,199 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -/** Given a monotonic non-decreasing function f(x), which satisfies f(x) = c for any value x within [b, e). - * We could convert it into its equivalent form, x >= b AND x < e, which is free from the invocation of the function. - * And we could apply the similar transformation to other comparisons. The suggested transformations list: - * - * f(x) == c -> x >= b AND x < e - * f(x) != c -> x < b OR x >= e - * f(x) > c -> x >= e - * f(x) >= c -> x >= b - * f(x) < c -> x < b - * f(x) <= c -> x < e - * - * This function generates a new AST with the transformed relation. - */ -ASTPtr generateOptimizedDateFilterAST(const String & comparator, const NameAndTypePair & column, const std::pair& range) -{ - const DateLUTImpl & date_lut = DateLUT::instance(); - - const String & column_name = column.name; - String start_date_or_date_time; - String end_date_or_date_time; - - if (isDateOrDate32(column.type.get())) - { - start_date_or_date_time = date_lut.dateToString(range.first.get()); - end_date_or_date_time = date_lut.dateToString(range.second.get()); - } - else if (isDateTime(column.type.get()) || isDateTime64(column.type.get())) - { - start_date_or_date_time = date_lut.timeToString(range.first.get()); - end_date_or_date_time = date_lut.timeToString(range.second.get()); - } - else [[unlikely]] return {}; - - if (comparator == "equals") - { - return makeASTFunction("and", - makeASTFunction("greaterOrEquals", - std::make_shared(column_name), - std::make_shared(start_date_or_date_time) - ), - makeASTFunction("less", - std::make_shared(column_name), - std::make_shared(end_date_or_date_time) - ) - ); - } - else if (comparator == "notEquals") - { - return makeASTFunction("or", - makeASTFunction("less", - std::make_shared(column_name), - std::make_shared(start_date_or_date_time) - ), - makeASTFunction("greaterOrEquals", - std::make_shared(column_name), - std::make_shared(end_date_or_date_time) - ) - ); - } - else if (comparator == "greater") - { - return makeASTFunction("greaterOrEquals", - std::make_shared(column_name), - std::make_shared(end_date_or_date_time) - ); - } - else if (comparator == "lessOrEquals") - { - return makeASTFunction("less", - std::make_shared(column_name), - std::make_shared(end_date_or_date_time) - ); - } - else if (comparator == "less" || comparator == "greaterOrEquals") - { - return makeASTFunction(comparator, - std::make_shared(column_name), - std::make_shared(start_date_or_date_time) - ); - } - else [[unlikely]] - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expected equals, notEquals, less, lessOrEquals, greater, greaterOrEquals. Actual {}", - comparator); - } -} - -void OptimizeDateOrDateTimeConverterWithPreimageMatcher::visit(const ASTFunction & function, ASTPtr & ast, const Data & data) -{ - const static std::unordered_map swap_relations = { - {"equals", "equals"}, - {"notEquals", "notEquals"}, - {"less", "greater"}, - {"greater", "less"}, - {"lessOrEquals", "greaterOrEquals"}, - {"greaterOrEquals", "lessOrEquals"}, - }; - - if (!swap_relations.contains(function.name)) return; - - if (!function.arguments || function.arguments->children.size() != 2) return; - - size_t func_id = function.arguments->children.size(); - - for (size_t i = 0; i < function.arguments->children.size(); i++) - { - if (const auto * func = function.arguments->children[i]->as()) - { - func_id = i; - } - } - - if (func_id == function.arguments->children.size()) return; - - size_t literal_id = 1 - func_id; - const auto * literal = function.arguments->children[literal_id]->as(); - - if (!literal || literal->value.getType() != Field::Types::UInt64) return; - - String comparator = literal_id > func_id ? function.name : swap_relations.at(function.name); - - const auto * ast_func = function.arguments->children[func_id]->as(); - /// Currently we only handle single-argument functions. - if (!ast_func || !ast_func->arguments || ast_func->arguments->children.size() != 1) return; - - const auto * column_id = ast_func->arguments->children.at(0)->as(); - if (!column_id) return; - - auto pos = IdentifierSemantic::getMembership(*column_id); - if (!pos) - pos = IdentifierSemantic::chooseTableColumnMatch(*column_id, data.tables, true); - if (!pos) - return; - - if (*pos >= data.tables.size()) - return; - - auto data_type_and_name = data.tables[*pos].columns.tryGetByName(column_id->shortName()); - if (!data_type_and_name) return; - - const auto & converter = FunctionFactory::instance().tryGet(ast_func->name, data.context); - if (!converter) return; - - ColumnsWithTypeAndName args; - args.emplace_back(data_type_and_name->type, "tmp"); - auto converter_base = converter->build(args); - if (!converter_base || !converter_base->hasInformationAboutPreimage()) return; - - auto preimage_range = converter_base->getPreimage(*(data_type_and_name->type), literal->value); - if (!preimage_range) return; - - const auto new_ast = generateOptimizedDateFilterAST(comparator, *data_type_and_name, *preimage_range); - if (!new_ast) return; - - ast = new_ast; -} - -bool OptimizeDateOrDateTimeConverterWithPreimageMatcher::needChildVisit(ASTPtr & ast, ASTPtr & /*child*/) -{ - const static std::unordered_set relations = { - "equals", - "notEquals", - "less", - "greater", - "lessOrEquals", - "greaterOrEquals", - }; - - if (const auto * ast_function = ast->as()) - { - return !relations.contains(ast_function->name); - } - - return true; -} - -} diff --git a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.h b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.h deleted file mode 100644 index 778fa462364..00000000000 --- a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.h +++ /dev/null @@ -1,37 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -class ASTFunction; - -/** Replace predicate having Date/DateTime converters with their preimages to improve performance. - * Given a Date column c, toYear(c) = 2023 -> c >= '2023-01-01' AND c < '2024-01-01' - * Or if c is a DateTime column, toYear(c) = 2023 -> c >= '2023-01-01 00:00:00' AND c < '2024-01-01 00:00:00'. - * The similar optimization also applies to other converters. - */ -class OptimizeDateOrDateTimeConverterWithPreimageMatcher -{ -public: - struct Data - { - const TablesWithColumns & tables; - ContextPtr context; - }; - - static void visit(ASTPtr & ast, Data & data) - { - if (const auto * ast_function = ast->as()) - visit(*ast_function, ast, data); - } - - static void visit(const ASTFunction & function, ASTPtr & ast, const Data & data); - - static bool needChildVisit(ASTPtr & ast, ASTPtr & child); -}; - -using OptimizeDateOrDateTimeConverterWithPreimageVisitor = InDepthNodeVisitor; -} diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index fd4d2c9d846..c38b3c79026 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -25,7 +25,6 @@ #include #include #include -#include #include #include @@ -678,21 +677,6 @@ void optimizeInjectiveFunctionsInsideUniq(ASTPtr & query, ContextPtr context) RemoveInjectiveFunctionsVisitor(data).visit(query); } -void optimizeDateFilters(ASTSelectQuery * select_query, const std::vector & tables_with_columns, ContextPtr context) -{ - /// Predicates in HAVING clause has been moved to WHERE clause. - if (select_query->where()) - { - OptimizeDateOrDateTimeConverterWithPreimageVisitor::Data data{tables_with_columns, context}; - OptimizeDateOrDateTimeConverterWithPreimageVisitor(data).visit(select_query->refWhere()); - } - if (select_query->prewhere()) - { - OptimizeDateOrDateTimeConverterWithPreimageVisitor::Data data{tables_with_columns, context}; - OptimizeDateOrDateTimeConverterWithPreimageVisitor(data).visit(select_query->refPrewhere()); - } -} - void transformIfStringsIntoEnum(ASTPtr & query) { std::unordered_set function_names = {"if", "transform"}; @@ -796,9 +780,6 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, tables_with_columns, result.storage_snapshot->metadata, result.storage); } - /// Rewrite date filters to avoid the calls of converters such as toYear, toYYYYMM, etc. - optimizeDateFilters(select_query, tables_with_columns, context); - /// GROUP BY injective function elimination. optimizeGroupBy(select_query, context); diff --git a/tests/queries/0_stateless/02783_date_predicate_optimizations.reference b/tests/queries/0_stateless/02783_date_predicate_optimizations.reference index 872a5dd1d7d..cd689b93034 100644 --- a/tests/queries/0_stateless/02783_date_predicate_optimizations.reference +++ b/tests/queries/0_stateless/02783_date_predicate_optimizations.reference @@ -1,54 +1,2 @@ 2021-12-31 23:00:00 0 2021-12-31 23:00:00 0 -Date -2 -3 -2 -4 -1 -3 -3 -2 -1 -4 -1 -4 -DateTime -2 -3 -2 -4 -1 -3 -3 -2 -1 -4 -1 -4 -Date32 -2 -3 -2 -4 -1 -3 -3 -2 -1 -4 -1 -4 -DateTime64 -2 -3 -2 -4 -1 -3 -3 -2 -1 -4 -1 -4 diff --git a/tests/queries/0_stateless/02783_date_predicate_optimizations.sql b/tests/queries/0_stateless/02783_date_predicate_optimizations.sql index 0a2fa6cc93b..abb13f1005e 100644 --- a/tests/queries/0_stateless/02783_date_predicate_optimizations.sql +++ b/tests/queries/0_stateless/02783_date_predicate_optimizations.sql @@ -11,79 +11,3 @@ INSERT INTO source values ('2021-12-31 23:00:00', 0); SELECT * FROM source WHERE toYYYYMM(ts) = 202112; SELECT * FROM source WHERE toYear(ts) = 2021; - -DROP TABLE IF EXISTS source; -CREATE TABLE source -( - `dt` Date, - `ts` DateTime, - `dt_32` Date32, - `ts_64` DateTime64(3), - `n` Int32 -) -ENGINE = MergeTree -PARTITION BY toYYYYMM(ts) -ORDER BY tuple(); - -INSERT INTO source values ('2022-12-31', '2022-12-31 23:59:59', '2022-12-31', '2022-12-31 23:59:59.123', 0); -INSERT INTO source values ('2023-01-01', '2023-01-01 00:00:00', '2023-01-01', '2023-01-01 00:00:00.000', 1); -INSERT INTO source values ('2023-12-01', '2023-12-01 00:00:00', '2023-12-01', '2023-12-01 00:00:00.000', 2); -INSERT INTO source values ('2023-12-31', '2023-12-31 23:59:59', '2023-12-31', '2023-12-31 23:59:59.123', 3); -INSERT INTO source values ('2024-01-01', '2024-01-01 00:00:00', '2024-01-01', '2024-01-01 00:00:00.000', 4); - -SELECT 'Date'; -SELECT count(*) FROM source WHERE toYYYYMM(dt) = 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt) <> 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt) < 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt) <= 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt) > 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt) >= 202312; -SELECT count(*) FROM source WHERE toYear(dt) = 2023; -SELECT count(*) FROM source WHERE toYear(dt) <> 2023; -SELECT count(*) FROM source WHERE toYear(dt) < 2023; -SELECT count(*) FROM source WHERE toYear(dt) <= 2023; -SELECT count(*) FROM source WHERE toYear(dt) > 2023; -SELECT count(*) FROM source WHERE toYear(dt) >= 2023; - -SELECT 'DateTime'; -SELECT count(*) FROM source WHERE toYYYYMM(ts) = 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts) <> 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts) < 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts) <= 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts) > 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts) >= 202312; -SELECT count(*) FROM source WHERE toYear(ts) = 2023; -SELECT count(*) FROM source WHERE toYear(ts) <> 2023; -SELECT count(*) FROM source WHERE toYear(ts) < 2023; -SELECT count(*) FROM source WHERE toYear(ts) <= 2023; -SELECT count(*) FROM source WHERE toYear(ts) > 2023; -SELECT count(*) FROM source WHERE toYear(ts) >= 2023; - -SELECT 'Date32'; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) = 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) <> 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) < 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) <= 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) > 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) >= 202312; -SELECT count(*) FROM source WHERE toYear(dt_32) = 2023; -SELECT count(*) FROM source WHERE toYear(dt_32) <> 2023; -SELECT count(*) FROM source WHERE toYear(dt_32) < 2023; -SELECT count(*) FROM source WHERE toYear(dt_32) <= 2023; -SELECT count(*) FROM source WHERE toYear(dt_32) > 2023; -SELECT count(*) FROM source WHERE toYear(dt_32) >= 2023; - -SELECT 'DateTime64'; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) = 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) <> 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) < 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) <= 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) > 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) >= 202312; -SELECT count(*) FROM source WHERE toYear(ts_64) = 2023; -SELECT count(*) FROM source WHERE toYear(ts_64) <> 2023; -SELECT count(*) FROM source WHERE toYear(ts_64) < 2023; -SELECT count(*) FROM source WHERE toYear(ts_64) <= 2023; -SELECT count(*) FROM source WHERE toYear(ts_64) > 2023; -SELECT count(*) FROM source WHERE toYear(ts_64) >= 2023; -DROP TABLE source; diff --git a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.reference b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.reference deleted file mode 100644 index 9235e7e106a..00000000000 --- a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.reference +++ /dev/null @@ -1,87 +0,0 @@ -SELECT value1 -FROM date_t -WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE ((date1 < \'1993-01-01\') OR (date1 >= \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 >= \'1994-01-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 < \'1994-01-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 >= \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1998-01-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) OR ((date1 >= \'1994-01-01\') AND (date1 < \'1995-01-01\'))) AND ((id >= 1) AND (id <= 3)) -SELECT - value1, - toYear(date1) AS year1 -FROM date_t -WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -PREWHERE (date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\') -WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE ((id >= 1) AND (id <= 3)) AND ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) -SELECT value1 -FROM date_t -WHERE (toYYYYMM(date1) = 199300) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (toYYYYMM(date1) = 199313) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE ((date1 >= \'1993-12-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE ((date1 >= \'1992-03-01\') AND (date1 < \'1992-04-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE ((date1 < \'1992-03-01\') OR (date1 >= \'1992-04-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 < \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 >= \'1992-04-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 < \'1992-04-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 >= \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE ((date1 >= \'1992-03-01\') OR ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\'))) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM datetime_t -WHERE ((date1 >= \'1993-01-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM datetime_t -WHERE ((date1 >= \'1993-12-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date32_t -WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date32_t -WHERE ((date1 >= \'1993-12-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM datetime64_t -WHERE ((date1 >= \'1993-01-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM datetime64_t -WHERE ((date1 >= \'1993-12-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) diff --git a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.sql b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.sql deleted file mode 100644 index 266be59b0a3..00000000000 --- a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.sql +++ /dev/null @@ -1,47 +0,0 @@ -DROP TABLE IF EXISTS date_t; -CREATE TABLE date_t (id UInt32, value1 String, date1 Date) ENGINE ReplacingMergeTree() ORDER BY id; - -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) <> 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) < 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) > 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) <= 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) >= 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) BETWEEN 1993 AND 1997 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE (toYear(date1) = 1993 OR toYear(date1) = 1994) AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1, toYear(date1) as year1 FROM date_t WHERE year1 = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE 1993 > toYear(date1) AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t PREWHERE toYear(date1) = 1993 WHERE id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE id BETWEEN 1 AND 3 HAVING toYear(date1) = 1993; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199300 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199313 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) <> 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) < 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) > 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) <= 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) >= 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE (toYYYYMM(date1) >= 199203 OR toYear(date1) = 1993) AND id BETWEEN 1 AND 3; -DROP TABLE date_t; - -DROP TABLE IF EXISTS datetime_t; -CREATE TABLE datetime_t (id UInt32, value1 String, date1 Datetime) ENGINE ReplacingMergeTree() ORDER BY id; - -EXPLAIN SYNTAX SELECT value1 FROM datetime_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM datetime_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; -DROP TABLE datetime_t; - -DROP TABLE IF EXISTS date32_t; -CREATE TABLE date32_t (id UInt32, value1 String, date1 Date32) ENGINE ReplacingMergeTree() ORDER BY id; - -EXPLAIN SYNTAX SELECT value1 FROM date32_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date32_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; -DROP TABLE date32_t; - -DROP TABLE IF EXISTS datetime64_t; -CREATE TABLE datetime64_t (id UInt32, value1 String, date1 Datetime64) ENGINE ReplacingMergeTree() ORDER BY id; - -EXPLAIN SYNTAX SELECT value1 FROM datetime64_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM datetime64_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; -DROP TABLE datetime64_t; From 4ee74ec213b333893ddeb89331970e0d8758adbd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 02:19:52 +0200 Subject: [PATCH 320/515] Two tests are twice longer in average with Analyzer and sometimes failing --- docker/test/util/process_functional_tests_result.py | 4 ++-- tests/{broken_tests.txt => analyzer_tech_debt.txt} | 2 ++ tests/ci/functional_test_check.py | 2 +- 3 files changed, 5 insertions(+), 3 deletions(-) rename tests/{broken_tests.txt => analyzer_tech_debt.txt} (97%) diff --git a/docker/test/util/process_functional_tests_result.py b/docker/test/util/process_functional_tests_result.py index c75a3500831..fd4cc9f4bf7 100755 --- a/docker/test/util/process_functional_tests_result.py +++ b/docker/test/util/process_functional_tests_result.py @@ -86,7 +86,7 @@ def process_test_log(log_path, broken_tests): test_name, "NOT_FAILED", test_time, - ["This test passed. Update broken_tests.txt.\n"], + ["This test passed. Update analyzer_tech_debt.txt.\n"], ) ) else: @@ -205,7 +205,7 @@ if __name__ == "__main__": parser.add_argument("--in-results-dir", default="/test_output/") parser.add_argument("--out-results-file", default="/test_output/test_results.tsv") parser.add_argument("--out-status-file", default="/test_output/check_status.tsv") - parser.add_argument("--broken-tests", default="/broken_tests.txt") + parser.add_argument("--broken-tests", default="/analyzer_tech_debt.txt") args = parser.parse_args() broken_tests = list() diff --git a/tests/broken_tests.txt b/tests/analyzer_tech_debt.txt similarity index 97% rename from tests/broken_tests.txt rename to tests/analyzer_tech_debt.txt index b3668b06e21..0872033aed0 100644 --- a/tests/broken_tests.txt +++ b/tests/analyzer_tech_debt.txt @@ -124,3 +124,5 @@ 02534_s3_cluster_insert_select_schema_inference 02765_parallel_replicas_final_modifier 02784_parallel_replicas_automatic_disabling +02581_share_big_sets_between_mutation_tasks_long +02581_share_big_sets_between_multiple_mutations_tasks_long diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index f77ef330ea2..9279b19b187 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -108,7 +108,7 @@ def get_run_command( env_str = " ".join(envs) volume_with_broken_test = ( - f"--volume={repo_tests_path}/broken_tests.txt:/broken_tests.txt" + f"--volume={repo_tests_path}/analyzer_tech_debt.txt:/analyzer_tech_debt.txt" if "analyzer" in check_name else "" ) From a487a1ab5acbe5fd2d5d1ee9872f895709408002 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 02:48:30 +0200 Subject: [PATCH 321/515] Fix bad test --- tests/queries/0_stateless/02479_mysql_connect_to_self.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql index c5c5dfb8c4a..cf2220073d3 100644 --- a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql @@ -2,6 +2,8 @@ SET send_logs_level = 'fatal'; -- failed connection tries are ok, if it succeeded after retry. +DROP TABLE IF EXISTS foo; + CREATE TABLE foo (key UInt32, a String, b Int64, c String) ENGINE = TinyLog; INSERT INTO foo VALUES (1, 'one', -1, 'een'), (2, 'two', -2, 'twee'), (3, 'three', -3, 'drie'), (4, 'four', -4, 'vier'), (5, 'five', -5, 'vijf'); @@ -40,3 +42,5 @@ SELECT * FROM mysql( SELECT '---'; SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connection_pool_size = 1, connect_timeout = 100, connection_wait_timeout = 100); SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connection_pool_size = 0); -- { serverError BAD_ARGUMENTS } + +DROP TABLE foo; From 0e88aae9d7ab53d4267324e17f5a48fae00ae72c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 04:27:50 +0200 Subject: [PATCH 322/515] Remove ConsoleCertificateHandler --- .../Poco/Net/ConsoleCertificateHandler.h | 53 ------------------- .../include/Poco/Net/SSLManager.h | 6 +-- .../src/CertificateHandlerFactoryMgr.cpp | 2 - .../src/ConsoleCertificateHandler.cpp | 53 ------------------- base/poco/NetSSL_OpenSSL/src/SSLManager.cpp | 2 +- .../settings.md | 2 +- .../settings.md | 2 +- .../settings.md | 2 +- 8 files changed, 7 insertions(+), 115 deletions(-) delete mode 100644 base/poco/NetSSL_OpenSSL/include/Poco/Net/ConsoleCertificateHandler.h delete mode 100644 base/poco/NetSSL_OpenSSL/src/ConsoleCertificateHandler.cpp diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/ConsoleCertificateHandler.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/ConsoleCertificateHandler.h deleted file mode 100644 index 8e09b6f18ae..00000000000 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/ConsoleCertificateHandler.h +++ /dev/null @@ -1,53 +0,0 @@ -// -// ConsoleCertificateHandler.h -// -// Library: NetSSL_OpenSSL -// Package: SSLCore -// Module: ConsoleCertificateHandler -// -// Definition of the ConsoleCertificateHandler class. -// -// Copyright (c) 2006-2009, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef NetSSL_ConsoleCertificateHandler_INCLUDED -#define NetSSL_ConsoleCertificateHandler_INCLUDED - - -#include "Poco/Net/InvalidCertificateHandler.h" -#include "Poco/Net/NetSSL.h" - - -namespace Poco -{ -namespace Net -{ - - - class NetSSL_API ConsoleCertificateHandler : public InvalidCertificateHandler - /// A ConsoleCertificateHandler is invoked whenever an error occurs verifying the certificate. - /// - /// The certificate is printed to stdout and the user is asked via console if he wants to accept it. - { - public: - ConsoleCertificateHandler(bool handleErrorsOnServerSide); - /// Creates the ConsoleCertificateHandler. - - virtual ~ConsoleCertificateHandler(); - /// Destroys the ConsoleCertificateHandler. - - void onInvalidCertificate(const void * pSender, VerificationErrorArgs & errorCert); - /// Prints the certificate to stdout and waits for user input on the console - /// to decide if a certificate should be accepted/rejected. - }; - - -} -} // namespace Poco::Net - - -#endif // NetSSL_ConsoleCertificateHandler_INCLUDED diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h index a4fde26286e..21a1ed685e5 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h @@ -85,7 +85,7 @@ namespace Net /// /// /// - /// ConsoleCertificateHandler + /// RejectCertificateHandler /// /// true|false /// someString @@ -186,7 +186,7 @@ namespace Net /// /// Valid initialization code would be: /// SharedPtr pConsoleHandler = new KeyConsoleHandler; - /// SharedPtr pInvalidCertHandler = new ConsoleCertificateHandler; + /// SharedPtr pInvalidCertHandler = new RejectCertificateHandler; /// Context::Ptr pContext = new Context(Context::SERVER_USE, "any.pem", "any.pem", "rootcert.pem", Context::VERIFY_RELAXED, 9, false, "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH"); /// SSLManager::instance().initializeServer(pConsoleHandler, pInvalidCertHandler, pContext); @@ -203,7 +203,7 @@ namespace Net /// /// Valid initialization code would be: /// SharedPtr pConsoleHandler = new KeyConsoleHandler; - /// SharedPtr pInvalidCertHandler = new ConsoleCertificateHandler; + /// SharedPtr pInvalidCertHandler = new RejectCertificateHandler; /// Context::Ptr pContext = new Context(Context::CLIENT_USE, "", "", "rootcert.pem", Context::VERIFY_RELAXED, 9, false, "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH"); /// SSLManager::instance().initializeClient(pConsoleHandler, pInvalidCertHandler, pContext); diff --git a/base/poco/NetSSL_OpenSSL/src/CertificateHandlerFactoryMgr.cpp b/base/poco/NetSSL_OpenSSL/src/CertificateHandlerFactoryMgr.cpp index a89bbea11f2..f570e2d3599 100644 --- a/base/poco/NetSSL_OpenSSL/src/CertificateHandlerFactoryMgr.cpp +++ b/base/poco/NetSSL_OpenSSL/src/CertificateHandlerFactoryMgr.cpp @@ -13,7 +13,6 @@ #include "Poco/Net/CertificateHandlerFactoryMgr.h" -#include "Poco/Net/ConsoleCertificateHandler.h" #include "Poco/Net/AcceptCertificateHandler.h" #include "Poco/Net/RejectCertificateHandler.h" @@ -24,7 +23,6 @@ namespace Net { CertificateHandlerFactoryMgr::CertificateHandlerFactoryMgr() { - setFactory("ConsoleCertificateHandler", new CertificateHandlerFactoryImpl()); setFactory("AcceptCertificateHandler", new CertificateHandlerFactoryImpl()); setFactory("RejectCertificateHandler", new CertificateHandlerFactoryImpl()); } diff --git a/base/poco/NetSSL_OpenSSL/src/ConsoleCertificateHandler.cpp b/base/poco/NetSSL_OpenSSL/src/ConsoleCertificateHandler.cpp deleted file mode 100644 index db64752e70c..00000000000 --- a/base/poco/NetSSL_OpenSSL/src/ConsoleCertificateHandler.cpp +++ /dev/null @@ -1,53 +0,0 @@ -// -// ConsoleCertificateHandler.cpp -// -// Library: NetSSL_OpenSSL -// Package: SSLCore -// Module: ConsoleCertificateHandler -// -// Copyright (c) 2006-2009, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/Net/ConsoleCertificateHandler.h" -#include - - -namespace Poco { -namespace Net { - - -ConsoleCertificateHandler::ConsoleCertificateHandler(bool server): InvalidCertificateHandler(server) -{ -} - - -ConsoleCertificateHandler::~ConsoleCertificateHandler() -{ -} - - -void ConsoleCertificateHandler::onInvalidCertificate(const void*, VerificationErrorArgs& errorCert) -{ - const X509Certificate& aCert = errorCert.certificate(); - std::cout << "\n"; - std::cout << "WARNING: Certificate verification failed\n"; - std::cout << "----------------------------------------\n"; - std::cout << "Issuer Name: " << aCert.issuerName() << "\n"; - std::cout << "Subject Name: " << aCert.subjectName() << "\n\n"; - std::cout << "The certificate yielded the error: " << errorCert.errorMessage() << "\n\n"; - std::cout << "The error occurred in the certificate chain at position " << errorCert.errorDepth() << "\n"; - std::cout << "Accept the certificate (y,n)? "; - char c = 0; - std::cin >> c; - if (c == 'y' || c == 'Y') - errorCert.setIgnoreError(true); - else - errorCert.setIgnoreError(false); -} - - -} } // namespace Poco::Net diff --git a/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp b/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp index 82eed1a29eb..927602ca658 100644 --- a/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp +++ b/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp @@ -46,7 +46,7 @@ const std::string SSLManager::CFG_PREFER_SERVER_CIPHERS("preferServerCiphers"); const std::string SSLManager::CFG_DELEGATE_HANDLER("privateKeyPassphraseHandler.name"); const std::string SSLManager::VAL_DELEGATE_HANDLER("KeyConsoleHandler"); const std::string SSLManager::CFG_CERTIFICATE_HANDLER("invalidCertificateHandler.name"); -const std::string SSLManager::VAL_CERTIFICATE_HANDLER("ConsoleCertificateHandler"); +const std::string SSLManager::VAL_CERTIFICATE_HANDLER("RejectCertificateHandler"); const std::string SSLManager::CFG_SERVER_PREFIX("openSSL.server."); const std::string SSLManager::CFG_CLIENT_PREFIX("openSSL.client."); const std::string SSLManager::CFG_CACHE_SESSIONS("cacheSessions"); diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 5643e273d39..40c1b8d64a1 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1602,7 +1602,7 @@ Keys for server/client settings: - requireTLSv1_2 (default: false) – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. - fips (default: false) – Activates OpenSSL FIPS mode. Supported if the library’s OpenSSL version supports FIPS. - privateKeyPassphraseHandler (default: `KeyConsoleHandler`)– Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ``, `KeyFileHandler`, `test`, ``. -- invalidCertificateHandler (default: `ConsoleCertificateHandler`) – Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: ` ConsoleCertificateHandler ` . +- invalidCertificateHandler (default: `RejectCertificateHandler`) – Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: ` RejectCertificateHandler ` . - disableProtocols (default: "") – Protocols that are not allowed to use. - preferServerCiphers (default: false) – Preferred server ciphers on the client. diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 33db6df0fdd..5430469ea18 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1067,7 +1067,7 @@ ClickHouse использует потоки из глобального пул - requireTLSv1_2 - Требование соединения TLSv1.2. Допустимые значения: `true`, `false`. - fips - Активация режима OpenSSL FIPS. Поддерживается, если версия OpenSSL, с которой собрана библиотека поддерживает fips. - privateKeyPassphraseHandler - Класс (подкласс PrivateKeyPassphraseHandler)запрашивающий кодовую фразу доступа к секретному ключу. Например, ``, `KeyFileHandler`, `test`, ``. -- invalidCertificateHandler - Класс (подкласс CertificateHandler) для подтверждения не валидных сертификатов. Например, ` ConsoleCertificateHandler `. +- invalidCertificateHandler - Класс (подкласс CertificateHandler) для подтверждения не валидных сертификатов. Например, ` RejectCertificateHandler `. - disableProtocols - Запрещенные к использованию протоколы. - preferServerCiphers - Предпочтение серверных шифров на клиенте. diff --git a/docs/zh/operations/server-configuration-parameters/settings.md b/docs/zh/operations/server-configuration-parameters/settings.md index 52142eda2e8..f6106d8734e 100644 --- a/docs/zh/operations/server-configuration-parameters/settings.md +++ b/docs/zh/operations/server-configuration-parameters/settings.md @@ -466,7 +466,7 @@ SSL客户端/服务器配置。 - requireTLSv1_2 – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. - fips – Activates OpenSSL FIPS mode. Supported if the library’s OpenSSL version supports FIPS. - privateKeyPassphraseHandler – Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ``, `KeyFileHandler`, `test`, ``. -- invalidCertificateHandler – Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: ` ConsoleCertificateHandler ` . +- invalidCertificateHandler – Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: ` RejectCertificateHandler ` . - disableProtocols – Protocols that are not allowed to use. - preferServerCiphers – Preferred server ciphers on the client. From 88048153d7af70dcfc27dc1c9bd480d9cad99d53 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 04:37:14 +0200 Subject: [PATCH 323/515] Fix 00899_long_attach_memory_limit --- tests/queries/0_stateless/00899_long_attach_memory_limit.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00899_long_attach_memory_limit.sql b/tests/queries/0_stateless/00899_long_attach_memory_limit.sql index aa507cda2ac..d4aa2a0eb7b 100644 --- a/tests/queries/0_stateless/00899_long_attach_memory_limit.sql +++ b/tests/queries/0_stateless/00899_long_attach_memory_limit.sql @@ -1,4 +1,5 @@ --- Tags: long, no-debug, no-parallel, no-fasttest +-- Tags: long, no-debug, no-parallel, no-fasttest, no-msan, no-tsan +-- This test is slow under MSan or TSan. DROP TABLE IF EXISTS index_memory; CREATE TABLE index_memory (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS index_granularity = 1; From cf2e110c134c6934fd22e0c9ec5a598ac5b107cc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 05:20:35 +0200 Subject: [PATCH 324/515] Fix test 01293_optimize_final_force --- tests/queries/0_stateless/01293_optimize_final_force.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01293_optimize_final_force.sh b/tests/queries/0_stateless/01293_optimize_final_force.sh index 994d5952dbc..eb3a2756899 100755 --- a/tests/queries/0_stateless/01293_optimize_final_force.sh +++ b/tests/queries/0_stateless/01293_optimize_final_force.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-debug, no-s3-storage +# This test is too slow with S3 storage and debug modes. CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 4cef0342979ced426f4823016ebc65f42067aab7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 05:27:45 +0200 Subject: [PATCH 325/515] Fix 02481_parquet_list_monotonically_increasing_offsets.sh --- .../02481_parquet_list_monotonically_increasing_offsets.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh b/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh index 47245eeb940..55e6ac2f758 100755 --- a/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh +++ b/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-ubsan, no-fasttest +# Tags: no-ubsan, no-fasttest, no-tsan +# It is too slow under TSan CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -13,4 +14,4 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_load (list Array(Int64), json cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO parquet_load FORMAT Parquet" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load" | md5sum ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM parquet_load" -${CLICKHOUSE_CLIENT} --query="drop table parquet_load" \ No newline at end of file +${CLICKHOUSE_CLIENT} --query="drop table parquet_load" From e3d999b023cad921dd740354125a8cbac9c30da5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 05:43:27 +0200 Subject: [PATCH 326/515] Fix test 02497_trace_events_stress_long --- .../02497_trace_events_stress_long.sh | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02497_trace_events_stress_long.sh b/tests/queries/0_stateless/02497_trace_events_stress_long.sh index 7dc72807c5a..f1dbb9c0399 100755 --- a/tests/queries/0_stateless/02497_trace_events_stress_long.sh +++ b/tests/queries/0_stateless/02497_trace_events_stress_long.sh @@ -44,4 +44,18 @@ timeout $TIMEOUT bash -c thread2 >/dev/null & wait $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%' SYNC" >/dev/null -$CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%'" + +# After this moment, the server can still run another query. +# For example, the 'timeout' command killed all threads of thread1, +# and the 'timeout' itself has finished, and we have successfully 'wait'-ed for it, +# but just before that, one of the threads successfully sent a query to the server, +# but the server didn't start to run this query yet, +# and even when the KILL QUERY was run, the query from the thread didn't start, +# but only started after the KILL QUERY has been already processed. + +# That's why we have to run the next command in a loop. + +for _ in {1..10} +do + $CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%'" | rg '^0$' && break +done From 21c9feeeb3af922fb621688d8bfc7459d9a50f1a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 05:46:10 +0200 Subject: [PATCH 327/515] Fix test --- .../02497_trace_events_stress_long.sh | 26 ++++++++++--------- 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02497_trace_events_stress_long.sh b/tests/queries/0_stateless/02497_trace_events_stress_long.sh index f1dbb9c0399..3ec729079b8 100755 --- a/tests/queries/0_stateless/02497_trace_events_stress_long.sh +++ b/tests/queries/0_stateless/02497_trace_events_stress_long.sh @@ -43,19 +43,21 @@ timeout $TIMEOUT bash -c thread2 >/dev/null & wait -$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%' SYNC" >/dev/null - -# After this moment, the server can still run another query. -# For example, the 'timeout' command killed all threads of thread1, -# and the 'timeout' itself has finished, and we have successfully 'wait'-ed for it, -# but just before that, one of the threads successfully sent a query to the server, -# but the server didn't start to run this query yet, -# and even when the KILL QUERY was run, the query from the thread didn't start, -# but only started after the KILL QUERY has been already processed. - -# That's why we have to run the next command in a loop. - for _ in {1..10} do + $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%' SYNC" >/dev/null + + # After this moment, the server can still run another query. + # For example, the 'timeout' command killed all threads of thread1, + # and the 'timeout' itself has finished, and we have successfully 'wait'-ed for it, + # but just before that, one of the threads successfully sent a query to the server, + # but the server didn't start to run this query yet, + # and even when the KILL QUERY was run, the query from the thread didn't start, + # but only started after the KILL QUERY has been already processed. + + # That's why we have to run this in a loop. + $CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%'" | rg '^0$' && break + + sleep 1 done From 8e6f6655853cfd1106f96c75fddb4cd57f39f8a0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 05:57:38 +0200 Subject: [PATCH 328/515] Fix build --- src/Functions/IFunction.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index 1e4f8bf1102..c5b9a78015d 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -11,6 +11,11 @@ #include +#if USE_EMBEDDED_COMPILER +# include +#endif + + /// This file contains user interface for functions. namespace llvm From ae08fb20198a953a31ea5930e5890eded39b0642 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 26 Jun 2023 12:02:09 +0200 Subject: [PATCH 329/515] no finalize in d-tor WriteBufferFromOStream --- src/IO/WriteBufferFromOStream.cpp | 14 +------------- src/IO/WriteBufferFromOStream.h | 2 -- 2 files changed, 1 insertion(+), 15 deletions(-) diff --git a/src/IO/WriteBufferFromOStream.cpp b/src/IO/WriteBufferFromOStream.cpp index e0ec0b770e2..ffc3e62e9a6 100644 --- a/src/IO/WriteBufferFromOStream.cpp +++ b/src/IO/WriteBufferFromOStream.cpp @@ -19,14 +19,7 @@ void WriteBufferFromOStream::nextImpl() ostr->flush(); if (!ostr->good()) - { - /// FIXME do not call finalize in dtors (and remove iostreams) - bool avoid_throwing_exceptions = std::uncaught_exceptions(); - if (avoid_throwing_exceptions) - LOG_ERROR(&Poco::Logger::get("WriteBufferFromOStream"), "Cannot write to ostream at offset {}. Stack trace: {}", count(), StackTrace().toString()); - else - throw Exception(ErrorCodes::CANNOT_WRITE_TO_OSTREAM, "Cannot write to ostream at offset {}", count()); - } + throw Exception(ErrorCodes::CANNOT_WRITE_TO_OSTREAM, "Cannot write to ostream at offset {}", count()); } WriteBufferFromOStream::WriteBufferFromOStream( @@ -46,9 +39,4 @@ WriteBufferFromOStream::WriteBufferFromOStream( { } -WriteBufferFromOStream::~WriteBufferFromOStream() -{ - finalize(); -} - } diff --git a/src/IO/WriteBufferFromOStream.h b/src/IO/WriteBufferFromOStream.h index f8b45c2fa59..5a933739cb1 100644 --- a/src/IO/WriteBufferFromOStream.h +++ b/src/IO/WriteBufferFromOStream.h @@ -18,8 +18,6 @@ public: char * existing_memory = nullptr, size_t alignment = 0); - ~WriteBufferFromOStream() override; - protected: explicit WriteBufferFromOStream(size_t size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0); From 1f60a6ed4e1040623f7482a64a2ae493996be3e7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 26 Jun 2023 12:34:11 +0200 Subject: [PATCH 330/515] Fix --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 5 +---- src/Interpreters/Cache/FileSegment.cpp | 2 +- src/Interpreters/Cache/FileSegment.h | 2 -- 3 files changed, 2 insertions(+), 7 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 76d54f9d27c..960d2a72410 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -510,9 +510,6 @@ bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext() current_file_segment->use(); implementation_buffer = getImplementationBuffer(*current_file_segment); - if (read_type == ReadType::CACHED) - current_file_segment->incrementHitsCount(); - LOG_TEST( log, "New segment range: {}, old range: {}", current_file_segment->range().toString(), completed_range.toString()); @@ -857,7 +854,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() implementation_buffer = getImplementationBuffer(file_segments->front()); if (read_type == ReadType::CACHED) - file_segments->front().incrementHitsCount(); + file_segments->front().use(); } chassert(!internal_buffer.empty()); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 95592fc7c12..a77f0726d74 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -898,7 +898,7 @@ void FileSegment::use() if (it) { auto cache_lock = cache->lockCache(); - it->use(cache_lock); + hits_count = it->use(cache_lock); } } diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 681c0d719e4..2e6bbe5657e 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -180,8 +180,6 @@ public: size_t getRefCount() const { return ref_count; } - void incrementHitsCount() { ++hits_count; } - size_t getCurrentWriteOffset(bool sync) const; size_t getFirstNonDownloadedOffset(bool sync) const; From e92035072b7b3367da12089b28041893eb90e636 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 26 Jun 2023 13:44:09 +0300 Subject: [PATCH 331/515] Update MergeTreeTransaction.cpp --- src/Interpreters/MergeTreeTransaction.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 1358e3ed3c2..6b8e09a64f5 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -326,6 +326,8 @@ void MergeTreeTransaction::afterFinalize() is_read_only = storages.empty(); /// Release shared pointers just in case + creating_parts.clear(); + removing_parts.clear(); storages.clear(); mutations.clear(); finalized = true; From eb649873b379a29fec5584205558f70641f16bba Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 26 Jun 2023 13:48:21 +0300 Subject: [PATCH 332/515] Unify merge predicate (#51344) * unify merge predicate * Update BaseDaemon.cpp --- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 234 +++++++++--------- .../MergeTree/ReplicatedMergeTreeQueue.h | 95 +++---- 2 files changed, 175 insertions(+), 154 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 03ded2ef260..3ba3048b812 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1530,7 +1530,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( Int64 ReplicatedMergeTreeQueue::getCurrentMutationVersion( - const String & partition_id, Int64 data_version, std::lock_guard & /* state_lock */) const + const String & partition_id, Int64 data_version) const { auto in_partition = mutations_by_partition.find(partition_id); if (in_partition == mutations_by_partition.end()) @@ -2108,24 +2108,19 @@ ReplicatedMergeTreeQueue::QueueLocks ReplicatedMergeTreeQueue::lockQueue() } LocalMergePredicate::LocalMergePredicate(ReplicatedMergeTreeQueue & queue_) - : queue(queue_) { + /// Use only information that can be quickly accessed locally without querying ZooKeeper + virtual_parts_ = &queue_.virtual_parts; + mutations_state_ = &queue_; + virtual_parts_mutex = &queue_.state_mutex; } -ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( - ReplicatedMergeTreeQueue & queue_, zkutil::ZooKeeperPtr & zookeeper, std::optional && partition_ids_hint_) - : nested_pred(queue_) - , queue(queue_) - , partition_ids_hint(std::move(partition_ids_hint_)) - , prev_virtual_parts(queue.format_version) -{ - { - std::lock_guard lock(queue.state_mutex); - prev_virtual_parts = queue.virtual_parts; - } - /// Load current quorum status. - auto quorum_status_future = zookeeper->asyncTryGet(fs::path(queue.zookeeper_path) / "quorum" / "status"); +template +CommittingBlocks BaseMergePredicate::getCommittingBlocks( + zkutil::ZooKeeperPtr & zookeeper, const std::string & zookeeper_path, Poco::Logger * log_) +{ + CommittingBlocks committing_blocks; /// Load current inserts /// Hint avoids listing partitions that we don't really need. @@ -2133,14 +2128,14 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( /// so without hint it can do a few thousands requests (if not using MultiRead). Strings partitions; if (!partition_ids_hint) - partitions = zookeeper->getChildren(fs::path(queue.zookeeper_path) / "block_numbers"); + partitions = zookeeper->getChildren(fs::path(zookeeper_path) / "block_numbers"); else std::copy(partition_ids_hint->begin(), partition_ids_hint->end(), std::back_inserter(partitions)); std::vector paths; paths.reserve(partitions.size()); for (const String & partition : partitions) - paths.push_back(fs::path(queue.zookeeper_path) / "block_numbers" / partition); + paths.push_back(fs::path(zookeeper_path) / "block_numbers" / partition); auto locks_children = zookeeper->tryGetChildren(paths); @@ -2153,22 +2148,40 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( if (response.error != Coordination::Error::ZOK) { /// Probably a wrong hint was provided (it's ok if a user passed non-existing partition to OPTIMIZE) - LOG_WARNING(queue.log, "Partition id '{}' was provided as a hint, but there's not such partition in ZooKeeper", partitions[i]); + LOG_WARNING(log_, "Partition id '{}' was provided as a hint, but there's not such partition in ZooKeeper", partitions[i]); partition_ids_hint->erase(partitions[i]); continue; } - Strings partition_block_numbers = response.names; + Strings partition_block_numbers = locks_children[i].names; for (const String & entry : partition_block_numbers) { if (!startsWith(entry, "block-")) continue; + Int64 block_number = parse(entry.substr(strlen("block-"))); - String zk_path = fs::path(queue.zookeeper_path) / "block_numbers" / partitions[i] / entry; committing_blocks[partitions[i]].insert(block_number); } } + return committing_blocks; +} + +ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( + ReplicatedMergeTreeQueue & queue_, zkutil::ZooKeeperPtr & zookeeper, std::optional && partition_ids_hint_) + : BaseMergePredicate(std::move(partition_ids_hint_)) + , queue(queue_) +{ + { + std::lock_guard lock(queue.state_mutex); + prev_virtual_parts = std::make_shared(queue.virtual_parts); + } + + /// Load current quorum status. + auto quorum_status_future = zookeeper->asyncTryGet(fs::path(queue.zookeeper_path) / "quorum" / "status"); + + committing_blocks = std::make_shared(getCommittingBlocks(zookeeper, queue.zookeeper_path, queue.log)); + merges_version = queue_.pullLogsToQueue(zookeeper, {}, ReplicatedMergeTreeQueue::MERGE_PREDICATE); { @@ -2179,7 +2192,8 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( /// /// If pinned parts are fetched after logs are pulled then we can safely say that it contains all locks up to `merges_version`. String s = zookeeper->get(queue.zookeeper_path + "/pinned_part_uuids"); - pinned_part_uuids.fromString(s); + pinned_part_uuids = std::make_shared(); + pinned_part_uuids->fromString(s); } Coordination::GetResponse quorum_status_response = quorum_status_future.get(); @@ -2187,13 +2201,21 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( { ReplicatedMergeTreeQuorumEntry quorum_status; quorum_status.fromString(quorum_status_response.data); - inprogress_quorum_part = quorum_status.part_name; + inprogress_quorum_part = std::make_shared(quorum_status.part_name); } - else - inprogress_quorum_part.clear(); + + /// Use all information about parts + prev_virtual_parts_ = prev_virtual_parts.get(); + virtual_parts_ = &queue.virtual_parts; + committing_blocks_ = committing_blocks.get(); + pinned_part_uuids_ = pinned_part_uuids.get(); + inprogress_quorum_part_ = inprogress_quorum_part.get(); + mutations_state_ = &queue; + virtual_parts_mutex = &queue.state_mutex; } -bool LocalMergePredicate::operator()( +template +bool BaseMergePredicate::operator()( const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, const MergeTreeTransaction *, @@ -2205,20 +2227,8 @@ bool LocalMergePredicate::operator()( return canMergeSinglePart(right, out_reason); } -bool ReplicatedMergeTreeMergePredicate::operator()( - const MergeTreeData::DataPartPtr & left, - const MergeTreeData::DataPartPtr & right, - const MergeTreeTransaction *, - String * out_reason) const -{ - if (left) - return canMergeTwoParts(left, right, out_reason); - else - return canMergeSinglePart(right, out_reason); -} - - -bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( +template +bool BaseMergePredicate::canMergeTwoParts( const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, String * out_reason) const @@ -2263,21 +2273,21 @@ bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( for (const MergeTreeData::DataPartPtr & part : {left, right}) { - if (pinned_part_uuids.part_uuids.contains(part->uuid)) + if (pinned_part_uuids_ && pinned_part_uuids_->part_uuids.contains(part->uuid)) { if (out_reason) *out_reason = "Part " + part->name + " has uuid " + toString(part->uuid) + " which is currently pinned"; return false; } - if (part->name == inprogress_quorum_part) + if (inprogress_quorum_part_ && part->name == *inprogress_quorum_part_) { if (out_reason) *out_reason = "Quorum insert for part " + part->name + " is currently in progress"; return false; } - if (prev_virtual_parts.getContainingPart(part->info).empty()) + if (prev_virtual_parts_ && prev_virtual_parts_->getContainingPart(part->info).empty()) { if (out_reason) *out_reason = "Entry for part " + part->name + " hasn't been read from the replication log yet"; @@ -2290,7 +2300,7 @@ bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( if (left_max_block > right_min_block) std::swap(left_max_block, right_min_block); - if (left_max_block + 1 < right_min_block) + if (committing_blocks_ && left_max_block + 1 < right_min_block) { if (partition_ids_hint && !partition_ids_hint->contains(left->info.partition_id)) { @@ -2299,8 +2309,8 @@ bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( return false; } - auto committing_blocks_in_partition = committing_blocks.find(left->info.partition_id); - if (committing_blocks_in_partition != committing_blocks.end()) + auto committing_blocks_in_partition = committing_blocks_->find(left->info.partition_id); + if (committing_blocks_in_partition != committing_blocks_->end()) { const std::set & block_numbers = committing_blocks_in_partition->second; @@ -2316,109 +2326,107 @@ bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( } } - return nested_pred.canMergeTwoParts(left, right, out_reason); -} + std::unique_lock lock; + if (virtual_parts_mutex) + lock = std::unique_lock(*virtual_parts_mutex); -bool LocalMergePredicate::canMergeTwoParts( - const MergeTreeData::DataPartPtr & left, - const MergeTreeData::DataPartPtr & right, - String * out_reason) const -{ - Int64 left_max_block = left->info.max_block; - Int64 right_min_block = right->info.min_block; - - std::lock_guard lock(queue.state_mutex); - - for (const MergeTreeData::DataPartPtr & part : {left, right}) + if (virtual_parts_) { - /// We look for containing parts in queue.virtual_parts (and not in prev_virtual_parts) because queue.virtual_parts is newer - /// and it is guaranteed that it will contain all merges assigned before this object is constructed. - String containing_part = queue.virtual_parts.getContainingPart(part->info); - if (containing_part != part->name) + for (const MergeTreeData::DataPartPtr & part : {left, right}) { - if (out_reason) - *out_reason = "Part " + part->name + " has already been assigned a merge into " + containing_part; - return false; + /// We look for containing parts in queue.virtual_parts (and not in prev_virtual_parts) because queue.virtual_parts is newer + /// and it is guaranteed that it will contain all merges assigned before this object is constructed. + String containing_part = virtual_parts_->getContainingPart(part->info); + if (containing_part != part->name) + { + if (out_reason) + *out_reason = "Part " + part->name + " has already been assigned a merge into " + containing_part; + return false; + } + } + + if (left_max_block + 1 < right_min_block) + { + /// Fake part which will appear as merge result + MergeTreePartInfo gap_part_info( + left->info.partition_id, left_max_block + 1, right_min_block - 1, + MergeTreePartInfo::MAX_LEVEL, MergeTreePartInfo::MAX_BLOCK_NUMBER); + + /// We don't select parts if any smaller part covered by our merge must exist after + /// processing replication log up to log_pointer. + Strings covered = virtual_parts_->getPartsCoveredBy(gap_part_info); + if (!covered.empty()) + { + if (out_reason) + *out_reason = "There are " + toString(covered.size()) + " parts (from " + covered.front() + + " to " + covered.back() + ") that are still not present or being processed by " + + " other background process on this replica between " + left->name + " and " + right->name; + return false; + } } } - if (left_max_block + 1 < right_min_block) + if (mutations_state_) { - /// Fake part which will appear as merge result - MergeTreePartInfo gap_part_info( - left->info.partition_id, left_max_block + 1, right_min_block - 1, - MergeTreePartInfo::MAX_LEVEL, MergeTreePartInfo::MAX_BLOCK_NUMBER); + Int64 left_mutation_ver = mutations_state_->getCurrentMutationVersion( + left->info.partition_id, left->info.getDataVersion()); - /// We don't select parts if any smaller part covered by our merge must exist after - /// processing replication log up to log_pointer. - Strings covered = queue.virtual_parts.getPartsCoveredBy(gap_part_info); - if (!covered.empty()) + Int64 right_mutation_ver = mutations_state_->getCurrentMutationVersion( + left->info.partition_id, right->info.getDataVersion()); + + if (left_mutation_ver != right_mutation_ver) { if (out_reason) - *out_reason = "There are " + toString(covered.size()) + " parts (from " + covered.front() - + " to " + covered.back() + ") that are still not present or being processed by " - + " other background process on this replica between " + left->name + " and " + right->name; + *out_reason = "Current mutation versions of parts " + left->name + " and " + right->name + " differ: " + + toString(left_mutation_ver) + " and " + toString(right_mutation_ver) + " respectively"; return false; } } - Int64 left_mutation_ver = queue.getCurrentMutationVersion( - left->info.partition_id, left->info.getDataVersion(), lock); - - Int64 right_mutation_ver = queue.getCurrentMutationVersion( - left->info.partition_id, right->info.getDataVersion(), lock); - - if (left_mutation_ver != right_mutation_ver) - { - if (out_reason) - *out_reason = "Current mutation versions of parts " + left->name + " and " + right->name + " differ: " - + toString(left_mutation_ver) + " and " + toString(right_mutation_ver) + " respectively"; - return false; - } - return MergeTreeData::partsContainSameProjections(left, right); } -bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart( +template +bool BaseMergePredicate::canMergeSinglePart( const MergeTreeData::DataPartPtr & part, String * out_reason) const { - if (pinned_part_uuids.part_uuids.contains(part->uuid)) + if (pinned_part_uuids_ && pinned_part_uuids_->part_uuids.contains(part->uuid)) { if (out_reason) *out_reason = fmt::format("Part {} has uuid {} which is currently pinned", part->name, part->uuid); return false; } - if (part->name == inprogress_quorum_part) + if (inprogress_quorum_part_ && part->name == *inprogress_quorum_part_) { if (out_reason) *out_reason = fmt::format("Quorum insert for part {} is currently in progress", part->name); return false; } - if (prev_virtual_parts.getContainingPart(part->info).empty()) + if (prev_virtual_parts_ && prev_virtual_parts_->getContainingPart(part->info).empty()) { if (out_reason) *out_reason = fmt::format("Entry for part {} hasn't been read from the replication log yet", part->name); return false; } - return nested_pred.canMergeSinglePart(part, out_reason); -} + std::unique_lock lock; + if (virtual_parts_mutex) + lock = std::unique_lock(*virtual_parts_mutex); -bool LocalMergePredicate::canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String * out_reason) const -{ - std::lock_guard lock(queue.state_mutex); - - /// We look for containing parts in queue.virtual_parts (and not in prev_virtual_parts) because queue.virtual_parts is newer - /// and it is guaranteed that it will contain all merges assigned before this object is constructed. - String containing_part = queue.virtual_parts.getContainingPart(part->info); - if (containing_part != part->name) + if (virtual_parts_) { - if (out_reason) - *out_reason = fmt::format("Part {} has already been assigned a merge into {}", part->name, containing_part); - return false; + /// We look for containing parts in queue.virtual_parts (and not in prev_virtual_parts) because queue.virtual_parts is newer + /// and it is guaranteed that it will contain all merges assigned before this object is constructed. + String containing_part = virtual_parts_->getContainingPart(part->info); + if (containing_part != part->name) + { + if (out_reason) + *out_reason = fmt::format("Part {} has already been assigned a merge into {}", part->name, containing_part); + return false; + } } return true; @@ -2459,7 +2467,7 @@ std::optional> ReplicatedMergeTreeMergePredicate::getDesir /// We cannot mutate part if it's being inserted with quorum and it's not /// already reached. - if (part->name == inprogress_quorum_part) + if (inprogress_quorum_part && part->name == *inprogress_quorum_part) return {}; std::lock_guard lock(queue.state_mutex); @@ -2474,7 +2482,7 @@ std::optional> ReplicatedMergeTreeMergePredicate::getDesir UInt64 mutations_limit = queue.storage.getSettings()->replicated_max_mutations_in_one_entry; UInt64 mutations_count = 0; - Int64 current_version = queue.getCurrentMutationVersion(part->info.partition_id, part->info.getDataVersion(), lock); + Int64 current_version = queue.getCurrentMutationVersion(part->info.partition_id, part->info.getDataVersion()); Int64 max_version = in_partition->second.begin()->first; int alter_version = -1; @@ -2548,8 +2556,8 @@ bool ReplicatedMergeTreeMergePredicate::isMutationFinished(const std::string & z if (partition_ids_hint && !partition_ids_hint->contains(partition_id)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Partition id {} was not provided as hint, it's a bug", partition_id); - auto partition_it = committing_blocks.find(partition_id); - if (partition_it != committing_blocks.end()) + auto partition_it = committing_blocks->find(partition_id); + if (partition_it != committing_blocks->end()) { size_t blocks_count = std::distance( partition_it->second.begin(), partition_it->second.lower_bound(block_num)); @@ -2671,4 +2679,6 @@ void ReplicatedMergeTreeQueue::removeCurrentPartsFromMutations() removeCoveredPartsFromMutations(part_name, /*remove_part = */ false, /*remove_covered_parts = */ true); } +template class BaseMergePredicate; + } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 79572e13963..f205526a660 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -34,6 +34,7 @@ private: friend class CurrentlyExecuting; friend class LocalMergePredicate; friend class ReplicatedMergeTreeMergePredicate; + template friend class BaseMergePredicate; friend class MergeFromLogEntryTask; friend class ReplicatedMergeMutateTaskBase; @@ -212,7 +213,7 @@ private: /// with getDataVersion() == data_version. (Either this mutation was already applied or the part /// was created after the mutation). /// If there is no such mutation or it has already been executed and deleted, return 0. - Int64 getCurrentMutationVersion(const String & partition_id, Int64 data_version, std::lock_guard & /* state_lock */) const; + Int64 getCurrentMutationVersion(const String & partition_id, Int64 data_version) const; /** Check that part isn't in currently generating parts and isn't covered by them. * Should be called under state_mutex. @@ -491,33 +492,14 @@ public: void createLogEntriesToFetchBrokenParts(); }; -/// Lightweight version of ReplicatedMergeTreeMergePredicate that do not make any ZooKeeper requests, -/// but may return false-positive results. Checks only a subset of required conditions. -class LocalMergePredicate +using CommittingBlocks = std::unordered_map>; + +template +class BaseMergePredicate { public: - LocalMergePredicate(ReplicatedMergeTreeQueue & queue_); - - bool operator()(const MergeTreeData::DataPartPtr & left, - const MergeTreeData::DataPartPtr & right, - const MergeTreeTransaction * txn, - String * out_reason = nullptr) const; - - bool canMergeTwoParts(const MergeTreeData::DataPartPtr & left, - const MergeTreeData::DataPartPtr & right, - String * out_reason = nullptr) const; - - bool canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String * out_reason) const; - -private: - const ReplicatedMergeTreeQueue & queue; -}; - -class ReplicatedMergeTreeMergePredicate -{ -public: - ReplicatedMergeTreeMergePredicate(ReplicatedMergeTreeQueue & queue_, zkutil::ZooKeeperPtr & zookeeper, - std::optional && partition_ids_hint_); + BaseMergePredicate() = default; + BaseMergePredicate(std::optional && partition_ids_hint_) : partition_ids_hint(std::move(partition_ids_hint_)) {} /// Depending on the existence of left part checks a merge predicate for two parts or for single part. bool operator()(const MergeTreeData::DataPartPtr & left, @@ -537,6 +519,46 @@ public: /// This predicate is checked for the first part of each range. bool canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String * out_reason) const; + CommittingBlocks getCommittingBlocks(zkutil::ZooKeeperPtr & zookeeper, const std::string & zookeeper_path, Poco::Logger * log_); + +protected: + /// A list of partitions that can be used in the merge predicate + std::optional partition_ids_hint; + + /// A snapshot of active parts that would appear if the replica executes all log entries in its queue. + const VirtualPartsT * prev_virtual_parts_ = nullptr; + const VirtualPartsT * virtual_parts_ = nullptr; + + /// partition ID -> block numbers of the inserts and mutations that are about to commit + /// (loaded at some later time than prev_virtual_parts). + const CommittingBlocks * committing_blocks_ = nullptr; + + /// List of UUIDs for parts that have their identity "pinned". + const PinnedPartUUIDs * pinned_part_uuids_ = nullptr; + + /// Quorum state taken at some later time than prev_virtual_parts. + const String * inprogress_quorum_part_ = nullptr; + + /// An object that provides current mutation version for a part + const MutationsStateT * mutations_state_ = nullptr; + + std::mutex * virtual_parts_mutex = nullptr; +}; + +/// Lightweight version of ReplicatedMergeTreeMergePredicate that do not make any ZooKeeper requests, +/// but may return false-positive results. Checks only a subset of required conditions. +class LocalMergePredicate : public BaseMergePredicate +{ +public: + LocalMergePredicate(ReplicatedMergeTreeQueue & queue_); +}; + +class ReplicatedMergeTreeMergePredicate : public BaseMergePredicate +{ +public: + ReplicatedMergeTreeMergePredicate(ReplicatedMergeTreeQueue & queue_, zkutil::ZooKeeperPtr & zookeeper, + std::optional && partition_ids_hint_); + /// Returns true if part is needed for some REPLACE_RANGE entry. /// We should not drop part in this case, because replication queue may stuck without that part. bool partParticipatesInReplaceRange(const MergeTreeData::DataPartPtr & part, String * out_reason) const; @@ -561,28 +583,17 @@ public: String getCoveringVirtualPart(const String & part_name) const; private: - LocalMergePredicate nested_pred; - const ReplicatedMergeTreeQueue & queue; - std::optional partition_ids_hint; - - /// A snapshot of active parts that would appear if the replica executes all log entries in its queue. - ActiveDataPartSet prev_virtual_parts; - /// partition ID -> block numbers of the inserts and mutations that are about to commit - /// (loaded at some later time than prev_virtual_parts). - std::unordered_map> committing_blocks; - - /// List of UUIDs for parts that have their identity "pinned". - PinnedPartUUIDs pinned_part_uuids; - - /// Quorum state taken at some later time than prev_virtual_parts. - String inprogress_quorum_part; + /// We copy a merge predicate when we cast it to AllowedMergingPredicate, let's keep the pointers valid + std::shared_ptr prev_virtual_parts; + std::shared_ptr committing_blocks; + std::shared_ptr pinned_part_uuids; + std::shared_ptr inprogress_quorum_part; int32_t merges_version = -1; }; - /** Convert a number to a string in the format of the suffixes of auto-incremental nodes in ZooKeeper. * Negative numbers are also supported - for them the name of the node looks somewhat silly * and does not match any auto-incremented node in ZK. From c64f929b9cb57ca171dbe28e82b60a956004daf8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 26 Jun 2023 12:10:58 +0200 Subject: [PATCH 333/515] Fix broken labeling for `manual approve` --- tests/ci/workflow_approve_rerun_lambda/app.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 3db62430d85..1a3874ad01a 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -224,8 +224,8 @@ def approve_run(workflow_description: WorkflowDescription, token: str) -> None: def label_manual_approve(pull_request, token): - url = f"{pull_request['url']}/labels" - data = {"labels": "manual approve"} + url = f"{pull_request['issue_url']}/labels" + data = {"labels": ["manual approve"]} _exec_post_with_retry(url, token, data) From 065b87aa30d6e6a3f7e64ef3fdfabdc79039a91d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 26 Jun 2023 12:21:20 +0200 Subject: [PATCH 334/515] Improve logging a little bit, avoid double labeling --- tests/ci/workflow_approve_rerun_lambda/app.py | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 1a3874ad01a..5e2331ece3c 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -376,11 +376,10 @@ def main(event): changed_files = get_changed_files_for_pull_request(pull_request, token) print(f"Totally have {len(changed_files)} changed files in PR:", changed_files) if check_suspicious_changed_files(changed_files): - print( - f"Pull Request {pull_request['number']} has suspicious changes, " - "label it for manuall approve" - ) - label_manual_approve(pull_request, token) + print(f"Pull Request {pull_request['number']} has suspicious changes") + if "manual approve" not in labels: + print("Label the PR as needed for manuall approve") + label_manual_approve(pull_request, token) else: print(f"Pull Request {pull_request['number']} has no suspicious changes") approve_run(workflow_description, token) From 9af56624f523b63a078815fc5e31bc16f49821c9 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 26 Jun 2023 13:30:51 +0200 Subject: [PATCH 335/515] Fix flaky test test_skip_empty_files --- tests/integration/test_storage_s3/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index a7293337a9e..6c251d2f84e 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1776,7 +1776,7 @@ def test_skip_empty_files(started_cluster): assert int(res) == 0 res = instance.query( - f"select * from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files{{11|1|22}}.parquet') settings engine_url_skip_empty_files=1" + f"select * from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files{{11|1|22}}.parquet', auto, 'number UInt64') settings engine_url_skip_empty_files=1" ) assert len(res.strip()) == 0 From 7d8d19d8003ebaaac910a6af802ee4874e1821f8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 26 Jun 2023 14:27:13 +0200 Subject: [PATCH 336/515] Add test --- .../tests/gtest_lru_file_cache.cpp | 77 +++++++++++++++++++ 1 file changed, 77 insertions(+) diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 3cba1e48e1e..9ff9f92afe4 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -22,6 +22,8 @@ #include #include +#include +#include #include namespace fs = std::filesystem; @@ -862,3 +864,78 @@ TEST_F(FileCacheTest, temporaryData) ASSERT_LE(file_cache.getUsedCacheSize(), size_used_before_temporary_data); ASSERT_LE(file_cache.getFileSegmentsNum(), segments_used_before_temporary_data); } + +TEST_F(FileCacheTest, CachedReadBuffer) +{ + DB::ThreadStatus thread_status; + + /// To work with cache need query_id and query context. + std::string query_id = "query_id"; + + Poco::XML::DOMParser dom_parser; + std::string xml(R"CONFIG( +)CONFIG"); + Poco::AutoPtr document = dom_parser.parseString(xml); + Poco::AutoPtr config = new Poco::Util::XMLConfiguration(document); + getMutableContext().context->setConfig(config); + + auto query_context = DB::Context::createCopy(getContext().context); + query_context->makeQueryContext(); + query_context->setCurrentQueryId(query_id); + chassert(&DB::CurrentThread::get() == &thread_status); + DB::CurrentThread::QueryScope query_scope_holder(query_context); + + DB::FileCacheSettings settings; + settings.base_path = cache_base_path; + settings.max_file_segment_size = 5; + settings.max_size = 30; + settings.max_elements = 10; + settings.boundary_alignment = 1; + + ReadSettings read_settings; + read_settings.enable_filesystem_cache = 1; + read_settings.local_fs_method = LocalFSReadMethod::pread; + + std::string file_path = fs::current_path() / "test"; + auto read_buffer_creator = [&]() + { + return createReadBufferFromFileBase(file_path, read_settings, std::nullopt, std::nullopt); + }; + + auto wb = std::make_unique(file_path, DBMS_DEFAULT_BUFFER_SIZE); + std::string s(30, '*'); + wb->write(s.data(), s.size()); + wb->next(); + wb->finalize(); + + auto cache = std::make_shared(settings); + cache->initialize(); + auto key = cache->createKeyForPath(file_path); + + { + auto cached_buffer = std::make_shared( + file_path, key, cache, read_buffer_creator, read_settings, "test", s.size(), false, false, std::nullopt, nullptr); + + WriteBufferFromOwnString result; + copyData(*cached_buffer, result); + ASSERT_EQ(result.str(), s); + + assertEqual(cache->dumpQueue(), { Range(0, 4), Range(5, 9), Range(10, 14), Range(15, 19), Range(20, 24), Range(25, 29) }); + } + + { + ReadSettings modified_settings{read_settings}; + modified_settings.local_fs_buffer_size = 10; + modified_settings.remote_fs_buffer_size = 10; + + auto cached_buffer = std::make_shared( + file_path, key, cache, read_buffer_creator, modified_settings, "test", s.size(), false, false, std::nullopt, nullptr); + + cached_buffer->next(); + assertEqual(cache->dumpQueue(), { Range(5, 9), Range(10, 14), Range(15, 19), Range(20, 24), Range(25, 29), Range(0, 4) }); + + cached_buffer->position() = cached_buffer->buffer().end(); + cached_buffer->next(); + assertEqual(cache->dumpQueue(), {Range(10, 14), Range(15, 19), Range(20, 24), Range(25, 29), Range(0, 4), Range(5, 9) }); + } +} From 179a7ce20263fd3bc50431ee234a99520595aebe Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Sun, 25 Jun 2023 11:07:15 +0800 Subject: [PATCH 337/515] debug --- src/Interpreters/GraceHashJoin.cpp | 30 +++++++++++++++++++++++------- tests/ci/stress.py | 3 +++ 2 files changed, 26 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 197b64865e1..4218a8ea4e1 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -385,11 +385,23 @@ GraceHashJoin::Buckets GraceHashJoin::rehashBuckets(size_t to_size) void GraceHashJoin::addBucket(Buckets & destination) { - auto & left_file = tmp_data->createStream(left_sample_block); - auto & right_file = tmp_data->createStream(prepareRightBlock(right_sample_block)); + // There could be exceptions from createStream, In ci tests + // there is a certain probability of failure in allocating memory, see memory_tracker_fault_probability. + // It may terminate this thread and leave a broken hash_join, and another thread cores when it tries to + // use the broken hash_join. So we print an exception message here to help debug. + try + { + auto & left_file = tmp_data->createStream(left_sample_block); + auto & right_file = tmp_data->createStream(prepareRightBlock(right_sample_block)); - BucketPtr new_bucket = std::make_shared(destination.size(), left_file, right_file, log); - destination.emplace_back(std::move(new_bucket)); + BucketPtr new_bucket = std::make_shared(destination.size(), left_file, right_file, log); + destination.emplace_back(std::move(new_bucket)); + } + catch (...) + { + LOG_ERROR(&Poco::Logger::get("GraceHashJoin"), "Can't create bucket. current buckets size: {}", destination.size()); + throw; + } } void GraceHashJoin::checkTypesOfKeys(const Block & block) const @@ -626,7 +638,11 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) if (current_block.rows() > 0) { std::lock_guard lock(hash_join_mutex); - + auto current_buckets = getCurrentBuckets(); + if (!isPowerOf2(current_buckets.size())) [[unlikely]] + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Broken buckets. its size({}) is not power of 2", current_buckets.size()); + } if (!hash_join) hash_join = makeInMemoryJoin(); @@ -637,11 +653,11 @@ void GraceHashJoin::addJoinedBlockImpl(Block block) current_block = {}; + // Must use the latest buckets snapshot in case that it has been rehashed by other threads. + buckets_snapshot = rehashBuckets(current_buckets.size() * 2); auto right_blocks = hash_join->releaseJoinedBlocks(/* restructure */ false); hash_join = nullptr; - buckets_snapshot = rehashBuckets(buckets_snapshot.size() * 2); - { Blocks current_blocks; current_blocks.reserve(right_blocks.size()); diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 1aa483854fc..6d17384c63f 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -38,6 +38,9 @@ def get_options(i, upgrade_check): client_options.append("join_algorithm='partial_merge'") if join_alg_num % 5 == 2: client_options.append("join_algorithm='full_sorting_merge'") + if join_alg_num % 5 == 3 and not upgrade_check: + # Some crashes are not fixed in 23.2 yet, so ignore the setting in Upgrade check + client_options.append("join_algorithm='grace_hash'") if join_alg_num % 5 == 4: client_options.append("join_algorithm='auto'") client_options.append("max_rows_in_join=1000") From aab86da4e9bc1840d740c00fa02c599a36c3d04c Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Mon, 26 Jun 2023 20:42:21 +0800 Subject: [PATCH 338/515] increase max_bytes_in_join --- tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 index 98cc46c9cb4..7276e77dc16 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 +++ b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 @@ -30,7 +30,7 @@ SELECT 'skipped'; {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} -SET max_bytes_in_join = '{% if join_algorithm == 'grace_hash' %}1M{% else %}0{% endif %}'; +SET max_bytes_in_join = '{% if join_algorithm == 'grace_hash' %}16M{% else %}0{% endif %}'; SELECT '-- {{ join_algorithm }} --'; SET join_algorithm = '{{ join_algorithm }}'; From c4d190617c72436bfeae8563c636ed8e925e1849 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 26 Jun 2023 14:46:51 +0200 Subject: [PATCH 339/515] fix flacky test test_profile_events_s3 --- tests/integration/test_profile_events_s3/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_profile_events_s3/test.py b/tests/integration/test_profile_events_s3/test.py index 10c9385f865..de18142d4f6 100644 --- a/tests/integration/test_profile_events_s3/test.py +++ b/tests/integration/test_profile_events_s3/test.py @@ -139,7 +139,7 @@ def test_profile_events(cluster): ) stat1 = get_query_stat(instance, query1) for metric in stat1: - assert stat1[metric] == metrics1[metric] - metrics0[metric] + assert stat1[metric] == metrics1.get(metric, 0) - metrics0.get(metric, 0) assert ( metrics1["WriteBufferFromS3Bytes"] - metrics0["WriteBufferFromS3Bytes"] == size1 ) @@ -163,7 +163,7 @@ def test_profile_events(cluster): stat2 = get_query_stat(instance, query2) for metric in stat2: - assert stat2[metric] == metrics2[metric] - metrics1[metric] + assert stat2[metric] == metrics2.get(metric, 0)- metrics1.get(metric, 0) assert ( metrics2["WriteBufferFromS3Bytes"] - metrics1["WriteBufferFromS3Bytes"] @@ -189,4 +189,4 @@ def test_profile_events(cluster): # With async reads profile events are not updated fully because reads are done in a separate thread. # for metric in stat3: # print(metric) - # assert stat3[metric] == metrics3[metric] - metrics2[metric] + # assert stat3[metric] == metrics3.get(metric, 0) - metrics2.get(metric, 0) From 047f656980f01cc339043c1b5ba225bb6f4ce07e Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 26 Jun 2023 15:04:50 +0200 Subject: [PATCH 340/515] add explicit finalize calls --- src/Client/QueryFuzzer.cpp | 2 +- src/Parsers/MySQL/tests/gtest_create_parser.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index bfcfe659982..39ca7a5ed88 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -1244,7 +1244,7 @@ void QueryFuzzer::fuzzMain(ASTPtr & ast) std::cout << std::endl; WriteBufferFromOStream ast_buf(std::cout, 4096); formatAST(*ast, ast_buf, false /*highlight*/); - ast_buf.next(); + ast_buf.finalize(); std::cout << std::endl << std::endl; } diff --git a/src/Parsers/MySQL/tests/gtest_create_parser.cpp b/src/Parsers/MySQL/tests/gtest_create_parser.cpp index 554b3f0a67d..2f65eb6e592 100644 --- a/src/Parsers/MySQL/tests/gtest_create_parser.cpp +++ b/src/Parsers/MySQL/tests/gtest_create_parser.cpp @@ -40,5 +40,5 @@ TEST(CreateTableParser, SS) ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); WriteBufferFromOStream buf(std::cerr, 4096); ast->dumpTree(buf); - + buf.finalize(); } From 4fabce9cc85cf649af55c0bb5341dad15aa91231 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Mon, 26 Jun 2023 15:10:56 +0200 Subject: [PATCH 341/515] Update README.md --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 87fb4920222..eac036c2d9b 100644 --- a/README.md +++ b/README.md @@ -27,6 +27,7 @@ curl https://clickhouse.com/ | sh * [**ClickHouse Meetup in Boston**](https://www.meetup.com/clickhouse-boston-user-group/events/293913596) - Jul 18 * [**ClickHouse Meetup in NYC**](https://www.meetup.com/clickhouse-new-york-user-group/events/293913441) - Jul 19 * [**ClickHouse Meetup in Toronto**](https://www.meetup.com/clickhouse-toronto-user-group/events/294183127) - Jul 20 +* [**ClickHouse Meetup in Singapore**](https://www.meetup.com/clickhouse-singapore-meetup-group/events/294428050/) - Jul 27 Also, keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler clickhouse com. From 237805705c0239d3b252b3fa3b59f0214452ccd0 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Mon, 26 Jun 2023 15:44:30 +0200 Subject: [PATCH 342/515] Update test.py --- tests/integration/test_profile_events_s3/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_profile_events_s3/test.py b/tests/integration/test_profile_events_s3/test.py index de18142d4f6..a8e037d667f 100644 --- a/tests/integration/test_profile_events_s3/test.py +++ b/tests/integration/test_profile_events_s3/test.py @@ -163,7 +163,7 @@ def test_profile_events(cluster): stat2 = get_query_stat(instance, query2) for metric in stat2: - assert stat2[metric] == metrics2.get(metric, 0)- metrics1.get(metric, 0) + assert stat2[metric] == metrics2.get(metric, 0) - metrics1.get(metric, 0) assert ( metrics2["WriteBufferFromS3Bytes"] - metrics1["WriteBufferFromS3Bytes"] From d4d85d9fe92430e08c1006f2d6c9d935445473e8 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 26 Jun 2023 16:03:47 +0200 Subject: [PATCH 343/515] add explicit finalize calls --- programs/client/Client.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 14516bfa939..19b601b9a7b 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -790,7 +790,7 @@ bool Client::processWithFuzzing(const String & full_query) WriteBufferFromOStream cerr_buf(std::cerr, 4096); fuzz_base->dumpTree(cerr_buf); - cerr_buf.next(); + cerr_buf.finalize(); fmt::print( stderr, @@ -928,7 +928,7 @@ bool Client::processWithFuzzing(const String & full_query) std::cout << std::endl; WriteBufferFromOStream ast_buf(std::cout, 4096); formatAST(*query, ast_buf, false /*highlight*/); - ast_buf.next(); + ast_buf.finalize(); if (const auto * insert = query->as()) { /// For inserts with data it's really useful to have the data itself available in the logs, as formatAST doesn't print it From e6535b1f875931d46421166897a2a912b6b8a17d Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 26 Jun 2023 22:22:22 +0800 Subject: [PATCH 344/515] update libhdfs3 version --- contrib/libhdfs3 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libhdfs3 b/contrib/libhdfs3 index 164b89253fa..377220ef351 160000 --- a/contrib/libhdfs3 +++ b/contrib/libhdfs3 @@ -1 +1 @@ -Subproject commit 164b89253fad7991bce77882f01b51ab81d19f3d +Subproject commit 377220ef351ae24994a5fcd2b5fa3930d00c4db0 From 7cadfeac29c68dc9ac749cd48d115da02ae48f62 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 26 Jun 2023 14:55:03 +0000 Subject: [PATCH 345/515] Add retries to flaky tests --- tests/integration/test_tlsv1_3/test.py | 67 ++++++++++++++++++-------- 1 file changed, 47 insertions(+), 20 deletions(-) diff --git a/tests/integration/test_tlsv1_3/test.py b/tests/integration/test_tlsv1_3/test.py index e25216c67df..a29dcb5fc1c 100644 --- a/tests/integration/test_tlsv1_3/test.py +++ b/tests/integration/test_tlsv1_3/test.py @@ -4,6 +4,7 @@ from helpers.ssl_context import WrapSSLContextWithSNI import urllib.request, urllib.parse import ssl import os.path +import logging # The test cluster is configured with certificate for that host name, see 'server-ext.cnf'. @@ -11,6 +12,7 @@ import os.path SSL_HOST = "integration-tests.clickhouse.com" HTTPS_PORT = 8443 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +MAX_RETRY = 5 cluster = ClickHouseCluster(__file__) instance = cluster.add_instance( @@ -88,10 +90,18 @@ def test_https_wrong_cert(): execute_query_https("SELECT currentUser()", user="john", cert_name="client2") assert "HTTP Error 403" in str(err.value) + count = 0 # Wrong certificate: self-signed certificate. - with pytest.raises(Exception) as err: - execute_query_https("SELECT currentUser()", user="john", cert_name="wrong") - assert "unknown ca" in str(err.value) + while count <= MAX_RETRY: + with pytest.raises(Exception) as err: + execute_query_https("SELECT currentUser()", user="john", cert_name="wrong") + err_str = str(err.value) + if count < MAX_RETRY and "Broken pipe" in err_str: + count = count + 1 + logging.warning(f"Failed attempt with wrong cert, err: {err_str}") + continue + assert "unknown ca" in err_str + break # No certificate. with pytest.raises(Exception) as err: @@ -181,24 +191,41 @@ def test_https_non_ssl_auth(): == "jane\n" ) + count = 0 # However if we send a certificate it must not be wrong. - with pytest.raises(Exception) as err: - execute_query_https( - "SELECT currentUser()", - user="peter", - enable_ssl_auth=False, - cert_name="wrong", - ) - assert "unknown ca" in str(err.value) - with pytest.raises(Exception) as err: - execute_query_https( - "SELECT currentUser()", - user="jane", - enable_ssl_auth=False, - password="qwe123", - cert_name="wrong", - ) - assert "unknown ca" in str(err.value) + while count <= MAX_RETRY: + with pytest.raises(Exception) as err: + execute_query_https( + "SELECT currentUser()", + user="peter", + enable_ssl_auth=False, + cert_name="wrong", + ) + err_str = str(err.value) + if count < MAX_RETRY and "Broken pipe" in err_str: + count = count + 1 + logging.warning(f"Failed attempt with wrong cert, user: peter, err: {err_str}") + continue + assert "unknown ca" in err_str + break + + count = 0 + while count <= MAX_RETRY: + with pytest.raises(Exception) as err: + execute_query_https( + "SELECT currentUser()", + user="jane", + enable_ssl_auth=False, + password="qwe123", + cert_name="wrong", + ) + err_str = str(err.value) + if count < MAX_RETRY and "Broken pipe" in err_str: + count = count + 1 + logging.warning(f"Failed attempt with wrong cert, user: jane, err: {err_str}") + continue + assert "unknown ca" in err_str + break def test_create_user(): From 45c263cbb551c27e303f6a050df558ec63a3b463 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 26 Jun 2023 15:00:46 +0000 Subject: [PATCH 346/515] Replace try/catch logic in hasTokenOrNull() by sth more lightweight --- src/Common/StringSearcher.h | 16 ++++- src/Common/Volnitsky.h | 3 +- src/Functions/HasTokenImpl.h | 74 +++++++++++------------ src/Functions/hasToken.cpp | 12 ++-- src/Functions/hasTokenCaseInsensitive.cpp | 7 +-- 5 files changed, 59 insertions(+), 53 deletions(-) diff --git a/src/Common/StringSearcher.h b/src/Common/StringSearcher.h index ae440f9151b..120a79120bb 100644 --- a/src/Common/StringSearcher.h +++ b/src/Common/StringSearcher.h @@ -811,15 +811,24 @@ class TokenSearcher : public StringSearcherBase size_t needle_size; public: + + template + requires (sizeof(CharT) == 1) + static bool isValidNeedle(const CharT * needle_, size_t needle_size_) + { + if (std::any_of(needle_, needle_ + needle_size_, isTokenSeparator)) + return false; + return true; + } + template requires (sizeof(CharT) == 1) TokenSearcher(const CharT * needle_, size_t needle_size_) : searcher(needle_, needle_size_) , needle_size(needle_size_) { - if (std::any_of(needle_, needle_ + needle_size_, isTokenSeparator)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle must not contain whitespace or separator characters"); - + /// The caller is responsible for calling isValidNeedle() + chassert(isValidNeedle(needle_, needle_size_)); } template @@ -880,6 +889,7 @@ using ASCIICaseSensitiveStringSearcher = impl::StringSearcher; using ASCIICaseInsensitiveStringSearcher = impl::StringSearcher; using UTF8CaseSensitiveStringSearcher = impl::StringSearcher; using UTF8CaseInsensitiveStringSearcher = impl::StringSearcher; + using ASCIICaseSensitiveTokenSearcher = impl::TokenSearcher; using ASCIICaseInsensitiveTokenSearcher = impl::TokenSearcher; diff --git a/src/Common/Volnitsky.h b/src/Common/Volnitsky.h index 64c5bf4c0d3..8f9aa23a38a 100644 --- a/src/Common/Volnitsky.h +++ b/src/Common/Volnitsky.h @@ -406,8 +406,7 @@ public: /// And also adding from the end guarantees that we will find first occurrence because we will lookup bigger offsets first. for (auto i = static_cast(needle_size - sizeof(VolnitskyTraits::Ngram)); i >= 0; --i) { - bool ok = VolnitskyTraits::putNGram( - needle + i, static_cast(i + 1), needle, needle_size, callback); + bool ok = VolnitskyTraits::putNGram(needle + i, static_cast(i + 1), needle, needle_size, callback); /** `putNGramUTF8CaseInsensitive` does not work if characters with lower and upper cases * are represented by different number of bytes or code points. diff --git a/src/Functions/HasTokenImpl.h b/src/Functions/HasTokenImpl.h index 18e629b58fb..8cacdfff99d 100644 --- a/src/Functions/HasTokenImpl.h +++ b/src/Functions/HasTokenImpl.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -9,6 +10,7 @@ namespace DB namespace ErrorCodes { + extern const int BAD_ARGUMENTS; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -44,51 +46,47 @@ struct HasTokenImpl const UInt8 * const end = haystack_data.data() + haystack_data.size(); const UInt8 * pos = begin; - try + if (!ASCIICaseSensitiveTokenSearcher::isValidNeedle(pattern.data(), pattern.size())) { - /// Parameter `pattern` is supposed to be a literal of letters and/or numbers. - /// Otherwise, an exception from the constructor of `TokenSearcher` is thrown. - /// If no exception is thrown at that point, then no further error cases may occur. - TokenSearcher searcher(pattern.data(), pattern.size(), end - pos); if (res_null) - std::ranges::fill(res_null->getData(), false); - - /// The current index in the array of strings. - size_t i = 0; - /// We will search for the next occurrence in all rows at once. - while (pos < end && end != (pos = searcher.search(pos, end - pos))) - { - /// Let's determine which index it refers to. - while (begin + haystack_offsets[i] <= pos) - { - res[i] = negate; - ++i; - } - - /// We check that the entry does not pass through the boundaries of strings. - if (pos + pattern.size() < begin + haystack_offsets[i]) - res[i] = !negate; - else - res[i] = negate; - - pos = begin + haystack_offsets[i]; - ++i; - } - - /// Tail, in which there can be no substring. - if (i < res.size()) - memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); - } - catch (...) - { - if (!res_null) - throw; - else { std::ranges::fill(res, 0); std::ranges::fill(res_null->getData(), true); + return; } + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle must not contain whitespace or separator characters"); } + + TokenSearcher searcher(pattern.data(), pattern.size(), end - pos); + if (res_null) + std::ranges::fill(res_null->getData(), false); + + /// The current index in the array of strings. + size_t i = 0; + /// We will search for the next occurrence in all rows at once. + while (pos < end && end != (pos = searcher.search(pos, end - pos))) + { + /// Let's determine which index it refers to. + while (begin + haystack_offsets[i] <= pos) + { + res[i] = negate; + ++i; + } + + /// We check that the entry does not pass through the boundaries of strings. + if (pos + pattern.size() < begin + haystack_offsets[i]) + res[i] = !negate; + else + res[i] = negate; + + pos = begin + haystack_offsets[i]; + ++i; + } + + /// Tail, in which there can be no substring. + if (i < res.size()) + memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); } template diff --git a/src/Functions/hasToken.cpp b/src/Functions/hasToken.cpp index de67e80e752..b90750ea233 100644 --- a/src/Functions/hasToken.cpp +++ b/src/Functions/hasToken.cpp @@ -16,18 +16,18 @@ struct NameHasTokenOrNull static constexpr auto name = "hasTokenOrNull"; }; -using FunctionHasToken = DB::FunctionsStringSearch>; -using FunctionHasTokenOrNull = DB:: - FunctionsStringSearch, DB::ExecutionErrorPolicy::Null>; +using FunctionHasToken + = FunctionsStringSearch>; +using FunctionHasTokenOrNull + = FunctionsStringSearch, ExecutionErrorPolicy::Null>; REGISTER_FUNCTION(HasToken) { factory.registerFunction(FunctionDocumentation - {.description="Performs lookup of needle in haystack using tokenbf_v1 index."}, DB::FunctionFactory::CaseSensitive); + {.description="Performs lookup of needle in haystack using tokenbf_v1 index."}, FunctionFactory::CaseSensitive); factory.registerFunction(FunctionDocumentation - {.description="Performs lookup of needle in haystack using tokenbf_v1 index. Returns null if needle is ill-formed."}, - DB::FunctionFactory::CaseSensitive); + {.description="Performs lookup of needle in haystack using tokenbf_v1 index. Returns null if needle is ill-formed."}, FunctionFactory::CaseSensitive); } } diff --git a/src/Functions/hasTokenCaseInsensitive.cpp b/src/Functions/hasTokenCaseInsensitive.cpp index a6e8ecf3e9d..d7381e336b5 100644 --- a/src/Functions/hasTokenCaseInsensitive.cpp +++ b/src/Functions/hasTokenCaseInsensitive.cpp @@ -17,10 +17,9 @@ struct NameHasTokenCaseInsensitiveOrNull }; using FunctionHasTokenCaseInsensitive - = DB::FunctionsStringSearch>; -using FunctionHasTokenCaseInsensitiveOrNull = DB::FunctionsStringSearch< - DB::HasTokenImpl, - DB::ExecutionErrorPolicy::Null>; + = FunctionsStringSearch>; +using FunctionHasTokenCaseInsensitiveOrNull + = FunctionsStringSearch, ExecutionErrorPolicy::Null>; REGISTER_FUNCTION(HasTokenCaseInsensitive) { From 5de8c4ac679c0c79f0554681a3113be4a5c000d7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 26 Jun 2023 15:27:05 +0000 Subject: [PATCH 347/515] Automatic style fix --- tests/integration/test_tlsv1_3/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_tlsv1_3/test.py b/tests/integration/test_tlsv1_3/test.py index a29dcb5fc1c..f5c2be51ed7 100644 --- a/tests/integration/test_tlsv1_3/test.py +++ b/tests/integration/test_tlsv1_3/test.py @@ -204,7 +204,9 @@ def test_https_non_ssl_auth(): err_str = str(err.value) if count < MAX_RETRY and "Broken pipe" in err_str: count = count + 1 - logging.warning(f"Failed attempt with wrong cert, user: peter, err: {err_str}") + logging.warning( + f"Failed attempt with wrong cert, user: peter, err: {err_str}" + ) continue assert "unknown ca" in err_str break @@ -222,7 +224,9 @@ def test_https_non_ssl_auth(): err_str = str(err.value) if count < MAX_RETRY and "Broken pipe" in err_str: count = count + 1 - logging.warning(f"Failed attempt with wrong cert, user: jane, err: {err_str}") + logging.warning( + f"Failed attempt with wrong cert, user: jane, err: {err_str}" + ) continue assert "unknown ca" in err_str break From 4da82d10d0a7eba0a10fc8f02889d1bd533f7b82 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 26 Jun 2023 17:57:46 +0200 Subject: [PATCH 348/515] Update gtest_lru_file_cache.cpp --- src/Interpreters/tests/gtest_lru_file_cache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 9ff9f92afe4..58b1302a72c 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -893,7 +893,7 @@ TEST_F(FileCacheTest, CachedReadBuffer) settings.boundary_alignment = 1; ReadSettings read_settings; - read_settings.enable_filesystem_cache = 1; + read_settings.enable_filesystem_cache = true; read_settings.local_fs_method = LocalFSReadMethod::pread; std::string file_path = fs::current_path() / "test"; From a1eb27ce3038df0921d6667e65074168776988dc Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 26 Jun 2023 18:12:41 +0200 Subject: [PATCH 349/515] Update log message --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 28 ++++++++++++------- 1 file changed, 18 insertions(+), 10 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 76d54f9d27c..b65a368b936 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -878,10 +878,11 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() LOG_TEST( log, - "Current read type: {}, read offset: {}, impl offset: {}, file segment: {}", + "Current read type: {}, read offset: {}, impl offset: {}, impl position: {}, file segment: {}", toString(read_type), file_offset_of_buffer_end, implementation_buffer->getFileOffsetOfBufferEnd(), + implementation_buffer->getPosition(), file_segment.getInfoForLog()); chassert(current_read_range.left <= file_offset_of_buffer_end); @@ -940,7 +941,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() // We don't support implementation_buffer implementations that use nextimpl_working_buffer_offset. chassert(implementation_buffer->position() == implementation_buffer->buffer().begin()); - size = implementation_buffer->buffer().size(); + if (result) + size = implementation_buffer->buffer().size(); LOG_TEST( log, @@ -954,15 +956,21 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheBytes, size); ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheMicroseconds, elapsed); - const size_t new_file_offset = file_offset_of_buffer_end + size; - const size_t file_segment_write_offset = file_segment.getCurrentWriteOffset(true); - if (new_file_offset > file_segment.range().right + 1 || new_file_offset > file_segment_write_offset) + if (result) { - auto file_segment_path = file_segment.getPathInLocalCache(); - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Read unexpected size. File size: {}, file path: {}, file segment info: {}", - fs::file_size(file_segment_path), file_segment_path, file_segment.getInfoForLog()); + const size_t new_file_offset = file_offset_of_buffer_end + size; + const size_t file_segment_write_offset = file_segment.getCurrentWriteOffset(true); + if (new_file_offset > file_segment.range().right + 1 || new_file_offset > file_segment_write_offset) + { + auto file_segment_path = file_segment.getPathInLocalCache(); + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Read unexpected size. " + "File size: {}, file segment path: {}, impl size: {}, impl path: {}" + "file segment info: {}", + fs::file_size(file_segment_path), file_segment_path, + implementation_buffer->getFileSize(), implementation_buffer->getFileName(), + file_segment.getInfoForLog()); + } } } else From d8cee8a34c4f3376b64977e222f46bd6abf0ffd5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 26 Jun 2023 19:00:25 +0200 Subject: [PATCH 350/515] Fix style --- src/Common/StringSearcher.h | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Common/StringSearcher.h b/src/Common/StringSearcher.h index 120a79120bb..24b707c97ae 100644 --- a/src/Common/StringSearcher.h +++ b/src/Common/StringSearcher.h @@ -21,12 +21,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - - /** Variants for searching a substring in a string. * In most cases, performance is less than Volnitsky (see Volnitsky.h). */ From 1ed33b193628dd69ccfefb7ca343066608399bbb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 26 Jun 2023 19:22:46 +0200 Subject: [PATCH 351/515] Update src/Common/StringSearcher.h Co-authored-by: ltrk2 <107155950+ltrk2@users.noreply.github.com> --- src/Common/StringSearcher.h | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Common/StringSearcher.h b/src/Common/StringSearcher.h index 24b707c97ae..ff32218b64b 100644 --- a/src/Common/StringSearcher.h +++ b/src/Common/StringSearcher.h @@ -810,9 +810,7 @@ public: requires (sizeof(CharT) == 1) static bool isValidNeedle(const CharT * needle_, size_t needle_size_) { - if (std::any_of(needle_, needle_ + needle_size_, isTokenSeparator)) - return false; - return true; + return std::none_of(needle_, needle_ + needle_size_, isTokenSeparator)); } template From 61136680e403ae88469f66157e847168996be786 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 26 Jun 2023 20:05:58 +0200 Subject: [PATCH 352/515] Update src/Common/StringSearcher.h Co-authored-by: ltrk2 <107155950+ltrk2@users.noreply.github.com> --- src/Common/StringSearcher.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/StringSearcher.h b/src/Common/StringSearcher.h index ff32218b64b..3ed192d05f3 100644 --- a/src/Common/StringSearcher.h +++ b/src/Common/StringSearcher.h @@ -810,7 +810,7 @@ public: requires (sizeof(CharT) == 1) static bool isValidNeedle(const CharT * needle_, size_t needle_size_) { - return std::none_of(needle_, needle_ + needle_size_, isTokenSeparator)); + return std::none_of(needle_, needle_ + needle_size_, isTokenSeparator); } template From 01607f0b32291f75d86e55d17a9baa0ef82c1075 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 26 Jun 2023 20:21:25 +0200 Subject: [PATCH 353/515] Add check for intersecting ranges --- src/Interpreters/Cache/FileCache.cpp | 24 +++++++++++++++++++----- src/Interpreters/Cache/FileSegment.h | 2 ++ src/Interpreters/Cache/Metadata.cpp | 22 ++++++++++++++++++++++ src/Interpreters/Cache/Metadata.h | 2 ++ 4 files changed, 45 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index b601c4029f3..9bd1bc503ff 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -499,12 +499,14 @@ KeyMetadata::iterator FileCache::addFileSegment( chassert(size > 0); /// Empty file segments in cache are not allowed. const auto & key = locked_key.getKey(); - if (locked_key.tryGetByOffset(offset)) + const FileSegment::Range range(offset, offset + size - 1); + + if (auto intersecting_range = locked_key.hasIntersectingRange(range)) { throw Exception( ErrorCodes::LOGICAL_ERROR, - "Cache entry already exists for key: `{}`, offset: {}, size: {}.", - key, offset, size); + "Attempt to add intersecting file segment in cache ({} intersects {})", + range.toString(), intersecting_range->toString()); } FileSegment::State result_state; @@ -964,8 +966,20 @@ void FileCache::loadMetadata() if ((main_priority->getSizeLimit() == 0 || main_priority->getSize(lock) + size <= main_priority->getSizeLimit()) && (main_priority->getElementsLimit() == 0 || main_priority->getElementsCount(lock) + 1 <= main_priority->getElementsLimit())) { - auto file_segment_metadata_it = addFileSegment( - *locked_key, offset, size, FileSegment::State::DOWNLOADED, CreateFileSegmentSettings(segment_kind), &lock); + KeyMetadata::iterator file_segment_metadata_it; + try + { + file_segment_metadata_it = addFileSegment( + *locked_key, offset, size, FileSegment::State::DOWNLOADED, CreateFileSegmentSettings(segment_kind), &lock); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + chassert(false); + + fs::remove(offset_it->path()); + continue; + } const auto & file_segment_metadata = file_segment_metadata_it->second; chassert(file_segment_metadata->file_segment->assertCorrectness()); diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 681c0d719e4..75053ff5dad 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -134,6 +134,8 @@ public: bool operator==(const Range & other) const { return left == other.left && right == other.right; } + bool operator<(const Range & other) const { return right < other.left; } + size_t size() const { return right - left + 1; } String toString() const { return fmt::format("[{}, {}]", std::to_string(left), std::to_string(right)); } diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index f799bae1e10..5be3bb6490a 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -451,6 +451,28 @@ void LockedKey::shrinkFileSegmentToDownloadedSize( chassert(file_segment->assertCorrectnessUnlocked(segment_lock)); } +std::optional LockedKey::hasIntersectingRange(const FileSegment::Range & range) const +{ + if (key_metadata->empty()) + return {}; + + auto it = key_metadata->lower_bound(range.left); + if (it == key_metadata->end()) + { + auto check_range = std::prev(it)->second->file_segment->range(); + return check_range < range ? std::nullopt : std::optional(check_range); + } + + std::optional check_range; + if (range < (check_range = it->second->file_segment->range()) + && (it == key_metadata->begin() || (check_range = std::prev(it)->second->file_segment->range()) < range)) + { + return {}; + } + + return check_range; +} + std::shared_ptr LockedKey::getByOffset(size_t offset) const { auto it = key_metadata->find(offset); diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index 635594a289a..8ee40aa977f 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -164,6 +164,8 @@ struct LockedKey : private boost::noncopyable bool isLastOwnerOfFileSegment(size_t offset) const; + std::optional hasIntersectingRange(const FileSegment::Range & range) const; + void removeFromCleanupQueue(); void markAsRemoved(); From 72e407eb06e506a03f3d491e81cf9b8982b469c0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 26 Jun 2023 20:40:18 +0200 Subject: [PATCH 354/515] Better --- src/Interpreters/Cache/Metadata.cpp | 21 +++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 5be3bb6490a..cde11a071f5 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -457,20 +457,21 @@ std::optional LockedKey::hasIntersectingRange(const FileSegm return {}; auto it = key_metadata->lower_bound(range.left); - if (it == key_metadata->end()) + if (it != key_metadata->end()) /// has next range { - auto check_range = std::prev(it)->second->file_segment->range(); - return check_range < range ? std::nullopt : std::optional(check_range); + auto next_range = it->second->file_segment->range(); + if (!(range < next_range)) + return next_range; + + if (it == key_metadata->begin()) + return {}; } - std::optional check_range; - if (range < (check_range = it->second->file_segment->range()) - && (it == key_metadata->begin() || (check_range = std::prev(it)->second->file_segment->range()) < range)) - { - return {}; - } + auto prev_range = std::prev(it)->second->file_segment->range(); + if (!(prev_range < range)) + return prev_range; - return check_range; + return {}; } std::shared_ptr LockedKey::getByOffset(size_t offset) const From c01d5ed004bbc64f95a7bfbacba1bda3bf2aa273 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 27 Jun 2023 00:27:51 +0200 Subject: [PATCH 355/515] Fix after review. --- src/Interpreters/ProcessList.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index c8e33b477a1..daa8d434ab6 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -593,10 +593,13 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even res.profile_counters = std::make_shared(thread_group->performance_counters.getPartiallyAtomicSnapshot()); } - if (get_settings && getContext()) + if (get_settings) { - res.query_settings = std::make_shared(getContext()->getSettings()); - res.current_database = getContext()->getCurrentDatabase(); + if (auto ctx = context.lock()) + { + res.query_settings = std::make_shared(ctx->getSettings()); + res.current_database = ctx->getCurrentDatabase(); + } } return res; From bcb106e1385b55b046643b55bb6683a666db81e4 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Mon, 26 Jun 2023 19:39:47 -0400 Subject: [PATCH 356/515] add missing aggregate functions (#51443) * add missing aggregate functions * add sparkBar --- .../aggregate-functions/reference/index.md | 20 +++++++++++++++++++ .../aspell-ignore/en/aspell-dict.txt | 1 + 2 files changed, 21 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/reference/index.md b/docs/en/sql-reference/aggregate-functions/reference/index.md index 17ef494e9ad..6c56aefd51d 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/index.md +++ b/docs/en/sql-reference/aggregate-functions/reference/index.md @@ -19,8 +19,19 @@ Standard aggregate functions: - [stddevSamp](/docs/en/sql-reference/aggregate-functions/reference/stddevsamp.md) - [varPop](/docs/en/sql-reference/aggregate-functions/reference/varpop.md) - [varSamp](/docs/en/sql-reference/aggregate-functions/reference/varsamp.md) +- [corr](./corr.md) - [covarPop](/docs/en/sql-reference/aggregate-functions/reference/covarpop.md) - [covarSamp](/docs/en/sql-reference/aggregate-functions/reference/covarsamp.md) +- [entropy](./entropy.md) +- [exponentialMovingAverage](./exponentialmovingaverage.md) +- [intervalLengthSum](./intervalLengthSum.md) +- [kolmogorovSmirnovTest](./kolmogorovsmirnovtest.md) +- [mannwhitneyutest](./mannwhitneyutest.md) +- [median](./median.md) +- [rankCorr](./rankCorr.md) +- [sumKahan](./sumkahan.md) +- [studentTTest](./studentttest.md) +- [welchTTest](./welchttest.md) ClickHouse-specific aggregate functions: @@ -34,12 +45,15 @@ ClickHouse-specific aggregate functions: - [avgWeighted](/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md) - [topK](/docs/en/sql-reference/aggregate-functions/reference/topk.md) - [topKWeighted](/docs/en/sql-reference/aggregate-functions/reference/topkweighted.md) +- [deltaSum](./deltasum.md) +- [deltaSumTimestamp](./deltasumtimestamp.md) - [groupArray](/docs/en/sql-reference/aggregate-functions/reference/grouparray.md) - [groupArrayLast](/docs/en/sql-reference/aggregate-functions/reference/grouparraylast.md) - [groupUniqArray](/docs/en/sql-reference/aggregate-functions/reference/groupuniqarray.md) - [groupArrayInsertAt](/docs/en/sql-reference/aggregate-functions/reference/grouparrayinsertat.md) - [groupArrayMovingAvg](/docs/en/sql-reference/aggregate-functions/reference/grouparraymovingavg.md) - [groupArrayMovingSum](/docs/en/sql-reference/aggregate-functions/reference/grouparraymovingsum.md) +- [groupArraySample](./grouparraysample.md) - [groupBitAnd](/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md) - [groupBitOr](/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md) - [groupBitXor](/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md) @@ -84,3 +98,9 @@ ClickHouse-specific aggregate functions: - [theilsU](./theilsu.md) - [maxIntersections](./maxintersections.md) - [maxIntersectionsPosition](./maxintersectionsposition.md) +- [meanZTest](./meanztest.md) +- [quantileGK](./quantileGK.md) +- [quantileInterpolatedWeighted](./quantileinterpolatedweighted.md) +- [sparkBar](./sparkbar.md) +- [sumCount](./sumcount.md) + diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index bf988295644..00d047121e6 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2169,6 +2169,7 @@ snowflakeToDateTime socketcache soundex sparkbar +sparkBar sparsehash speedscope splitByChar From 45f40cf2220fd7e37195fbed8d83db7f7cfa75e4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Jun 2023 04:16:26 +0200 Subject: [PATCH 357/515] Update system_warnings test --- tests/queries/0_stateless/01945_system_warnings.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01945_system_warnings.sh b/tests/queries/0_stateless/01945_system_warnings.sh index bf11cee2911..c9bd0fd4b8a 100755 --- a/tests/queries/0_stateless/01945_system_warnings.sh +++ b/tests/queries/0_stateless/01945_system_warnings.sh @@ -21,5 +21,4 @@ ${CLICKHOUSE_CLIENT} --multiple_joins_rewriter_version=42 -q "SELECT message FRO ${CLICKHOUSE_CLIENT} -q "SELECT count() = countDistinct(message) FROM system.warnings" # Avoid too many warnings, especially in CI -${CLICKHOUSE_CLIENT} -q "SELECT count() < 5 FROM system.warnings" - +${CLICKHOUSE_CLIENT} -q "SELECT count() < 10 FROM system.warnings" From 8c38d10bb4e838a156a2c1031cf2b60548e8cc42 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Jun 2023 05:49:03 +0200 Subject: [PATCH 358/515] A fix for clang-17 --- src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp index 4ad3cc7373a..188956b34fc 100644 --- a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp @@ -144,9 +144,9 @@ Pipe ReadFromMemoryStorageStep::makePipe() storage_snapshot, nullptr /* data */, nullptr /* parallel execution index */, - [storage = storage](std::shared_ptr & data_to_initialize) + [my_storage = storage](std::shared_ptr & data_to_initialize) { - data_to_initialize = static_cast(*storage).data.get(); + data_to_initialize = assert_cast(*my_storage).data.get(); })); } From 9d8b996f684ff5f669db49a616bd3ec3bf0e4c23 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 27 Jun 2023 04:30:56 +0000 Subject: [PATCH 359/515] Automatic style fix --- tests/ci/fast_test_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 7bb09d30177..2849759a3ee 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -150,7 +150,7 @@ def main(): os.makedirs(logs_path) run_log_path = os.path.join(logs_path, "run.log") - with TeePopen(run_cmd, run_log_path, timeout = 90 * 60) as process: + with TeePopen(run_cmd, run_log_path, timeout=90 * 60) as process: retcode = process.wait() if retcode == 0: logging.info("Run successfully") From 76108b955cdab459ba98dd660912f6e90aad57f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Jun 2023 07:06:50 +0200 Subject: [PATCH 360/515] Fix test 01605_adaptive_granularity_block_borders --- .../01605_adaptive_granularity_block_borders.sql | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql index ca7d0f3c950..4623c456475 100644 --- a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql +++ b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql @@ -5,9 +5,9 @@ SET allow_prefetched_read_pool_for_remote_filesystem=0; DROP TABLE IF EXISTS adaptive_table; ---- If granularity of consequent blocks differs a lot, then adaptive ---- granularity will adjust amout of marks correctly. Data for test empirically ---- derived, it's quite hard to get good parameters. +-- If granularity of consequent blocks differs a lot, then adaptive +-- granularity will adjust the amount of marks correctly. +-- Data for test was empirically derived, it's quite hard to get good parameters. CREATE TABLE adaptive_table( key UInt64, @@ -32,6 +32,7 @@ SET enable_filesystem_cache = 0; -- If we have computed granularity incorrectly than we will exceed this limit. SET max_memory_usage='30M'; +SET max_threads = 1; SELECT max(length(value)) FROM adaptive_table; From b4b21c0cf2e0c9c128a1558b543787297059748a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 27 Jun 2023 10:05:14 +0200 Subject: [PATCH 361/515] Try fix flaky 02497_storage_file_reader_selection --- .../queries/0_stateless/02497_storage_file_reader_selection.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02497_storage_file_reader_selection.sh b/tests/queries/0_stateless/02497_storage_file_reader_selection.sh index 20bde68718d..66b894404cf 100755 --- a/tests/queries/0_stateless/02497_storage_file_reader_selection.sh +++ b/tests/queries/0_stateless/02497_storage_file_reader_selection.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) DATA_FILE=test_02497_$CLICKHOUSE_TEST_UNIQUE_NAME.tsv echo -e 'key\nfoo\nbar' > $DATA_FILE -$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferMMap" +$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferMMap:" $CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferOrdinary" $CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferMMap" From dc2c1a42267ec8c0edf86b3ebbad00d7bc4fa42b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 27 Jun 2023 13:21:34 +0200 Subject: [PATCH 362/515] Convert 02003_memory_limit_in_client from expect to sh test (to fix flakiness) I don't see the reason for using expect here, and plus now the test is only 2 lines instead. Also this conversion should fix the test flakiness, since sometimes 60 seconds is not enough [1]: 2023.06.26 17:42:29.582168 [ 13456 ] {8a324061-a504-4843-be65-6abb5db4bc5c} executeQuery: Query span trace_id for opentelemetry log: 00000000-0000-0000-0000-000000000000 2023.06.26 17:42:29.584019 [ 13456 ] {8a324061-a504-4843-be65-6abb5db4bc5c} executeQuery: (from [::1]:38130) (comment: 02003_memory_limit_in_client.expect) SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) (stage: Complete) 2023.06.26 17:42:29.588131 [ 13456 ] {8a324061-a504-4843-be65-6abb5db4bc5c} InterpreterSelectQuery: FetchColumns -> Complete 2023.06.26 17:42:29.780980 [ 2566 ] {8a324061-a504-4843-be65-6abb5db4bc5c} MemoryTracker: Current memory usage (for query): 1.24 GiB. 2023.06.26 17:43:29.179098 [ 13456 ] {8a324061-a504-4843-be65-6abb5db4bc5c} TCPHandler: Client has dropped the connection, cancel the query. 2023.06.26 17:44:05.523345 [ 13456 ] {8a324061-a504-4843-be65-6abb5db4bc5c} executeQuery: Code: 394. DB::Exception: Query was cancelled or a client has unexpectedly dropped the connection. (QUERY_WAS_CANCELLED) (version 23.6.1.1) (from [::1]:38130) (comment: 02003_memory_limit_in_client.expect) (in query: SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000)) 2023.06.26 17:44:05.680631 [ 13456 ] {8a324061-a504-4843-be65-6abb5db4bc5c} MemoryTracker: Peak memory usage (for query): 1.25 GiB. 2023.06.26 17:44:05.680747 [ 13456 ] {8a324061-a504-4843-be65-6abb5db4bc5c} TCPHandler: Processed in 96.108615451 sec. [1]: https://s3.amazonaws.com/clickhouse-test-reports/51407/e92035072b7b3367da12089b28041893eb90e636/stateless_tests__tsan__[2_5].html And it is either expect is too slow, or server did not respond for awhile, but this was a TSan build and trace_log is empty, so it is not possible to check which one was the problem. Signed-off-by: Azat Khuzhin --- .../02003_memory_limit_in_client.expect | 64 ------------------- .../02003_memory_limit_in_client.reference | 1 + .../02003_memory_limit_in_client.sh | 8 +++ 3 files changed, 9 insertions(+), 64 deletions(-) delete mode 100755 tests/queries/0_stateless/02003_memory_limit_in_client.expect create mode 100755 tests/queries/0_stateless/02003_memory_limit_in_client.sh diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.expect b/tests/queries/0_stateless/02003_memory_limit_in_client.expect deleted file mode 100755 index 377656fa641..00000000000 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.expect +++ /dev/null @@ -1,64 +0,0 @@ -#!/usr/bin/expect -f - -# This is a test for system.warnings. Testing in interactive mode is necessary, -# as we want to see certain warnings from client - -set basedir [file dirname $argv0] -set basename [file tail $argv0] -exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 -set history_file $env(CLICKHOUSE_TMP)/$basename.history - -log_user 0 -set timeout 60 -match_max 100000 - -expect_after { - # Do not ignore eof from expect - -i $any_spawn_id eof { exp_continue } - # A default timeout action is to do nothing, change it to fail - -i $any_spawn_id timeout { exit 1 } -} - -# -# Check that the query will fail in clickhouse-client -# -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=$history_file" -expect ":) " - -send -- "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000)\r" -expect "Code: 241" - -expect ":) " - -# Exit. -send -- "\4" -expect eof - -# -# Check that the query will fail in clickhouse-client -# -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=$history_file" -expect ":) " - -send -- "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000)\r" -expect "Code: 241" - -expect ":) " - -# Exit. -send -- "\4" -expect eof - -# -# Check that the query will not fail (due to max_untracked_memory) -# -spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_in_client=1 --history_file=$history_file" -expect ":) " - -send -- "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000\r" -expect "60000" -expect ":) " - -# Exit. -send -- "\4" -expect eof diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.reference b/tests/queries/0_stateless/02003_memory_limit_in_client.reference index e69de29bb2d..541b3a18e90 100644 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.reference +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.reference @@ -0,0 +1 @@ +60000 diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.sh b/tests/queries/0_stateless/02003_memory_limit_in_client.sh new file mode 100755 index 00000000000..2d2493828c8 --- /dev/null +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.sh @@ -0,0 +1,8 @@ +#!/usr/bin/bash -f + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --max_memory_usage_in_client=1 -n -q "SELECT arrayMap(x -> range(x), range(number)) FROM numbers(1000) -- { clientError MEMORY_LIMIT_EXCEEDED }" +$CLICKHOUSE_CLIENT --max_memory_usage_in_client=0 -n -q "SELECT * FROM (SELECT * FROM system.numbers LIMIT 600000) as num WHERE num.number=60000" From acea2d66f0c4abad3c700ef2153977a5bcfba819 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Jun 2023 13:40:07 +0200 Subject: [PATCH 363/515] Fix --- tests/integration/test_disk_over_web_server/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index 719de5e8bef..d62adfb3343 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -45,6 +45,8 @@ def cluster(): f"CREATE TABLE data{i} (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'def', min_bytes_for_wide_part=1;" ) + node.query("SYSTEM STOP MERGES") + for _ in range(10): node.query( f"INSERT INTO data{i} SELECT number FROM numbers(500000 * {i+1})" From d4c8021f8078425ae3e7a74e32477f78307a0999 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 27 Jun 2023 12:41:21 +0000 Subject: [PATCH 364/515] Extend consistency check for ColumnObject + check that there arer subcolumns when there are rows --- src/Columns/ColumnObject.cpp | 14 ++++++++++++-- .../02789_object_type_invalid_num_of_rows.sql | 2 +- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 07872774559..4fa0c3ee41a 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -626,6 +626,9 @@ ColumnObject::ColumnObject(Subcolumns && subcolumns_, bool is_nullable_) void ColumnObject::checkConsistency() const { + if (num_rows && subcolumns.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ColumnObject is inconsistent: it has no subcolumns, but has {} rows", num_rows); + if (subcolumns.empty()) return; @@ -642,9 +645,9 @@ void ColumnObject::checkConsistency() const size_t ColumnObject::size() const { -#ifndef NDEBUG +// #ifndef NDEBUG checkConsistency(); -#endif +// #endif return num_rows; } @@ -711,6 +714,8 @@ void ColumnObject::insert(const Field & field) } ++num_rows; + + checkConsistency(); } void ColumnObject::insertDefault() @@ -719,6 +724,8 @@ void ColumnObject::insertDefault() entry->data.insertDefault(); ++num_rows; + + checkConsistency(); } Field ColumnObject::operator[](size_t n) const @@ -776,6 +783,8 @@ void ColumnObject::insertRangeFrom(const IColumn & src, size_t start, size_t len num_rows += length; finalize(); + + checkConsistency(); } void ColumnObject::popBack(size_t length) @@ -784,6 +793,7 @@ void ColumnObject::popBack(size_t length) entry->data.popBack(length); num_rows -= length; + checkConsistency(); } template diff --git a/tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.sql b/tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.sql index 8ced133c8eb..a9c8a844aa0 100644 --- a/tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.sql +++ b/tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.sql @@ -1,2 +1,2 @@ set allow_experimental_object_type=1; -SELECT '0.02' GROUP BY GROUPING SETS (('6553.6'), (CAST(CAST('{"x" : 1}', 'Object(Nullable(\'json\'))'), 'Object(\'json\')'))) settings max_threads=1; -- { serverError NOT_IMPLEMENTED } +SELECT '0.02' GROUP BY GROUPING SETS (('6553.6'), (CAST('{"x" : 1}', 'Object(\'json\')'))) settings max_threads=1; -- { serverError NOT_IMPLEMENTED } From 46a69829ff5a50867c21013b2ae9d28f2331876e Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Jun 2023 15:04:47 +0200 Subject: [PATCH 365/515] Delay shutdown of system and temporary databases --- src/Interpreters/DatabaseCatalog.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 8d3fa91a7fe..9a96bd09d82 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -216,8 +216,22 @@ void DatabaseCatalog::shutdownImpl() /// We still hold "databases" (instead of std::move) for Buffer tables to flush data correctly. + /// Delay shutdown of temporary and system databases. They will be shutdown last. + std::vector databases_with_delayed_shutdown; for (auto & database : current_databases) + { + if (database.first == TEMPORARY_DATABASE || database.first == SYSTEM_DATABASE) + { + databases_with_delayed_shutdown.push_back(database.second); + continue; + } database.second->shutdown(); + } + + for (auto & database : databases_with_delayed_shutdown) + { + database->shutdown(); + } { std::lock_guard lock(tables_marked_dropped_mutex); From 89de7a64eb3d36c3cbfed7388338e7352d216c78 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 27 Jun 2023 15:06:31 +0200 Subject: [PATCH 366/515] Fix memory leakage in CompressionCodecDeflateQpl. --- src/Compression/CompressionCodecDeflateQpl.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 4a2866d152d..859a9c2463a 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -9,6 +9,7 @@ #include #include "libaccel_config.h" #include +#include namespace DB { @@ -34,6 +35,7 @@ DeflateQplJobHWPool::DeflateQplJobHWPool() // loop all configured workqueue size to get maximum job number. accfg_ctx * ctx_ptr = nullptr; auto ctx_status = accfg_new(&ctx_ptr); + SCOPE_EXIT({ accfg_unref(ctx_ptr); }); if (ctx_status == 0) { auto * dev_ptr = accfg_device_get_first(ctx_ptr); From ddca9ece501e7e5554fbf439008f990997c7f2d1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Jun 2023 15:32:14 +0200 Subject: [PATCH 367/515] Increase retries --- tests/integration/test_multiple_disks/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 54e7f6dd8ee..b5606ee8bc2 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -846,7 +846,7 @@ def test_start_stop_moves(start_cluster, name, engine): node1.query("SYSTEM START MOVES {}".format(name)) # wait sometime until background backoff finishes - retry = 30 + retry = 60 i = 0 while not sum(1 for x in used_disks if x == "jbod1") <= 2 and i < retry: time.sleep(1) From 99c9830737941e133814c75cd811ce4c5576b4fa Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 27 Jun 2023 13:34:58 +0000 Subject: [PATCH 368/515] Docs: Add more blog posts to join docs --- docs/en/sql-reference/statements/select/join.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index 08ffae838f8..7971b3ba275 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -21,6 +21,9 @@ Expressions from `ON` clause and columns from `USING` clause are called “join ## Related Content - Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Part 1](https://clickhouse.com/blog/clickhouse-fully-supports-joins) +- Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Under the Hood - Part 2](https://clickhouse.com/blog/clickhouse-fully-supports-joins-hash-joins-part2) +- Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Under the Hood - Part 3](https://clickhouse.com/blog/clickhouse-fully-supports-joins-full-sort-partial-merge-part3) +- Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Under the Hood - Part 4](https://clickhouse.com/blog/clickhouse-fully-supports-joins-direct-join-part4) ## Supported Types of JOIN From 648b647f5f878c4da0265cfae3dc39e50021170b Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Jun 2023 15:49:26 +0200 Subject: [PATCH 369/515] Fix race --- src/IO/BoundedReadBuffer.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/IO/BoundedReadBuffer.h b/src/IO/BoundedReadBuffer.h index 183dbedb78b..eb65857e83a 100644 --- a/src/IO/BoundedReadBuffer.h +++ b/src/IO/BoundedReadBuffer.h @@ -31,7 +31,8 @@ public: private: std::optional read_until_position; - size_t file_offset_of_buffer_end = 0; + /// atomic because can be used in log or exception messages while being updated. + std::atomic file_offset_of_buffer_end = 0; }; } From b8866e01286c4e1adeb10554b823eb1a79b9119c Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Jun 2023 16:11:27 +0200 Subject: [PATCH 370/515] Fix flaky unit test --- src/Interpreters/tests/gtest_lru_file_cache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 3cba1e48e1e..429f940a7c7 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -475,7 +475,7 @@ TEST_F(FileCacheTest, get) } cv.notify_one(); - file_segment2.wait(file_segment2.range().left); + file_segment2.wait(file_segment2.range().right); file_segment2.complete(); ASSERT_TRUE(file_segment2.state() == State::DOWNLOADED); }); From fe9702192961fe2073713d9b9f033c6177666d22 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 27 Jun 2023 16:54:14 +0200 Subject: [PATCH 371/515] add missing finalize calls in buffers --- src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.cpp | 1 + src/Dictionaries/HTTPDictionarySource.cpp | 2 ++ 2 files changed, 3 insertions(+) diff --git a/src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.cpp b/src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.cpp index 284e7740a58..fcb8ebd1f22 100644 --- a/src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.cpp +++ b/src/BridgeHelper/ExternalDictionaryLibraryBridgeHelper.cpp @@ -239,6 +239,7 @@ QueryPipeline ExternalDictionaryLibraryBridgeHelper::loadKeys(const Block & requ WriteBufferFromOStream out_buffer(os); auto output_format = getContext()->getOutputFormat(ExternalDictionaryLibraryBridgeHelper::DEFAULT_FORMAT, out_buffer, requested_block.cloneEmpty()); formatBlock(output_format, requested_block); + out_buffer.finalize(); }; return QueryPipeline(loadBase(uri, out_stream_callback)); } diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 757a3b1819b..55bff868dee 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -135,6 +135,7 @@ QueryPipeline HTTPDictionarySource::loadIds(const std::vector & ids) WriteBufferFromOStream out_buffer(ostr); auto output_format = context->getOutputFormatParallelIfPossible(configuration.format, out_buffer, block.cloneEmpty()); formatBlock(output_format, block); + out_buffer.finalize(); }; Poco::URI uri(configuration.url); @@ -164,6 +165,7 @@ QueryPipeline HTTPDictionarySource::loadKeys(const Columns & key_columns, const WriteBufferFromOStream out_buffer(ostr); auto output_format = context->getOutputFormatParallelIfPossible(configuration.format, out_buffer, block.cloneEmpty()); formatBlock(output_format, block); + out_buffer.finalize(); }; Poco::URI uri(configuration.url); From 98721ca36c87379e6899837b46cf4b9b5810a969 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 27 Jun 2023 16:44:50 +0000 Subject: [PATCH 372/515] New epoch From d3b8b454f853c63da4b94ec97afdcb1528ffdc22 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Tue, 27 Jun 2023 19:19:58 +0200 Subject: [PATCH 373/515] Fix segfault in MathUnary --- src/Functions/FunctionMathUnary.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Functions/FunctionMathUnary.h b/src/Functions/FunctionMathUnary.h index 6e4bff7122d..9f400932356 100644 --- a/src/Functions/FunctionMathUnary.h +++ b/src/Functions/FunctionMathUnary.h @@ -154,6 +154,8 @@ private: using ColVecType = ColumnVectorOrDecimal; const auto col_vec = checkAndGetColumn(col.column.get()); + if (col_vec == nullptr) + return false; return (res = execute(col_vec)) != nullptr; }; From 5a39960e0374f355b22260bea1095676b50e92c6 Mon Sep 17 00:00:00 2001 From: Nikifor Seriakov Date: Tue, 27 Jun 2023 21:32:39 +0400 Subject: [PATCH 374/515] Update docs/en/interfaces/formats.md Fixed RawBLOB comparison lists formatting. --- docs/en/interfaces/formats.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 79baf04d75d..378a1c46d93 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -2454,18 +2454,22 @@ In this format, all input data is read to a single value. It is possible to pars The result is output in binary format without delimiters and escaping. If more than one value is output, the format is ambiguous, and it will be impossible to read the data back. Below is a comparison of the formats `RawBLOB` and [TabSeparatedRaw](#tabseparatedraw). + `RawBLOB`: - data is output in binary format, no escaping; - there are no delimiters between values; - no newline at the end of each value. -[TabSeparatedRaw] (#tabseparatedraw): + +`TabSeparatedRaw`: - data is output without escaping; - the rows contain values separated by tabs; - there is a line feed after the last value in every row. The following is a comparison of the `RawBLOB` and [RowBinary](#rowbinary) formats. + `RawBLOB`: - String fields are output without being prefixed by length. + `RowBinary`: - String fields are represented as length in varint format (unsigned [LEB128] (https://en.wikipedia.org/wiki/LEB128)), followed by the bytes of the string. From f5327e79bb37c3cc061b8704ffdb85aa4f0b31c4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 27 Jun 2023 20:18:38 +0200 Subject: [PATCH 375/515] fix a logical error on mutation --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 1 + src/Storages/MergeTree/IMergeTreeDataPart.h | 3 +++ src/Storages/MergeTree/MutateTask.cpp | 16 +++++++++------- 3 files changed, 13 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index cf2d1b19326..85edba84296 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1362,6 +1362,7 @@ void IMergeTreeDataPart::loadColumns(bool require) else { loaded_metadata_version = metadata_snapshot->getMetadataVersion(); + old_part_with_no_metadata_version_on_disk = true; } setColumns(loaded_columns, infos, loaded_metadata_version); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index fd73d802579..6dd82493398 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -249,6 +249,9 @@ public: /// Flag for keep S3 data when zero-copy replication over S3 turned on. mutable bool force_keep_shared_data = false; + /// Some old parts don't have metadata version, so we set it to the current table's version when loading the part + bool old_part_with_no_metadata_version_on_disk = false; + using TTLInfo = MergeTreeDataPartTTLInfo; using TTLInfos = MergeTreeDataPartTTLInfos; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index b98b0844ee7..a19b9daca0e 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -153,20 +153,22 @@ static void splitAndModifyMutationCommands( /// But we don't know for sure what happened. auto part_metadata_version = part->getMetadataVersion(); auto table_metadata_version = metadata_snapshot->getMetadataVersion(); - /// StorageMergeTree does not have metadata version - if (table_metadata_version <= part_metadata_version && part->storage.supportsReplication()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} with metadata version {} contains column {} that is absent " - "in table {} with metadata version {}", - part->name, part_metadata_version, column.name, - part->storage.getStorageID().getNameForLogs(), table_metadata_version); - if (part_metadata_version < table_metadata_version) + bool allow_equal_versions = part_metadata_version == table_metadata_version && part->old_part_with_no_metadata_version_on_disk; + if (part_metadata_version < table_metadata_version || allow_equal_versions) { LOG_WARNING(log, "Ignoring column {} from part {} with metadata version {} because there is no such column " "in table {} with metadata version {}. Assuming the column was dropped", column.name, part->name, part_metadata_version, part->storage.getStorageID().getNameForLogs(), table_metadata_version); continue; } + + /// StorageMergeTree does not have metadata version + if (part->storage.supportsReplication()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} with metadata version {} contains column {} that is absent " + "in table {} with metadata version {}", + part->name, part_metadata_version, column.name, + part->storage.getStorageID().getNameForLogs(), table_metadata_version); } for_interpreter.emplace_back( From 575f3513977a21a8fea5ff30116636f2fc9ac2f1 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 27 Jun 2023 18:34:12 +0000 Subject: [PATCH 376/515] add test --- tests/queries/0_stateless/02807_math_unary_crash.reference | 2 ++ tests/queries/0_stateless/02807_math_unary_crash.sql | 4 ++++ 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/02807_math_unary_crash.reference create mode 100644 tests/queries/0_stateless/02807_math_unary_crash.sql diff --git a/tests/queries/0_stateless/02807_math_unary_crash.reference b/tests/queries/0_stateless/02807_math_unary_crash.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/02807_math_unary_crash.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/02807_math_unary_crash.sql b/tests/queries/0_stateless/02807_math_unary_crash.sql new file mode 100644 index 00000000000..16c3ba1e0ae --- /dev/null +++ b/tests/queries/0_stateless/02807_math_unary_crash.sql @@ -0,0 +1,4 @@ +CREATE TABLE t10 (`c0` Int32) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO t10 (c0) FORMAT Values (-1); +SELECT 1 FROM t10 GROUP BY erf(-sign(t10.c0)); +SELECT 1 FROM t10 GROUP BY -sign(t10.c0); From 3e5abbbf48953288d5bcea4fab9f2431bd05873d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Jun 2023 22:47:19 +0300 Subject: [PATCH 377/515] Update 02807_math_unary_crash.sql --- tests/queries/0_stateless/02807_math_unary_crash.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02807_math_unary_crash.sql b/tests/queries/0_stateless/02807_math_unary_crash.sql index 16c3ba1e0ae..fb693ac70f7 100644 --- a/tests/queries/0_stateless/02807_math_unary_crash.sql +++ b/tests/queries/0_stateless/02807_math_unary_crash.sql @@ -1,4 +1,6 @@ +DROP TABLE IF EXISTS t10; CREATE TABLE t10 (`c0` Int32) ENGINE = MergeTree ORDER BY tuple(); INSERT INTO t10 (c0) FORMAT Values (-1); SELECT 1 FROM t10 GROUP BY erf(-sign(t10.c0)); SELECT 1 FROM t10 GROUP BY -sign(t10.c0); +DROP TABLE t10; From 1ed104417a2cb49fc3c677d9193e2b676de95d77 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 27 Jun 2023 19:41:24 +0200 Subject: [PATCH 378/515] fix race between executeMetadataAlter and initializeReplication (cherry picked from commit 1d47783e857ed4dd7550de0728913e0144657a52) --- src/Databases/DatabaseReplicated.cpp | 10 ++++++++++ src/Databases/DatabaseReplicated.h | 2 ++ src/Databases/DatabaseReplicatedWorker.cpp | 5 ++++- src/Databases/IDatabase.h | 3 +++ src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 9 +++++++++ 5 files changed, 28 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index de40ee4d82d..661afc6bf1f 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -1293,6 +1293,16 @@ void DatabaseReplicated::commitAlterTable(const StorageID & table_id, assert(checkDigestValid(query_context)); } + +bool DatabaseReplicated::canExecuteReplicatedMetadataAlter() const +{ + /// ReplicatedMergeTree may call commitAlterTable from its background threads when executing ALTER_METADATA entries. + /// It may update the metadata digest (both locally and in ZooKeeper) + /// before DatabaseReplicatedDDLWorker::initializeReplication() has finished. + /// We should not update metadata until the database is initialized. + return ddl_worker && ddl_worker->isCurrentlyActive(); +} + void DatabaseReplicated::detachTablePermanently(ContextPtr local_context, const String & table_name) { auto txn = local_context->getZooKeeperMetadataTransaction(); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 1da181de030..ff1a4aba41c 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -48,6 +48,8 @@ public: /// then it will be executed on all replicas. BlockIO tryEnqueueReplicatedDDL(const ASTPtr & query, ContextPtr query_context, bool internal) override; + bool canExecuteReplicatedMetadataAlter() const override; + bool hasReplicationThread() const override { return true; } void stopReplication() override; diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index ff2675dfd6b..593d0655777 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -91,12 +91,12 @@ void DatabaseReplicatedDDLWorker::initializeReplication() if (zookeeper->tryGet(database->replica_path + "/digest", digest_str)) { digest = parse(digest_str); - LOG_TRACE(log, "Metadata digest in ZooKeeper: {}", digest); std::lock_guard lock{database->metadata_mutex}; local_digest = database->tables_metadata_digest; } else { + LOG_WARNING(log, "Did not find digest in ZooKeeper, creating it"); /// Database was created by old ClickHouse versions, let's create the node std::lock_guard lock{database->metadata_mutex}; digest = local_digest = database->tables_metadata_digest; @@ -104,6 +104,9 @@ void DatabaseReplicatedDDLWorker::initializeReplication() zookeeper->create(database->replica_path + "/digest", digest_str, zkutil::CreateMode::Persistent); } + LOG_TRACE(log, "Trying to initialize replication: our_log_ptr={}, max_log_ptr={}, local_digest={}, zk_digest={}", + our_log_ptr, max_log_ptr, local_digest, digest); + bool is_new_replica = our_log_ptr == 0; bool lost_according_to_log_ptr = our_log_ptr + logs_to_keep < max_log_ptr; bool lost_according_to_digest = database->db_settings.check_consistency && local_digest != digest; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 53a2f372814..aadae3e2491 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -254,6 +254,9 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{}: alterTable() is not supported", getEngineName()); } + /// Special method for ReplicatedMergeTree and DatabaseReplicated + virtual bool canExecuteReplicatedMetadataAlter() const { return true; } + /// Returns time of table's metadata change, 0 if there is no corresponding metadata file. virtual time_t getObjectMetadataModificationTime(const String & /*name*/) const { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 3ba3048b812..792843cbe18 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1448,6 +1448,15 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( LOG_TRACE(LogToStr(out_postpone_reason, log), fmt_string, entry.znode_name, entry.alter_version, head_alter); return false; } + + auto database_name = storage.getStorageID().database_name; + auto database = DatabaseCatalog::instance().getDatabase(database_name); + if (!database->canExecuteReplicatedMetadataAlter()) + { + LOG_TRACE(LogToStr(out_postpone_reason, log), "Cannot execute alter metadata {} with version {} " + "because database {} cannot process metadata alters now", entry.znode_name, entry.alter_version, database_name); + return false; + } } /// If this MUTATE_PART is part of alter modify/drop query, than we have to execute them one by one From b95e8704d3e0f255cb5a8830b87f4cb4ca3ebe4c Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Wed, 28 Jun 2023 02:27:31 +0200 Subject: [PATCH 379/515] Fix test_host_regexp_multiple_ptr_records --- .../coredns_config/Corefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile index 0dd198441dc..3edf37dafa5 100644 --- a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile +++ b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile @@ -1,6 +1,6 @@ . { hosts /example.com { - reload "200ms" + reload "20ms" fallthrough } forward . 127.0.0.11 From 42f3871833993ffb3fd135c47ee9ee81d2897116 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Wed, 28 Jun 2023 02:29:27 +0200 Subject: [PATCH 380/515] Fix test_host_regexp_multiple_ptr_records_concurrent --- .../coredns_config/Corefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/Corefile b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/Corefile index 0dd198441dc..3edf37dafa5 100644 --- a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/Corefile +++ b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/Corefile @@ -1,6 +1,6 @@ . { hosts /example.com { - reload "200ms" + reload "20ms" fallthrough } forward . 127.0.0.11 From 6e769237a736d5f7c9e795451a4dddf675fdf85c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 28 Jun 2023 01:39:06 +0000 Subject: [PATCH 381/515] added setting 'enable_job_stack_trace', default is off --- src/Common/Exception.cpp | 1 + src/Common/Exception.h | 1 + src/Common/ThreadPool.cpp | 10 +++++++--- src/Common/ThreadPool.h | 3 ++- src/Common/ThreadStatus.h | 1 + src/Core/Settings.h | 1 + src/Interpreters/ThreadStatusExt.cpp | 14 ++++++++++++++ 7 files changed, 27 insertions(+), 4 deletions(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index af48ce8fd99..ee268be45f6 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -200,6 +200,7 @@ Exception::FramePointers Exception::getStackFramePointers() const return frame_pointers; } +thread_local bool Exception::enable_job_stack_trace = false; thread_local std::vector Exception::thread_frame_pointers = {}; diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 4514df2159b..f80dfe7f0a2 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -43,6 +43,7 @@ public: } /// Collect call stacks of all previous jobs' schedulings leading to this thread job's execution + static thread_local bool enable_job_stack_trace; static thread_local std::vector thread_frame_pointers; protected: diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 0bd4fcb0455..979e53a72c0 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -191,7 +191,7 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, Priority priority, std: /// Tracing context on this thread is used as parent context for the sub-thread that runs the job propagate_opentelemetry_tracing_context ? DB::OpenTelemetry::CurrentContext() : DB::OpenTelemetry::TracingContextOnThread(), /// capture_frame_pointers - true); + DB::Exception::enable_job_stack_trace); ++scheduled_jobs; } @@ -397,7 +397,9 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ /// to prevent us from modifying its priority. We have to use const_cast to force move semantics on JobWithPriority::job. job = std::move(const_cast(jobs.top().job)); parent_thread_trace_context = std::move(const_cast(jobs.top().thread_trace_context)); - thread_frame_pointers = std::move(const_cast &>(jobs.top().frame_pointers)); + DB::Exception::enable_job_stack_trace = jobs.top().enable_job_stack_trace; + if (DB::Exception::enable_job_stack_trace) + thread_frame_pointers = std::move(const_cast &>(jobs.top().frame_pointers)); jobs.pop(); /// We don't run jobs after `shutdown` is set, but we have to properly dequeue all jobs and finish them. @@ -416,7 +418,9 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ /// Run the job. try { - DB::Exception::thread_frame_pointers = std::move(thread_frame_pointers); + if (DB::Exception::enable_job_stack_trace) + DB::Exception::thread_frame_pointers = std::move(thread_frame_pointers); + CurrentMetrics::Increment metric_active_pool_threads(metric_active_threads); diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 57188572a9d..f5721146e09 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -131,9 +131,10 @@ private: /// Call stacks of all jobs' schedulings leading to this one std::vector frame_pointers; + bool enable_job_stack_trace = false; JobWithPriority(Job job_, Priority priority_, const DB::OpenTelemetry::TracingContextOnThread & thread_trace_context_, bool capture_frame_pointers = false) - : job(job_), priority(priority_), thread_trace_context(thread_trace_context_) + : job(job_), priority(priority_), thread_trace_context(thread_trace_context_), enable_job_stack_trace(capture_frame_pointers) { if (!capture_frame_pointers) return; diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 061959d9f1f..3b6b947471e 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -290,6 +290,7 @@ public: void flushUntrackedMemory(); private: + void applyGlobalSettings(); void applyQuerySettings(); void initPerformanceCounters(); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c14bd420c5c..d4003124303 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -465,6 +465,7 @@ class IColumn; M(UInt64, max_fetch_partition_retries_count, 5, "Amount of retries while fetching partition from another host.", 0) \ M(UInt64, http_max_multipart_form_data_size, 1024 * 1024 * 1024, "Limit on size of multipart/form-data content. This setting cannot be parsed from URL parameters and should be set in user profile. Note that content is parsed and external tables are created in memory before start of query execution. And this is the only limit that has effect on that stage (limits on max memory usage and max execution time have no effect while reading HTTP form data).", 0) \ M(Bool, calculate_text_stack_trace, true, "Calculate text stack trace in case of exceptions during query execution. This is the default. It requires symbol lookups that may slow down fuzzing tests when huge amount of wrong queries are executed. In normal cases you should not disable this option.", 0) \ + M(Bool, enable_job_stack_trace, false, "Output stack trace of a job creator when job results in exception", 0) \ M(Bool, allow_ddl, true, "If it is set to true, then a user is allowed to executed DDL queries.", 0) \ M(Bool, parallel_view_processing, false, "Enables pushing to attached views concurrently instead of sequentially.", 0) \ M(Bool, enable_unaligned_array_join, false, "Allow ARRAY JOIN with multiple arrays that have different sizes. When this settings is enabled, arrays will be resized to the longest one.", 0) \ diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 6a4f4576eca..5acfe500b1d 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -158,6 +158,17 @@ void CurrentThread::attachQueryForLog(const String & query_) current_thread->attachQueryForLog(query_); } +void ThreadStatus::applyGlobalSettings() +{ + auto global_context_ptr = global_context.lock(); + if (!global_context_ptr) + return; + + const Settings & settings = global_context_ptr->getSettingsRef(); + + DB::Exception::enable_job_stack_trace = settings.enable_job_stack_trace; +} + void ThreadStatus::applyQuerySettings() { auto query_context_ptr = query_context.lock(); @@ -166,6 +177,8 @@ void ThreadStatus::applyQuerySettings() const Settings & settings = query_context_ptr->getSettingsRef(); + DB::Exception::enable_job_stack_trace = settings.enable_job_stack_trace; + query_id_from_query_context = query_context_ptr->getCurrentQueryId(); initQueryProfiler(); @@ -204,6 +217,7 @@ void ThreadStatus::attachToGroupImpl(const ThreadGroupPtr & thread_group_) local_data = thread_group->getSharedData(); + applyGlobalSettings(); applyQuerySettings(); initPerformanceCounters(); } From be852d554bb592cb66cacd0feef75b496b2465f1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 27 Jun 2023 13:33:39 +0000 Subject: [PATCH 382/515] Use magic_enum for values --- src/Common/ZooKeeper/IKeeper.h | 2 + src/Common/ZooKeeper/ZooKeeper.h | 2 + src/Common/ZooKeeper/ZooKeeperImpl.h | 2 + src/Coordination/FourLetterCommand.cpp | 11 +++-- src/Coordination/KeeperContext.cpp | 20 +++++++-- src/Coordination/KeeperFeatureFlags.cpp | 17 +++----- src/Coordination/KeeperFeatureFlags.h | 14 +------ .../StorageSystemZooKeeperConnection.cpp | 41 ++++++++++++++++--- .../test_keeper_feature_flags_config/test.py | 4 +- ...2735_system_zookeeper_connection.reference | 4 +- .../02735_system_zookeeper_connection.sql | 2 +- 11 files changed, 80 insertions(+), 39 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 369aacf16c7..2703c1079c0 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -532,6 +532,8 @@ public: virtual bool isFeatureEnabled(DB::KeeperFeatureFlag feature_flag) const = 0; + virtual const DB::KeeperFeatureFlags * getKeeperFeatureFlags() const { return nullptr; } + /// Expire session and finish all pending requests virtual void finalize(const String & reason) = 0; }; diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 03200771e4a..1fcb048add2 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -529,6 +529,8 @@ public: size_t getConnectedZooKeeperIndex() const { return connected_zk_index; } UInt64 getConnectedTime() const { return connected_time; } + const DB::KeeperFeatureFlags * getKeeperFeatureFlags() const { return impl->getKeeperFeatureFlags(); } + private: void init(ZooKeeperArgs args_); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index ae6bef067e3..44ea993947e 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -202,6 +202,8 @@ public: void setServerCompletelyStarted(); + const KeeperFeatureFlags * getKeeperFeatureFlags() const override { return &keeper_feature_flags; } + private: ACLs default_acls; Poco::Net::SocketAddress connected_zk_address; diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 10d13657fb0..34540902d47 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -545,7 +546,7 @@ String FeatureFlagsCommand::run() StringBuffer ret; - auto append = [&ret] (String key, uint8_t value) -> void + auto append = [&ret] (const String & key, uint8_t value) -> void { writeText(key, ret); writeText('\t', ret); @@ -553,8 +554,12 @@ String FeatureFlagsCommand::run() writeText('\n', ret); }; - for (const auto feature : all_keeper_feature_flags) - append(SettingFieldKeeperFeatureFlagTraits::toString(feature), feature_flags.isEnabled(feature)); + for (const auto & [feature_flag, name] : magic_enum::enum_entries()) + { + std::string feature_flag_string(name); + boost::to_lower(feature_flag_string); + append(feature_flag_string, feature_flags.isEnabled(feature_flag)); + } return ret.str(); diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index e6f30c81310..e1c3a138646 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -1,10 +1,19 @@ #include #include #include +#include +#include namespace DB { +namespace ErrorCodes +{ + +extern const int BAD_ARGUMENTS; + +} + KeeperContext::KeeperContext() { /// enable by default some feature flags @@ -29,12 +38,17 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) config.keys(feature_flags_key, keys); for (const auto & key : keys) { - auto feature_flag = SettingFieldKeeperFeatureFlagTraits::fromString(key); + auto feature_flag_string = boost::to_upper_copy(key); + auto feature_flag = magic_enum::enum_cast(feature_flag_string); + + if (!feature_flag.has_value()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid feature flag defined in config for Keeper: {}", key); + auto is_enabled = config.getBool(feature_flags_key + "." + key); if (is_enabled) - feature_flags.enableFeatureFlag(feature_flag); + feature_flags.enableFeatureFlag(feature_flag.value()); else - feature_flags.disableFeatureFlag(feature_flag); + feature_flags.disableFeatureFlag(feature_flag.value()); } system_nodes_with_data[keeper_api_feature_flags_path] = feature_flags.getFeatureFlags(); diff --git a/src/Coordination/KeeperFeatureFlags.cpp b/src/Coordination/KeeperFeatureFlags.cpp index 216dca014d4..3aff87bcea9 100644 --- a/src/Coordination/KeeperFeatureFlags.cpp +++ b/src/Coordination/KeeperFeatureFlags.cpp @@ -23,15 +23,10 @@ std::pair getByteAndBitIndex(size_t num) } -IMPLEMENT_SETTING_ENUM(KeeperFeatureFlag, ErrorCodes::BAD_ARGUMENTS, - {{"filtered_list", KeeperFeatureFlag::FILTERED_LIST}, - {"multi_read", KeeperFeatureFlag::MULTI_READ}, - {"check_not_exists", KeeperFeatureFlag::CHECK_NOT_EXISTS}}); - KeeperFeatureFlags::KeeperFeatureFlags() { /// get byte idx of largest value - auto [byte_idx, _] = getByteAndBitIndex(all_keeper_feature_flags.size() - 1); + auto [byte_idx, _] = getByteAndBitIndex(magic_enum::enum_count() - 1); feature_flags = std::string(byte_idx + 1, 0); } @@ -56,7 +51,7 @@ void KeeperFeatureFlags::fromApiVersion(KeeperApiVersion keeper_api_version) bool KeeperFeatureFlags::isEnabled(KeeperFeatureFlag feature_flag) const { - auto [byte_idx, bit_idx] = getByteAndBitIndex(feature_flag); + auto [byte_idx, bit_idx] = getByteAndBitIndex(magic_enum::enum_integer(feature_flag)); if (byte_idx > feature_flags.size()) return false; @@ -71,7 +66,7 @@ void KeeperFeatureFlags::setFeatureFlags(std::string feature_flags_) void KeeperFeatureFlags::enableFeatureFlag(KeeperFeatureFlag feature_flag) { - auto [byte_idx, bit_idx] = getByteAndBitIndex(feature_flag); + auto [byte_idx, bit_idx] = getByteAndBitIndex(magic_enum::enum_integer(feature_flag)); chassert(byte_idx < feature_flags.size()); feature_flags[byte_idx] |= (1 << bit_idx); @@ -79,7 +74,7 @@ void KeeperFeatureFlags::enableFeatureFlag(KeeperFeatureFlag feature_flag) void KeeperFeatureFlags::disableFeatureFlag(KeeperFeatureFlag feature_flag) { - auto [byte_idx, bit_idx] = getByteAndBitIndex(feature_flag); + auto [byte_idx, bit_idx] = getByteAndBitIndex(magic_enum::enum_integer(feature_flag)); chassert(byte_idx < feature_flags.size()); feature_flags[byte_idx] &= ~(1 << bit_idx); @@ -92,10 +87,10 @@ const std::string & KeeperFeatureFlags::getFeatureFlags() const void KeeperFeatureFlags::logFlags(Poco::Logger * log) const { - for (const auto & feature_flag : all_keeper_feature_flags) + for (const auto & [feature_flag, feature_flag_name] : magic_enum::enum_entries()) { auto is_enabled = isEnabled(feature_flag); - LOG_INFO(log, "Keeper feature flag {}: {}", SettingFieldKeeperFeatureFlagTraits::toString(feature_flag), is_enabled ? "enabled" : "disabled"); + LOG_INFO(log, "Keeper feature flag {}: {}", feature_flag_name, is_enabled ? "enabled" : "disabled"); } } diff --git a/src/Coordination/KeeperFeatureFlags.h b/src/Coordination/KeeperFeatureFlags.h index cdd4704a7ca..6c48915f60c 100644 --- a/src/Coordination/KeeperFeatureFlags.h +++ b/src/Coordination/KeeperFeatureFlags.h @@ -1,28 +1,18 @@ #pragma once -#include -#include #include namespace DB { -enum KeeperFeatureFlag +/// these values cannot be reordered or removed, only new values can be added +enum class KeeperFeatureFlag : size_t { FILTERED_LIST = 0, MULTI_READ, CHECK_NOT_EXISTS, }; -static inline constexpr std::array all_keeper_feature_flags -{ - KeeperFeatureFlag::FILTERED_LIST, - KeeperFeatureFlag::MULTI_READ, - KeeperFeatureFlag::CHECK_NOT_EXISTS, -}; - -DECLARE_SETTING_ENUM(KeeperFeatureFlag); - class KeeperFeatureFlags { public: diff --git a/src/Storages/System/StorageSystemZooKeeperConnection.cpp b/src/Storages/System/StorageSystemZooKeeperConnection.cpp index cd78ae01457..33268d58358 100644 --- a/src/Storages/System/StorageSystemZooKeeperConnection.cpp +++ b/src/Storages/System/StorageSystemZooKeeperConnection.cpp @@ -1,8 +1,11 @@ #include +#include +#include #include #include #include #include +#include #include namespace DB @@ -10,6 +13,13 @@ namespace DB NamesAndTypesList StorageSystemZooKeeperConnection::getNamesAndTypes() { + DataTypeEnum16::Values feature_flags_enum_values; + feature_flags_enum_values.reserve(magic_enum::enum_count()); + for (const auto & [feature_flag, feature_flag_string] : magic_enum::enum_entries()) + feature_flags_enum_values.push_back(std::pair{std::string{feature_flag_string}, static_cast(feature_flag)}); + + auto feature_flags_enum = std::make_shared(std::move(feature_flags_enum_values)); + return { {"name", std::make_shared()}, {"host", std::make_shared()}, @@ -19,7 +29,8 @@ NamesAndTypesList StorageSystemZooKeeperConnection::getNamesAndTypes() {"session_uptime_elapsed_seconds", std::make_shared()}, {"is_expired", std::make_shared()}, {"keeper_api_version", std::make_shared()}, - {"client_id", std::make_shared()} + {"client_id", std::make_shared()}, + {"enabled_feature_flags", std::make_shared(std::move(feature_flags_enum))} }; } @@ -36,17 +47,37 @@ void StorageSystemZooKeeperConnection::fillData(MutableColumns & res_columns, Co res_columns[7]->insert(0); res_columns[8]->insert(context->getZooKeeper()->getClientID()); + const auto add_enabled_feature_flags = [&](const auto & zookeeper) + { + Array enabled_feature_flags; + const auto * feature_flags = zookeeper->getKeeperFeatureFlags(); + if (feature_flags) + { + for (const auto & feature_flag : magic_enum::enum_values()) + { + if (feature_flags->isEnabled(feature_flag)) + { + enabled_feature_flags.push_back(feature_flag); + } + } + } + res_columns[9]->insert(std::move(enabled_feature_flags)); + }; + + add_enabled_feature_flags(context->getZooKeeper()); + for (const auto & elem : context->getAuxiliaryZooKeepers()) { res_columns[0]->insert(elem.first); res_columns[1]->insert(elem.second->getConnectedZooKeeperHost()); res_columns[2]->insert(elem.second->getConnectedZooKeeperPort()); res_columns[3]->insert(elem.second->getConnectedZooKeeperIndex()); - res_columns[4]->insert(elem.second->getSessionUptime()); - res_columns[5]->insert(elem.second->expired()); - res_columns[6]->insert(0); - res_columns[7]->insert(elem.second->getClientID()); + res_columns[4]->insert(elem.second->getConnectedTime()); + res_columns[5]->insert(elem.second->getSessionUptime()); + res_columns[6]->insert(elem.second->expired()); + res_columns[7]->insert(0); res_columns[8]->insert(elem.second->getClientID()); + add_enabled_feature_flags(elem.second); } } diff --git a/tests/integration/test_keeper_feature_flags_config/test.py b/tests/integration/test_keeper_feature_flags_config/test.py index bb7252e9ec8..93ac6cbd3bd 100644 --- a/tests/integration/test_keeper_feature_flags_config/test.py +++ b/tests/integration/test_keeper_feature_flags_config/test.py @@ -69,12 +69,12 @@ def test_keeper_feature_flags(started_cluster): for feature, is_enabled in feature_flags: node.wait_for_log_line( - f"ZooKeeperClient: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", + f"ZooKeeperClient: Keeper feature flag {feature.upper()}: {'enabled' if is_enabled else 'disabled'}", look_behind_lines=1000, ) node.wait_for_log_line( - f"KeeperContext: Keeper feature flag {feature}: {'enabled' if is_enabled else 'disabled'}", + f"KeeperContext: Keeper feature flag {feature.upper()}: {'enabled' if is_enabled else 'disabled'}", look_behind_lines=1000, ) diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference index eddd4829829..380da27cde6 100644 --- a/tests/queries/0_stateless/02735_system_zookeeper_connection.reference +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.reference @@ -1,2 +1,2 @@ -default ::1 9181 0 0 0 1 1 -zookeeper2 ::1 9181 0 0 0 1 \ No newline at end of file +default ::1 9181 0 0 0 1 1 ['FILTERED_LIST','MULTI_READ','CHECK_NOT_EXISTS'] +zookeeper2 ::1 9181 0 0 0 1 diff --git a/tests/queries/0_stateless/02735_system_zookeeper_connection.sql b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql index 863d90e1654..f999da51225 100644 --- a/tests/queries/0_stateless/02735_system_zookeeper_connection.sql +++ b/tests/queries/0_stateless/02735_system_zookeeper_connection.sql @@ -9,7 +9,7 @@ ENGINE ReplicatedMergeTree('zookeeper2:/clickhouse/{database}/02731_zk_connectio ORDER BY tuple(); select name, host, port, index, is_expired, keeper_api_version, (connected_time between yesterday() and now()), - (abs(session_uptime_elapsed_seconds - zookeeperSessionUptime()) < 10) + (abs(session_uptime_elapsed_seconds - zookeeperSessionUptime()) < 10), enabled_feature_flags from system.zookeeper_connection where name='default'; -- keeper_api_version will by 0 for auxiliary_zookeeper2, because we fail to get /api_version due to chroot From eff1cc0e5df2af76b5687fce03901686e1a8360c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 28 Jun 2023 10:20:33 +0200 Subject: [PATCH 383/515] Remove unused errorcode --- src/Coordination/KeeperFeatureFlags.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Coordination/KeeperFeatureFlags.cpp b/src/Coordination/KeeperFeatureFlags.cpp index 3aff87bcea9..d0cd1c86b55 100644 --- a/src/Coordination/KeeperFeatureFlags.cpp +++ b/src/Coordination/KeeperFeatureFlags.cpp @@ -6,11 +6,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - namespace { From 04a08b47f3ca5d250600775adf1c307c09af929b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 27 Jun 2023 22:25:56 +0200 Subject: [PATCH 384/515] Reorganize installation of clickhouse to have separated layers --- docker/server/Dockerfile.ubuntu | 46 +++++++++++++++++++++------------ 1 file changed, 30 insertions(+), 16 deletions(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index f393b98cfe6..abb3f387330 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -11,14 +11,15 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list && apt-get update \ && apt-get upgrade -yq \ && apt-get install --yes --no-install-recommends \ - apt-transport-https \ ca-certificates \ - dirmngr \ - gnupg2 \ - wget \ locales \ tzdata \ - && apt-get clean + wget \ + && apt-get clean \ + && rm -rf \ + /var/lib/apt/lists/* \ + /var/cache/debconf \ + /tmp/* ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" @@ -43,7 +44,8 @@ ARG single_binary_location_url="" ARG TARGETARCH -RUN arch=${TARGETARCH:-amd64} \ +# install from a web location with deb packages +RUN arch="${TARGETARCH:-amd64}" \ && if [ -n "${deb_location_url}" ]; then \ echo "installing from custom url with deb packages: ${deb_location_url}" \ rm -rf /tmp/clickhouse_debs \ @@ -54,15 +56,27 @@ RUN arch=${TARGETARCH:-amd64} \ || exit 1 \ ; done \ && dpkg -i /tmp/clickhouse_debs/*.deb ; \ - elif [ -n "${single_binary_location_url}" ]; then \ + fi + +# install from a single binary +RUN if [ -n "${single_binary_location_url}" ]; then \ echo "installing from single binary url: ${single_binary_location_url}" \ && rm -rf /tmp/clickhouse_binary \ && mkdir -p /tmp/clickhouse_binary \ && wget --progress=bar:force:noscroll "${single_binary_location_url}" -O /tmp/clickhouse_binary/clickhouse \ && chmod +x /tmp/clickhouse_binary/clickhouse \ && /tmp/clickhouse_binary/clickhouse install --user "clickhouse" --group "clickhouse" ; \ - else \ - mkdir -p /etc/apt/sources.list.d \ + fi + +# A fallback to installation from ClickHouse repository +RUN if ! clickhouse local -q "SELECT ''" > /dev/null; then \ + apt-get update \ + && apt-get install --yes --no-install-recommends \ + apt-transport-https \ + ca-certificates \ + dirmngr \ + gnupg2 \ + && mkdir -p /etc/apt/sources.list.d \ && apt-key adv --keyserver keyserver.ubuntu.com --recv 8919F6BD2B48D754 \ && echo ${REPOSITORY} > /etc/apt/sources.list.d/clickhouse.list \ && echo "installing from repository: ${REPOSITORY}" \ @@ -72,20 +86,20 @@ RUN arch=${TARGETARCH:-amd64} \ packages="${packages} ${package}=${VERSION}" \ ; done \ && apt-get install --allow-unauthenticated --yes --no-install-recommends ${packages} || exit 1 \ - ; fi \ - && clickhouse-local -q 'SELECT * FROM system.build_options' \ && rm -rf \ /var/lib/apt/lists/* \ /var/cache/debconf \ /tmp/* \ - && mkdir -p /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client \ - && chmod ugo+Xrw -R /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client - -RUN apt-get autoremove --purge -yq libksba8 && \ - apt-get autoremove -yq + && apt-get autoremove --purge -yq libksba8 \ + && apt-get autoremove -yq \ + ; fi +# post install # we need to allow "others" access to clickhouse folder, because docker container # can be started with arbitrary uid (openshift usecase) +RUN clickhouse-local -q 'SELECT * FROM system.build_options' \ + && mkdir -p /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client \ + && chmod ugo+Xrw -R /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client RUN locale-gen en_US.UTF-8 ENV LANG en_US.UTF-8 From 6145baade8dad30207935c81a844fadd618cb912 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 27 Jun 2023 22:27:33 +0200 Subject: [PATCH 385/515] Upgrade OS only once The ubuntu images are updated twice a month, it makes sense to do upgrade https://github.com/docker-library/official-images/commits/master/library/ubuntu --- docker/server/Dockerfile.ubuntu | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index abb3f387330..a563149ba8e 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -81,7 +81,6 @@ RUN if ! clickhouse local -q "SELECT ''" > /dev/null; then \ && echo ${REPOSITORY} > /etc/apt/sources.list.d/clickhouse.list \ && echo "installing from repository: ${REPOSITORY}" \ && apt-get update \ - && apt-get --yes -o "Dpkg::Options::=--force-confdef" -o "Dpkg::Options::=--force-confold" upgrade \ && for package in ${PACKAGES}; do \ packages="${packages} ${package}=${VERSION}" \ ; done \ From a2c4546c00f181ddc11a59c9e1b259819432e3c8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 27 Jun 2023 22:53:30 +0200 Subject: [PATCH 386/515] Downgrade the base image to address issues on old dockerd --- docker/server/Dockerfile.ubuntu | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index a563149ba8e..e2347945cf6 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -1,4 +1,4 @@ -FROM ubuntu:22.04 +FROM ubuntu:20.04 # see https://github.com/moby/moby/issues/4032#issuecomment-192327844 ARG DEBIAN_FRONTEND=noninteractive From 6d2b5166b930e77e53c9bb608d56d372c0ecd0b3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 27 Jun 2023 23:10:56 +0200 Subject: [PATCH 387/515] Update golang version in builder --- docker/packager/binary/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index dd21c8552d3..e824161a688 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -89,7 +89,7 @@ RUN arch=${TARGETARCH:-amd64} \ && dpkg -i /tmp/nfpm.deb \ && rm /tmp/nfpm.deb -ARG GO_VERSION=1.19.5 +ARG GO_VERSION=1.19.10 # We need go for clickhouse-diagnostics RUN arch=${TARGETARCH:-amd64} \ && curl -Lo /tmp/go.tgz "https://go.dev/dl/go${GO_VERSION}.linux-${arch}.tar.gz" \ From 1cd5ae36bd89215d7c1af99d2a8b9193d0f79081 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 27 Jun 2023 23:20:47 +0200 Subject: [PATCH 388/515] Get rid of deprecated apt-key --- docker/server/Dockerfile.ubuntu | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index e2347945cf6..1bb0cfdc700 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -22,7 +22,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list /tmp/* ARG REPO_CHANNEL="stable" -ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" +ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" ARG VERSION="23.5.3.24" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" @@ -77,8 +77,13 @@ RUN if ! clickhouse local -q "SELECT ''" > /dev/null; then \ dirmngr \ gnupg2 \ && mkdir -p /etc/apt/sources.list.d \ - && apt-key adv --keyserver keyserver.ubuntu.com --recv 8919F6BD2B48D754 \ - && echo ${REPOSITORY} > /etc/apt/sources.list.d/clickhouse.list \ + && GNUPGHOME=$(mktemp -d) \ + && GNUPGHOME="$GNUPGHOME" gpg --no-default-keyring \ + --keyring /usr/share/keyrings/clickhouse-keyring.gpg \ + --keyserver hkp://keyserver.ubuntu.com:80 --recv-keys 8919F6BD2B48D754 \ + && rm -r "$GNUPGHOME" \ + && chmod +r /usr/share/keyrings/clickhouse-keyring.gpg \ + && echo "${REPOSITORY}" > /etc/apt/sources.list.d/clickhouse.list \ && echo "installing from repository: ${REPOSITORY}" \ && apt-get update \ && for package in ${PACKAGES}; do \ From 0e72c6dcc934a52184909e3c6b32b084678c185a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 28 Jun 2023 10:33:07 +0200 Subject: [PATCH 389/515] fix --- .../0_stateless/01111_create_drop_replicated_db_stress.sh | 2 +- tests/queries/0_stateless/01293_optimize_final_force.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh index 770a0780ca2..4d341e5b8a3 100755 --- a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh +++ b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh @@ -64,7 +64,7 @@ function alter_table() if [ -z "$table" ]; then continue; fi $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=0 -q \ "alter table $table update n = n + (select max(n) from merge(REGEXP('${CLICKHOUSE_DATABASE}.*'), '.*')) where 1 settings allow_nondeterministic_mutations=1" \ - 2>&1| grep -Fa "Exception: " | grep -Fv "Cannot enqueue query" | grep -Fv "ZooKeeper session expired" | grep -Fv UNKNOWN_DATABASE | grep -Fv UNKNOWN_TABLE | grep -Fv TABLE_IS_READ_ONLY | grep -Fv TABLE_IS_DROPPED + 2>&1| grep -Fa "Exception: " | grep -Fv "Cannot enqueue query" | grep -Fv "ZooKeeper session expired" | grep -Fv UNKNOWN_DATABASE | grep -Fv UNKNOWN_TABLE | grep -Fv TABLE_IS_READ_ONLY | grep -Fv TABLE_IS_DROPPED | grep -Fv "Error while executing table function merge" sleep 0.$RANDOM done } diff --git a/tests/queries/0_stateless/01293_optimize_final_force.sh b/tests/queries/0_stateless/01293_optimize_final_force.sh index eb3a2756899..9b9ed6272a1 100755 --- a/tests/queries/0_stateless/01293_optimize_final_force.sh +++ b/tests/queries/0_stateless/01293_optimize_final_force.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-debug, no-s3-storage +# Tags: no-fasttest, long, no-debug, no-s3-storage # This test is too slow with S3 storage and debug modes. CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From 67a7bdbb3a1c8fa2341e1627261d311cba1308a7 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 28 Jun 2023 10:45:07 +0200 Subject: [PATCH 390/515] Rollback note about 22.04 in #50958 --- docker/server/README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/server/README.md b/docker/server/README.md index 18dce492123..67646a262f5 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -20,7 +20,6 @@ For more information and documentation see https://clickhouse.com/. - The amd64 image requires support for [SSE3 instructions](https://en.wikipedia.org/wiki/SSE3). Virtually all x86 CPUs after 2005 support SSE3. - The arm64 image requires support for the [ARMv8.2-A architecture](https://en.wikipedia.org/wiki/AArch64#ARMv8.2-A). Most ARM CPUs after 2017 support ARMv8.2-A. A notable exception is Raspberry Pi 4 from 2019 whose CPU only supports ARMv8.0-A. -- Since the Clickhouse 23.3 Ubuntu image started using `ubuntu:22.04` as its base image, it requires docker version >= `20.10.10`, or use `docker run -- privileged` instead. Alternatively, try the Clickhouse Alpine image. ## How to use this image From 1596b09dc2888690acafa397936d0d7051e95fba Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 28 Jun 2023 10:51:15 +0200 Subject: [PATCH 391/515] add missing finalize calls --- programs/format/Format.cpp | 5 +++-- src/Client/ClientBase.cpp | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index 392ecefac0f..43c66a32302 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -151,6 +151,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv) WriteBufferFromFileDescriptor out(STDOUT_FILENO); obfuscateQueries(query, out, obfuscated_words_map, used_nouns, hash_func, is_known_identifier); + out.finalize(); } else { @@ -175,7 +176,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv) { WriteBufferFromOStream res_buf(std::cout, 4096); formatAST(*res, res_buf, hilite, oneline); - res_buf.next(); + res_buf.finalize(); if (multiple) std::cout << "\n;\n"; std::cout << std::endl; @@ -199,7 +200,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv) res_cout.write(*s_pos++); } - res_cout.next(); + res_cout.finalize(); if (multiple) std::cout << " \\\n;\n"; std::cout << std::endl; diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 34b3b1e228a..cf1c2ed8779 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -362,7 +362,7 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_mu std::cout << std::endl; WriteBufferFromOStream res_buf(std::cout, 4096); formatAST(*res, res_buf); - res_buf.next(); + res_buf.finalize(); std::cout << std::endl << std::endl; } From 0c1f24db77354c7982eaac60618da3b15589e4a4 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 28 Jun 2023 12:13:16 +0200 Subject: [PATCH 392/515] Add a comment (#51517) * Add a comment * Update DatabaseCatalog.cpp --- src/Interpreters/DatabaseCatalog.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 40ef2bd4775..4cb2f6e3b3d 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -217,6 +217,8 @@ void DatabaseCatalog::shutdownImpl() /// We still hold "databases" (instead of std::move) for Buffer tables to flush data correctly. /// Delay shutdown of temporary and system databases. They will be shutdown last. + /// Because some databases might use them until their shutdown is called, but calling shutdown + /// on temporary database means clearing its set of tables, which will lead to unnecessary errors like "table not found". std::vector databases_with_delayed_shutdown; for (auto & database : current_databases) { From 0320ab4a7056ac099145648504505cb1c7225858 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 28 Jun 2023 12:16:16 +0200 Subject: [PATCH 393/515] Try making Keeper in `DatabaseReplicated` tests more stable (#51473) * Try making Keeper in DatabaseReplicated less flaky * lower log level to debug * Revert back to information --- tests/config/config.d/database_replicated.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/config.d/database_replicated.xml b/tests/config/config.d/database_replicated.xml index 9a405f85908..2504a7ca526 100644 --- a/tests/config/config.d/database_replicated.xml +++ b/tests/config/config.d/database_replicated.xml @@ -40,7 +40,7 @@ 10000 30000 1000 - 4000 + 2000 5000 information false From 451694d8b6ed7630842020ca132a9d7548352cad Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 28 Jun 2023 10:24:54 +0000 Subject: [PATCH 394/515] Make `test_ssl_cert_authentication` similar to `test_tlvs1_3` --- .../test_ssl_cert_authentication/test.py | 92 +++++++++++++------ 1 file changed, 62 insertions(+), 30 deletions(-) diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index b05a6acc16b..91faf8b0ce3 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -2,10 +2,10 @@ import pytest from helpers.client import Client from helpers.cluster import ClickHouseCluster from helpers.ssl_context import WrapSSLContextWithSNI +import urllib.request, urllib.parse import ssl import os.path from os import remove -import urllib3 # The test cluster is configured with certificate for that host name, see 'server-ext.cnf'. @@ -14,6 +14,7 @@ SSL_HOST = "integration-tests.clickhouse.com" HTTPS_PORT = 8443 # It's important for the node to work at this IP because 'server-cert.pem' requires that (see server-ext.cnf). SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +MAX_RETRY = 5 cluster = ClickHouseCluster(__file__) instance = cluster.add_instance( @@ -164,17 +165,19 @@ def get_ssl_context(cert_name): def execute_query_https( query, user, enable_ssl_auth=True, cert_name=None, password=None ): - url = f"https://{instance.ip_address}:{HTTPS_PORT}/?query={query}" - headers = {"X-ClickHouse-User": user} + url = ( + f"https://{instance.ip_address}:{HTTPS_PORT}/?query={urllib.parse.quote(query)}" + ) + request = urllib.request.Request(url) + request.add_header("X-ClickHouse-User", user) if enable_ssl_auth: - headers["X-ClickHouse-SSL-Certificate-Auth"] = "on" + request.add_header("X-ClickHouse-SSL-Certificate-Auth", "on") if password: - headers["X-ClickHouse-Key"] = password - http_client = urllib3.PoolManager(ssl_context=get_ssl_context(cert_name)) - response = http_client.request("GET", url, headers=headers) - if response.status != 200: - raise Exception(response.status) - return response.data.decode("utf-8") + request.add_header("X-ClickHouse-Key", password) + response = urllib.request.urlopen( + request, context=get_ssl_context(cert_name) + ).read() + return response.decode("utf-8") def test_https(): @@ -198,10 +201,18 @@ def test_https_wrong_cert(): execute_query_https("SELECT currentUser()", user="john", cert_name="client2") assert "403" in str(err.value) + count = 0 # Wrong certificate: self-signed certificate. - with pytest.raises(Exception) as err: - execute_query_https("SELECT currentUser()", user="john", cert_name="wrong") - assert "unknown ca" in str(err.value) + while count <= MAX_RETRY: + with pytest.raises(Exception) as err: + execute_query_https("SELECT currentUser()", user="john", cert_name="wrong") + err_str = str(err.value) + if count < MAX_RETRY and "Broken pipe" in err_str: + count = count + 1 + logging.warning(f"Failed attempt with wrong cert, err: {err_str}") + continue + assert "unknown ca" in err_str + break # No certificate. with pytest.raises(Exception) as err: @@ -291,24 +302,45 @@ def test_https_non_ssl_auth(): == "jane\n" ) + count = 0 # However if we send a certificate it must not be wrong. - with pytest.raises(Exception) as err: - execute_query_https( - "SELECT currentUser()", - user="peter", - enable_ssl_auth=False, - cert_name="wrong", - ) - assert "unknown ca" in str(err.value) - with pytest.raises(Exception) as err: - execute_query_https( - "SELECT currentUser()", - user="jane", - enable_ssl_auth=False, - password="qwe123", - cert_name="wrong", - ) - assert "unknown ca" in str(err.value) + while count <= MAX_RETRY: + with pytest.raises(Exception) as err: + execute_query_https( + "SELECT currentUser()", + user="peter", + enable_ssl_auth=False, + cert_name="wrong", + ) + err_str = str(err.value) + if count < MAX_RETRY and "Broken pipe" in err_str: + count = count + 1 + logging.warning( + f"Failed attempt with wrong cert, user: peter, err: {err_str}" + ) + continue + assert "unknown ca" in err_str + break + + count = 0 + while count <= MAX_RETRY: + with pytest.raises(Exception) as err: + execute_query_https( + "SELECT currentUser()", + user="jane", + enable_ssl_auth=False, + password="qwe123", + cert_name="wrong", + ) + err_str = str(err.value) + if count < MAX_RETRY and "Broken pipe" in err_str: + count = count + 1 + logging.warning( + f"Failed attempt with wrong cert, user: jane, err: {err_str}" + ) + continue + assert "unknown ca" in err_str + break def test_create_user(): From b8b0fd3abfee08db6b31b393fb828915940ae13d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 28 Jun 2023 12:30:15 +0200 Subject: [PATCH 395/515] Slightly better code around packets for parallel replicas (#51451) --- src/Client/Connection.cpp | 8 +--- src/Client/IServerConnection.h | 10 +++-- src/QueryPipeline/RemoteQueryExecutor.cpp | 6 ++- src/Storages/MergeTree/MergeTreeReadPool.cpp | 21 ++++++---- src/Storages/MergeTree/MergeTreeReadPool.h | 18 ++++---- .../ParallelReplicasReadingCoordinator.cpp | 2 - src/Storages/MergeTree/RequestResponse.cpp | 26 +++++++++++- src/Storages/MergeTree/RequestResponse.h | 42 +++++++++++++++++-- 8 files changed, 98 insertions(+), 35 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index be226bfd9dd..cd102f46ffe 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -1175,16 +1175,12 @@ ProfileInfo Connection::receiveProfileInfo() const ParallelReadRequest Connection::receiveParallelReadRequest() const { - ParallelReadRequest request; - request.deserialize(*in); - return request; + return ParallelReadRequest::deserialize(*in); } InitialAllRangesAnnouncement Connection::receiveInitialParallelReadAnnounecement() const { - InitialAllRangesAnnouncement announcement; - announcement.deserialize(*in); - return announcement; + return InitialAllRangesAnnouncement::deserialize(*in); } diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index 52382ff9d45..a0c029c79fb 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -16,6 +16,10 @@ #include +#include +#include +#include +#include namespace DB { @@ -34,9 +38,9 @@ struct Packet ProfileInfo profile_info; std::vector part_uuids; - InitialAllRangesAnnouncement announcement; - ParallelReadRequest request; - ParallelReadResponse response; + /// The part of parallel replicas protocol + std::optional announcement; + std::optional request; std::string server_timezone; diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 233dfe13339..cd6f65b7b43 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -434,11 +434,13 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::processPacket(Packet packet switch (packet.type) { case Protocol::Server::MergeTreeReadTaskRequest: - processMergeTreeReadTaskRequest(packet.request); + chassert(packet.request.has_value()); + processMergeTreeReadTaskRequest(packet.request.value()); return ReadResult(ReadResult::Type::ParallelReplicasToken); case Protocol::Server::MergeTreeAllRangesAnnounecement: - processMergeTreeInitialReadAnnounecement(packet.announcement); + chassert(packet.announcement.has_value()); + processMergeTreeInitialReadAnnounecement(packet.announcement.value()); return ReadResult(ReadResult::Type::ParallelReplicasToken); case Protocol::Server::ReadTaskRequest: diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index ba8c2c6385f..2ab90189f9d 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace ProfileEvents @@ -433,8 +434,12 @@ MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t thread) if (buffered_ranges.empty()) { - auto result = extension.callback(ParallelReadRequest{ - .replica_num = extension.number_of_current_replica, .min_number_of_marks = min_marks_for_concurrent_read * threads}); + auto result = extension.callback(ParallelReadRequest( + CoordinationMode::Default, + extension.number_of_current_replica, + min_marks_for_concurrent_read * threads, + /// For Default coordination mode we don't need to pass part names. + RangesInDataPartsDescription{})); if (!result || result->finish) { @@ -529,12 +534,12 @@ MarkRanges MergeTreeInOrderReadPoolParallelReplicas::getNewTask(RangesInDataPart if (no_more_tasks) return {}; - auto response = extension.callback(ParallelReadRequest{ - .mode = mode, - .replica_num = extension.number_of_current_replica, - .min_number_of_marks = min_marks_for_concurrent_read * request.size(), - .description = request, - }); + auto response = extension.callback(ParallelReadRequest( + mode, + extension.number_of_current_replica, + min_marks_for_concurrent_read * request.size(), + request + )); if (!response || response->description.empty() || response->finish) { diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index 21273904e00..68d5438cb3d 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -193,10 +193,11 @@ public: predict_block_size_bytes, column_names, virtual_column_names, prewhere_info, actions_settings, reader_settings, per_part_params); - extension.all_callback({ - .description = parts_ranges.getDescriptions(), - .replica_num = extension.number_of_current_replica - }); + extension.all_callback(InitialAllRangesAnnouncement( + CoordinationMode::Default, + parts_ranges.getDescriptions(), + extension.number_of_current_replica + )); } ~MergeTreeReadPoolParallelReplicas() override; @@ -253,10 +254,11 @@ public: for (const auto & part : parts_ranges) buffered_tasks.push_back({part.data_part->info, MarkRanges{}}); - extension.all_callback({ - .description = parts_ranges.getDescriptions(), - .replica_num = extension.number_of_current_replica - }); + extension.all_callback(InitialAllRangesAnnouncement( + mode, + parts_ranges.getDescriptions(), + extension.number_of_current_replica + )); } MarkRanges getNewTask(RangesInDataPartDescription description); diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 2814d13cff0..57cd91cc995 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -102,7 +102,6 @@ public: explicit DefaultCoordinator(size_t replicas_count_) : ParallelReplicasReadingCoordinator::ImplInterface(replicas_count_) - , announcements(replicas_count_) , reading_state(replicas_count_) { } @@ -119,7 +118,6 @@ public: PartitionToBlockRanges partitions; size_t sent_initial_requests{0}; - std::vector announcements; Parts all_parts_to_read; /// Contains only parts which we haven't started to read from diff --git a/src/Storages/MergeTree/RequestResponse.cpp b/src/Storages/MergeTree/RequestResponse.cpp index 05930d5a4c4..2ce0e20dcd2 100644 --- a/src/Storages/MergeTree/RequestResponse.cpp +++ b/src/Storages/MergeTree/RequestResponse.cpp @@ -51,7 +51,7 @@ String ParallelReadRequest::describe() const return result; } -void ParallelReadRequest::deserialize(ReadBuffer & in) +ParallelReadRequest ParallelReadRequest::deserialize(ReadBuffer & in) { UInt64 version; readIntBinary(version, in); @@ -60,12 +60,24 @@ void ParallelReadRequest::deserialize(ReadBuffer & in) "from replicas differ. Got: {}, supported version: {}", version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION); + CoordinationMode mode; + size_t replica_num; + size_t min_number_of_marks; + RangesInDataPartsDescription description; + uint8_t mode_candidate; readIntBinary(mode_candidate, in); mode = validateAndGet(mode_candidate); readIntBinary(replica_num, in); readIntBinary(min_number_of_marks, in); description.deserialize(in); + + return ParallelReadRequest( + mode, + replica_num, + min_number_of_marks, + std::move(description) + ); } void ParallelReadRequest::merge(ParallelReadRequest & other) @@ -125,7 +137,7 @@ String InitialAllRangesAnnouncement::describe() return result; } -void InitialAllRangesAnnouncement::deserialize(ReadBuffer & in) +InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffer & in) { UInt64 version; readIntBinary(version, in); @@ -134,11 +146,21 @@ void InitialAllRangesAnnouncement::deserialize(ReadBuffer & in) "from replicas differ. Got: {}, supported version: {}", version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION); + CoordinationMode mode; + RangesInDataPartsDescription description; + size_t replica_num; + uint8_t mode_candidate; readIntBinary(mode_candidate, in); mode = validateAndGet(mode_candidate); description.deserialize(in); readIntBinary(replica_num, in); + + return InitialAllRangesAnnouncement { + mode, + description, + replica_num + }; } } diff --git a/src/Storages/MergeTree/RequestResponse.h b/src/Storages/MergeTree/RequestResponse.h index 7e5563c0553..3a5bfde6c20 100644 --- a/src/Storages/MergeTree/RequestResponse.h +++ b/src/Storages/MergeTree/RequestResponse.h @@ -40,21 +40,40 @@ struct PartBlockRange } }; +/// ParallelReadRequest is used by remote replicas during parallel read +/// to signal an initiator that they need more marks to read. struct ParallelReadRequest { + /// No default constructor, you must initialize all fields at once. + + ParallelReadRequest( + CoordinationMode mode_, + size_t replica_num_, + size_t min_number_of_marks_, + RangesInDataPartsDescription description_) + : mode(mode_) + , replica_num(replica_num_) + , min_number_of_marks(min_number_of_marks_) + , description(std::move(description_)) + {} + CoordinationMode mode; size_t replica_num; size_t min_number_of_marks; - - /// Extension for ordered mode + /// Extension for Ordered (InOrder or ReverseOrder) mode + /// Contains only data part names without mark ranges. RangesInDataPartsDescription description; void serialize(WriteBuffer & out) const; String describe() const; - void deserialize(ReadBuffer & in); + static ParallelReadRequest deserialize(ReadBuffer & in); void merge(ParallelReadRequest & other); }; +/// ParallelReadResponse is used by an initiator to tell +/// remote replicas about what to read during parallel reading. +/// Additionally contains information whether there are more available +/// marks to read (whether it is the last packet or not). struct ParallelReadResponse { bool finish{false}; @@ -66,15 +85,30 @@ struct ParallelReadResponse }; +/// The set of parts (their names) along with ranges to read which is sent back +/// to the initiator by remote replicas during parallel reading. +/// Additionally contains an identifier (replica_num) plus +/// the reading algorithm chosen (Default, InOrder or ReverseOrder). struct InitialAllRangesAnnouncement { + /// No default constructor, you must initialize all fields at once. + + InitialAllRangesAnnouncement( + CoordinationMode mode_, + RangesInDataPartsDescription description_, + size_t replica_num_) + : mode(mode_) + , description(description_) + , replica_num(replica_num_) + {} + CoordinationMode mode; RangesInDataPartsDescription description; size_t replica_num; void serialize(WriteBuffer & out) const; String describe(); - void deserialize(ReadBuffer & in); + static InitialAllRangesAnnouncement deserialize(ReadBuffer & in); }; From a029105fd51044d692c0b2895dd9d5ce09b28036 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 28 Jun 2023 10:35:19 +0000 Subject: [PATCH 396/515] Fix duplicate storage set logical error. --- src/Interpreters/ActionsVisitor.cpp | 3 +++ .../02809_storage_set_analysis_bug.reference | 0 .../02809_storage_set_analysis_bug.sql | 15 +++++++++++++++ 3 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/02809_storage_set_analysis_bug.reference create mode 100644 tests/queries/0_stateless/02809_storage_set_analysis_bug.sql diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index cfbe53b5e4d..10502b7e66d 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1435,6 +1435,9 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool if (table) { + if (auto set = data.prepared_sets->findStorage(set_key)) + return set; + if (StorageSet * storage_set = dynamic_cast(table.get())) return data.prepared_sets->addFromStorage(set_key, storage_set->getSet()); } diff --git a/tests/queries/0_stateless/02809_storage_set_analysis_bug.reference b/tests/queries/0_stateless/02809_storage_set_analysis_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02809_storage_set_analysis_bug.sql b/tests/queries/0_stateless/02809_storage_set_analysis_bug.sql new file mode 100644 index 00000000000..212170571f7 --- /dev/null +++ b/tests/queries/0_stateless/02809_storage_set_analysis_bug.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS test_set; +DROP TABLE IF EXISTS null_in__fuzz_6; + +set allow_suspicious_low_cardinality_types = 1; + + +CREATE TABLE null_in__fuzz_6 (`dt` LowCardinality(UInt16), `idx` Int32, `i` Nullable(Int256), `s` Int32) ENGINE = MergeTree PARTITION BY dt ORDER BY idx; +insert into null_in__fuzz_6 select * from generateRandom() limit 1; + +SET transform_null_in = 0; + +CREATE TABLE test_set (i Nullable(int)) ENGINE = Set(); +INSERT INTO test_set VALUES (1), (NULL); + +SELECT count() = 1 FROM null_in__fuzz_6 PREWHERE 71 WHERE i IN (test_set); -- { serverError CANNOT_CONVERT_TYPE } From d3677adcdf367ec87aba49ec844c607a9d6c987d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 28 Jun 2023 10:37:38 +0000 Subject: [PATCH 397/515] Import missing module --- tests/integration/test_ssl_cert_authentication/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index 91faf8b0ce3..ff2de7491e1 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -6,6 +6,7 @@ import urllib.request, urllib.parse import ssl import os.path from os import remove +import logging # The test cluster is configured with certificate for that host name, see 'server-ext.cnf'. From 82fd863ad4eb1209ca66e660d7cf3b646f88fff7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Jun 2023 13:39:47 +0300 Subject: [PATCH 398/515] Update 02809_storage_set_analysis_bug.sql --- tests/queries/0_stateless/02809_storage_set_analysis_bug.sql | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02809_storage_set_analysis_bug.sql b/tests/queries/0_stateless/02809_storage_set_analysis_bug.sql index 212170571f7..531946acda0 100644 --- a/tests/queries/0_stateless/02809_storage_set_analysis_bug.sql +++ b/tests/queries/0_stateless/02809_storage_set_analysis_bug.sql @@ -3,7 +3,6 @@ DROP TABLE IF EXISTS null_in__fuzz_6; set allow_suspicious_low_cardinality_types = 1; - CREATE TABLE null_in__fuzz_6 (`dt` LowCardinality(UInt16), `idx` Int32, `i` Nullable(Int256), `s` Int32) ENGINE = MergeTree PARTITION BY dt ORDER BY idx; insert into null_in__fuzz_6 select * from generateRandom() limit 1; @@ -13,3 +12,6 @@ CREATE TABLE test_set (i Nullable(int)) ENGINE = Set(); INSERT INTO test_set VALUES (1), (NULL); SELECT count() = 1 FROM null_in__fuzz_6 PREWHERE 71 WHERE i IN (test_set); -- { serverError CANNOT_CONVERT_TYPE } + +DROP TABLE test_set; +DROP TABLE null_in__fuzz_6; From 3c8c4ce449fb9bfa74de74a0756b85f9c149efbe Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 28 Jun 2023 13:16:26 +0200 Subject: [PATCH 399/515] Update test.py --- tests/integration/test_storage_postgresql/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index d60a90ed7ce..05d4b1e12ef 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -335,7 +335,7 @@ def test_concurrent_queries(started_cluster): ) ) print(count) - assert count <= 18 + assert count <= 19 busy_pool = Pool(30) p = busy_pool.map_async(node_insert_select, range(30)) @@ -347,7 +347,7 @@ def test_concurrent_queries(started_cluster): ) ) print(count) - assert count <= 18 + assert count <= 20 node1.query("DROP TABLE test.test_table;") node1.query("DROP TABLE test.stat;") From 06206d092d843fd3e6c005609008cd85597f9124 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 28 Jun 2023 13:18:53 +0200 Subject: [PATCH 400/515] Update test.py --- tests/integration/test_storage_postgresql/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 05d4b1e12ef..83eb4f6f02a 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -323,7 +323,7 @@ def test_concurrent_queries(started_cluster): ) ) print(count) - assert count <= 18 + assert count <= 18 # 16 for test.test_table + 1 for conn + 1 for test.stat busy_pool = Pool(30) p = busy_pool.map_async(node_insert, range(30)) @@ -335,7 +335,7 @@ def test_concurrent_queries(started_cluster): ) ) print(count) - assert count <= 19 + assert count <= 19 # 16 for test.test_table + 1 for conn + at most 2 for test.stat busy_pool = Pool(30) p = busy_pool.map_async(node_insert_select, range(30)) @@ -347,7 +347,7 @@ def test_concurrent_queries(started_cluster): ) ) print(count) - assert count <= 20 + assert count <= 20 # 16 for test.test_table + 1 for conn + at most 3 for test.stat node1.query("DROP TABLE test.test_table;") node1.query("DROP TABLE test.stat;") From 5443987113e0417c310eaf7bbfdfdf26de64f142 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 28 Jun 2023 11:33:45 +0000 Subject: [PATCH 401/515] Automatic style fix --- tests/integration/test_storage_postgresql/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 83eb4f6f02a..9f7c012e66f 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -323,7 +323,7 @@ def test_concurrent_queries(started_cluster): ) ) print(count) - assert count <= 18 # 16 for test.test_table + 1 for conn + 1 for test.stat + assert count <= 18 # 16 for test.test_table + 1 for conn + 1 for test.stat busy_pool = Pool(30) p = busy_pool.map_async(node_insert, range(30)) @@ -335,7 +335,7 @@ def test_concurrent_queries(started_cluster): ) ) print(count) - assert count <= 19 # 16 for test.test_table + 1 for conn + at most 2 for test.stat + assert count <= 19 # 16 for test.test_table + 1 for conn + at most 2 for test.stat busy_pool = Pool(30) p = busy_pool.map_async(node_insert_select, range(30)) @@ -347,7 +347,7 @@ def test_concurrent_queries(started_cluster): ) ) print(count) - assert count <= 20 # 16 for test.test_table + 1 for conn + at most 3 for test.stat + assert count <= 20 # 16 for test.test_table + 1 for conn + at most 3 for test.stat node1.query("DROP TABLE test.test_table;") node1.query("DROP TABLE test.stat;") From 5a1bbe5a8d2f07960161100e0a17527e4bda6de2 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 28 Jun 2023 14:05:14 +0200 Subject: [PATCH 402/515] Update CachedOnDiskReadBufferFromFile.cpp --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 960d2a72410..5b42f41fbf3 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -852,9 +852,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() else { implementation_buffer = getImplementationBuffer(file_segments->front()); - - if (read_type == ReadType::CACHED) - file_segments->front().use(); + file_segments->front().use(); } chassert(!internal_buffer.empty()); From 15f64a7cb9a07d48e92fd0db6ea1f9b8227b0e5e Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 28 Jun 2023 15:26:17 +0200 Subject: [PATCH 403/515] Add some checks --- src/Interpreters/Cache/FileSegment.cpp | 3 ++- src/Interpreters/Cache/IFileCachePriority.h | 2 +- .../Cache/LRUFileCachePriority.cpp | 19 +++++++++++++++---- src/Interpreters/Cache/LRUFileCachePriority.h | 4 +++- src/Interpreters/Cache/Metadata.cpp | 4 ++-- 5 files changed, 23 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 95592fc7c12..e97d708ba74 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -872,6 +872,7 @@ void FileSegment::setDetachedState(const FileSegmentGuard::Lock & lock) setDownloadState(State::DETACHED, lock); key_metadata.reset(); cache = nullptr; + queue_iterator = nullptr; } void FileSegment::detach(const FileSegmentGuard::Lock & lock, const LockedKey &) @@ -890,7 +891,7 @@ void FileSegment::use() if (!cache) { - chassert(isCompleted(true)); + chassert(isDetached()); return; } diff --git a/src/Interpreters/Cache/IFileCachePriority.h b/src/Interpreters/Cache/IFileCachePriority.h index 10f63fce21d..34c49653ab8 100644 --- a/src/Interpreters/Cache/IFileCachePriority.h +++ b/src/Interpreters/Cache/IFileCachePriority.h @@ -44,7 +44,7 @@ public: virtual size_t use(const CacheGuard::Lock &) = 0; - virtual std::shared_ptr remove(const CacheGuard::Lock &) = 0; + virtual void remove(const CacheGuard::Lock &) = 0; virtual const Entry & getEntry() const = 0; diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 6f142c0cc6d..18862e154da 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -166,15 +166,17 @@ void LRUFileCachePriority::iterate(IterateFunc && func, const CacheGuard::Lock & } } -LRUFileCachePriority::Iterator -LRUFileCachePriority::LRUFileCacheIterator::remove(const CacheGuard::Lock &) +void LRUFileCachePriority::LRUFileCacheIterator::remove(const CacheGuard::Lock &) { - return std::make_shared( - cache_priority, cache_priority->remove(queue_iter)); + checkUsable(); + cache_priority->remove(queue_iter); + queue_iter = LRUQueueIterator{}; } void LRUFileCachePriority::LRUFileCacheIterator::invalidate() { + checkUsable(); + LOG_TEST( cache_priority->log, "Invalidating entry in LRU queue. Key: {}, offset: {}, previous size: {}", @@ -187,6 +189,8 @@ void LRUFileCachePriority::LRUFileCacheIterator::invalidate() void LRUFileCachePriority::LRUFileCacheIterator::updateSize(int64_t size) { + checkUsable(); + LOG_TEST( cache_priority->log, "Update size with {} in LRU queue for key: {}, offset: {}, previous size: {}", @@ -198,8 +202,15 @@ void LRUFileCachePriority::LRUFileCacheIterator::updateSize(int64_t size) size_t LRUFileCachePriority::LRUFileCacheIterator::use(const CacheGuard::Lock &) { + checkUsable(); cache_priority->queue.splice(cache_priority->queue.end(), cache_priority->queue, queue_iter); return ++queue_iter->hits; } +void LRUFileCachePriority::LRUFileCacheIterator::checkUsable() const +{ + if (queue_iter == LRUQueueIterator{}) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to use invalid iterator"); +} + } diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 8dc4eb0a016..e0d7d45062a 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -60,13 +60,15 @@ public: size_t use(const CacheGuard::Lock &) override; - Iterator remove(const CacheGuard::Lock &) override; + void remove(const CacheGuard::Lock &) override; void invalidate() override; void updateSize(int64_t size) override; private: + void checkUsable() const; + LRUFileCachePriority * cache_priority; mutable LRUFileCachePriority::LRUQueueIterator queue_iter; }; diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 8c8524f7fa7..a77a9072cb6 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -147,7 +147,6 @@ String CacheMetadata::getFileNameForFileSegment(size_t offset, FileSegmentKind s file_suffix = "_temporary"; break; case FileSegmentKind::Regular: - file_suffix = ""; break; } return std::to_string(offset) + file_suffix; @@ -398,6 +397,8 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegm if (file_segment->queue_iterator) file_segment->queue_iterator->invalidate(); + file_segment->detach(segment_lock, *this); + const auto path = key_metadata->getFileSegmentPath(*file_segment); bool exists = fs::exists(path); if (exists) @@ -408,7 +409,6 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegm else if (file_segment->downloaded_size) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist", path); - file_segment->detach(segment_lock, *this); return key_metadata->erase(it); } From c42bf37a141decb206e405470c2af6d85145bf3f Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 28 Jun 2023 09:59:32 -0400 Subject: [PATCH 404/515] list the disk types --- .../engines/table-engines/mergetree-family/mergetree.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 42454af6feb..c67ac8fa4ef 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -756,6 +756,14 @@ If you perform the `SELECT` query between merges, you may get expired data. To a - [ttl_only_drop_parts](/docs/en/operations/settings/settings.md/#ttl_only_drop_parts) setting +## Disk types + +In addition to local block devices, ClickHouse supports other device types through table engines. These are the types: +- [S3](#table_engine-mergetree-s3) +- GCS (also supported using the [S3 table engine](#table_engine-mergetree-s3)) +- [Azure Blob Storage](#table_engine-mergetree-azure-blob-storage) +- [HDFS](/docs/en/sql-reference/table-functions/hdfs.md) + ## Using Multiple Block Devices for Data Storage {#table_engine-mergetree-multiple-volumes} ### Introduction {#introduction} From f57d91b678226094bc55074b3df1ce8ba630fb5f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 28 Jun 2023 17:14:58 +0300 Subject: [PATCH 405/515] Update 02803_remote_cannot_clone_block.sql --- tests/queries/0_stateless/02803_remote_cannot_clone_block.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql b/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql index 3e14aa6be96..6d79aa76d18 100644 --- a/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql +++ b/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql @@ -6,6 +6,8 @@ SELECT * FROM system.numbers LIMIT 10000; +SET allow_experimental_analyzer = 0; + SELECT * FROM ( From 0b19c1832a7afd9015e1872a932b2d28b326117c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 28 Jun 2023 14:15:03 +0000 Subject: [PATCH 406/515] Fix: detach from thread group --- src/Dictionaries/HashedDictionary.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index eb1d98a8f39..798f37cb516 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include @@ -69,6 +70,11 @@ public: shards_queues[shard].emplace(backlog); pool.scheduleOrThrowOnError([this, shard, thread_group = CurrentThread::getGroup()] { + SCOPE_EXIT_SAFE( + if (thread_group) + CurrentThread::detachFromGroupIfNotDetached(); + ); + /// Do not account memory that was occupied by the dictionaries for the query/user context. MemoryTrackerBlockerInThread memory_blocker; From bfcadabb927e5ea547c29df488d3fe6ea396a178 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 28 Jun 2023 10:30:49 -0400 Subject: [PATCH 407/515] add web disk type --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index c67ac8fa4ef..1f084fe075b 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -763,6 +763,7 @@ In addition to local block devices, ClickHouse supports other device types throu - GCS (also supported using the [S3 table engine](#table_engine-mergetree-s3)) - [Azure Blob Storage](#table_engine-mergetree-azure-blob-storage) - [HDFS](/docs/en/sql-reference/table-functions/hdfs.md) +- [Web (read-only)](#web-storage) ## Using Multiple Block Devices for Data Storage {#table_engine-mergetree-multiple-volumes} @@ -944,6 +945,8 @@ configuration files; all the settings are in the CREATE/ATTACH query. The example uses `type=web`, but any disk type can be configured as dynamic, even Local disk. Local disks require a path argument to be inside the server config parameter `custom_local_disks_base_directory`, which has no default, so set that also when using local disk. ::: +#### Example dynamic web storage {#web-storage} + ```sql ATTACH TABLE uk_price_paid UUID 'cf712b4f-2ca8-435c-ac23-c4393efe52f7' ( From b392127304d8c14ce34bd86d0b8ca561e1559919 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 28 Jun 2023 11:00:07 -0400 Subject: [PATCH 408/515] add example web config --- .../mergetree-family/mergetree.md | 53 ++++++++++++++++++- 1 file changed, 52 insertions(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 1f084fe075b..b87c4d216cf 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -945,7 +945,7 @@ configuration files; all the settings are in the CREATE/ATTACH query. The example uses `type=web`, but any disk type can be configured as dynamic, even Local disk. Local disks require a path argument to be inside the server config parameter `custom_local_disks_base_directory`, which has no default, so set that also when using local disk. ::: -#### Example dynamic web storage {#web-storage} +#### Example dynamic web storage ```sql ATTACH TABLE uk_price_paid UUID 'cf712b4f-2ca8-435c-ac23-c4393efe52f7' @@ -1249,6 +1249,57 @@ Examples of working configurations can be found in integration tests directory ( Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: +## Web storage (read-only) {#web-storage} + +Web storage can be used for read-only purposes. An example use is for hosting sample +data, or for migrating data. + +:::tip +Storage can also be configured temporarily within a query, if a web dataset is not expected +to be used routinely, see [dynamic storage](#dynamic-storage) and skip editing the +configuration file. +::: + +In this sample configuration: +- the disk is of type `web` +- the data is hosted at `http://nginx:80/test1/` +- a cache on local storage is used + +```xml + + + + + web + http://nginx:80/test1/ + + + cache + web + cached_web_cache/ + 100000000 + + + + + +
+ web +
+
+
+ + +
+ cached_web +
+
+
+
+
+
+``` + ## Virtual Columns {#virtual-columns} - `_part` — Name of a part. From dd3a744cef6e736bc68782fd79853a1535bdebb8 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 28 Jun 2023 11:17:16 -0400 Subject: [PATCH 409/515] add HDFS example --- .../mergetree-family/mergetree.md | 38 ++++++++++++++++++- 1 file changed, 37 insertions(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index b87c4d216cf..1b7f3263ab9 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -762,7 +762,7 @@ In addition to local block devices, ClickHouse supports other device types throu - [S3](#table_engine-mergetree-s3) - GCS (also supported using the [S3 table engine](#table_engine-mergetree-s3)) - [Azure Blob Storage](#table_engine-mergetree-azure-blob-storage) -- [HDFS](/docs/en/sql-reference/table-functions/hdfs.md) +- [HDFS](#hdfs-storage) - [Web (read-only)](#web-storage) ## Using Multiple Block Devices for Data Storage {#table_engine-mergetree-multiple-volumes} @@ -1249,6 +1249,42 @@ Examples of working configurations can be found in integration tests directory ( Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: +## HDFS storage {#hdfs-storage} + +In this sample configuration: +- the disk is of type `hdfs` +- the data is hosted at `hdfs://hdfs1:9000/clickhouse/` + +```xml + + + + + hdfs + hdfs://hdfs1:9000/clickhouse/ + true + + + local + / + + + + + +
+ hdfs +
+ + hdd + +
+
+
+
+
+``` + ## Web storage (read-only) {#web-storage} Web storage can be used for read-only purposes. An example use is for hosting sample From 594ec09edf0d63b73e02c2dbbee6cc500eb52d87 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 28 Jun 2023 18:19:16 +0200 Subject: [PATCH 410/515] fix missing metadata_version.txt --- .../MergeTree/DataPartStorageOnDiskBase.cpp | 26 ++++---- .../MergeTree/DataPartStorageOnDiskBase.h | 5 +- src/Storages/MergeTree/IDataPartStorage.h | 31 +++++++-- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 20 +++--- src/Storages/MergeTree/MergeTreeData.cpp | 35 +++++------ src/Storages/MergeTree/MergeTreeData.h | 13 +--- src/Storages/MergeTree/MutateTask.cpp | 11 +++- src/Storages/MergeTree/MutateTask.h | 2 +- src/Storages/StorageMergeTree.cpp | 6 +- src/Storages/StorageReplicatedMergeTree.cpp | 63 +++++++++++-------- 10 files changed, 120 insertions(+), 92 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index 30776a8bc50..c850ebaa6fd 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -415,41 +415,41 @@ void DataPartStorageOnDiskBase::backup( MutableDataPartStoragePtr DataPartStorageOnDiskBase::freeze( const std::string & to, const std::string & dir_path, - bool make_source_readonly, std::function save_metadata_callback, - bool copy_instead_of_hardlink, - const NameSet & files_to_copy_instead_of_hardlinks, - DiskTransactionPtr external_transaction) const + const ClonePartParams & params) const { auto disk = volume->getDisk(); - if (external_transaction) - external_transaction->createDirectories(to); + if (params.external_transaction) + params.external_transaction->createDirectories(to); else disk->createDirectories(to); - localBackup(disk, getRelativePath(), fs::path(to) / dir_path, make_source_readonly, {}, copy_instead_of_hardlink, files_to_copy_instead_of_hardlinks, external_transaction); + localBackup(disk, getRelativePath(), fs::path(to) / dir_path, params.make_source_readonly, {}, params.copy_instead_of_hardlink, + params.files_to_copy_instead_of_hardlinks, params.external_transaction); if (save_metadata_callback) save_metadata_callback(disk); - if (external_transaction) + if (params.external_transaction) { - external_transaction->removeFileIfExists(fs::path(to) / dir_path / "delete-on-destroy.txt"); - external_transaction->removeFileIfExists(fs::path(to) / dir_path / "txn_version.txt"); - external_transaction->removeFileIfExists(fs::path(to) / dir_path / IMergeTreeDataPart::METADATA_VERSION_FILE_NAME); + params.external_transaction->removeFileIfExists(fs::path(to) / dir_path / "delete-on-destroy.txt"); + params.external_transaction->removeFileIfExists(fs::path(to) / dir_path / "txn_version.txt"); + if (!params.keep_metadata_version) + params.external_transaction->removeFileIfExists(fs::path(to) / dir_path / IMergeTreeDataPart::METADATA_VERSION_FILE_NAME); } else { disk->removeFileIfExists(fs::path(to) / dir_path / "delete-on-destroy.txt"); disk->removeFileIfExists(fs::path(to) / dir_path / "txn_version.txt"); - disk->removeFileIfExists(fs::path(to) / dir_path / IMergeTreeDataPart::METADATA_VERSION_FILE_NAME); + if (!params.keep_metadata_version) + disk->removeFileIfExists(fs::path(to) / dir_path / IMergeTreeDataPart::METADATA_VERSION_FILE_NAME); } auto single_disk_volume = std::make_shared(disk->getName(), disk, 0); /// Do not initialize storage in case of DETACH because part may be broken. bool to_detached = dir_path.starts_with("detached/"); - return create(single_disk_volume, to, dir_path, /*initialize=*/ !to_detached && !external_transaction); + return create(single_disk_volume, to, dir_path, /*initialize=*/ !to_detached && !params.external_transaction); } MutableDataPartStoragePtr DataPartStorageOnDiskBase::clonePart( diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h index 043953eb20c..5f7dcc3fd32 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h @@ -62,11 +62,8 @@ public: MutableDataPartStoragePtr freeze( const std::string & to, const std::string & dir_path, - bool make_source_readonly, std::function save_metadata_callback, - bool copy_instead_of_hardlink, - const NameSet & files_to_copy_instead_of_hardlinks, - DiskTransactionPtr external_transaction) const override; + const ClonePartParams & params) const override; MutableDataPartStoragePtr clonePart( const std::string & to, diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index 9d6c5d0dcba..b40a9aa1b46 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -63,6 +63,9 @@ using DiskPtr = std::shared_ptr; class ISyncGuard; using SyncGuardPtr = std::unique_ptr; +class MergeTreeTransaction; +using MergeTreeTransactionPtr = std::shared_ptr; + class IBackupEntry; using BackupEntryPtr = std::shared_ptr; using BackupEntries = std::vector>; @@ -72,6 +75,17 @@ struct WriteSettings; class TemporaryFileOnDisk; + +struct HardlinkedFiles +{ + /// Shared table uuid where hardlinks live + std::string source_table_shared_id; + /// Hardlinked from part + std::string source_part_name; + /// Hardlinked files list + NameSet hardlinks_from_source_part; +}; + /// This is an abstraction of storage for data part files. /// Ideally, it is assumed to contain read-only methods from IDisk. /// It is not fulfilled now, but let's try our best. @@ -220,14 +234,23 @@ public: /// If `external_transaction` is provided, the disk operations (creating directories, hardlinking, /// etc) won't be applied immediately; instead, they'll be added to external_transaction, which the /// caller then needs to commit. + + struct ClonePartParams + { + MergeTreeTransactionPtr txn = NO_TRANSACTION_PTR; + HardlinkedFiles * hardlinked_files = nullptr; + bool copy_instead_of_hardlink = false; + NameSet files_to_copy_instead_of_hardlinks; + bool keep_metadata_version = false; + bool make_source_readonly = false; + DiskTransactionPtr external_transaction = nullptr; + }; + virtual std::shared_ptr freeze( const std::string & to, const std::string & dir_path, - bool make_source_readonly, std::function save_metadata_callback, - bool copy_instead_of_hardlink, - const NameSet & files_to_copy_instead_of_hardlinks, - DiskTransactionPtr external_transaction = nullptr) const = 0; + const ClonePartParams & params) const = 0; /// Make a full copy of a data part into 'to/dir_path' (possibly to a different disk). virtual std::shared_ptr clonePart( diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 85edba84296..1f105951757 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1363,6 +1363,8 @@ void IMergeTreeDataPart::loadColumns(bool require) { loaded_metadata_version = metadata_snapshot->getMetadataVersion(); old_part_with_no_metadata_version_on_disk = true; + LOG_WARNING(storage.log, "Part {} doesn't have metadata version on disk, setting it to {}. " + "It's okay if the part was created by an old version of ClickHouse", name, loaded_metadata_version); } setColumns(loaded_columns, infos, loaded_metadata_version); @@ -1765,12 +1767,6 @@ void IMergeTreeDataPart::renameToDetached(const String & prefix) DataPartStoragePtr IMergeTreeDataPart::makeCloneInDetached(const String & prefix, const StorageMetadataPtr & /*metadata_snapshot*/) const { - auto storage_settings = storage.getSettings(); - - /// In case of zero-copy replication we copy directory instead of hardlinks - /// because hardlinks tracking doesn't work for detached parts. - bool copy_instead_of_hardlink = isStoredOnRemoteDiskWithZeroCopySupport() && storage.supportsReplication() && storage_settings->allow_remote_fs_zero_copy_replication; - /// Avoid unneeded duplicates of broken parts if we try to detach the same broken part multiple times. /// Otherwise it may pollute detached/ with dirs with _tryN suffix and we will fail to remove broken part after 10 attempts. bool broken = !prefix.empty(); @@ -1778,13 +1774,19 @@ DataPartStoragePtr IMergeTreeDataPart::makeCloneInDetached(const String & prefix if (!maybe_path_in_detached) return nullptr; + /// In case of zero-copy replication we copy directory instead of hardlinks + /// because hardlinks tracking doesn't work for detached parts. + auto storage_settings = storage.getSettings(); + IDataPartStorage::ClonePartParams params + { + .copy_instead_of_hardlink = isStoredOnRemoteDiskWithZeroCopySupport() && storage.supportsReplication() && storage_settings->allow_remote_fs_zero_copy_replication, + .make_source_readonly = true + }; return getDataPartStorage().freeze( storage.relative_data_path, *maybe_path_in_detached, - /*make_source_readonly=*/ true, /*save_metadata_callback=*/ {}, - copy_instead_of_hardlink, - /*files_to_copy_instead_of_hardlinks=*/ {}); + params); } MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & disk, const String & directory_name) const diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e9c3a7f66ae..f3cf4a85953 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7289,10 +7289,7 @@ std::pair MergeTreeData::cloneAn const String & tmp_part_prefix, const MergeTreePartInfo & dst_part_info, const StorageMetadataPtr & metadata_snapshot, - const MergeTreeTransactionPtr & txn, - HardlinkedFiles * hardlinked_files, - bool copy_instead_of_hardlink, - const NameSet & files_to_copy_instead_of_hardlinks) + const IDataPartStorage::ClonePartParams & params) { /// Check that the storage policy contains the disk where the src_part is located. bool does_storage_policy_allow_same_disk = false; @@ -7343,16 +7340,14 @@ std::pair MergeTreeData::cloneAn } String with_copy; - if (copy_instead_of_hardlink) + if (params.copy_instead_of_hardlink) with_copy = " (copying data)"; auto dst_part_storage = src_part_storage->freeze( relative_data_path, tmp_dst_part_name, - /*make_source_readonly=*/ false, /*save_metadata_callback=*/ {}, - copy_instead_of_hardlink, - files_to_copy_instead_of_hardlinks); + params); LOG_DEBUG(log, "Clone{} part {} to {}{}", src_flushed_tmp_part ? " flushed" : "", @@ -7364,18 +7359,18 @@ std::pair MergeTreeData::cloneAn .withPartFormatFromDisk() .build(); - if (!copy_instead_of_hardlink && hardlinked_files) + if (!params.copy_instead_of_hardlink && params.hardlinked_files) { - hardlinked_files->source_part_name = src_part->name; - hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID(); + params.hardlinked_files->source_part_name = src_part->name; + params.hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID(); for (auto it = src_part->getDataPartStorage().iterate(); it->isValid(); it->next()) { - if (!files_to_copy_instead_of_hardlinks.contains(it->name()) + if (!params.files_to_copy_instead_of_hardlinks.contains(it->name()) && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) { - hardlinked_files->hardlinks_from_source_part.insert(it->name()); + params.hardlinked_files->hardlinks_from_source_part.insert(it->name()); } } @@ -7386,18 +7381,18 @@ std::pair MergeTreeData::cloneAn for (auto it = projection_storage.iterate(); it->isValid(); it->next()) { auto file_name_with_projection_prefix = fs::path(projection_storage.getPartDirectory()) / it->name(); - if (!files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix) + if (!params.files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix) && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) { - hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix); + params.hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix); } } } } /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. - TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; + TransactionID tid = params.txn ? params.txn->tid : Tx::PrehistoricTID; dst_data_part->version.setCreationTID(tid, nullptr); dst_data_part->storeVersionMetadata(); @@ -7579,13 +7574,15 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( createAndStoreFreezeMetadata(disk, part, fs::path(backup_part_path) / part->getDataPartStorage().getPartDirectory()); }; + IDataPartStorage::ClonePartParams params + { + .make_source_readonly = true + }; auto new_storage = data_part_storage->freeze( backup_part_path, part->getDataPartStorage().getPartDirectory(), - /*make_source_readonly=*/ true, callback, - /*copy_instead_of_hardlink=*/ false, - /*files_to_copy_instead_of_hardlinks=*/ {}); + params); part->is_frozen.store(true, std::memory_order_relaxed); result.push_back(PartitionCommandResultInfo{ diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index b27392b355b..43e59ccc392 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -828,21 +828,10 @@ public: MergeTreeData & checkStructureAndGetMergeTreeData(const StoragePtr & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const; MergeTreeData & checkStructureAndGetMergeTreeData(IStorage & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const; - struct HardlinkedFiles - { - /// Shared table uuid where hardlinks live - std::string source_table_shared_id; - /// Hardlinked from part - std::string source_part_name; - /// Hardlinked files list - NameSet hardlinks_from_source_part; - }; - std::pair cloneAndLoadDataPartOnSameDisk( const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix, const MergeTreePartInfo & dst_part_info, const StorageMetadataPtr & metadata_snapshot, - const MergeTreeTransactionPtr & txn, HardlinkedFiles * hardlinked_files, - bool copy_instead_of_hardlink, const NameSet & files_to_copy_instead_of_hardlinks); + const IDataPartStorage::ClonePartParams & params); virtual std::vector getMutationsStatus() const = 0; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index a19b9daca0e..f4a071b8f27 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -852,7 +852,7 @@ struct MutationContext MergeTreeTransactionPtr txn; - MergeTreeData::HardlinkedFiles hardlinked_files; + HardlinkedFiles hardlinked_files; bool need_prefix = true; @@ -1803,7 +1803,12 @@ bool MutateTask::prepare() if (ctx->need_prefix) prefix = "tmp_clone_"; - auto [part, lock] = ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, prefix, ctx->future_part->part_info, ctx->metadata_snapshot, ctx->txn, &ctx->hardlinked_files, false, files_to_copy_instead_of_hardlinks); + IDataPartStorage::ClonePartParams clone_params + { + .txn = ctx->txn, .hardlinked_files = &ctx->hardlinked_files, + .files_to_copy_instead_of_hardlinks = std::move(files_to_copy_instead_of_hardlinks), .keep_metadata_version = true + }; + auto [part, lock] = ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, prefix, ctx->future_part->part_info, ctx->metadata_snapshot, clone_params); part->getDataPartStorage().beginTransaction(); ctx->temporary_directory_lock = std::move(lock); @@ -1932,7 +1937,7 @@ bool MutateTask::prepare() return true; } -const MergeTreeData::HardlinkedFiles & MutateTask::getHardlinkedFiles() const +const HardlinkedFiles & MutateTask::getHardlinkedFiles() const { return ctx->hardlinked_files; } diff --git a/src/Storages/MergeTree/MutateTask.h b/src/Storages/MergeTree/MutateTask.h index 54ad996ad4c..dc21df018d7 100644 --- a/src/Storages/MergeTree/MutateTask.h +++ b/src/Storages/MergeTree/MutateTask.h @@ -45,7 +45,7 @@ public: return promise.get_future(); } - const MergeTreeData::HardlinkedFiles & getHardlinkedFiles() const; + const HardlinkedFiles & getHardlinkedFiles() const; private: diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c02c96f62be..4c0c0c8e3fa 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1932,7 +1932,8 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con Int64 temp_index = insert_increment.get(); MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level); - auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, my_metadata_snapshot, local_context->getCurrentTransaction(), {}, false, {}); + IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()}; + auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, my_metadata_snapshot, clone_params); dst_parts.emplace_back(std::move(dst_part)); dst_parts_locks.emplace_back(std::move(part_lock)); } @@ -2030,7 +2031,8 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const Int64 temp_index = insert_increment.get(); MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level); - auto [dst_part, part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot, local_context->getCurrentTransaction(), {}, false, {}); + IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()}; + auto [dst_part, part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot, clone_params); dst_parts.emplace_back(std::move(dst_part)); dst_parts_locks.emplace_back(std::move(part_lock)); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b1ba06c77f9..ecc2537f6ad 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1578,7 +1578,7 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil: } MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAndCommit(Transaction & transaction, - const MutableDataPartPtr & part, std::optional hardlinked_files, bool replace_zero_copy_lock) + const MutableDataPartPtr & part, std::optional hardlinked_files, bool replace_zero_copy_lock) { auto zookeeper = getZooKeeper(); @@ -2183,7 +2183,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) /// A replica that will be used to fetch part String replica; - MergeTreeData::HardlinkedFiles hardlinked_files; + HardlinkedFiles hardlinked_files; scope_guard temporary_part_lock; }; @@ -2433,8 +2433,6 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) static const String TMP_PREFIX = "tmp_replace_from_"; - std::vector hardlinked_files_for_parts; - auto obtain_part = [&] (PartDescriptionPtr & part_desc) { if (part_desc->src_table_part) @@ -2442,8 +2440,14 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) if (part_desc->checksum_hex != part_desc->src_table_part->checksums.getTotalChecksumHex()) throw Exception(ErrorCodes::UNFINISHED, "Checksums of {} is suddenly changed", part_desc->src_table_part->name); + bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication + || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; + IDataPartStorage::ClonePartParams clone_params + { + .copy_instead_of_hardlink = zero_copy_enabled && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport() + }; auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk( - part_desc->src_table_part, TMP_PREFIX + "clone_", part_desc->new_part_info, metadata_snapshot, NO_TRANSACTION_PTR, &part_desc->hardlinked_files, false, {}); + part_desc->src_table_part, TMP_PREFIX + "clone_", part_desc->new_part_info, metadata_snapshot, clone_params); part_desc->res_part = std::move(res_part); part_desc->temporary_part_lock = std::move(temporary_part_lock); } @@ -4270,6 +4274,11 @@ bool StorageReplicatedMergeTree::fetchPart( profile_events_scope.getSnapshot()); }; + auto is_zero_copy_part = [&settings_ptr](const auto & data_part) + { + return settings_ptr->allow_remote_fs_zero_copy_replication && data_part->isStoredOnRemoteDiskWithZeroCopySupport(); + }; + DataPartPtr part_to_clone; { /// If the desired part is a result of a part mutation, try to find the source part and compare @@ -4281,7 +4290,7 @@ bool StorageReplicatedMergeTree::fetchPart( auto source_part = getActiveContainingPart(covered_part_info); /// Fetch for zero-copy replication is cheap and straightforward, so we don't use local clone here - if (source_part && (!settings_ptr->allow_remote_fs_zero_copy_replication || !source_part->getDataPartStorage().supportZeroCopyReplication())) + if (source_part && !is_zero_copy_part(source_part)) { auto source_part_header = ReplicatedMergeTreePartHeader::fromColumnsAndChecksums( source_part->getColumns(), source_part->checksums); @@ -4330,14 +4339,15 @@ bool StorageReplicatedMergeTree::fetchPart( InterserverCredentialsPtr credentials; std::optional tagger_ptr; std::function get_part; - MergeTreeData::HardlinkedFiles hardlinked_files; scope_guard part_directory_lock; if (part_to_clone) { get_part = [&, part_to_clone]() { - auto [cloned_part, lock] = cloneAndLoadDataPartOnSameDisk(part_to_clone, "tmp_clone_", part_info, metadata_snapshot, NO_TRANSACTION_PTR, &hardlinked_files, false, {}); + chassert(!is_zero_copy_part(part_to_clone)); + IDataPartStorage::ClonePartParams clone_params{ .keep_metadata_version = true }; + auto [cloned_part, lock] = cloneAndLoadDataPartOnSameDisk(part_to_clone, "tmp_clone_", part_info, metadata_snapshot, clone_params); part_directory_lock = std::move(lock); return cloned_part; }; @@ -4387,7 +4397,8 @@ bool StorageReplicatedMergeTree::fetchPart( Transaction transaction(*this, NO_TRANSACTION_RAW); renameTempPartAndReplace(part, transaction); - replaced_parts = checkPartChecksumsAndCommit(transaction, part, hardlinked_files, !part_to_clone); + chassert(!part_to_clone || !is_zero_copy_part(part)); + replaced_parts = checkPartChecksumsAndCommit(transaction, part, /*hardlinked_files*/ {}, /*replace_zero_copy_lock*/ true); /** If a quorum is tracked for this part, you must update it. * If you do not have time, in case of losing the session, when you restart the server - see the `ReplicatedMergeTreeRestartingThread::updateQuorumIfWeHavePart` method. @@ -7255,7 +7266,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom( assert(replace == !LogEntry::ReplaceRangeEntry::isMovePartitionOrAttachFrom(drop_range)); String drop_range_fake_part_name = getPartNamePossiblyFake(format_version, drop_range); - std::vector hardlinked_files_for_parts; for (const auto & src_part : src_all_parts) { @@ -7286,19 +7296,21 @@ void StorageReplicatedMergeTree::replacePartitionFrom( UInt64 index = lock->getNumber(); MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - MergeTreeData::HardlinkedFiles hardlinked_files; + HardlinkedFiles hardlinked_files; - bool copy_instead_of_hardlink = storage_settings_ptr->allow_remote_fs_zero_copy_replication - && src_part->isStoredOnRemoteDiskWithZeroCopySupport(); - - auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, metadata_snapshot, NO_TRANSACTION_PTR, &hardlinked_files, copy_instead_of_hardlink, {}); + bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication + || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; + IDataPartStorage::ClonePartParams clone_params + { + .copy_instead_of_hardlink = zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport() + }; + auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, metadata_snapshot, clone_params); src_parts.emplace_back(src_part); dst_parts.emplace_back(dst_part); dst_parts_locks.emplace_back(std::move(part_lock)); ephemeral_locks.emplace_back(std::move(*lock)); block_id_paths.emplace_back(block_id_path); part_checksums.emplace_back(hash_hex); - hardlinked_files_for_parts.emplace_back(hardlinked_files); } ReplicatedMergeTreeLogEntryData entry; @@ -7360,7 +7372,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( } for (size_t i = 0; i < dst_parts.size(); ++i) - lockSharedData(*dst_parts[i], false, hardlinked_files_for_parts[i]); + lockSharedData(*dst_parts[i], false, /*hardlinked_files*/ {}); Coordination::Error code = zookeeper->tryMulti(ops, op_results); if (code == Coordination::Error::ZOK) @@ -7501,7 +7513,6 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta String dest_alter_partition_version_path = dest_table_storage->zookeeper_path + "/alter_partition_version"; Coordination::Stat dest_alter_partition_version_stat; zookeeper->get(dest_alter_partition_version_path, &dest_alter_partition_version_stat); - std::vector hardlinked_files_for_parts; std::vector temporary_parts_locks; for (const auto & src_part : src_all_parts) @@ -7524,12 +7535,15 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta UInt64 index = lock->getNumber(); MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - MergeTreeData::HardlinkedFiles hardlinked_files; + HardlinkedFiles hardlinked_files; - bool copy_instead_of_hardlink = storage_settings_ptr->allow_remote_fs_zero_copy_replication - && src_part->isStoredOnRemoteDiskWithZeroCopySupport(); - - auto [dst_part, dst_part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot, NO_TRANSACTION_PTR, &hardlinked_files, copy_instead_of_hardlink, {}); + bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication + || dynamic_cast(dest_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; + IDataPartStorage::ClonePartParams clone_params + { + .copy_instead_of_hardlink = zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport() + }; + auto [dst_part, dst_part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot, clone_params); src_parts.emplace_back(src_part); dst_parts.emplace_back(dst_part); @@ -7537,7 +7551,6 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta ephemeral_locks.emplace_back(std::move(*lock)); block_id_paths.emplace_back(block_id_path); part_checksums.emplace_back(hash_hex); - hardlinked_files_for_parts.emplace_back(hardlinked_files); } ReplicatedMergeTreeLogEntryData entry_delete; @@ -7606,7 +7619,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta dest_table_storage->renameTempPartAndReplaceUnlocked(part, transaction, dest_data_parts_lock); for (size_t i = 0; i < dst_parts.size(); ++i) - dest_table_storage->lockSharedData(*dst_parts[i], false, hardlinked_files_for_parts[i]); + dest_table_storage->lockSharedData(*dst_parts[i], false, /*hardlinked_files*/ {}); Coordination::Error code = zookeeper->tryMulti(ops, op_results); if (code == Coordination::Error::ZBADVERSION) From 9c2a9a60ea9a1f474495d74c399df5ffe5dddb45 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 28 Jun 2023 16:35:41 +0000 Subject: [PATCH 411/515] Update version_date.tsv and changelogs after v23.3.6.7-lts --- docs/changelogs/v23.3.6.7-lts.md | 19 +++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 20 insertions(+) create mode 100644 docs/changelogs/v23.3.6.7-lts.md diff --git a/docs/changelogs/v23.3.6.7-lts.md b/docs/changelogs/v23.3.6.7-lts.md new file mode 100644 index 00000000000..387cc126aba --- /dev/null +++ b/docs/changelogs/v23.3.6.7-lts.md @@ -0,0 +1,19 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.3.6.7-lts (7e3f0a271b7) FIXME as compared to v23.3.5.9-lts (f5fbc2fd2b3) + +#### Improvement +* Backported in [#51240](https://github.com/ClickHouse/ClickHouse/issues/51240): Improve the progress bar for file/s3/hdfs/url table functions by using chunk size from source data and using incremental total size counting in each thread. Fix the progress bar for *Cluster functions. This closes [#47250](https://github.com/ClickHouse/ClickHouse/issues/47250). [#51088](https://github.com/ClickHouse/ClickHouse/pull/51088) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Build/Testing/Packaging Improvement +* Backported in [#51529](https://github.com/ClickHouse/ClickHouse/issues/51529): Split huge `RUN` in Dockerfile into smaller conditional. Install the necessary tools on demand in the same `RUN` layer, and remove them after that. Upgrade the OS only once at the beginning. Use a modern way to check the signed repository. Downgrade the base repo to ubuntu:20.04 to address the issues on older docker versions. Upgrade golang version to address golang vulnerabilities. [#51504](https://github.com/ClickHouse/ClickHouse/pull/51504) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix type of LDAP server params hash in cache entry [#50865](https://github.com/ClickHouse/ClickHouse/pull/50865) ([Julian Maicher](https://github.com/jmaicher)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index dd21cc7e953..307ed97068f 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -5,6 +5,7 @@ v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.6.7-lts 2023-06-28 v23.3.5.9-lts 2023-06-22 v23.3.4.17-lts 2023-06-17 v23.3.3.52-lts 2023-06-12 From 372f5786c42fd3e96bbcafb80012c04fdaa96bbc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 28 Jun 2023 18:44:54 +0200 Subject: [PATCH 412/515] more optimal REPLACE_RANGE with zero-copy --- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ecc2537f6ad..b9d48fc75f3 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2435,13 +2435,17 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) auto obtain_part = [&] (PartDescriptionPtr & part_desc) { - if (part_desc->src_table_part) + /// Fetches with zero-copy-replication are cheap, but cloneAndLoadDataPartOnSameDisk will do full copy. + /// It's okay to check the setting for current table and disk for the source table, because src and dst part are on the same disk. + bool prefer_fetch_from_other_replica = !part_desc->replica.empty() && storage_settings_ptr->allow_remote_fs_zero_copy_replication + && part_desc->src_table_part && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport(); + + if (part_desc->src_table_part && !prefer_fetch_from_other_replica) { if (part_desc->checksum_hex != part_desc->src_table_part->checksums.getTotalChecksumHex()) throw Exception(ErrorCodes::UNFINISHED, "Checksums of {} is suddenly changed", part_desc->src_table_part->name); - bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication - || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; + bool zero_copy_enabled = dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; IDataPartStorage::ClonePartParams clone_params { .copy_instead_of_hardlink = zero_copy_enabled && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport() From 56354b72514e79dfcd9a322c4cbf2aa2e81e6892 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 28 Jun 2023 16:55:22 +0000 Subject: [PATCH 413/515] Fix yet another place --- src/Dictionaries/HashedDictionary.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 798f37cb516..5f25600db8f 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -236,6 +236,11 @@ HashedDictionary::~HashedDictionary() pool.trySchedule([&container, thread_group = CurrentThread::getGroup()] { + SCOPE_EXIT_SAFE( + if (thread_group) + CurrentThread::detachFromGroupIfNotDetached(); + ); + /// Do not account memory that was occupied by the dictionaries for the query/user context. MemoryTrackerBlockerInThread memory_blocker; From 39f8b92e24ea8e695583469dfc42d2c0238e6356 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 28 Jun 2023 19:05:35 +0200 Subject: [PATCH 414/515] Allow to add disk name for custom disk --- src/Disks/getOrCreateDiskFromAST.cpp | 23 ++++++++++---- src/Parsers/ParserSetQuery.cpp | 4 +-- src/Parsers/isDiskFunction.cpp | 2 +- ...stom_disk_with_user_defined_name.reference | 3 ++ ...2808_custom_disk_with_user_defined_name.sh | 31 +++++++++++++++++++ 5 files changed, 54 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.reference create mode 100755 tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index 637acff7b95..c5ec0f5d91b 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -26,12 +26,20 @@ namespace { std::string getOrCreateDiskFromDiskAST(const ASTFunction & function, ContextPtr context) { - /// We need a unique name for a created custom disk, but it needs to be the same - /// after table is reattached or server is restarted, so take a hash of the disk - /// configuration serialized ast as a disk name suffix. - auto disk_setting_string = serializeAST(function, true); - auto disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX - + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); + std::string disk_name; + if (function.name == "disk") + { + /// We need a unique name for a created custom disk, but it needs to be the same + /// after table is reattached or server is restarted, so take a hash of the disk + /// configuration serialized ast as a disk name suffix. + auto disk_setting_string = serializeAST(function, true); + disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX + + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); + } + else + { + disk_name = function.name.substr(std::strlen("disk_")); + } auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { const auto * function_args_expr = assert_cast(function.arguments.get()); @@ -43,6 +51,9 @@ namespace return disk; }); + if (!result_disk->isCustomDisk()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk with name `{}` already exist", disk_name); + if (!result_disk->isRemote()) { static constexpr auto custom_disks_base_dir_in_config = "custom_local_disks_base_directory"; diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index 4df74c2dd82..727d037112f 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -215,7 +215,7 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p else if (ParserKeyword("FALSE").ignore(pos, expected)) value = std::make_shared(Field(static_cast(0))); /// for SETTINGS disk=disk(type='s3', path='', ...) - else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") + else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name.starts_with("disk")) { tryGetIdentifierNameInto(name, change.name); change.value = createFieldFromAST(function_ast); @@ -280,7 +280,7 @@ bool ParserSetQuery::parseNameValuePairWithParameterOrDefault( node = std::make_shared(Field(static_cast(1))); else if (ParserKeyword("FALSE").ignore(pos, expected)) node = std::make_shared(Field(static_cast(0))); - else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") + else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name.starts_with("disk")) { change.name = name; change.value = createFieldFromAST(function_ast); diff --git a/src/Parsers/isDiskFunction.cpp b/src/Parsers/isDiskFunction.cpp index e60229cb3f7..5ba626a8b2c 100644 --- a/src/Parsers/isDiskFunction.cpp +++ b/src/Parsers/isDiskFunction.cpp @@ -10,7 +10,7 @@ bool isDiskFunction(ASTPtr ast) return false; const auto * function = ast->as(); - return function && function->name == "disk" && function->arguments->as(); + return function && function->name.starts_with("disk") && function->arguments->as(); } } diff --git a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.reference b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.reference new file mode 100644 index 00000000000..713dde3527d --- /dev/null +++ b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.reference @@ -0,0 +1,3 @@ +OK +0 +1 diff --git a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh new file mode 100755 index 00000000000..99b9a0ed7b3 --- /dev/null +++ b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh @@ -0,0 +1,31 @@ +# Tags: no-fasttest, no-parallel + +# set -x + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT -nm --query """ +DROP TABLE IF EXISTS test; +CREATE TABLE test (a Int32, b String) +ENGINE = MergeTree() ORDER BY tuple() +SETTINGS disk = disk_s3disk(type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3disk); +""" 2>&1 | grep -q "Disk with name \`s3disk\` already exist" && echo 'OK' || echo 'FAIL' + +$CLICKHOUSE_CLIENT -nm --query """ +SELECT count() FROM system.disks WHERE name = '$disk_name' +""" + +disk_name="${CLICKHOUSE_TEST_UNIQUE_NAME}" +$CLICKHOUSE_CLIENT -nm --query """ +DROP TABLE IF EXISTS test; +CREATE TABLE test (a Int32, b String) +ENGINE = MergeTree() ORDER BY tuple() +SETTINGS disk = disk_$disk_name(type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3disk); +""" + +$CLICKHOUSE_CLIENT -nm --query """ +SELECT count() FROM system.disks WHERE name = '$disk_name' +""" From eea3c39959876a78639d53d4a5f84354cc53135b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 28 Jun 2023 10:49:46 +0000 Subject: [PATCH 415/515] Cosmetics --- .../functions/tuple-functions.md | 7 +- src/Common/assert_cast.h | 2 +- src/Functions/tupleElement.cpp | 140 +++++++----------- .../0_stateless/02116_tuple_element.sql | 22 +-- .../02354_tuple_element_with_default.sql | 10 +- 5 files changed, 73 insertions(+), 108 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 1739920c9f0..7ed2deaeda6 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -22,14 +22,15 @@ tuple(x, y, …) A function that allows getting a column from a tuple. -If the second argument is a number `n`, it is the column index, starting from 1. If the second argument is a string `s`, it represents the name of the element. Besides, we can provide the third optional argument, such that when index out of bounds or element for such name does not exist, the default value returned instead of throw exception. The second and third arguments if provided are always must be constant. There is no cost to execute the function. +If the second argument is a number `index`, it is the column index, starting from 1. If the second argument is a string `name`, it represents the name of the element. Besides, we can provide the third optional argument, such that when index out of bounds or no element exist for the name, the default value returned instead of throwing an exception. The second and third arguments, if provided, must be constants. There is no cost to execute the function. -The function implements the operator `x.n` and `x.s`. +The function implements operators `x.index` and `x.name`. **Syntax** ``` sql -tupleElement(tuple, n/s [, default_value]) +tupleElement(tuple, index, [, default_value]) +tupleElement(tuple, name, [, default_value]) ``` ## untuple diff --git a/src/Common/assert_cast.h b/src/Common/assert_cast.h index 604cfaed6e2..0b73ba1cc12 100644 --- a/src/Common/assert_cast.h +++ b/src/Common/assert_cast.h @@ -23,7 +23,7 @@ namespace DB * The exact match of the type is checked. That is, cast to the ancestor will be unsuccessful. */ template -To assert_cast(From && from) +inline To assert_cast(From && from) { #ifndef NDEBUG try diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index b1fd200f5cd..fb8f1d3b48d 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -34,32 +34,14 @@ class FunctionTupleElement : public IFunction { public: static constexpr auto name = "tupleElement"; - static FunctionPtr create(ContextPtr) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + String getName() const override { return name; } bool isVariadic() const override { return true; } - - size_t getNumberOfArguments() const override - { - return 0; - } - - bool useDefaultImplementationForConstants() const override - { - return true; - } - + size_t getNumberOfArguments() const override { return 0; } + bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - bool useDefaultImplementationForNulls() const override { return false; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override @@ -72,107 +54,98 @@ public: getName(), number_of_arguments); size_t count_arrays = 0; - const IDataType * tuple_col = arguments[0].type.get(); - while (const DataTypeArray * array = checkAndGetDataType(tuple_col)) + const IDataType * input_type = arguments[0].type.get(); + while (const DataTypeArray * array = checkAndGetDataType(input_type)) { - tuple_col = array->getNestedType().get(); + input_type = array->getNestedType().get(); ++count_arrays; } - const DataTypeTuple * tuple = checkAndGetDataType(tuple_col); + const DataTypeTuple * tuple = checkAndGetDataType(input_type); if (!tuple) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be tuple or array of tuple. Actual {}", getName(), arguments[0].type->getName()); - auto index = getElementNum(arguments[1].column, *tuple, number_of_arguments); + std::optional index = getElementIndex(arguments[1].column, *tuple, number_of_arguments); if (index.has_value()) { - DataTypePtr out_return_type = tuple->getElements()[index.value()]; + DataTypePtr return_type = tuple->getElements()[index.value()]; for (; count_arrays; --count_arrays) - out_return_type = std::make_shared(out_return_type); + return_type = std::make_shared(return_type); - return out_return_type; + return return_type; } else { - const IDataType * default_col = arguments[2].type.get(); - size_t default_argument_count_arrays = 0; - if (const DataTypeArray * array = checkAndGetDataType(default_col)) - { - default_argument_count_arrays = array->getNumberOfDimensions(); - } + const IDataType * default_type = arguments[2].type.get(); + size_t default_count_arrays = 0; - if (count_arrays != default_argument_count_arrays) - { + if (const DataTypeArray * default_type_as_array = checkAndGetDataType(default_type)) + default_count_arrays = default_type_as_array->getNumberOfDimensions(); + + if (count_arrays != default_count_arrays) throw Exception(ErrorCodes::NUMBER_OF_DIMENSIONS_MISMATCHED, "Dimension of types mismatched between first argument and third argument. " "Dimension of 1st argument: {}. " - "Dimension of 3rd argument: {}.",count_arrays, default_argument_count_arrays); - } + "Dimension of 3rd argument: {}", count_arrays, default_count_arrays); + return arguments[2].type; } } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - Columns array_offsets; + const auto & input_arg = arguments[0]; + const IDataType * input_type = input_arg.type.get(); + const IColumn * input_col = input_arg.column.get(); - const auto & first_arg = arguments[0]; - - const IDataType * tuple_type = first_arg.type.get(); - const IColumn * tuple_col = first_arg.column.get(); - bool first_arg_is_const = false; - if (typeid_cast(tuple_col)) + bool input_arg_is_const = false; + if (typeid_cast(input_col)) { - tuple_col = assert_cast(tuple_col)->getDataColumnPtr().get(); - first_arg_is_const = true; + input_col = assert_cast(input_col)->getDataColumnPtr().get(); + input_arg_is_const = true; } - while (const DataTypeArray * array_type = checkAndGetDataType(tuple_type)) - { - const ColumnArray * array_col = assert_cast(tuple_col); - tuple_type = array_type->getNestedType().get(); - tuple_col = &array_col->getData(); + Columns array_offsets; + while (const DataTypeArray * array_type = checkAndGetDataType(input_type)) + { + const ColumnArray * array_col = assert_cast(input_col); + + input_type = array_type->getNestedType().get(); + input_col = &array_col->getData(); array_offsets.push_back(array_col->getOffsetsPtr()); } - const DataTypeTuple * tuple_type_concrete = checkAndGetDataType(tuple_type); - const ColumnTuple * tuple_col_concrete = checkAndGetColumn(tuple_col); - if (!tuple_type_concrete || !tuple_col_concrete) + const DataTypeTuple * input_type_as_tuple = checkAndGetDataType(input_type); + const ColumnTuple * input_col_as_tuple = checkAndGetColumn(input_col); + if (!input_type_as_tuple || !input_col_as_tuple) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "First argument for function {} must be tuple or array of tuple. Actual {}", - getName(), - first_arg.type->getName()); + "First argument for function {} must be tuple or array of tuple. Actual {}", getName(), input_arg.type->getName()); - auto index = getElementNum(arguments[1].column, *tuple_type_concrete, arguments.size()); + std::optional index = getElementIndex(arguments[1].column, *input_type_as_tuple, arguments.size()); if (!index.has_value()) { if (!array_offsets.empty()) - { recursiveCheckArrayOffsets(arguments[0].column, arguments[2].column, array_offsets.size()); - } return arguments[2].column; } - ColumnPtr res = tuple_col_concrete->getColumns()[index.value()]; + ColumnPtr res = input_col_as_tuple->getColumns()[index.value()]; /// Wrap into Arrays for (auto it = array_offsets.rbegin(); it != array_offsets.rend(); ++it) res = ColumnArray::create(res, *it); - if (first_arg_is_const) - { + if (input_arg_is_const) res = ColumnConst::create(res, input_rows_count); - } return res; } private: - void recursiveCheckArrayOffsets(ColumnPtr col_x, ColumnPtr col_y, size_t depth) const { for (size_t i = 1; i < depth; ++i) @@ -187,22 +160,16 @@ private: void checkArrayOffsets(ColumnPtr col_x, ColumnPtr col_y) const { if (isColumnConst(*col_x)) - { checkArrayOffsetsWithFirstArgConst(col_x, col_y); - } else if (isColumnConst(*col_y)) - { checkArrayOffsetsWithFirstArgConst(col_y, col_x); - } else { const auto & array_x = *assert_cast(col_x.get()); const auto & array_y = *assert_cast(col_y.get()); if (!array_x.hasEqualOffsets(array_y)) - { throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "The argument 1 and argument 3 of function {} have different array sizes", getName()); - } } } @@ -220,23 +187,21 @@ private: size_t row_size = offsets_y.size(); for (size_t row = 0; row < row_size; ++row) { - if (unlikely(offsets_x[0] != offsets_y[row] - prev_offset)) - { + if (offsets_x[0] != offsets_y[row] - prev_offset) throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "The argument 1 and argument 3 of function {} have different array sizes", getName()); - } prev_offset = offsets_y[row]; } } - std::optional getElementNum(const ColumnPtr & index_column, const DataTypeTuple & tuple, const size_t argument_size) const + std::optional getElementIndex(const ColumnPtr & index_column, const DataTypeTuple & tuple, size_t argument_size) const { if (checkAndGetColumnConst(index_column.get()) || checkAndGetColumnConst(index_column.get()) || checkAndGetColumnConst(index_column.get()) || checkAndGetColumnConst(index_column.get())) { - size_t index = index_column->getUInt(0); + const size_t index = index_column->getUInt(0); if (index == 0) throw Exception(ErrorCodes::ILLEGAL_INDEX, "Indices in tuples are 1-based."); @@ -244,21 +209,20 @@ private: if (index > tuple.getElements().size()) throw Exception(ErrorCodes::ILLEGAL_INDEX, "Index for tuple element is out of range."); - return std::optional(index - 1); + return {index - 1}; } else if (const auto * name_col = checkAndGetColumnConst(index_column.get())) { - auto index = tuple.tryGetPositionByName(name_col->getValue()); - if (index.has_value()) - { - return index; - } + std::optional index = tuple.tryGetPositionByName(name_col->getValue()); - if (argument_size == 2) + if (index.has_value()) + return index; + else { - throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Tuple doesn't have element with name '{}'", name_col->getValue()); + if (argument_size == 2) + throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Tuple doesn't have element with name '{}'", name_col->getValue()); + return std::nullopt; } - return std::nullopt; } else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, diff --git a/tests/queries/0_stateless/02116_tuple_element.sql b/tests/queries/0_stateless/02116_tuple_element.sql index c911712684d..bedfedd7c2d 100644 --- a/tests/queries/0_stateless/02116_tuple_element.sql +++ b/tests/queries/0_stateless/02116_tuple_element.sql @@ -14,12 +14,12 @@ EXPLAIN SYNTAX SELECT tupleElement(t1, 2) FROM t_tuple_element; SELECT tupleElement(t1, 'a') FROM t_tuple_element; EXPLAIN SYNTAX SELECT tupleElement(t1, 'a') FROM t_tuple_element; -SELECT tupleElement(number, 1) FROM numbers(1); -- { serverError 43 } -SELECT tupleElement(t1) FROM t_tuple_element; -- { serverError 42 } -SELECT tupleElement(t1, 'b') FROM t_tuple_element; -- { serverError 10, 47 } -SELECT tupleElement(t1, 0) FROM t_tuple_element; -- { serverError 127 } -SELECT tupleElement(t1, 3) FROM t_tuple_element; -- { serverError 127 } -SELECT tupleElement(t1, materialize('a')) FROM t_tuple_element; -- { serverError 43 } +SELECT tupleElement(number, 1) FROM numbers(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT tupleElement(t1) FROM t_tuple_element; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tupleElement(t1, 'b') FROM t_tuple_element; -- { serverError NOT_FOUND_COLUMN_IN_BLOCK, UNKNOWN_IDENTIFIER } +SELECT tupleElement(t1, 0) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX } +SELECT tupleElement(t1, 3) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX } +SELECT tupleElement(t1, materialize('a')) FROM t_tuple_element; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT t2.1 FROM t_tuple_element; EXPLAIN SYNTAX SELECT t2.1 FROM t_tuple_element; @@ -27,11 +27,11 @@ EXPLAIN SYNTAX SELECT t2.1 FROM t_tuple_element; SELECT tupleElement(t2, 1) FROM t_tuple_element; EXPLAIN SYNTAX SELECT tupleElement(t2, 1) FROM t_tuple_element; -SELECT tupleElement(t2) FROM t_tuple_element; -- { serverError 42 } -SELECT tupleElement(t2, 'a') FROM t_tuple_element; -- { serverError 10, 47 } -SELECT tupleElement(t2, 0) FROM t_tuple_element; -- { serverError 127 } -SELECT tupleElement(t2, 3) FROM t_tuple_element; -- { serverError 127 } -SELECT tupleElement(t2, materialize(1)) FROM t_tuple_element; -- { serverError 43 } +SELECT tupleElement(t2) FROM t_tuple_element; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tupleElement(t2, 'a') FROM t_tuple_element; -- { serverError NOT_FOUND_COLUMN_IN_BLOCK, UNKNOWN_IDENTIFIER } +SELECT tupleElement(t2, 0) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX } +SELECT tupleElement(t2, 3) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX } +SELECT tupleElement(t2, materialize(1)) FROM t_tuple_element; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } DROP TABLE t_tuple_element; diff --git a/tests/queries/0_stateless/02354_tuple_element_with_default.sql b/tests/queries/0_stateless/02354_tuple_element_with_default.sql index 908a869885b..ba1388cfa57 100644 --- a/tests/queries/0_stateless/02354_tuple_element_with_default.sql +++ b/tests/queries/0_stateless/02354_tuple_element_with_default.sql @@ -10,16 +10,16 @@ EXPLAIN SYNTAX SELECT tupleElement(t1, 'z', 0) FROM t_tuple_element_default; SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element_default; EXPLAIN SYNTAX SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element_default; -SELECT tupleElement(t1, 3, 'z') FROM t_tuple_element_default; -- { serverError 127 } -SELECT tupleElement(t1, 0, 'z') FROM t_tuple_element_default; -- { serverError 127 } +SELECT tupleElement(t1, 3, 'z') FROM t_tuple_element_default; -- { serverError ILLEGAL_INDEX } +SELECT tupleElement(t1, 0, 'z') FROM t_tuple_element_default; -- { serverError ILLEGAL_INDEX } DROP TABLE t_tuple_element_default; SELECT '--------------------'; -SELECT tupleElement(array(tuple(1, 2)), 'a', 0); -- { serverError 645 } -SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(1, 2), tuple(3, 4))); -- { serverError 190 } -SELECT tupleElement(array(array(tuple(1))), 'a', array(array(1, 2, 3))); -- { serverError 190 } +SELECT tupleElement(array(tuple(1, 2)), 'a', 0); -- { serverError NUMBER_OF_DIMENSIONS_MISMATCHED } +SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(1, 2), tuple(3, 4))); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } +SELECT tupleElement(array(array(tuple(1))), 'a', array(array(1, 2, 3))); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(3, 4))); EXPLAIN SYNTAX SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(3, 4))); From bf54fb4caad774bb1efe1fd39e7b7abaef1a9869 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 28 Jun 2023 11:06:41 +0000 Subject: [PATCH 416/515] tupleElement(): Return default value for out-of-bounds-index Makes the actual and the documented behavior consistent. --- src/Functions/tupleElement.cpp | 14 ++++++++------ .../02354_tuple_element_with_default.reference | 2 ++ .../02354_tuple_element_with_default.sql | 4 ++-- 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index fb8f1d3b48d..8689a095809 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -203,13 +203,15 @@ private: { const size_t index = index_column->getUInt(0); - if (index == 0) - throw Exception(ErrorCodes::ILLEGAL_INDEX, "Indices in tuples are 1-based."); + if (index > 0 && index <= tuple.getElements().size()) + return {index - 1}; + else + { + if (argument_size == 2) + throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Tuple doesn't have element with index '{}'", index); + return std::nullopt; + } - if (index > tuple.getElements().size()) - throw Exception(ErrorCodes::ILLEGAL_INDEX, "Index for tuple element is out of range."); - - return {index - 1}; } else if (const auto * name_col = checkAndGetColumnConst(index_column.get())) { diff --git a/tests/queries/0_stateless/02354_tuple_element_with_default.reference b/tests/queries/0_stateless/02354_tuple_element_with_default.reference index d5dfff17ef1..08a1f60e163 100644 --- a/tests/queries/0_stateless/02354_tuple_element_with_default.reference +++ b/tests/queries/0_stateless/02354_tuple_element_with_default.reference @@ -7,6 +7,8 @@ FROM t_tuple_element_default z SELECT tupleElement(t2, \'z\', \'z\') FROM t_tuple_element_default +z +z -------------------- [(3,4)] SELECT tupleElement([(1, 2)], \'a\', [(3, 4)]) diff --git a/tests/queries/0_stateless/02354_tuple_element_with_default.sql b/tests/queries/0_stateless/02354_tuple_element_with_default.sql index ba1388cfa57..de281c0a868 100644 --- a/tests/queries/0_stateless/02354_tuple_element_with_default.sql +++ b/tests/queries/0_stateless/02354_tuple_element_with_default.sql @@ -10,8 +10,8 @@ EXPLAIN SYNTAX SELECT tupleElement(t1, 'z', 0) FROM t_tuple_element_default; SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element_default; EXPLAIN SYNTAX SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element_default; -SELECT tupleElement(t1, 3, 'z') FROM t_tuple_element_default; -- { serverError ILLEGAL_INDEX } -SELECT tupleElement(t1, 0, 'z') FROM t_tuple_element_default; -- { serverError ILLEGAL_INDEX } +SELECT tupleElement(t1, 3, 'z') FROM t_tuple_element_default; +SELECT tupleElement(t1, 0, 'z') FROM t_tuple_element_default; DROP TABLE t_tuple_element_default; From 5b78b3903b468575dc36af84bbb90f673d50a7c0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 28 Jun 2023 12:26:34 +0000 Subject: [PATCH 417/515] Fix logical error in tupleElement() --- src/Functions/tupleElement.cpp | 69 ------------------- .../02286_tuple_numeric_identifier.sql | 4 +- ...02354_tuple_element_with_default.reference | 43 ++++-------- .../02354_tuple_element_with_default.sql | 61 +++++----------- 4 files changed, 34 insertions(+), 143 deletions(-) diff --git a/src/Functions/tupleElement.cpp b/src/Functions/tupleElement.cpp index 8689a095809..96b5a047419 100644 --- a/src/Functions/tupleElement.cpp +++ b/src/Functions/tupleElement.cpp @@ -17,11 +17,8 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ILLEGAL_INDEX; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NOT_FOUND_COLUMN_IN_BLOCK; - extern const int NUMBER_OF_DIMENSIONS_MISMATCHED; - extern const int SIZES_OF_ARRAYS_DONT_MATCH; } namespace @@ -79,21 +76,7 @@ public: return return_type; } else - { - const IDataType * default_type = arguments[2].type.get(); - size_t default_count_arrays = 0; - - if (const DataTypeArray * default_type_as_array = checkAndGetDataType(default_type)) - default_count_arrays = default_type_as_array->getNumberOfDimensions(); - - if (count_arrays != default_count_arrays) - throw Exception(ErrorCodes::NUMBER_OF_DIMENSIONS_MISMATCHED, - "Dimension of types mismatched between first argument and third argument. " - "Dimension of 1st argument: {}. " - "Dimension of 3rd argument: {}", count_arrays, default_count_arrays); - return arguments[2].type; - } } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override @@ -128,11 +111,7 @@ public: std::optional index = getElementIndex(arguments[1].column, *input_type_as_tuple, arguments.size()); if (!index.has_value()) - { - if (!array_offsets.empty()) - recursiveCheckArrayOffsets(arguments[0].column, arguments[2].column, array_offsets.size()); return arguments[2].column; - } ColumnPtr res = input_col_as_tuple->getColumns()[index.value()]; @@ -146,54 +125,6 @@ public: } private: - void recursiveCheckArrayOffsets(ColumnPtr col_x, ColumnPtr col_y, size_t depth) const - { - for (size_t i = 1; i < depth; ++i) - { - checkArrayOffsets(col_x, col_y); - col_x = assert_cast(col_x.get())->getDataPtr(); - col_y = assert_cast(col_y.get())->getDataPtr(); - } - checkArrayOffsets(col_x, col_y); - } - - void checkArrayOffsets(ColumnPtr col_x, ColumnPtr col_y) const - { - if (isColumnConst(*col_x)) - checkArrayOffsetsWithFirstArgConst(col_x, col_y); - else if (isColumnConst(*col_y)) - checkArrayOffsetsWithFirstArgConst(col_y, col_x); - else - { - const auto & array_x = *assert_cast(col_x.get()); - const auto & array_y = *assert_cast(col_y.get()); - if (!array_x.hasEqualOffsets(array_y)) - throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, - "The argument 1 and argument 3 of function {} have different array sizes", getName()); - } - } - - void checkArrayOffsetsWithFirstArgConst(ColumnPtr col_x, ColumnPtr col_y) const - { - col_x = assert_cast(col_x.get())->getDataColumnPtr(); - col_y = col_y->convertToFullColumnIfConst(); - const auto & array_x = *assert_cast(col_x.get()); - const auto & array_y = *assert_cast(col_y.get()); - - const auto & offsets_x = array_x.getOffsets(); - const auto & offsets_y = array_y.getOffsets(); - - ColumnArray::Offset prev_offset = 0; - size_t row_size = offsets_y.size(); - for (size_t row = 0; row < row_size; ++row) - { - if (offsets_x[0] != offsets_y[row] - prev_offset) - throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, - "The argument 1 and argument 3 of function {} have different array sizes", getName()); - prev_offset = offsets_y[row]; - } - } - std::optional getElementIndex(const ColumnPtr & index_column, const DataTypeTuple & tuple, size_t argument_size) const { if (checkAndGetColumnConst(index_column.get()) diff --git a/tests/queries/0_stateless/02286_tuple_numeric_identifier.sql b/tests/queries/0_stateless/02286_tuple_numeric_identifier.sql index a5fd8e57ad5..f723284ad61 100644 --- a/tests/queries/0_stateless/02286_tuple_numeric_identifier.sql +++ b/tests/queries/0_stateless/02286_tuple_numeric_identifier.sql @@ -12,8 +12,8 @@ SELECT * FROM t_tuple_numeric FORMAT JSONEachRow; SELECT `t`.`1`.`2`, `t`.`1`.`3`, `t`.`4` FROM t_tuple_numeric; SELECT t.1.1, t.1.2, t.2 FROM t_tuple_numeric; -SELECT t.1.3 FROM t_tuple_numeric; -- {serverError ILLEGAL_INDEX} -SELECT t.4 FROM t_tuple_numeric; -- {serverError ILLEGAL_INDEX} +SELECT t.1.3 FROM t_tuple_numeric; -- {serverError NOT_FOUND_COLUMN_IN_BLOCK} +SELECT t.4 FROM t_tuple_numeric; -- {serverError NOT_FOUND_COLUMN_IN_BLOCK} SELECT `t`.`1`.`1`, `t`.`1`.`2`, `t`.`2` FROM t_tuple_numeric; -- {serverError UNKNOWN_IDENTIFIER} DROP TABLE t_tuple_numeric; diff --git a/tests/queries/0_stateless/02354_tuple_element_with_default.reference b/tests/queries/0_stateless/02354_tuple_element_with_default.reference index 08a1f60e163..499b4c36a10 100644 --- a/tests/queries/0_stateless/02354_tuple_element_with_default.reference +++ b/tests/queries/0_stateless/02354_tuple_element_with_default.reference @@ -1,28 +1,15 @@ -z -SELECT tupleElement(t1, \'z\', \'z\') -FROM t_tuple_element_default -0 -SELECT tupleElement(t1, \'z\', 0) -FROM t_tuple_element_default -z -SELECT tupleElement(t2, \'z\', \'z\') -FROM t_tuple_element_default -z -z --------------------- -[(3,4)] -SELECT tupleElement([(1, 2)], \'a\', [(3, 4)]) --------------------- -SELECT tupleElement(t1, \'a\', [tuple(1)]) -FROM t_tuple_element_default --------------------- -[(0)] -SELECT tupleElement(t1, \'a\', [tuple(0)]) -FROM t_tuple_element_default -[0] -SELECT tupleElement(t1, \'a\', [0]) -FROM t_tuple_element_default -[0] -[0] -SELECT tupleElement(t1, \'a\', [0]) -FROM t_tuple_element_default +hello +world +default +default +[(['a'],1)] +[1,3] +[2,4] +default +-------- +hello +world +default +default +[(['a'],1)] +[[1,2,3]] diff --git a/tests/queries/0_stateless/02354_tuple_element_with_default.sql b/tests/queries/0_stateless/02354_tuple_element_with_default.sql index de281c0a868..89320f4d210 100644 --- a/tests/queries/0_stateless/02354_tuple_element_with_default.sql +++ b/tests/queries/0_stateless/02354_tuple_element_with_default.sql @@ -1,50 +1,23 @@ -DROP TABLE IF EXISTS t_tuple_element_default; +-- const tuple argument -CREATE TABLE t_tuple_element_default(t1 Tuple(a UInt32, s String), t2 Tuple(UInt32, String)) ENGINE = Memory; -INSERT INTO t_tuple_element_default VALUES ((1, 'a'), (2, 'b')); +SELECT tupleElement(('hello', 'world'), 1, 'default'); +SELECT tupleElement(('hello', 'world'), 2, 'default'); +SELECT tupleElement(('hello', 'world'), 3, 'default'); +SELECT tupleElement(('hello', 'world'), 'xyz', 'default'); +SELECT tupleElement(('hello', 'world'), 3, [([('a')], 1)]); -- arbitrary default value -SELECT tupleElement(t1, 'z', 'z') FROM t_tuple_element_default; -EXPLAIN SYNTAX SELECT tupleElement(t1, 'z', 'z') FROM t_tuple_element_default; -SELECT tupleElement(t1, 'z', 0) FROM t_tuple_element_default; -EXPLAIN SYNTAX SELECT tupleElement(t1, 'z', 0) FROM t_tuple_element_default; -SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element_default; -EXPLAIN SYNTAX SELECT tupleElement(t2, 'z', 'z') FROM t_tuple_element_default; +SELECT tupleElement([(1, 2), (3, 4)], 1, 'default'); +SELECT tupleElement([(1, 2), (3, 4)], 2, 'default'); +SELECT tupleElement([(1, 2), (3, 4)], 3, 'default'); -SELECT tupleElement(t1, 3, 'z') FROM t_tuple_element_default; -SELECT tupleElement(t1, 0, 'z') FROM t_tuple_element_default; +SELECT '--------'; -DROP TABLE t_tuple_element_default; +-- non-const tuple argument -SELECT '--------------------'; - -SELECT tupleElement(array(tuple(1, 2)), 'a', 0); -- { serverError NUMBER_OF_DIMENSIONS_MISMATCHED } -SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(1, 2), tuple(3, 4))); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } -SELECT tupleElement(array(array(tuple(1))), 'a', array(array(1, 2, 3))); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } - -SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(3, 4))); -EXPLAIN SYNTAX SELECT tupleElement(array(tuple(1, 2)), 'a', array(tuple(3, 4))); - -SELECT '--------------------'; - -CREATE TABLE t_tuple_element_default(t1 Array(Tuple(UInt32)), t2 UInt32) ENGINE = Memory; - -SELECT tupleElement(t1, 'a', array(tuple(1))) FROM t_tuple_element_default; -EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(tuple(1))) FROM t_tuple_element_default; - -SELECT '--------------------'; - -INSERT INTO t_tuple_element_default VALUES ([(1)], 100); - -SELECT tupleElement(t1, 'a', array(tuple(0))) FROM t_tuple_element_default; -EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(tuple(0))) FROM t_tuple_element_default; - -SELECT tupleElement(t1, 'a', array(0)) FROM t_tuple_element_default; -EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(0)) FROM t_tuple_element_default; - -INSERT INTO t_tuple_element_default VALUES ([(2)], 200); - -SELECT tupleElement(t1, 'a', array(0)) FROM t_tuple_element_default; -EXPLAIN SYNTAX SELECT tupleElement(t1, 'a', array(0)) FROM t_tuple_element_default; - -DROP TABLE t_tuple_element_default; +SELECT tupleElement(materialize(('hello', 'world')), 1, 'default'); +SELECT tupleElement(materialize(('hello', 'world')), 2, 'default'); +SELECT tupleElement(materialize(('hello', 'world')), 3, 'default'); +SELECT tupleElement(materialize(('hello', 'world')), 'xzy', 'default'); +SELECT tupleElement(materialize(('hello', 'world')), 'xzy', [([('a')], 1)]); -- arbitrary default value +SELECT tupleElement([[(count('2147483646'), 1)]], 'aaaa', [[1, 2, 3]]) -- bug #51525 From 8854f05a7534fc844b5a7b00af72ad0ae7c0ac1b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 28 Jun 2023 19:53:49 +0000 Subject: [PATCH 418/515] style fix --- src/Common/ThreadPool.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 979e53a72c0..cc88594d84f 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -420,7 +420,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ { if (DB::Exception::enable_job_stack_trace) DB::Exception::thread_frame_pointers = std::move(thread_frame_pointers); - + CurrentMetrics::Increment metric_active_pool_threads(metric_active_threads); From 45cd7f35cdfb3c1ed17cd2451468761aa25a6bfb Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 22 Jun 2023 02:16:56 +0200 Subject: [PATCH 419/515] Use clickhouse/integration-helper from changed images --- docker/test/integration/runner/dockerd-entrypoint.sh | 2 ++ tests/integration/helpers/network.py | 11 +++++++---- tests/integration/runner | 2 ++ 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index fe47fc90951..347d904d5c0 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -52,6 +52,8 @@ export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-config export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge export CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH=/clickhouse-library-bridge +export DOCKER_BASE_TAG=${DOCKER_BASE_TAG:=latest} +export DOCKER_HELPER_TAG=${DOCKER_HELPER_TAG:=latest} export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} export DOCKER_DOTNET_CLIENT_TAG=${DOCKER_DOTNET_CLIENT_TAG:=latest} export DOCKER_MYSQL_JAVA_CLIENT_TAG=${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest} diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index 471aa2bdc2e..4859a8c5946 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -231,6 +231,9 @@ class _NetworkManager: def _ensure_container(self): if self._container is None or self._container_expire_time <= time.time(): + image_name = "clickhouse/integration-helper:" + os.getenv( + "DOCKER_HELPER_TAG", "latest" + ) for i in range(5): if self._container is not None: try: @@ -247,7 +250,7 @@ class _NetworkManager: time.sleep(i) image = subprocess.check_output( - "docker images -q clickhouse/integration-helper 2>/dev/null", shell=True + f"docker images -q {image_name} 2>/dev/null", shell=True ) if not image.strip(): print("No network image helper, will try download") @@ -256,16 +259,16 @@ class _NetworkManager: for i in range(5): try: subprocess.check_call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL - "docker pull clickhouse/integration-helper", shell=True + f"docker pull {image_name}", shell=True ) break except: time.sleep(i) else: - raise Exception("Cannot pull clickhouse/integration-helper image") + raise Exception(f"Cannot pull {image_name} image") self._container = self._docker_client.containers.run( - "clickhouse/integration-helper", + image_name, auto_remove=True, command=("sleep %s" % self.container_exit_timeout), # /run/xtables.lock passed inside for correct iptables --wait diff --git a/tests/integration/runner b/tests/integration/runner index f658bac412b..301a707a78d 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -336,6 +336,8 @@ if __name__ == "__main__": env_tags += "-e {}={} ".format("DOCKER_MYSQL_PHP_CLIENT_TAG", tag) elif image == "clickhouse/postgresql-java-client": env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) + elif image == "clickhouse/integration-helper": + env_tags += "-e {}={} ".format("DOCKER_HELPER_TAG", tag) elif image == "clickhouse/integration-test": env_tags += "-e {}={} ".format("DOCKER_BASE_TAG", tag) elif image == "clickhouse/kerberized-hadoop": From eeb8cdbc19e69aab64b0da2de0b569d6e31f438f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 22 Jun 2023 02:36:30 +0200 Subject: [PATCH 420/515] Add way to define additional urls in test reports --- tests/ci/upload_result_helper.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/ci/upload_result_helper.py b/tests/ci/upload_result_helper.py index 150af7aff4a..fbb89ef8078 100644 --- a/tests/ci/upload_result_helper.py +++ b/tests/ci/upload_result_helper.py @@ -1,5 +1,5 @@ from pathlib import Path -from typing import Dict, List +from typing import Dict, List, Optional import os import logging @@ -58,14 +58,19 @@ def upload_results( test_results: TestResults, additional_files: List[str], check_name: str, + additional_urls: Optional[List[str]] = None, ) -> str: normalized_check_name = check_name.lower() for r in ((" ", "_"), ("(", "_"), (")", "_"), (",", "_"), ("/", "_")): normalized_check_name = normalized_check_name.replace(*r) + + # Preserve additional_urls to not modify the original one + original_additional_urls = additional_urls or [] s3_path_prefix = f"{pr_number}/{commit_sha}/{normalized_check_name}" additional_urls = process_logs( s3_client, additional_files, s3_path_prefix, test_results ) + additional_urls.extend(original_additional_urls) branch_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/commits/master" branch_name = "master" From 7b4e6faece3b172763f3237bf8a42282a289f059 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 19 Jun 2023 18:32:31 +0200 Subject: [PATCH 421/515] Speedup the submodules cloning --- docker/test/fasttest/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index dab873377ce..be9d569e65c 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -151,7 +151,7 @@ function clone_submodules ) git submodule sync - git submodule update --jobs=16 --depth 1 --init "${SUBMODULES_TO_UPDATE[@]}" + git submodule update --jobs=16 --depth 1 --single-branch --init "${SUBMODULES_TO_UPDATE[@]}" git submodule foreach git reset --hard git submodule foreach git checkout @ -f git submodule foreach git clean -xfd From b68d8fa76e7be0ef1ae58ce41c6a326027c3afe6 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 19 Jun 2023 18:49:54 +0200 Subject: [PATCH 422/515] Update the OS version for builder and testers docker images --- docker/test/fasttest/Dockerfile | 1 + docker/test/integration/base/Dockerfile | 2 +- docker/test/stateful/Dockerfile | 5 +++-- docker/test/stateless/Dockerfile | 7 ++++--- docker/test/util/Dockerfile | 2 +- 5 files changed, 10 insertions(+), 7 deletions(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index ffb13fc774d..da4baa8c687 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -9,6 +9,7 @@ RUN apt-get update \ expect \ file \ lsof \ + odbcinst \ psmisc \ python3 \ python3-lxml \ diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index de8efa20af4..ff50626b6aa 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -49,7 +49,7 @@ RUN arch=${TARGETARCH:-amd64} \ && curl -o mysql-odbc.rpm "https://cdn.mysql.com/archives/mysql-connector-odbc-8.0/mysql-connector-odbc-8.0.27-1.el8.${rarch}.rpm" \ && rpm2archive mysql-odbc.rpm \ && tar xf mysql-odbc.rpm.tgz -C / ./usr/lib64/ \ - && LINK_DIR=$(dpkg -L libodbc1 | rg '^/usr/lib/.*-linux-gnu/odbc$') \ + && LINK_DIR=$(dpkg -L odbc-postgresql | rg '^/usr/lib/.*-linux-gnu/odbc$') \ && ln -s /usr/lib64/libmyodbc8a.so "$LINK_DIR" \ && ln -s /usr/lib64/libmyodbc8a.so "$LINK_DIR"/libmyodbc.so diff --git a/docker/test/stateful/Dockerfile b/docker/test/stateful/Dockerfile index 71a2e92e3a8..f513735a2d0 100644 --- a/docker/test/stateful/Dockerfile +++ b/docker/test/stateful/Dockerfile @@ -16,8 +16,9 @@ COPY s3downloader /s3downloader ENV S3_URL="https://clickhouse-datasets.s3.amazonaws.com" ENV DATASETS="hits visits" -RUN npm install -g azurite -RUN npm install tslib +# The following is already done in clickhouse/stateless-test +# RUN npm install -g azurite +# RUN npm install tslib COPY run.sh / CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 40109255a7e..32996140521 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -20,6 +20,7 @@ RUN apt-get update -y \ netcat-openbsd \ nodejs \ npm \ + odbcinst \ openjdk-11-jre-headless \ openssl \ postgresql-client \ @@ -71,7 +72,7 @@ RUN arch=${TARGETARCH:-amd64} \ && chmod +x ./mc ./minio -RUN wget 'https://dlcdn.apache.org/hadoop/common/hadoop-3.3.1/hadoop-3.3.1.tar.gz' \ +RUN wget --no-verbose 'https://dlcdn.apache.org/hadoop/common/hadoop-3.3.1/hadoop-3.3.1.tar.gz' \ && tar -xvf hadoop-3.3.1.tar.gz \ && rm -rf hadoop-3.3.1.tar.gz @@ -79,8 +80,8 @@ ENV MINIO_ROOT_USER="clickhouse" ENV MINIO_ROOT_PASSWORD="clickhouse" ENV EXPORT_S3_STORAGE_POLICIES=1 -RUN npm install -g azurite -RUN npm install tslib +RUN npm install -g azurite \ + && npm install -g tslib COPY run.sh / COPY setup_minio.sh / diff --git a/docker/test/util/Dockerfile b/docker/test/util/Dockerfile index a49278e960b..85e888f1df7 100644 --- a/docker/test/util/Dockerfile +++ b/docker/test/util/Dockerfile @@ -1,5 +1,5 @@ # docker build -t clickhouse/test-util . -FROM ubuntu:20.04 +FROM ubuntu:22.04 # ARG for quick switch to a given ubuntu mirror ARG apt_archive="http://archive.ubuntu.com" From 5a8ce1f2fe5a7048066e9883ec49828e27ab7bd8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 19 Jun 2023 22:26:27 +0200 Subject: [PATCH 423/515] Do not upload the binaries for fast-tests to tests-reports --- docker/test/fasttest/run.sh | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index be9d569e65c..989ed9d2fbb 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -80,7 +80,7 @@ function start_server function clone_root { - git config --global --add safe.directory "$FASTTEST_SOURCE" + [ "$UID" -eq 0 ] && git config --global --add safe.directory "$FASTTEST_SOURCE" git clone --depth 1 https://github.com/ClickHouse/ClickHouse.git -- "$FASTTEST_SOURCE" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/clone_log.txt" ( @@ -202,10 +202,11 @@ function build | ts '%Y-%m-%d %H:%M:%S' \ | tee "$FASTTEST_OUTPUT/test_result.txt" if [ "$COPY_CLICKHOUSE_BINARY_TO_OUTPUT" -eq "1" ]; then - cp programs/clickhouse "$FASTTEST_OUTPUT/clickhouse" + mkdir -p "$FASTTEST_OUTPUT/binaries/" + cp programs/clickhouse "$FASTTEST_OUTPUT/binaries/clickhouse" - strip programs/clickhouse -o "$FASTTEST_OUTPUT/clickhouse-stripped" - zstd --threads=0 "$FASTTEST_OUTPUT/clickhouse-stripped" + strip programs/clickhouse -o programs/clickhouse-stripped + zstd --threads=0 programs/clickhouse-stripped -o "$FASTTEST_OUTPUT/binaries/clickhouse-stripped.zst" fi ccache_status ccache --evict-older-than 1d ||: From edcf981c4837468bf4d08dd86183b38c120913c2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 20 Jun 2023 13:11:11 +0200 Subject: [PATCH 424/515] Update mysql odbc connector --- docker/test/integration/base/Dockerfile | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index ff50626b6aa..270b40e23a6 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -46,12 +46,13 @@ RUN arch=${TARGETARCH:-amd64} \ arm64) rarch=aarch64 ;; \ esac \ && cd /tmp \ - && curl -o mysql-odbc.rpm "https://cdn.mysql.com/archives/mysql-connector-odbc-8.0/mysql-connector-odbc-8.0.27-1.el8.${rarch}.rpm" \ + && curl -o mysql-odbc.rpm "https://cdn.mysql.com/archives/mysql-connector-odbc-8.0/mysql-connector-odbc-8.0.32-1.el9.${rarch}.rpm" \ && rpm2archive mysql-odbc.rpm \ && tar xf mysql-odbc.rpm.tgz -C / ./usr/lib64/ \ - && LINK_DIR=$(dpkg -L odbc-postgresql | rg '^/usr/lib/.*-linux-gnu/odbc$') \ - && ln -s /usr/lib64/libmyodbc8a.so "$LINK_DIR" \ - && ln -s /usr/lib64/libmyodbc8a.so "$LINK_DIR"/libmyodbc.so + && rm mysql-odbc.rpm mysql-odbc.rpm.tgz \ + && ODBC_DIR=$(dpkg -L odbc-postgresql | rg '^/usr/lib/.*-linux-gnu/odbc$') \ + && ln -s /usr/lib64/libmyodbc8a.so "$ODBC_DIR" \ + && ln -s /usr/lib64/libmyodbc8a.so "$ODBC_DIR"/libmyodbc.so # Unfortunately this is required for a single test for conversion data from zookeeper to clickhouse-keeper. # ZooKeeper is not started by default, but consumes some space in containers. From ead81879b43d8b86f9b73f97fc0ac4277a79f047 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 21 Jun 2023 16:33:43 +0200 Subject: [PATCH 425/515] Update integration tests runner too --- .../integration/helper_container/Dockerfile | 5 +- docker/test/integration/runner/Dockerfile | 50 ++++++++++--------- tests/integration/helpers/cluster.py | 2 - tests/integration/helpers/network.py | 4 -- tests/integration/pytest.ini | 3 ++ tests/integration/runner | 46 +++++++---------- tests/integration/test_storage_hudi/test.py | 2 +- 7 files changed, 51 insertions(+), 61 deletions(-) diff --git a/docker/test/integration/helper_container/Dockerfile b/docker/test/integration/helper_container/Dockerfile index 6a093081bf2..60adaea1796 100644 --- a/docker/test/integration/helper_container/Dockerfile +++ b/docker/test/integration/helper_container/Dockerfile @@ -2,4 +2,7 @@ # Helper docker container to run iptables without sudo FROM alpine -RUN apk add -U iproute2 +RUN apk add --no-cache -U iproute2 \ + && for bin in iptables iptables-restore iptables-save; \ + do ln -sf xtables-nft-multi "/sbin/$bin"; \ + done diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 14c97e479f6..40627354f70 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -1,5 +1,5 @@ # docker build -t clickhouse/integration-tests-runner . -FROM ubuntu:20.04 +FROM ubuntu:22.04 # ARG for quick switch to a given ubuntu mirror ARG apt_archive="http://archive.ubuntu.com" @@ -56,17 +56,19 @@ RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add - \ /var/lib/apt/lists/* \ /var/cache/debconf \ /tmp/* \ - && apt-get clean + && apt-get clean \ + && dockerd --version; docker --version -RUN dockerd --version; docker --version RUN python3 -m pip install --no-cache-dir \ PyMySQL \ - aerospike==4.0.0 \ - avro==1.10.2 \ + aerospike==11.1.0 \ asyncio \ + avro==1.10.2 \ + azure-storage-blob \ cassandra-driver \ - confluent-kafka==1.5.0 \ + confluent-kafka==1.9.2 \ + delta-spark==2.3.0 \ dict2xml \ dicttoxml \ docker \ @@ -76,47 +78,47 @@ RUN python3 -m pip install --no-cache-dir \ kafka-python \ kazoo \ lz4 \ + meilisearch==0.18.3 \ minio \ nats-py \ protobuf \ - psycopg2-binary==2.8.6 \ + psycopg2-binary==2.9.6 \ + pyhdfs \ pymongo==3.11.0 \ + pyspark==3.3.2 \ pytest \ pytest-order==1.0.0 \ - pytest-timeout \ pytest-random \ - pytest-xdist \ pytest-repeat \ + pytest-timeout \ + pytest-xdist \ pytz \ redis \ - tzlocal==2.1 \ - urllib3 \ requests-kerberos \ - pyspark==3.3.2 \ - delta-spark==2.2.0 \ - pyhdfs \ - azure-storage-blob \ - meilisearch==0.18.3 - -COPY modprobe.sh /usr/local/bin/modprobe -COPY dockerd-entrypoint.sh /usr/local/bin/ -COPY compose/ /compose/ -COPY misc/ /misc/ + tzlocal==2.1 \ + urllib3 +# Hudi supports only spark 3.3.*, not 3.4 RUN curl -fsSL -O https://dlcdn.apache.org/spark/spark-3.3.2/spark-3.3.2-bin-hadoop3.tgz \ && tar xzvf spark-3.3.2-bin-hadoop3.tgz -C / \ && rm spark-3.3.2-bin-hadoop3.tgz # download spark and packages # if you change packages, don't forget to update them in tests/integration/helpers/cluster.py -RUN echo ":quit" | /spark-3.3.2-bin-hadoop3/bin/spark-shell --packages "org.apache.hudi:hudi-spark3.3-bundle_2.12:0.13.0,io.delta:delta-core_2.12:2.2.0,org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.1.0" > /dev/null +RUN echo ":quit" | /spark-3.3.2-bin-hadoop3/bin/spark-shell --packages "org.apache.hudi:hudi-spark3.3-bundle_2.12:0.13.0,io.delta:delta-core_2.12:2.3.0,org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.1.0" > /dev/null RUN set -x \ && addgroup --system dockremap \ - && adduser --system dockremap \ + && adduser --system dockremap \ && adduser dockremap dockremap \ && echo 'dockremap:165536:65536' >> /etc/subuid \ - && echo 'dockremap:165536:65536' >> /etc/subgid + && echo 'dockremap:165536:65536' >> /etc/subgid + +COPY modprobe.sh /usr/local/bin/modprobe +COPY dockerd-entrypoint.sh /usr/local/bin/ +COPY compose/ /compose/ +COPY misc/ /misc/ + # Same options as in test/base/Dockerfile # (in case you need to override them in tests) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c77e67062a1..967eaaa78a5 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3054,7 +3054,6 @@ CLICKHOUSE_STAY_ALIVE_COMMAND = "bash -c \"trap 'pkill tail' INT TERM; {} --daem CLICKHOUSE_START_COMMAND ) -# /run/xtables.lock passed inside for correct iptables --wait DOCKER_COMPOSE_TEMPLATE = """ version: '2.3' services: @@ -3066,7 +3065,6 @@ services: - {db_dir}:/var/lib/clickhouse/ - {logs_dir}:/var/log/clickhouse-server/ - /etc/passwd:/etc/passwd:ro - - /run/xtables.lock:/run/xtables.lock:ro {binary_volume} {odbc_bridge_volume} {library_bridge_volume} diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index 4859a8c5946..2df560708e0 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -271,10 +271,6 @@ class _NetworkManager: image_name, auto_remove=True, command=("sleep %s" % self.container_exit_timeout), - # /run/xtables.lock passed inside for correct iptables --wait - volumes={ - "/run/xtables.lock": {"bind": "/run/xtables.lock", "mode": "ro"} - }, detach=True, network_mode="host", ) diff --git a/tests/integration/pytest.ini b/tests/integration/pytest.ini index 772c96f7361..e40959bd37b 100644 --- a/tests/integration/pytest.ini +++ b/tests/integration/pytest.ini @@ -19,3 +19,6 @@ markers = long_run: marks tests which run for a long time addopts = -m 'not long_run' +; 'The asyncore module is deprecated' comes from casandra driver +filterwarnings = + ignore:The asyncore module is deprecated:DeprecationWarning diff --git a/tests/integration/runner b/tests/integration/runner index 301a707a78d..df52f587eee 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -395,39 +395,27 @@ if __name__ == "__main__": if args.keyword_expression: args.pytest_args += ["-k", args.keyword_expression] - cmd_base = "docker run {net} {tty} --rm --name {name} --privileged \ - --volume={odbc_bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ - --volume={library_bridge_bin}:/clickhouse-library-bridge \ - --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ - --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ - --volume=/run:/run/host:ro \ - {dockerd_internal_volume} -e DOCKER_CLIENT_TIMEOUT=300 -e COMPOSE_HTTP_TIMEOUT=600 \ - -e XTABLES_LOCKFILE=/run/host/xtables.lock \ - -e PYTHONUNBUFFERED=1 \ - {env_tags} {env_cleanup} -e PYTEST_OPTS='{parallel} {opts} {tests_list} {rand} -vvv' {img}".format( - net=net, - tty=tty, - bin=args.binary, - odbc_bridge_bin=args.odbc_bridge_binary, - library_bridge_bin=args.library_bridge_binary, - base_cfg=args.base_configs_dir, - cases_dir=args.cases_dir, - src_dir=args.src_dir, - env_tags=env_tags, - env_cleanup=env_cleanup, - parallel=parallel_args, - rand=rand_args, - opts=" ".join(args.pytest_args).replace("'", "\\'"), - tests_list=" ".join(args.tests_list), - dockerd_internal_volume=dockerd_internal_volume, - img=DIND_INTEGRATION_TESTS_IMAGE_NAME + ":" + args.docker_image_version, - name=CONTAINER_NAME, + pytest_opts = " ".join(args.pytest_args).replace("'", "\\'") + tests_list = " ".join(args.tests_list) + cmd_base = ( + f"docker run {net} {tty} --rm --name {CONTAINER_NAME} " + "--privileged --dns-search='.' " # since recent dns search leaks from host + f"--volume={args.odbc_bridge_binary}:/clickhouse-odbc-bridge " + f"--volume={args.binary}:/clickhouse " + f"--volume={args.library_bridge_binary}:/clickhouse-library-bridge " + f"--volume={args.base_configs_dir}:/clickhouse-config " + f"--volume={args.cases_dir}:/ClickHouse/tests/integration " + f"--volume={args.src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos " + f"--volume=/run:/run/host:ro {dockerd_internal_volume} {env_tags} {env_cleanup} " + "-e DOCKER_CLIENT_TIMEOUT=300 -e COMPOSE_HTTP_TIMEOUT=600 -e PYTHONUNBUFFERED=1 " + f"-e PYTEST_OPTS='{parallel_args} {pytest_opts} {tests_list} {rand_args} -vvv'" + f" {DIND_INTEGRATION_TESTS_IMAGE_NAME}:{args.docker_image_version}" ) cmd = cmd_base + " " + args.command cmd_pre_pull = ( - cmd_base - + " find /compose -name docker_compose_*.yml -exec docker-compose -f '{}' pull \;" + f"{cmd_base} find /compose -name docker_compose_*.yml " + r"-exec docker-compose -f '{}' pull \;" ) containers = subprocess.check_output( diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index de9cde43609..2b77f4d6d61 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -79,7 +79,7 @@ def run_query(instance, query, stdin=None, settings=None): def write_hudi_from_df(spark, table_name, df, result_path, mode="overwrite"): - if mode is "overwrite": + if mode == "overwrite": hudi_write_mode = "insert_overwrite" else: hudi_write_mode = "upsert" From 953f1c78855500bdd2a6a603749a65082e0a4067 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 22 Jun 2023 02:36:30 +0200 Subject: [PATCH 426/515] Upload fast test binaries to builds bucket --- tests/ci/fast_test_check.py | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 2849759a3ee..d5198e5c3d7 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -11,6 +11,7 @@ from typing import List, Tuple from github import Github +from build_check import get_release_or_pr from clickhouse_helper import ( ClickHouseHelper, mark_flaky_tests, @@ -31,6 +32,7 @@ from s3_helper import S3Helper from stopwatch import Stopwatch from tee_popen import TeePopen from upload_result_helper import upload_results +from version_helper import get_version_from_repo NAME = "Fast test" @@ -189,6 +191,17 @@ def main(): ch_helper = ClickHouseHelper() mark_flaky_tests(ch_helper, NAME, test_results) + s3_path_prefix = os.path.join( + get_release_or_pr(pr_info, get_version_from_repo())[0], + pr_info.sha, + "fast_tests", + ) + build_urls = s3_helper.upload_build_folder_to_s3( + os.path.join(output_path, "binaries"), + s3_path_prefix, + keep_dirs_in_s3_path=False, + upload_symlinks=False, + ) report_url = upload_results( s3_helper, @@ -197,6 +210,7 @@ def main(): test_results, [run_log_path] + additional_logs, NAME, + build_urls, ) print(f"::notice ::Report url: {report_url}") post_commit_status(commit, state, report_url, description, NAME, pr_info) From 91dc6dfe34f1239b936506324d552f578cf39fa3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 22 Jun 2023 18:20:23 +0200 Subject: [PATCH 427/515] Update mysql-php-client to the recent rolling version --- docker/test/integration/mysql_php_client/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/integration/mysql_php_client/Dockerfile b/docker/test/integration/mysql_php_client/Dockerfile index 55db4d15a7f..0e11ae023e6 100644 --- a/docker/test/integration/mysql_php_client/Dockerfile +++ b/docker/test/integration/mysql_php_client/Dockerfile @@ -1,7 +1,7 @@ # docker build -t clickhouse/mysql-php-client . # MySQL PHP client docker container -FROM php:8.0.18-cli +FROM php:8-cli-alpine COPY ./client.crt client.crt COPY ./client.key client.key From b2bfe2eb87e30e217839ec9b6c063a2e234c6f1f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 22 Jun 2023 19:10:44 +0200 Subject: [PATCH 428/515] The error is changed in psycopg2=2.9.6 --- tests/integration/test_postgresql_protocol/test.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index e1d8cbf9bcc..de01bba6862 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -111,7 +111,7 @@ def test_psql_client(started_cluster): def test_python_client(started_cluster): node = cluster.instances["node"] - with pytest.raises(py_psql.InternalError) as exc_info: + with pytest.raises(py_psql.OperationalError) as exc_info: ch = py_psql.connect( host=node.ip_address, port=server_port, @@ -122,9 +122,7 @@ def test_python_client(started_cluster): cur = ch.cursor() cur.execute("select name from tables;") - assert exc_info.value.args == ( - "Query execution failed.\nDB::Exception: Table default.tables doesn't exist\nSSL connection has been closed unexpectedly\n", - ) + assert exc_info.value.args == ("SSL connection has been closed unexpectedly\n",) ch = py_psql.connect( host=node.ip_address, From 22c8f1c0be46f5cc5115d6e8c55a963fa466b12e Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 23 Jun 2023 01:46:29 +0200 Subject: [PATCH 429/515] Fix test_jemalloc_percpu_arena after upgrade --- tests/integration/test_jemalloc_percpu_arena/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_jemalloc_percpu_arena/test.py b/tests/integration/test_jemalloc_percpu_arena/test.py index 0dccde3776e..8de3dcd7ea2 100755 --- a/tests/integration/test_jemalloc_percpu_arena/test.py +++ b/tests/integration/test_jemalloc_percpu_arena/test.py @@ -26,7 +26,7 @@ def run_command_in_container(cmd, *args): "run", "--rm", *args, - "ubuntu:20.04", + "ubuntu:22.04", "sh", "-c", cmd, From 406896f9a95658a27f298c4c8b154467f8d5cbb8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 23 Jun 2023 02:10:47 +0200 Subject: [PATCH 430/515] Create cgroupsv2 subtree for docker-in-docker --- docker/test/integration/runner/dockerd-entrypoint.sh | 11 +++++++++++ tests/integration/test_cgroup_limit/test.py | 2 +- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 347d904d5c0..3c4ff522b36 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -12,6 +12,17 @@ echo '{ "registry-mirrors" : ["http://dockerhub-proxy.dockerhub-proxy-zone:5000"] }' | dd of=/etc/docker/daemon.json 2>/dev/null +if [ -f /sys/fs/cgroup/cgroup.controllers ]; then + # move the processes from the root group to the /init group, + # otherwise writing subtree_control fails with EBUSY. + # An error during moving non-existent process (i.e., "cat") is ignored. + mkdir -p /sys/fs/cgroup/init + xargs -rn1 < /sys/fs/cgroup/cgroup.procs > /sys/fs/cgroup/init/cgroup.procs || : + # enable controllers + sed -e 's/ / +/g' -e 's/^/+/' < /sys/fs/cgroup/cgroup.controllers \ + > /sys/fs/cgroup/cgroup.subtree_control +fi + # In case of test hung it is convenient to use pytest --pdb to debug it, # and on hung you can simply press Ctrl-C and it will spawn a python pdb, # but on SIGINT dockerd will exit, so ignore it to preserve the daemon. diff --git a/tests/integration/test_cgroup_limit/test.py b/tests/integration/test_cgroup_limit/test.py index f6392eca4d7..4ab12436dc9 100644 --- a/tests/integration/test_cgroup_limit/test.py +++ b/tests/integration/test_cgroup_limit/test.py @@ -22,7 +22,7 @@ def run_command_in_container(cmd, *args): "run", "--rm", *args, - "ubuntu:20.04", + "ubuntu:22.04", "sh", "-c", cmd, From a18300c497a051762f1ffcb5f366d50591a812f1 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 23 Jun 2023 02:28:01 +0200 Subject: [PATCH 431/515] Improve logging in integration/test_cgroup_limit --- tests/integration/test_cgroup_limit/test.py | 26 +++++++++++---------- 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_cgroup_limit/test.py b/tests/integration/test_cgroup_limit/test.py index 4ab12436dc9..e77b0f70960 100644 --- a/tests/integration/test_cgroup_limit/test.py +++ b/tests/integration/test_cgroup_limit/test.py @@ -1,5 +1,6 @@ #!/usr/bin/env python3 +import logging import os import math import subprocess @@ -16,18 +17,19 @@ def run_command_in_container(cmd, *args): f"{alternative_binary}:/usr/bin/clickhouse", ) - return subprocess.check_output( - [ - "docker", - "run", - "--rm", - *args, - "ubuntu:22.04", - "sh", - "-c", - cmd, - ] - ) + command = [ + "docker", + "run", + "--rm", + *args, + "ubuntu:22.04", + "sh", + "-c", + cmd, + ] + + logging.debug("Command: %s", " ".join(command)) + return subprocess.check_output(command) def run_with_cpu_limit(cmd, num_cpus, *args): From 4f1982e25e9313442f312feb70bc25bd88dfae2c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 23 Jun 2023 02:51:18 +0200 Subject: [PATCH 432/515] API of pika for rabbitmq has changed --- tests/integration/test_storage_rabbitmq/test.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index b4dcf86e0ba..943e7742018 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -858,7 +858,7 @@ def test_rabbitmq_insert(rabbitmq_cluster): if len(insert_messages) == 50: channel.stop_consuming() - consumer.basic_consume(onReceived, queue_name) + consumer.basic_consume(queue_name, onReceived) consumer.start_consuming() consumer_connection.close() @@ -921,7 +921,7 @@ def test_rabbitmq_insert_headers_exchange(rabbitmq_cluster): if len(insert_messages) == 50: channel.stop_consuming() - consumer.basic_consume(onReceived, queue_name) + consumer.basic_consume(queue_name, onReceived) consumer.start_consuming() consumer_connection.close() @@ -2991,7 +2991,7 @@ def test_format_with_prefix_and_suffix(rabbitmq_cluster): if len(insert_messages) == 2: channel.stop_consuming() - consumer.basic_consume(onReceived, queue_name) + consumer.basic_consume(queue_name, onReceived) consumer.start_consuming() consumer_connection.close() @@ -3050,7 +3050,7 @@ def test_max_rows_per_message(rabbitmq_cluster): if len(insert_messages) == 2: channel.stop_consuming() - consumer.basic_consume(onReceived, queue_name) + consumer.basic_consume(queue_name, onReceived) consumer.start_consuming() consumer_connection.close() @@ -3148,7 +3148,7 @@ def test_row_based_formats(rabbitmq_cluster): if insert_messages == 2: channel.stop_consuming() - consumer.basic_consume(onReceived, queue_name) + consumer.basic_consume(queue_name, onReceived) consumer.start_consuming() consumer_connection.close() @@ -3211,7 +3211,7 @@ def test_block_based_formats_1(rabbitmq_cluster): if len(insert_messages) == 3: channel.stop_consuming() - consumer.basic_consume(onReceived, queue_name) + consumer.basic_consume(queue_name, onReceived) consumer.start_consuming() consumer_connection.close() @@ -3296,7 +3296,7 @@ def test_block_based_formats_2(rabbitmq_cluster): if insert_messages == 9: channel.stop_consuming() - consumer.basic_consume(onReceived, queue_name) + consumer.basic_consume(queue_name, onReceived) consumer.start_consuming() consumer_connection.close() From f85460d8ef6e5996f9ec56296b3a155344772dd8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 23 Jun 2023 02:56:06 +0200 Subject: [PATCH 433/515] Enable tests with broken channel.start_consuming --- tests/integration/test_storage_rabbitmq/test.py | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 943e7742018..14764dd3835 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2953,7 +2953,6 @@ def test_rabbitmq_address(rabbitmq_cluster): instance2.query("drop table rabbit_out sync") -@pytest.mark.skip(reason="FIXME: flaky (something with channel.start_consuming()") def test_format_with_prefix_and_suffix(rabbitmq_cluster): instance.query( """ @@ -3002,7 +3001,6 @@ def test_format_with_prefix_and_suffix(rabbitmq_cluster): ) -@pytest.mark.skip(reason="FIXME: flaky (something with channel.start_consuming()") def test_max_rows_per_message(rabbitmq_cluster): num_rows = 5 @@ -3075,7 +3073,6 @@ def test_max_rows_per_message(rabbitmq_cluster): assert result == "0\t0\n10\t100\n20\t200\n30\t300\n40\t400\n" -@pytest.mark.skip(reason="FIXME: flaky (something with channel.start_consuming()") def test_row_based_formats(rabbitmq_cluster): num_rows = 10 @@ -3172,7 +3169,6 @@ def test_row_based_formats(rabbitmq_cluster): assert result == expected -@pytest.mark.skip(reason="FIXME: flaky (something with channel.start_consuming()") def test_block_based_formats_1(rabbitmq_cluster): instance.query( """ @@ -3234,7 +3230,6 @@ def test_block_based_formats_1(rabbitmq_cluster): ] -@pytest.mark.skip(reason="FIXME: flaky (something with channel.start_consuming()") def test_block_based_formats_2(rabbitmq_cluster): num_rows = 100 From 0680f0988c906a01567fae54b2097f5ad60f749d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 23 Jun 2023 03:02:03 +0200 Subject: [PATCH 434/515] Enable other working tests --- tests/integration/test_storage_rabbitmq/test.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 14764dd3835..751279f5e5a 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -1020,7 +1020,6 @@ def test_rabbitmq_many_inserts(rabbitmq_cluster): ), "ClickHouse lost some messages: {}".format(result) -@pytest.mark.skip(reason="Flaky") def test_rabbitmq_overloaded_insert(rabbitmq_cluster): instance.query( """ @@ -2050,7 +2049,6 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): ) -@pytest.mark.skip(reason="Timeout: FIXME") def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): logging.getLogger("pika").propagate = False instance.query( From f728f9735048d5cea37bb422fefc72ffcab649ce Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 27 Jun 2023 13:54:01 +0200 Subject: [PATCH 435/515] Add cgroupsv2 for getCGroupLimitedCPUCores --- src/Common/getNumberOfPhysicalCPUCores.cpp | 86 +++++++++++++++++++++- 1 file changed, 83 insertions(+), 3 deletions(-) diff --git a/src/Common/getNumberOfPhysicalCPUCores.cpp b/src/Common/getNumberOfPhysicalCPUCores.cpp index ed82c59140d..8fbb32e911f 100644 --- a/src/Common/getNumberOfPhysicalCPUCores.cpp +++ b/src/Common/getNumberOfPhysicalCPUCores.cpp @@ -1,4 +1,5 @@ #include "getNumberOfPhysicalCPUCores.h" +#include #include "config.h" #if defined(OS_LINUX) @@ -7,6 +8,8 @@ #endif #include +#include +#include #include #include @@ -15,7 +18,7 @@ namespace { #if defined(OS_LINUX) -int32_t readFrom(const char * filename, int default_value) +int32_t readFrom(const std::filesystem::path & filename, int default_value) { std::ifstream infile(filename); if (!infile.is_open()) @@ -31,10 +34,87 @@ int32_t readFrom(const char * filename, int default_value) uint32_t getCGroupLimitedCPUCores(unsigned default_cpu_count) { uint32_t quota_count = default_cpu_count; + std::filesystem::path prefix = "/sys/fs/cgroup"; + /// cgroupsv2 + std::ifstream contr_file(prefix / "cgroup.controllers"); + if (contr_file.is_open()) + { + /// First, we identify the cgroup the process belongs + std::ifstream cgroup_name_file("/proc/self/cgroup"); + if (!cgroup_name_file.is_open()) + return default_cpu_count; + + // cgroup_name_file always starts with '0::/' for v2 + cgroup_name_file.ignore(4); + std::string cgroup_name; + cgroup_name_file >> cgroup_name; + + std::filesystem::path current_cgroup; + if (cgroup_name.empty()) + current_cgroup = prefix; + else + current_cgroup = prefix / cgroup_name; + + // Looking for cpu.max in directories from the current cgroup to the top level + // It does not stop on the first time since the child could have a greater value than parent + while (current_cgroup != prefix.parent_path()) + { + std::ifstream cpu_max_file(current_cgroup / "cpu.max"); + current_cgroup = current_cgroup.parent_path(); + if (cpu_max_file.is_open()) + { + std::string cpu_limit_str; + float cpu_period; + cpu_max_file >> cpu_limit_str >> cpu_period; + if (cpu_limit_str != "max" && cpu_period != 0) + { + float cpu_limit = std::stof(cpu_limit_str); + quota_count = std::min(static_cast(ceil(cpu_limit / cpu_period)), quota_count); + } + } + } + current_cgroup = prefix / cgroup_name; + // Looking for cpuset.cpus.effective in directories from the current cgroup to the top level + while (current_cgroup != prefix.parent_path()) + { + std::ifstream cpuset_cpus_file(current_cgroup / "cpuset.cpus.effective"); + current_cgroup = current_cgroup.parent_path(); + if (cpuset_cpus_file.is_open()) + { + // The line in the file is "0,2-4,6,9-14" cpu numbers + // It's always grouped and ordered + std::vector cpu_ranges; + std::string cpuset_line; + cpuset_cpus_file >> cpuset_line; + if (cpuset_line.empty()) + continue; + boost::split(cpu_ranges, cpuset_line, boost::is_any_of(",")); + uint32_t cpus_count = 0; + for (const std::string& cpu_number_or_range : cpu_ranges) + { + std::vector cpu_range; + boost::split(cpu_range, cpu_number_or_range, boost::is_any_of("-")); + + if (cpu_range.size() == 2) + { + int start = std::stoi(cpu_range[0]); + int end = std::stoi(cpu_range[1]); + cpus_count += (end - start) + 1; + } + else + cpus_count++; + } + quota_count = std::min(cpus_count, quota_count); + break; + } + } + return quota_count; + } + /// cgroupsv1 /// Return the number of milliseconds per period process is guaranteed to run. /// -1 for no quota - int cgroup_quota = readFrom("/sys/fs/cgroup/cpu/cpu.cfs_quota_us", -1); - int cgroup_period = readFrom("/sys/fs/cgroup/cpu/cpu.cfs_period_us", -1); + int cgroup_quota = readFrom(prefix / "cpu/cpu.cfs_quota_us", -1); + int cgroup_period = readFrom(prefix / "cpu/cpu.cfs_period_us", -1); if (cgroup_quota > -1 && cgroup_period > 0) quota_count = static_cast(ceil(static_cast(cgroup_quota) / static_cast(cgroup_period))); From 29a0220e63420a60d1aef9cfec0f9c6c0bba8160 Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Wed, 28 Jun 2023 14:42:01 -0600 Subject: [PATCH 436/515] Update functions-for-nulls.md Fixing broken link to HTML anchor on page --- docs/en/sql-reference/functions/functions-for-nulls.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/functions-for-nulls.md b/docs/en/sql-reference/functions/functions-for-nulls.md index 6f82fedaab7..d57b799e94c 100644 --- a/docs/en/sql-reference/functions/functions-for-nulls.md +++ b/docs/en/sql-reference/functions/functions-for-nulls.md @@ -8,7 +8,7 @@ sidebar_label: Nullable ## isNull -Returns whether the argument is [NULL](../../sql-reference/syntax.md#null-literal). +Returns whether the argument is [NULL](../../sql-reference/syntax.md#null). ``` sql isNull(x) From 296f9968c04f6ca49599e281b4092f6cffef5bfd Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 28 Jun 2023 23:20:02 +0200 Subject: [PATCH 437/515] fix --- docker/test/upgrade/run.sh | 1 + src/Storages/MergeTree/IDataPartStorage.h | 1 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 3 ++- src/Storages/MergeTree/MergeTreeData.cpp | 10 ++++++++++ src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 9 +++++++++ src/Storages/StorageReplicatedMergeTree.cpp | 9 ++++++--- 6 files changed, 29 insertions(+), 4 deletions(-) diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index e72c28b7167..8fd514eaa93 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -189,6 +189,7 @@ rg -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \ -e "Authentication failed" \ -e "Cannot flush" \ -e "Container already exists" \ + -e "doesn't have metadata version on disk" \ clickhouse-server.upgrade.log \ | grep -av -e "_repl_01111_.*Mapping for table with UUID" \ | zgrep -Fa "" > /test_output/upgrade_error_messages.txt \ diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index b40a9aa1b46..a2ca30bf73a 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -244,6 +244,7 @@ public: bool keep_metadata_version = false; bool make_source_readonly = false; DiskTransactionPtr external_transaction = nullptr; + std::optional metadata_version_to_write = std::nullopt; }; virtual std::shared_ptr freeze( diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 1f105951757..b9591864869 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1363,7 +1363,8 @@ void IMergeTreeDataPart::loadColumns(bool require) { loaded_metadata_version = metadata_snapshot->getMetadataVersion(); old_part_with_no_metadata_version_on_disk = true; - LOG_WARNING(storage.log, "Part {} doesn't have metadata version on disk, setting it to {}. " + if (storage.supportsReplication()) + LOG_WARNING(storage.log, "Part {} doesn't have metadata version on disk, setting it to {}. " "It's okay if the part was created by an old version of ClickHouse", name, loaded_metadata_version); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f3cf4a85953..9f2fa6c0434 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7349,6 +7349,16 @@ std::pair MergeTreeData::cloneAn /*save_metadata_callback=*/ {}, params); + if (params.metadata_version_to_write.has_value()) + { + chassert(!params.keep_metadata_version); + auto out_metadata = dst_part_storage->writeFile(IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, getContext()->getWriteSettings()); + writeText(metadata_snapshot->getMetadataVersion(), *out_metadata); + out_metadata->finalize(); + if (getSettings()->fsync_after_insert) + out_metadata->sync(); + } + LOG_DEBUG(log, "Clone{} part {} to {}{}", src_flushed_tmp_part ? " flushed" : "", src_part_storage->getFullPath(), diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index c93077fb4fb..1c896c4e2a6 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -604,6 +604,15 @@ void ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData: { /// NOTE: No delay in this case. That's Ok. + part->getDataPartStorage().removeFileIfExists(IMergeTreeDataPart::METADATA_VERSION_FILE_NAME); + { + auto out_metadata = part->getDataPartStorage().writeFile(IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, context->getWriteSettings()); + writeText(metadata_snapshot->getMetadataVersion(), *out_metadata); + out_metadata->finalize(); + if (storage.getSettings()->fsync_after_insert) + out_metadata->sync(); + } + auto origin_zookeeper = storage.getZooKeeper(); assertSessionIsNotExpired(origin_zookeeper); auto zookeeper = std::make_shared(origin_zookeeper); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b9d48fc75f3..0361d46612b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2448,7 +2448,8 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) bool zero_copy_enabled = dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; IDataPartStorage::ClonePartParams clone_params { - .copy_instead_of_hardlink = zero_copy_enabled && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport() + .copy_instead_of_hardlink = zero_copy_enabled && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport(), + .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk( part_desc->src_table_part, TMP_PREFIX + "clone_", part_desc->new_part_info, metadata_snapshot, clone_params); @@ -7306,7 +7307,8 @@ void StorageReplicatedMergeTree::replacePartitionFrom( || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; IDataPartStorage::ClonePartParams clone_params { - .copy_instead_of_hardlink = zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport() + .copy_instead_of_hardlink = zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport(), + .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, metadata_snapshot, clone_params); src_parts.emplace_back(src_part); @@ -7545,7 +7547,8 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta || dynamic_cast(dest_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; IDataPartStorage::ClonePartParams clone_params { - .copy_instead_of_hardlink = zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport() + .copy_instead_of_hardlink = zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport(), + .metadata_version_to_write = dest_metadata_snapshot->getMetadataVersion() }; auto [dst_part, dst_part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot, clone_params); From 92d7b067b0ae9eff9b28fb41c8ffbeebc8b2c31c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 28 Jun 2023 23:36:15 +0200 Subject: [PATCH 438/515] Bring back lost /tmp cleanup in clickhouse-server docker image --- docker/server/Dockerfile.ubuntu | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 1bb0cfdc700..42ae81655d2 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -48,14 +48,15 @@ ARG TARGETARCH RUN arch="${TARGETARCH:-amd64}" \ && if [ -n "${deb_location_url}" ]; then \ echo "installing from custom url with deb packages: ${deb_location_url}" \ - rm -rf /tmp/clickhouse_debs \ + && rm -rf /tmp/clickhouse_debs \ && mkdir -p /tmp/clickhouse_debs \ && for package in ${PACKAGES}; do \ { wget --progress=bar:force:noscroll "${deb_location_url}/${package}_${VERSION}_${arch}.deb" -P /tmp/clickhouse_debs || \ wget --progress=bar:force:noscroll "${deb_location_url}/${package}_${VERSION}_all.deb" -P /tmp/clickhouse_debs ; } \ || exit 1 \ ; done \ - && dpkg -i /tmp/clickhouse_debs/*.deb ; \ + && dpkg -i /tmp/clickhouse_debs/*.deb \ + && rm -rf /tmp/* ; \ fi # install from a single binary @@ -65,11 +66,12 @@ RUN if [ -n "${single_binary_location_url}" ]; then \ && mkdir -p /tmp/clickhouse_binary \ && wget --progress=bar:force:noscroll "${single_binary_location_url}" -O /tmp/clickhouse_binary/clickhouse \ && chmod +x /tmp/clickhouse_binary/clickhouse \ - && /tmp/clickhouse_binary/clickhouse install --user "clickhouse" --group "clickhouse" ; \ + && /tmp/clickhouse_binary/clickhouse install --user "clickhouse" --group "clickhouse" \ + && rm -rf /tmp/* ; \ fi # A fallback to installation from ClickHouse repository -RUN if ! clickhouse local -q "SELECT ''" > /dev/null; then \ +RUN if ! clickhouse local -q "SELECT ''" > /dev/null 2>&1; then \ apt-get update \ && apt-get install --yes --no-install-recommends \ apt-transport-https \ @@ -90,12 +92,12 @@ RUN if ! clickhouse local -q "SELECT ''" > /dev/null; then \ packages="${packages} ${package}=${VERSION}" \ ; done \ && apt-get install --allow-unauthenticated --yes --no-install-recommends ${packages} || exit 1 \ - && rm -rf \ - /var/lib/apt/lists/* \ - /var/cache/debconf \ - /tmp/* \ - && apt-get autoremove --purge -yq libksba8 \ - && apt-get autoremove -yq \ + && rm -rf \ + /var/lib/apt/lists/* \ + /var/cache/debconf \ + /tmp/* \ + && apt-get autoremove --purge -yq libksba8 \ + && apt-get autoremove -yq \ ; fi # post install From 45232770e06079b2b346209ae7b412597ff5996f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 29 Jun 2023 00:41:23 +0300 Subject: [PATCH 439/515] Try to fix deadlock in ZooKeeper client --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 14b31c6a411..dd555e39529 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -362,6 +362,16 @@ ZooKeeper::ZooKeeper( { tryLogCurrentException(log, "Failed to connect to ZooKeeper"); + try + { + requests_queue.finish(); + socket.shutdown(); + } + catch (...) + { + tryLogCurrentException(log); + } + send_thread.join(); receive_thread.join(); From b27cf4e7ba0d9d579722202f693f820777c6bfc7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 29 Jun 2023 00:15:45 +0200 Subject: [PATCH 440/515] fix --- src/Storages/MergeTree/MergeTreeData.cpp | 11 +++++++++-- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 10 ---------- 2 files changed, 9 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9f2fa6c0434..add48e2cf03 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4707,12 +4707,19 @@ MergeTreeData::DataPartPtr MergeTreeData::getPartIfExistsUnlocked(const MergeTre return nullptr; } -static void loadPartAndFixMetadataImpl(MergeTreeData::MutableDataPartPtr part) +static void loadPartAndFixMetadataImpl(MergeTreeData::MutableDataPartPtr part, ContextPtr local_context, int32_t metadata_version, bool sync) { /// Remove metadata version file and take it from table. /// Currently we cannot attach parts with different schema, so /// we can assume that it's equal to table's current schema. part->removeMetadataVersion(); + { + auto out_metadata = part->getDataPartStorage().writeFile(IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, local_context->getWriteSettings()); + writeText(metadata_version, *out_metadata); + out_metadata->finalize(); + if (sync) + out_metadata->sync(); + } part->loadColumnsChecksumsIndexes(false, true); part->modification_time = part->getDataPartStorage().getLastModified().epochTime(); @@ -5844,7 +5851,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const .withPartFormatFromDisk() .build(); - loadPartAndFixMetadataImpl(part); + loadPartAndFixMetadataImpl(part, local_context, getInMemoryMetadataPtr()->getMetadataVersion(), getSettings()->fsync_after_insert); loaded_parts.push_back(part); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 1c896c4e2a6..9c5890383dc 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -603,16 +603,6 @@ template void ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData::MutableDataPartPtr & part) { /// NOTE: No delay in this case. That's Ok. - - part->getDataPartStorage().removeFileIfExists(IMergeTreeDataPart::METADATA_VERSION_FILE_NAME); - { - auto out_metadata = part->getDataPartStorage().writeFile(IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, context->getWriteSettings()); - writeText(metadata_snapshot->getMetadataVersion(), *out_metadata); - out_metadata->finalize(); - if (storage.getSettings()->fsync_after_insert) - out_metadata->sync(); - } - auto origin_zookeeper = storage.getZooKeeper(); assertSessionIsNotExpired(origin_zookeeper); auto zookeeper = std::make_shared(origin_zookeeper); From 37ad50bd637b8bfd6612e0b80b8f6dd388a19595 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 28 Jun 2023 19:40:40 -0300 Subject: [PATCH 441/515] Update settings.md --- .../server-configuration-parameters/settings.md | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 40c1b8d64a1..bad7e388377 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -2120,7 +2120,13 @@ This section contains the following parameters: - `operation_timeout_ms` — Maximum timeout for one operation in milliseconds. - `root` — The [znode](http://zookeeper.apache.org/doc/r3.5.5/zookeeperOver.html#Nodes+and+ephemeral+nodes) that is used as the root for znodes used by the ClickHouse server. Optional. - `identity` — User and password, that can be required by ZooKeeper to give access to requested znodes. Optional. - +- zookeeper_load_balancing - Specifies the algorithm of ZooKeeper node selection. + * random - randomly selects one of ZooKeeper nodes. + * in_order - selects the first ZooKeeper node, if it's not available then the second, and so on. + * nearest_hostname - selects a ZooKeeper node with a hostname that is most similar to the server’s hostname. + * first_or_random - selects the first ZooKeeper node, if it's not available then randomly selects one of remaining ZooKeeper nodes. + * round_robin - selects the first ZooKeeper node, if reconnection happens selects the next. + **Example configuration** ``` xml @@ -2139,6 +2145,8 @@ This section contains the following parameters: /path/to/zookeeper/node user:password + + random ``` From f23bf9c5acdd885d61ec7f68bdf8be1b1fa79dee Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 29 Jun 2023 10:04:36 +0800 Subject: [PATCH 442/515] Update redis.md --- .../engines/table-engines/integrations/redis.md | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/redis.md b/docs/en/engines/table-engines/integrations/redis.md index 568179eb690..7ef87927bfd 100644 --- a/docs/en/engines/table-engines/integrations/redis.md +++ b/docs/en/engines/table-engines/integrations/redis.md @@ -44,9 +44,10 @@ Create a table in ClickHouse which allows to read data from Redis: ``` sql CREATE TABLE redis_table ( - `k` String, - `m` String, - `n` UInt32 + `key` String, + `v1` UInt32, + `v2` String, + `v3` Float32 ) ENGINE = Redis('redis1:6379') PRIMARY KEY(k); ``` @@ -111,9 +112,16 @@ Flush Redis db asynchronously. Also `Truncate` support SYNC mode. TRUNCATE TABLE redis_table SYNC; ``` +Join: + +Join with other tables. + +``` +SELECT * FROM redis_table JOIN merge_tree_table ON redis_table.key=merge_tree_table.key; +``` ## Limitations {#limitations} Redis engine also supports scanning queries, such as `where k > xx`, but it has some limitations: -1. Scanning query may produce some duplicated keys in a very rare case when it is rehashing. See details in [Redis Scan](https://github.com/redis/redis/blob/e4d183afd33e0b2e6e8d1c79a832f678a04a7886/src/dict.c#L1186-L1269) +1. Scanning query may produce some duplicated keys in a very rare case when it is rehashing. See details in [Redis Scan](https://github.com/redis/redis/blob/e4d183afd33e0b2e6e8d1c79a832f678a04a7886/src/dict.c#L1186-L1269). 2. During the scanning, keys could be created and deleted, so the resulting dataset can not represent a valid point in time. From f8f0b7d086d23f70a49be88233b19d152e99e3f4 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 29 Jun 2023 10:09:27 +0800 Subject: [PATCH 443/515] fix typo --- docs/en/engines/table-engines/integrations/redis.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/redis.md b/docs/en/engines/table-engines/integrations/redis.md index 7ef87927bfd..2697abcf30e 100644 --- a/docs/en/engines/table-engines/integrations/redis.md +++ b/docs/en/engines/table-engines/integrations/redis.md @@ -49,7 +49,7 @@ CREATE TABLE redis_table `v2` String, `v3` Float32 ) -ENGINE = Redis('redis1:6379') PRIMARY KEY(k); +ENGINE = Redis('redis1:6379') PRIMARY KEY(key); ``` Insert: From 635ab9f9af3894c42b69ef093cc34e64cefce219 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 29 Jun 2023 12:53:23 +0800 Subject: [PATCH 444/515] move redis_table to right --- docs/en/engines/table-engines/integrations/redis.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/redis.md b/docs/en/engines/table-engines/integrations/redis.md index 2697abcf30e..8086a6503b8 100644 --- a/docs/en/engines/table-engines/integrations/redis.md +++ b/docs/en/engines/table-engines/integrations/redis.md @@ -117,7 +117,7 @@ Join: Join with other tables. ``` -SELECT * FROM redis_table JOIN merge_tree_table ON redis_table.key=merge_tree_table.key; +SELECT * FROM redis_table JOIN merge_tree_table ON merge_tree_table.key=redis_table.key; ``` ## Limitations {#limitations} From 2392dda4126f00ed968f1496e6b61ab8089832f9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 29 Jun 2023 09:38:59 +0200 Subject: [PATCH 445/515] Changelog for 23.6 --- CHANGELOG.md | 102 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 102 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 72372c8fac4..a2e7b021081 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ### Table of Contents +**[ClickHouse release v23.6, 2023-06-30](#236)**
**[ClickHouse release v23.5, 2023-06-08](#235)**
**[ClickHouse release v23.4, 2023-04-26](#234)**
**[ClickHouse release v23.3 LTS, 2023-03-30](#233)**
@@ -8,6 +9,107 @@ # 2023 Changelog +### ClickHouse release 23.6, 2023-06-29 + +#### Backward Incompatible Change +* Delete feature `do_not_evict_index_and_mark_files` in the fs cache. This feature was only making things worse. [#51253](https://github.com/ClickHouse/ClickHouse/pull/51253) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Remove ALTER support for experimental LIVE VIEW. [#51287](https://github.com/ClickHouse/ClickHouse/pull/51287) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Decrease the default values for `http_max_field_value_size` and `http_max_field_name_size` to 128 KiB. [#51163](https://github.com/ClickHouse/ClickHouse/pull/51163) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* CGroups metrics related to CPU are replaced with one metric, `CGroupMaxCPU` for better usability. The `Normalized` CPU usage metrics will be normalized to CGroups limits instead of the total number of CPUs when they are set. This closes [#50836](https://github.com/ClickHouse/ClickHouse/issues/50836). [#50835](https://github.com/ClickHouse/ClickHouse/pull/50835) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* Added `Overlay` database engine to combine multiple databases into one. Added `Filesystem` database engine to represent a directory in the filesystem as a set of implicitly available tables with auto-detected formats and structures. A new `S3` database engine allows to read-only interact with s3 storage by representing a prefix as a set of tables. A new `HDFS` database engine allows to interact with HDFS storage in the same way. [#48821](https://github.com/ClickHouse/ClickHouse/pull/48821) ([alekseygolub](https://github.com/alekseygolub)). +* The function `transform` as well as `CASE` with value matching started to support all data types. This closes [#29730](https://github.com/ClickHouse/ClickHouse/issues/29730). This closes [#32387](https://github.com/ClickHouse/ClickHouse/issues/32387). This closes [#50827](https://github.com/ClickHouse/ClickHouse/issues/50827). This closes [#31336](https://github.com/ClickHouse/ClickHouse/issues/31336). This closes [#40493](https://github.com/ClickHouse/ClickHouse/issues/40493). [#51351](https://github.com/ClickHouse/ClickHouse/pull/51351) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Added option `--rename_files_after_processing `. This closes [#34207](https://github.com/ClickHouse/ClickHouse/issues/34207). [#49626](https://github.com/ClickHouse/ClickHouse/pull/49626) ([alekseygolub](https://github.com/alekseygolub)). +* Add support for `APPEND` modifier in `INTO OUTFILE` clause. Suggest using `APPEND` or `TRUNCATE` for `INTO OUTFILE` when file exists. [#50950](https://github.com/ClickHouse/ClickHouse/pull/50950) ([alekar](https://github.com/alekar)). +* Add table engine `Redis` and table function `redis`. It allows querying external Redis servers. [#50150](https://github.com/ClickHouse/ClickHouse/pull/50150) ([JackyWoo](https://github.com/JackyWoo)). +* Allow to skip empty files in file/s3/url/hdfs table functions using settings `s3_skip_empty_files`, `hdfs_skip_empty_files`, `engine_file_skip_empty_files`, `engine_url_skip_empty_files`. [#50364](https://github.com/ClickHouse/ClickHouse/pull/50364) ([Kruglov Pavel](https://github.com/Avogar)). +* Add a new setting named `use_mysql_types_in_show_columns` to alter the `SHOW COLUMNS` SQL statement to display MySQL equivalent types when a client is connected via the MySQL compatibility port. [#49577](https://github.com/ClickHouse/ClickHouse/pull/49577) ([Thomas Panetti](https://github.com/tpanetti)). +* Clickhouse-client can now be called with a connection string instead of "--host", "--port", "--user" etc. [#50689](https://github.com/ClickHouse/ClickHouse/pull/50689) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Add setting `session_timezone`, it is used as default timezone for session when not explicitly specified. [#44149](https://github.com/ClickHouse/ClickHouse/pull/44149) ([Andrey Zvonov](https://github.com/zvonand)). +* Codec DEFLATE_QPL is now controlled via server setting "enable_deflate_qpl_codec" (default: false) instead of setting "allow_experimental_codecs". This marks DEFLATE_QPL non-experimental. [#50775](https://github.com/ClickHouse/ClickHouse/pull/50775) ([Robert Schulze](https://github.com/rschu1ze)). + +#### Performance Improvement +* Improved scheduling of merge selecting and cleanup tasks in `ReplicatedMergeTree`. The tasks will not be executed too frequently when there's nothing to merge or cleanup. Added settings `max_merge_selecting_sleep_ms`, `merge_selecting_sleep_slowdown_factor`, `max_cleanup_delay_period` and `cleanup_thread_preferred_points_per_iteration`. It should close [#31919](https://github.com/ClickHouse/ClickHouse/issues/31919). [#50107](https://github.com/ClickHouse/ClickHouse/pull/50107) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Make filter push down through cross join. [#50605](https://github.com/ClickHouse/ClickHouse/pull/50605) ([Han Fei](https://github.com/hanfei1991)). +* Improve performance with enabled QueryProfiler using thread-local timer_id instead of global object. [#48778](https://github.com/ClickHouse/ClickHouse/pull/48778) ([Jiebin Sun](https://github.com/jiebinn)). +* Rewrite CapnProto input/output format to improve its performance. Map column names and CapnProto fields case insensitive, fix reading/writing of nested structure fields. [#49752](https://github.com/ClickHouse/ClickHouse/pull/49752) ([Kruglov Pavel](https://github.com/Avogar)). +* Optimize parquet write performance for parallel threads. [#50102](https://github.com/ClickHouse/ClickHouse/pull/50102) ([Hongbin Ma](https://github.com/binmahone)). +* Disable `parallelize_output_from_storages` for processing MATERIALIZED VIEWs and storages with one block only. [#50214](https://github.com/ClickHouse/ClickHouse/pull/50214) ([Azat Khuzhin](https://github.com/azat)). +* Merge PR [#46558](https://github.com/ClickHouse/ClickHouse/pull/46558). Avoid block permutation during sort if the block is already sorted. [#50697](https://github.com/ClickHouse/ClickHouse/pull/50697) ([Alexey Milovidov](https://github.com/alexey-milovidov), [Maksim Kita](https://github.com/kitaisreal)). +* Make multiple list requests to ZooKeeper in parallel to speed up reading from system.zookeeper table. [#51042](https://github.com/ClickHouse/ClickHouse/pull/51042) ([Alexander Gololobov](https://github.com/davenger)). +* Speedup initialization of DateTime lookup tables for time zones. This should reduce startup/connect time of clickhouse-client especially in debug build as it is rather heavy. [#51347](https://github.com/ClickHouse/ClickHouse/pull/51347) ([Alexander Gololobov](https://github.com/davenger)). +* Fix data lakes slowness because of synchronous head requests. (Related to Iceberg/Deltalake/Hudi being slow with a lot of files). [#50976](https://github.com/ClickHouse/ClickHouse/pull/50976) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Do not replicate `ALTER PARTITION` queries and mutations through `Replicated` database if it has only one shard and the underlying table is `ReplicatedMergeTree`. [#51049](https://github.com/ClickHouse/ClickHouse/pull/51049) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Do not read all the columns from right GLOBAL JOIN table. [#50721](https://github.com/ClickHouse/ClickHouse/pull/50721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### Experimental Feature +* Support parallel replicas with the analyzer. [#50441](https://github.com/ClickHouse/ClickHouse/pull/50441) ([Raúl Marín](https://github.com/Algunenano)). +* Add random sleep before large merges/mutations execution to split load more evenly between replicas in case of zero-copy replication. [#51282](https://github.com/ClickHouse/ClickHouse/pull/51282) ([alesapin](https://github.com/alesapin)). + +#### Improvement +* Relax the thresholds for "too many parts" to be more modern. Return the backpressure during long-running insert queries. [#50856](https://github.com/ClickHouse/ClickHouse/pull/50856) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Allow to cast IPv6 to IPv4 address for CIDR ::ffff:0:0/96 (IPv4-mapped addresses). [#49759](https://github.com/ClickHouse/ClickHouse/pull/49759) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Update MongoDB protocol to support MongoDB 5.1 version and newer. Support for the versions with the old protocol (<3.6) is preserved. Closes [#45621](https://github.com/ClickHouse/ClickHouse/issues/45621), [#49879](https://github.com/ClickHouse/ClickHouse/issues/49879). [#50061](https://github.com/ClickHouse/ClickHouse/pull/50061) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add setting `input_format_max_bytes_to_read_for_schema_inference` to limit the number of bytes to read in schema inference. Closes [#50577](https://github.com/ClickHouse/ClickHouse/issues/50577). [#50592](https://github.com/ClickHouse/ClickHouse/pull/50592) ([Kruglov Pavel](https://github.com/Avogar)). +* Respect setting `input_format_null_as_default` in schema inference. [#50602](https://github.com/ClickHouse/ClickHouse/pull/50602) ([Kruglov Pavel](https://github.com/Avogar)). +* Allow to skip trailing empty lines in CSV/TSV/CustomSeparated formats via settings `input_format_csv_skip_trailing_empty_lines`, `input_format_tsv_skip_trailing_empty_lines` and `input_format_custom_skip_trailing_empty_lines` (disabled by default). Closes [#49315](https://github.com/ClickHouse/ClickHouse/issues/49315). [#50635](https://github.com/ClickHouse/ClickHouse/pull/50635) ([Kruglov Pavel](https://github.com/Avogar)). +* Functions "toDateOrDefault|OrNull" and "accuateCast[OrDefault|OrNull]" now correctly parse numeric arguments. [#50709](https://github.com/ClickHouse/ClickHouse/pull/50709) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Support CSV with whitespace or `\t` field delimiters, and these delimiters are supported in Spark. [#50712](https://github.com/ClickHouse/ClickHouse/pull/50712) ([KevinyhZou](https://github.com/KevinyhZou)). +* Settings `number_of_mutations_to_delay` and `number_of_mutations_to_throw` are enabled by default now with values 500 and 1000 respectively. [#50726](https://github.com/ClickHouse/ClickHouse/pull/50726) ([Anton Popov](https://github.com/CurtizJ)). +* The dashboard correctly shows missing values. This closes [#50831](https://github.com/ClickHouse/ClickHouse/issues/50831). [#50832](https://github.com/ClickHouse/ClickHouse/pull/50832) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Added the possibility to use date and time arguments in the syslog timestamp format in functions `parseDateTimeBestEffort*` and `parseDateTime64BestEffort*`. [#50925](https://github.com/ClickHouse/ClickHouse/pull/50925) ([Victor Krasnov](https://github.com/sirvickr)). +* Command line parameter "--password" in clickhouse-client can now be specified only once. [#50966](https://github.com/ClickHouse/ClickHouse/pull/50966) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Use `hash_of_all_files` from `system.parts` to check identity of parts during on-cluster backups. [#50997](https://github.com/ClickHouse/ClickHouse/pull/50997) ([Vitaly Baranov](https://github.com/vitlibar)). +* The system table zookeeper_connection connected_time identifies the time when the connection is established (standard format), and session_uptime_elapsed_seconds is added, which labels the duration of the established connection session (in seconds). [#51026](https://github.com/ClickHouse/ClickHouse/pull/51026) ([郭小龙](https://github.com/guoxiaolongzte)). +* Improve the progress bar for file/s3/hdfs/url table functions by using chunk size from source data and using incremental total size counting in each thread. Fix the progress bar for *Cluster functions. This closes [#47250](https://github.com/ClickHouse/ClickHouse/issues/47250). [#51088](https://github.com/ClickHouse/ClickHouse/pull/51088) ([Kruglov Pavel](https://github.com/Avogar)). +* Add total_bytes_to_read to the Progress packet in TCP protocol for better Progress bar. [#51158](https://github.com/ClickHouse/ClickHouse/pull/51158) ([Kruglov Pavel](https://github.com/Avogar)). +* Better checking of data parts on disks with filesystem cache. [#51164](https://github.com/ClickHouse/ClickHouse/pull/51164) ([Anton Popov](https://github.com/CurtizJ)). +* Fix sometimes not correct current_elements_num in fs cache. [#51242](https://github.com/ClickHouse/ClickHouse/pull/51242) ([Kseniia Sumarokova](https://github.com/kssenii)). + +#### Build/Testing/Packaging Improvement +* Add embedded keeper-client to standalone keeper binary. [#50964](https://github.com/ClickHouse/ClickHouse/pull/50964) ([pufit](https://github.com/pufit)). +* Actual LZ4 version is used now. [#50621](https://github.com/ClickHouse/ClickHouse/pull/50621) ([Nikita Taranov](https://github.com/nickitat)). +* ClickHouse server will print the list of changed settings on fatal errors. This closes [#51137](https://github.com/ClickHouse/ClickHouse/issues/51137). [#51138](https://github.com/ClickHouse/ClickHouse/pull/51138) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Allow building ClickHouse with clang-17. [#51300](https://github.com/ClickHouse/ClickHouse/pull/51300) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* [SQLancer](https://github.com/sqlancer/sqlancer) check is considered stable as bugs that were triggered by it are fixed. Now failures of SQLancer check will be reported as failed check status. [#51340](https://github.com/ClickHouse/ClickHouse/pull/51340) ([Ilya Yatsishin](https://github.com/qoega)). +* Split huge `RUN` in Dockerfile into smaller conditional. Install the necessary tools on demand in the same `RUN` layer, and remove them after that. Upgrade the OS only once at the beginning. Use a modern way to check the signed repository. Downgrade the base repo to ubuntu:20.04 to address the issues on older docker versions. Upgrade golang version to address golang vulnerabilities. [#51504](https://github.com/ClickHouse/ClickHouse/pull/51504) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Report loading status for executable dictionaries correctly [#48775](https://github.com/ClickHouse/ClickHouse/pull/48775) ([Anton Kozlov](https://github.com/tonickkozlov)). +* Proper mutation of skip indices and projections [#50104](https://github.com/ClickHouse/ClickHouse/pull/50104) ([Amos Bird](https://github.com/amosbird)). +* Cleanup moving parts [#50489](https://github.com/ClickHouse/ClickHouse/pull/50489) ([vdimir](https://github.com/vdimir)). +* Fix backward compatibility for IP types hashing in aggregate functions [#50551](https://github.com/ClickHouse/ClickHouse/pull/50551) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix Log family table return wrong rows count after truncate [#50585](https://github.com/ClickHouse/ClickHouse/pull/50585) ([flynn](https://github.com/ucasfl)). +* Fix bug in `uniqExact` parallel merging [#50590](https://github.com/ClickHouse/ClickHouse/pull/50590) ([Nikita Taranov](https://github.com/nickitat)). +* Revert recent grace hash join changes [#50699](https://github.com/ClickHouse/ClickHouse/pull/50699) ([vdimir](https://github.com/vdimir)). +* Query Cache: Try to fix bad cast from `ColumnConst` to `ColumnVector` [#50704](https://github.com/ClickHouse/ClickHouse/pull/50704) ([Robert Schulze](https://github.com/rschu1ze)). +* Avoid storing logs in Keeper containing unknown operation [#50751](https://github.com/ClickHouse/ClickHouse/pull/50751) ([Antonio Andelic](https://github.com/antonio2368)). +* SummingMergeTree support for DateTime64 [#50797](https://github.com/ClickHouse/ClickHouse/pull/50797) ([Jordi Villar](https://github.com/jrdi)). +* Add compatibility setting for non-const timezones [#50834](https://github.com/ClickHouse/ClickHouse/pull/50834) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix hashing of LDAP params in the cache entries [#50865](https://github.com/ClickHouse/ClickHouse/pull/50865) ([Julian Maicher](https://github.com/jmaicher)). +* Fallback to parsing big integer from String instead of exception in Parquet format [#50873](https://github.com/ClickHouse/ClickHouse/pull/50873) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix checking the lock file too often while writing a backup [#50889](https://github.com/ClickHouse/ClickHouse/pull/50889) ([Vitaly Baranov](https://github.com/vitlibar)). +* Do not apply projection if read-in-order was enabled. [#50923](https://github.com/ClickHouse/ClickHouse/pull/50923) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix race in the Azure blob storage iterator [#50936](https://github.com/ClickHouse/ClickHouse/pull/50936) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix erroneous `sort_description` propagation in `CreatingSets` [#50955](https://github.com/ClickHouse/ClickHouse/pull/50955) ([Nikita Taranov](https://github.com/nickitat)). +* Fix Iceberg v2 optional metadata parsing [#50974](https://github.com/ClickHouse/ClickHouse/pull/50974) ([Kseniia Sumarokova](https://github.com/kssenii)). +* MaterializedMySQL: Keep parentheses for empty table overrides [#50977](https://github.com/ClickHouse/ClickHouse/pull/50977) ([Val Doroshchuk](https://github.com/valbok)). +* Fix crash in BackupCoordinationStageSync::setError() [#51012](https://github.com/ClickHouse/ClickHouse/pull/51012) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix subtly broken copy-on-write of ColumnLowCardinality dictionary [#51064](https://github.com/ClickHouse/ClickHouse/pull/51064) ([Michael Kolupaev](https://github.com/al13n321)). +* Generate safe IVs [#51086](https://github.com/ClickHouse/ClickHouse/pull/51086) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Fix ineffective query cache for SELECTs with subqueries [#51132](https://github.com/ClickHouse/ClickHouse/pull/51132) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix Set index with constant nullable comparison. [#51205](https://github.com/ClickHouse/ClickHouse/pull/51205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix a crash in s3 and s3Cluster functions [#51209](https://github.com/ClickHouse/ClickHouse/pull/51209) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix a crash with compiled expressions [#51231](https://github.com/ClickHouse/ClickHouse/pull/51231) ([LiuNeng](https://github.com/liuneng1994)). +* Fix use-after-free in StorageURL when switching URLs [#51260](https://github.com/ClickHouse/ClickHouse/pull/51260) ([Michael Kolupaev](https://github.com/al13n321)). +* Updated check for parameterized view [#51272](https://github.com/ClickHouse/ClickHouse/pull/51272) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix multiple writing of same file to backup [#51299](https://github.com/ClickHouse/ClickHouse/pull/51299) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix fuzzer failure in ActionsDAG [#51301](https://github.com/ClickHouse/ClickHouse/pull/51301) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove garbage from function `transform` [#51350](https://github.com/ClickHouse/ClickHouse/pull/51350) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + + ### ClickHouse release 23.5, 2023-06-08 #### Upgrade Notes From 340262814a721ca7be1523ff1194c404b0f03de8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 29 Jun 2023 12:24:19 +0200 Subject: [PATCH 446/515] Avoid additional maven requests for spark tests --- docker/test/integration/runner/Dockerfile | 6 +++++- tests/integration/helpers/cluster.py | 10 ++++++---- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 40627354f70..38d8ed5f223 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -105,7 +105,11 @@ RUN curl -fsSL -O https://dlcdn.apache.org/spark/spark-3.3.2/spark-3.3.2-bin-had # download spark and packages # if you change packages, don't forget to update them in tests/integration/helpers/cluster.py -RUN echo ":quit" | /spark-3.3.2-bin-hadoop3/bin/spark-shell --packages "org.apache.hudi:hudi-spark3.3-bundle_2.12:0.13.0,io.delta:delta-core_2.12:2.3.0,org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.1.0" > /dev/null +RUN packages="org.apache.hudi:hudi-spark3.3-bundle_2.12:0.13.0,\ +io.delta:delta-core_2.12:2.3.0,\ +org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.1.0" \ + && /spark-3.3.2-bin-hadoop3/bin/spark-shell --packages "$packages" > /dev/null \ + && find /root/.ivy2/ -name '*.jar' -exec ln -sf {} /spark-3.3.2-bin-hadoop3/jars/ \; RUN set -x \ && addgroup --system dockremap \ diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 967eaaa78a5..21398790be3 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -624,10 +624,12 @@ class ClickHouseCluster: # if you change packages, don't forget to update them in docker/test/integration/runner/dockerd-entrypoint.sh ( pyspark.sql.SparkSession.builder.appName("spark_test") - .config( - "spark.jars.packages", - "org.apache.hudi:hudi-spark3.3-bundle_2.12:0.13.0,io.delta:delta-core_2.12:2.2.0,org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.1.0", - ) + # The jars are now linked to "$SPARK_HOME/jars" and we don't + # need packages to be downloaded once and once again + # .config( + # "spark.jars.packages", + # "org.apache.hudi:hudi-spark3.3-bundle_2.12:0.13.0,io.delta:delta-core_2.12:2.2.0,org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.1.0", + # ) .master("local") .getOrCreate() .stop() From 4ba334c98360bb24e7d21eb12453ac5cc765d9d8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 29 Jun 2023 13:01:31 +0200 Subject: [PATCH 447/515] Review fix --- src/Disks/getOrCreateDiskFromAST.cpp | 7 ++++++- .../02808_custom_disk_with_user_defined_name.sh | 4 +++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index c5ec0f5d91b..93d70738a82 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -38,7 +38,12 @@ namespace } else { - disk_name = function.name.substr(std::strlen("disk_")); + static constexpr std::string_view custom_disk_prefix = "disk_"; + + if (disk_name.size() <= custom_disk_prefix.size()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid disk name: {}", disk_name); + + disk_name = function.name.substr(custom_disk_prefix.size()); } auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { diff --git a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh index 99b9a0ed7b3..615da606519 100755 --- a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh +++ b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh @@ -1,3 +1,4 @@ +#!/usr/bin/env bash # Tags: no-fasttest, no-parallel # set -x @@ -14,11 +15,12 @@ ENGINE = MergeTree() ORDER BY tuple() SETTINGS disk = disk_s3disk(type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3disk); """ 2>&1 | grep -q "Disk with name \`s3disk\` already exist" && echo 'OK' || echo 'FAIL' +disk_name="${CLICKHOUSE_TEST_UNIQUE_NAME}" + $CLICKHOUSE_CLIENT -nm --query """ SELECT count() FROM system.disks WHERE name = '$disk_name' """ -disk_name="${CLICKHOUSE_TEST_UNIQUE_NAME}" $CLICKHOUSE_CLIENT -nm --query """ DROP TABLE IF EXISTS test; CREATE TABLE test (a Int32, b String) From 306ca66eb4d3e9ae8ee74004a09ab1d530f46658 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 29 Jun 2023 13:11:22 +0200 Subject: [PATCH 448/515] Fix --- src/Interpreters/Cache/FileCache.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index eb5b59a447d..e165c0914b6 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -757,12 +757,14 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) chassert(candidate->releasable()); const auto * segment = candidate->file_segment.get(); + auto queue_it = segment->getQueueIterator(); + chassert(queue_it); ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->range().size()); locked_key->removeFileSegment(segment->offset(), segment->lock()); - segment->getQueueIterator()->remove(cache_lock); + queue_it->remove(cache_lock); if (query_context) query_context->remove(current_key, segment->offset(), cache_lock); From ff25b72e6da94acb2af9214150131b0c4a8aad88 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 29 Jun 2023 13:30:53 +0200 Subject: [PATCH 449/515] Update 02808_custom_disk_with_user_defined_name.sh --- .../0_stateless/02808_custom_disk_with_user_defined_name.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh index 615da606519..50dee04f6a6 100755 --- a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh +++ b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel +# Tags: no-fasttest # set -x From f73d8786329237ad24f06505dcde23485a83a534 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 29 Jun 2023 13:31:41 +0200 Subject: [PATCH 450/515] Update src/Disks/getOrCreateDiskFromAST.cpp Co-authored-by: Antonio Andelic --- src/Disks/getOrCreateDiskFromAST.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index 93d70738a82..691a51d8b48 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -40,7 +40,7 @@ namespace { static constexpr std::string_view custom_disk_prefix = "disk_"; - if (disk_name.size() <= custom_disk_prefix.size()) + if (disk_name.size() <= custom_disk_prefix.size() || !disk_name.starts_with(custom_disk_prefix)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid disk name: {}", disk_name); disk_name = function.name.substr(custom_disk_prefix.size()); From 1d9d712984df7d63bbcaf232b73dde562619ba55 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 29 Jun 2023 13:32:17 +0200 Subject: [PATCH 451/515] use timeout instead trap in 01443_merge_truncate_long.sh --- .../0_stateless/01443_merge_truncate_long.sh | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01443_merge_truncate_long.sh b/tests/queries/0_stateless/01443_merge_truncate_long.sh index ce867d32c1a..65b9bcd366e 100755 --- a/tests/queries/0_stateless/01443_merge_truncate_long.sh +++ b/tests/queries/0_stateless/01443_merge_truncate_long.sh @@ -13,20 +13,20 @@ ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS t" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE t (x Int8) ENGINE = MergeTree ORDER BY ()" -function thread() +function thread_optimize() { - trap 'BREAK=1' 2 - - while [[ -z "${BREAK}" ]] + while true; do ${CLICKHOUSE_CLIENT} --query="OPTIMIZE TABLE t FINAL;" 2>&1 | tr -d '\n' | rg -v 'Cancelled merging parts' ||: done } -thread & -pid=$! +TIMEOUT=15 +export -f thread_optimize +timeout $TIMEOUT bash -c thread_optimize 2> /dev/null & -for i in {1..100}; do +for i in {1..100}; +do echo " INSERT INTO t VALUES (0); INSERT INTO t VALUES (0); @@ -36,7 +36,6 @@ for i in {1..100}; do " done | ${CLICKHOUSE_CLIENT} --multiquery -kill -2 "$pid" wait $CLICKHOUSE_CLIENT -q "DROP TABLE t" From b56d1602e841663f073c6f53a06e848b89c9214e Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 29 Jun 2023 14:25:13 +0200 Subject: [PATCH 452/515] Invalidate opened file cache when removing file --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- src/IO/OpenedFileCache.h | 9 +++++++-- src/Interpreters/Cache/FileSegment.h | 2 ++ src/Interpreters/Cache/Metadata.cpp | 5 +++++ 4 files changed, 15 insertions(+), 3 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 76d54f9d27c..e4dadb4eb21 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -160,7 +160,7 @@ CachedOnDiskReadBufferFromFile::getCacheReadBuffer(const FileSegment & file_segm if (use_external_buffer) local_read_settings.local_fs_buffer_size = 0; - auto buf = createReadBufferFromFileBase(path, local_read_settings); + auto buf = createReadBufferFromFileBase(path, local_read_settings, std::nullopt, std::nullopt, file_segment.getFlagsForLocalRead()); if (getFileSizeFromReadBuffer(*buf) == 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read from an empty cache file: {}", path); diff --git a/src/IO/OpenedFileCache.h b/src/IO/OpenedFileCache.h index 844e5b31d11..61e502a494b 100644 --- a/src/IO/OpenedFileCache.h +++ b/src/IO/OpenedFileCache.h @@ -72,6 +72,13 @@ public: return res; } + void remove(const std::string & path, int flags) + { + Key key(path, flags); + std::lock_guard lock(mutex); + files.erase(key); + } + static OpenedFileCache & instance() { static OpenedFileCache res; @@ -82,5 +89,3 @@ public: using OpenedFileCachePtr = std::shared_ptr; } - - diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 681c0d719e4..b8fa9486472 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -161,6 +161,8 @@ public: String getPathInLocalCache() const; + int getFlagsForLocalRead() const { return O_RDONLY | O_CLOEXEC; } + /** * ========== Methods for _any_ file segment's owner ======================== */ diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index f799bae1e10..f201455384b 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -406,6 +406,11 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegm if (exists) { fs::remove(path); + + int flags = file_segment->getFlagsForLocalRead(); + OpenedFileCache::instance().remove(path, flags); + OpenedFileCache::instance().remove(path, flags | O_DIRECT); + LOG_TEST(key_metadata->log, "Removed file segment at path: {}", path); } else if (file_segment->downloaded_size) From bac126fe3edbaec15834ba21ae57b3aec0f3d299 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 29 Jun 2023 15:11:02 +0200 Subject: [PATCH 453/515] fix --- src/Storages/StorageReplicatedMergeTree.cpp | 11 ++++------- tests/integration/test_merge_tree_hdfs/test.py | 1 - tests/integration/test_merge_tree_s3/test.py | 1 - 3 files changed, 4 insertions(+), 9 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0361d46612b..a967bf3c4c4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7301,7 +7301,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom( UInt64 index = lock->getNumber(); MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - HardlinkedFiles hardlinked_files; bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; @@ -7377,8 +7376,8 @@ void StorageReplicatedMergeTree::replacePartitionFrom( renameTempPartAndReplaceUnlocked(part, transaction, data_parts_lock); } - for (size_t i = 0; i < dst_parts.size(); ++i) - lockSharedData(*dst_parts[i], false, /*hardlinked_files*/ {}); + for (const auto & dst_part : dst_parts) + lockSharedData(*dst_part, false, /*hardlinked_files*/ {}); Coordination::Error code = zookeeper->tryMulti(ops, op_results); if (code == Coordination::Error::ZOK) @@ -7541,8 +7540,6 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta UInt64 index = lock->getNumber(); MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - HardlinkedFiles hardlinked_files; - bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication || dynamic_cast(dest_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; IDataPartStorage::ClonePartParams clone_params @@ -7625,8 +7622,8 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta for (auto & part : dst_parts) dest_table_storage->renameTempPartAndReplaceUnlocked(part, transaction, dest_data_parts_lock); - for (size_t i = 0; i < dst_parts.size(); ++i) - dest_table_storage->lockSharedData(*dst_parts[i], false, /*hardlinked_files*/ {}); + for (const auto & dst_part : dst_parts) + dest_table_storage->lockSharedData(*dst_part, false, /*hardlinked_files*/ {}); Coordination::Error code = zookeeper->tryMulti(ops, op_results); if (code == Coordination::Error::ZBADVERSION) diff --git a/tests/integration/test_merge_tree_hdfs/test.py b/tests/integration/test_merge_tree_hdfs/test.py index d1a145c00c1..ae3e2bde836 100644 --- a/tests/integration/test_merge_tree_hdfs/test.py +++ b/tests/integration/test_merge_tree_hdfs/test.py @@ -239,7 +239,6 @@ def test_attach_detach_partition(cluster): len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2 - - FILES_OVERHEAD_METADATA_VERSION ) node.query("ALTER TABLE hdfs_test DROP PARTITION '2020-01-03'") diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 303fff66166..d0be8a11a06 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -338,7 +338,6 @@ def test_attach_detach_partition(cluster, node_name): len(list_objects(cluster, "data/")) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2 - - FILES_OVERHEAD_METADATA_VERSION ) node.query("ALTER TABLE s3_test DROP PARTITION '2020-01-03'") From bbd4159beeca878616546b77a3dd36b3a9a391d6 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 29 Jun 2023 13:26:21 +0000 Subject: [PATCH 454/515] Automatic style fix --- tests/integration/test_merge_tree_hdfs/test.py | 6 +----- tests/integration/test_merge_tree_s3/test.py | 3 +-- 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_merge_tree_hdfs/test.py b/tests/integration/test_merge_tree_hdfs/test.py index ae3e2bde836..95b63a5c8a3 100644 --- a/tests/integration/test_merge_tree_hdfs/test.py +++ b/tests/integration/test_merge_tree_hdfs/test.py @@ -235,11 +235,7 @@ def test_attach_detach_partition(cluster): assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(8192)" hdfs_objects = fs.listdir("/clickhouse") - assert ( - len(hdfs_objects) - == FILES_OVERHEAD - + FILES_OVERHEAD_PER_PART_WIDE * 2 - ) + assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2 node.query("ALTER TABLE hdfs_test DROP PARTITION '2020-01-03'") assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(4096)" diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index d0be8a11a06..f754bc905bf 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -336,8 +336,7 @@ def test_attach_detach_partition(cluster, node_name): assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(8192)" assert ( len(list_objects(cluster, "data/")) - == FILES_OVERHEAD - + FILES_OVERHEAD_PER_PART_WIDE * 2 + == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2 ) node.query("ALTER TABLE s3_test DROP PARTITION '2020-01-03'") From 2a4cfeb01bcff1112d6615ddff529286ff024186 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 29 Jun 2023 15:27:40 +0200 Subject: [PATCH 455/515] Add comment --- src/Interpreters/Cache/Metadata.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index f201455384b..d4389e0d0b8 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -407,7 +407,10 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegm { fs::remove(path); + /// Clear OpenedFileCache to avoid reading from incorrect file descriptor. int flags = file_segment->getFlagsForLocalRead(); + /// Files are created with flags from file_segment->getFlagsForLocalRead() + /// plus optionally O_DIRECT is added, depends on query setting, so remove both. OpenedFileCache::instance().remove(path, flags); OpenedFileCache::instance().remove(path, flags | O_DIRECT); From 0cccba62cfdb6ee9c04a6ec0199d46f91df89160 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 25 Jun 2023 16:28:18 +0200 Subject: [PATCH 456/515] Support getHexUIntLowercase() with CityHash_v1_0_2::uint128 parameter. --- base/base/hex.h | 424 ++++++++++-------- src/Common/getHashOfLoadedBinary.cpp | 2 +- src/Compression/CompressedReadBufferBase.cpp | 4 +- .../DistributedAsyncInsertHeader.cpp | 5 +- .../MergeTree/MergeTreeDataPartChecksum.cpp | 14 +- .../PartMetadataManagerWithCache.cpp | 8 +- src/Storages/StorageReplicatedMergeTree.cpp | 3 +- src/Storages/System/StorageSystemParts.cpp | 6 +- .../System/StorageSystemProjectionParts.cpp | 6 +- utils/checksum-for-compressed-block/main.cpp | 2 +- 10 files changed, 267 insertions(+), 207 deletions(-) diff --git a/base/base/hex.h b/base/base/hex.h index b8cf95db893..0780e6159a1 100644 --- a/base/base/hex.h +++ b/base/base/hex.h @@ -4,212 +4,286 @@ #include #include "types.h" -/// Maps 0..15 to 0..9A..F or 0..9a..f correspondingly. +namespace CityHash_v1_0_2 { struct uint128; } -constexpr inline std::string_view hex_digit_to_char_uppercase_table = "0123456789ABCDEF"; -constexpr inline std::string_view hex_digit_to_char_lowercase_table = "0123456789abcdef"; +namespace wide +{ + template + class integer; +} + +namespace impl +{ + /// Maps 0..15 to 0..9A..F or 0..9a..f correspondingly. + constexpr inline std::string_view hex_digit_to_char_uppercase_table = "0123456789ABCDEF"; + constexpr inline std::string_view hex_digit_to_char_lowercase_table = "0123456789abcdef"; + + /// Maps 0..255 to 00..FF or 00..ff correspondingly. + constexpr inline std::string_view hex_byte_to_char_uppercase_table = // + "000102030405060708090A0B0C0D0E0F" + "101112131415161718191A1B1C1D1E1F" + "202122232425262728292A2B2C2D2E2F" + "303132333435363738393A3B3C3D3E3F" + "404142434445464748494A4B4C4D4E4F" + "505152535455565758595A5B5C5D5E5F" + "606162636465666768696A6B6C6D6E6F" + "707172737475767778797A7B7C7D7E7F" + "808182838485868788898A8B8C8D8E8F" + "909192939495969798999A9B9C9D9E9F" + "A0A1A2A3A4A5A6A7A8A9AAABACADAEAF" + "B0B1B2B3B4B5B6B7B8B9BABBBCBDBEBF" + "C0C1C2C3C4C5C6C7C8C9CACBCCCDCECF" + "D0D1D2D3D4D5D6D7D8D9DADBDCDDDEDF" + "E0E1E2E3E4E5E6E7E8E9EAEBECEDEEEF" + "F0F1F2F3F4F5F6F7F8F9FAFBFCFDFEFF"; + + constexpr inline std::string_view hex_byte_to_char_lowercase_table = // + "000102030405060708090a0b0c0d0e0f" + "101112131415161718191a1b1c1d1e1f" + "202122232425262728292a2b2c2d2e2f" + "303132333435363738393a3b3c3d3e3f" + "404142434445464748494a4b4c4d4e4f" + "505152535455565758595a5b5c5d5e5f" + "606162636465666768696a6b6c6d6e6f" + "707172737475767778797a7b7c7d7e7f" + "808182838485868788898a8b8c8d8e8f" + "909192939495969798999a9b9c9d9e9f" + "a0a1a2a3a4a5a6a7a8a9aaabacadaeaf" + "b0b1b2b3b4b5b6b7b8b9babbbcbdbebf" + "c0c1c2c3c4c5c6c7c8c9cacbcccdcecf" + "d0d1d2d3d4d5d6d7d8d9dadbdcdddedf" + "e0e1e2e3e4e5e6e7e8e9eaebecedeeef" + "f0f1f2f3f4f5f6f7f8f9fafbfcfdfeff"; + + /// Maps 0..255 to 00000000..11111111 correspondingly. + constexpr inline std::string_view bin_byte_to_char_table = // + "0000000000000001000000100000001100000100000001010000011000000111" + "0000100000001001000010100000101100001100000011010000111000001111" + "0001000000010001000100100001001100010100000101010001011000010111" + "0001100000011001000110100001101100011100000111010001111000011111" + "0010000000100001001000100010001100100100001001010010011000100111" + "0010100000101001001010100010101100101100001011010010111000101111" + "0011000000110001001100100011001100110100001101010011011000110111" + "0011100000111001001110100011101100111100001111010011111000111111" + "0100000001000001010000100100001101000100010001010100011001000111" + "0100100001001001010010100100101101001100010011010100111001001111" + "0101000001010001010100100101001101010100010101010101011001010111" + "0101100001011001010110100101101101011100010111010101111001011111" + "0110000001100001011000100110001101100100011001010110011001100111" + "0110100001101001011010100110101101101100011011010110111001101111" + "0111000001110001011100100111001101110100011101010111011001110111" + "0111100001111001011110100111101101111100011111010111111001111111" + "1000000010000001100000101000001110000100100001011000011010000111" + "1000100010001001100010101000101110001100100011011000111010001111" + "1001000010010001100100101001001110010100100101011001011010010111" + "1001100010011001100110101001101110011100100111011001111010011111" + "1010000010100001101000101010001110100100101001011010011010100111" + "1010100010101001101010101010101110101100101011011010111010101111" + "1011000010110001101100101011001110110100101101011011011010110111" + "1011100010111001101110101011101110111100101111011011111010111111" + "1100000011000001110000101100001111000100110001011100011011000111" + "1100100011001001110010101100101111001100110011011100111011001111" + "1101000011010001110100101101001111010100110101011101011011010111" + "1101100011011001110110101101101111011100110111011101111011011111" + "1110000011100001111000101110001111100100111001011110011011100111" + "1110100011101001111010101110101111101100111011011110111011101111" + "1111000011110001111100101111001111110100111101011111011011110111" + "1111100011111001111110101111101111111100111111011111111011111111"; + + /// Maps 0..9, A..F, a..f to 0..15. Other chars are mapped to implementation specific value. + constexpr inline std::string_view hex_char_to_digit_table + = {"\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\x00\x01\x02\x03\x04\x05\x06\x07\x08\x09\xff\xff\xff\xff\xff\xff" //0-9 + "\xff\x0a\x0b\x0c\x0d\x0e\x0f\xff\xff\xff\xff\xff\xff\xff\xff\xff" //A-Z + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\x0a\x0b\x0c\x0d\x0e\x0f\xff\xff\xff\xff\xff\xff\xff\xff\xff" //a-z + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff", + 256}; + + /// Converts a hex digit '0'..'f' or '0'..'F' to its value 0..15. + constexpr UInt8 unhexDigit(char c) + { + return hex_char_to_digit_table[static_cast(c)]; + } + + /// Converts an unsigned integer in the native endian to hexadecimal representation and back. Used as a base class for HexConversion. + template + struct HexConversionUInt + { + static const constexpr size_t num_hex_digits = sizeof(TUInt) * 2; + + static void hex(TUInt uint_, char * out, std::string_view table) + { + union + { + TUInt value; + UInt8 uint8[sizeof(TUInt)]; + }; + + value = uint_; + + for (size_t i = 0; i < sizeof(TUInt); ++i) + { + if constexpr (std::endian::native == std::endian::little) + memcpy(out + i * 2, &table[static_cast(uint8[sizeof(TUInt) - 1 - i]) * 2], 2); + else + memcpy(out + i * 2, &table[static_cast(uint8[i]) * 2], 2); + } + } + + static TUInt unhex(const char * data) + { + TUInt res; + if constexpr (sizeof(TUInt) == 1) + { + res = static_cast(unhexDigit(data[0])) * 0x10 + static_cast(unhexDigit(data[1])); + } + else if constexpr (sizeof(TUInt) == 2) + { + res = static_cast(unhexDigit(data[0])) * 0x1000 + static_cast(unhexDigit(data[1])) * 0x100 + + static_cast(unhexDigit(data[2])) * 0x10 + static_cast(unhexDigit(data[3])); + } + else if constexpr ((sizeof(TUInt) <= 8) || ((sizeof(TUInt) % 8) != 0)) + { + res = 0; + for (size_t i = 0; i < sizeof(TUInt) * 2; ++i, ++data) + { + res <<= 4; + res += unhexDigit(*data); + } + } + else + { + res = 0; + for (size_t i = 0; i < sizeof(TUInt) / 8; ++i, data += 16) + { + res <<= 64; + res += HexConversionUInt::unhex(data); + } + } + return res; + } + }; + + /// Helper template class to convert a value of any supported type to hexadecimal representation and back. + template + struct HexConversion; + + template + struct HexConversion>> : public HexConversionUInt {}; + + template + struct HexConversion> : public HexConversionUInt> {}; + + template + struct HexConversion>> + { + static const constexpr size_t num_hex_digits = 32; + + static void hex(const CityHashUInt128 & uint_, char * out, std::string_view table) + { + HexConversion::hex(uint_.high64, out, table); + HexConversion::hex(uint_.low64, out + 16, table); + } + + static CityHashUInt128 unhex(const char * data) + { + CityHashUInt128 res; + res.high64 = HexConversion::unhex(data); + res.low64 = HexConversion::unhex(data + 16); + return res; + } + }; +} + +/// Produces a hexadecimal representation of an integer value with leading zeros (for checksums). +/// The function supports native integer types, wide::integer, CityHash_v1_0_2::uint128. +template +void writeHexUIntUppercase(const T & value, char * out) +{ + impl::HexConversion::hex(value, out, impl::hex_byte_to_char_uppercase_table); +} + +template +void writeHexUIntLowercase(const T & value, char * out) +{ + impl::HexConversion::hex(value, out, impl::hex_byte_to_char_lowercase_table); +} + +template +std::string getHexUIntUppercase(const T & value) +{ + std::string res(impl::HexConversion::num_hex_digits, '\0'); + writeHexUIntUppercase(value, res.data()); + return res; +} + +template +std::string getHexUIntLowercase(const T & value) +{ + std::string res(impl::HexConversion::num_hex_digits, '\0'); + writeHexUIntLowercase(value, res.data()); + return res; +} constexpr char hexDigitUppercase(unsigned char c) { - return hex_digit_to_char_uppercase_table[c]; + return impl::hex_digit_to_char_uppercase_table[c]; } + constexpr char hexDigitLowercase(unsigned char c) { - return hex_digit_to_char_lowercase_table[c]; + return impl::hex_digit_to_char_lowercase_table[c]; } -/// Maps 0..255 to 00..FF or 00..ff correspondingly - -constexpr inline std::string_view hex_byte_to_char_uppercase_table = // - "000102030405060708090A0B0C0D0E0F" - "101112131415161718191A1B1C1D1E1F" - "202122232425262728292A2B2C2D2E2F" - "303132333435363738393A3B3C3D3E3F" - "404142434445464748494A4B4C4D4E4F" - "505152535455565758595A5B5C5D5E5F" - "606162636465666768696A6B6C6D6E6F" - "707172737475767778797A7B7C7D7E7F" - "808182838485868788898A8B8C8D8E8F" - "909192939495969798999A9B9C9D9E9F" - "A0A1A2A3A4A5A6A7A8A9AAABACADAEAF" - "B0B1B2B3B4B5B6B7B8B9BABBBCBDBEBF" - "C0C1C2C3C4C5C6C7C8C9CACBCCCDCECF" - "D0D1D2D3D4D5D6D7D8D9DADBDCDDDEDF" - "E0E1E2E3E4E5E6E7E8E9EAEBECEDEEEF" - "F0F1F2F3F4F5F6F7F8F9FAFBFCFDFEFF"; - -constexpr inline std::string_view hex_byte_to_char_lowercase_table = // - "000102030405060708090a0b0c0d0e0f" - "101112131415161718191a1b1c1d1e1f" - "202122232425262728292a2b2c2d2e2f" - "303132333435363738393a3b3c3d3e3f" - "404142434445464748494a4b4c4d4e4f" - "505152535455565758595a5b5c5d5e5f" - "606162636465666768696a6b6c6d6e6f" - "707172737475767778797a7b7c7d7e7f" - "808182838485868788898a8b8c8d8e8f" - "909192939495969798999a9b9c9d9e9f" - "a0a1a2a3a4a5a6a7a8a9aaabacadaeaf" - "b0b1b2b3b4b5b6b7b8b9babbbcbdbebf" - "c0c1c2c3c4c5c6c7c8c9cacbcccdcecf" - "d0d1d2d3d4d5d6d7d8d9dadbdcdddedf" - "e0e1e2e3e4e5e6e7e8e9eaebecedeeef" - "f0f1f2f3f4f5f6f7f8f9fafbfcfdfeff"; - inline void writeHexByteUppercase(UInt8 byte, void * out) { - memcpy(out, &hex_byte_to_char_uppercase_table[static_cast(byte) * 2], 2); + memcpy(out, &impl::hex_byte_to_char_uppercase_table[static_cast(byte) * 2], 2); } inline void writeHexByteLowercase(UInt8 byte, void * out) { - memcpy(out, &hex_byte_to_char_lowercase_table[static_cast(byte) * 2], 2); + memcpy(out, &impl::hex_byte_to_char_lowercase_table[static_cast(byte) * 2], 2); } -constexpr inline std::string_view bin_byte_to_char_table = // - "0000000000000001000000100000001100000100000001010000011000000111" - "0000100000001001000010100000101100001100000011010000111000001111" - "0001000000010001000100100001001100010100000101010001011000010111" - "0001100000011001000110100001101100011100000111010001111000011111" - "0010000000100001001000100010001100100100001001010010011000100111" - "0010100000101001001010100010101100101100001011010010111000101111" - "0011000000110001001100100011001100110100001101010011011000110111" - "0011100000111001001110100011101100111100001111010011111000111111" - "0100000001000001010000100100001101000100010001010100011001000111" - "0100100001001001010010100100101101001100010011010100111001001111" - "0101000001010001010100100101001101010100010101010101011001010111" - "0101100001011001010110100101101101011100010111010101111001011111" - "0110000001100001011000100110001101100100011001010110011001100111" - "0110100001101001011010100110101101101100011011010110111001101111" - "0111000001110001011100100111001101110100011101010111011001110111" - "0111100001111001011110100111101101111100011111010111111001111111" - "1000000010000001100000101000001110000100100001011000011010000111" - "1000100010001001100010101000101110001100100011011000111010001111" - "1001000010010001100100101001001110010100100101011001011010010111" - "1001100010011001100110101001101110011100100111011001111010011111" - "1010000010100001101000101010001110100100101001011010011010100111" - "1010100010101001101010101010101110101100101011011010111010101111" - "1011000010110001101100101011001110110100101101011011011010110111" - "1011100010111001101110101011101110111100101111011011111010111111" - "1100000011000001110000101100001111000100110001011100011011000111" - "1100100011001001110010101100101111001100110011011100111011001111" - "1101000011010001110100101101001111010100110101011101011011010111" - "1101100011011001110110101101101111011100110111011101111011011111" - "1110000011100001111000101110001111100100111001011110011011100111" - "1110100011101001111010101110101111101100111011011110111011101111" - "1111000011110001111100101111001111110100111101011111011011110111" - "1111100011111001111110101111101111111100111111011111111011111111"; - -inline void writeBinByte(UInt8 byte, void * out) +/// Converts a hex representation with leading zeros back to an integer value. +/// The function supports native integer types, wide::integer, CityHash_v1_0_2::uint128. +template +constexpr T unhexUInt(const char * data) { - memcpy(out, &bin_byte_to_char_table[static_cast(byte) * 8], 8); + return impl::HexConversion::unhex(data); } -/// Produces hex representation of an unsigned int with leading zeros (for checksums) -template -inline void writeHexUIntImpl(TUInt uint_, char * out, std::string_view table) -{ - union - { - TUInt value; - UInt8 uint8[sizeof(TUInt)]; - }; - - value = uint_; - - for (size_t i = 0; i < sizeof(TUInt); ++i) - { - if constexpr (std::endian::native == std::endian::little) - memcpy(out + i * 2, &table[static_cast(uint8[sizeof(TUInt) - 1 - i]) * 2], 2); - else - memcpy(out + i * 2, &table[static_cast(uint8[i]) * 2], 2); - } -} - -template -inline void writeHexUIntUppercase(TUInt uint_, char * out) -{ - writeHexUIntImpl(uint_, out, hex_byte_to_char_uppercase_table); -} - -template -inline void writeHexUIntLowercase(TUInt uint_, char * out) -{ - writeHexUIntImpl(uint_, out, hex_byte_to_char_lowercase_table); -} - -template -std::string getHexUIntUppercase(TUInt uint_) -{ - std::string res(sizeof(TUInt) * 2, '\0'); - writeHexUIntUppercase(uint_, res.data()); - return res; -} - -template -std::string getHexUIntLowercase(TUInt uint_) -{ - std::string res(sizeof(TUInt) * 2, '\0'); - writeHexUIntLowercase(uint_, res.data()); - return res; -} - -/// Maps 0..9, A..F, a..f to 0..15. Other chars are mapped to implementation specific value. - -constexpr inline std::string_view hex_char_to_digit_table - = {"\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\x00\x01\x02\x03\x04\x05\x06\x07\x08\x09\xff\xff\xff\xff\xff\xff" //0-9 - "\xff\x0a\x0b\x0c\x0d\x0e\x0f\xff\xff\xff\xff\xff\xff\xff\xff\xff" //A-Z - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\x0a\x0b\x0c\x0d\x0e\x0f\xff\xff\xff\xff\xff\xff\xff\xff\xff" //a-z - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff", - 256}; - +/// Converts a hexadecimal digit '0'..'f' or '0'..'F' to UInt8. constexpr UInt8 unhex(char c) { - return hex_char_to_digit_table[static_cast(c)]; + return impl::unhexDigit(c); } +/// Converts two hexadecimal digits to UInt8. constexpr UInt8 unhex2(const char * data) { - return static_cast(unhex(data[0])) * 0x10 + static_cast(unhex(data[1])); + return unhexUInt(data); } +/// Converts four hexadecimal digits to UInt16. constexpr UInt16 unhex4(const char * data) { - return static_cast(unhex(data[0])) * 0x1000 + static_cast(unhex(data[1])) * 0x100 - + static_cast(unhex(data[2])) * 0x10 + static_cast(unhex(data[3])); + return unhexUInt(data); } -template -constexpr TUInt unhexUInt(const char * data) +/// Produces a binary representation of a single byte. +inline void writeBinByte(UInt8 byte, void * out) { - TUInt res = 0; - if constexpr ((sizeof(TUInt) <= 8) || ((sizeof(TUInt) % 8) != 0)) - { - for (size_t i = 0; i < sizeof(TUInt) * 2; ++i, ++data) - { - res <<= 4; - res += unhex(*data); - } - } - else - { - for (size_t i = 0; i < sizeof(TUInt) / 8; ++i, data += 16) - { - res <<= 64; - res += unhexUInt(data); - } - } - return res; + memcpy(out, &impl::bin_byte_to_char_table[static_cast(byte) * 8], 8); } diff --git a/src/Common/getHashOfLoadedBinary.cpp b/src/Common/getHashOfLoadedBinary.cpp index cc0ad0d2143..b81300b8536 100644 --- a/src/Common/getHashOfLoadedBinary.cpp +++ b/src/Common/getHashOfLoadedBinary.cpp @@ -37,7 +37,7 @@ SipHash getHashOfLoadedBinary() std::string getHashOfLoadedBinaryHex() { SipHash hash = getHashOfLoadedBinary(); - std::array checksum; + UInt128 checksum; hash.get128(checksum); return getHexUIntUppercase(checksum); } diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index bae52c8bece..dd19955d010 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -49,8 +49,8 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c /// TODO mess up of endianness in error message. message << "Checksum doesn't match: corrupted data." - " Reference: " + getHexUIntLowercase(expected_checksum.high64) + getHexUIntLowercase(expected_checksum.low64) - + ". Actual: " + getHexUIntLowercase(calculated_checksum.high64) + getHexUIntLowercase(calculated_checksum.low64) + " Reference: " + getHexUIntLowercase(expected_checksum) + + ". Actual: " + getHexUIntLowercase(calculated_checksum) + ". Size of compressed block: " + toString(size); const char * message_hardware_failure = "This is most likely due to hardware failure. " diff --git a/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp b/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp index e1b54304f23..a8ed89e66f1 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp @@ -39,9 +39,8 @@ DistributedAsyncInsertHeader DistributedAsyncInsertHeader::read(ReadBufferFromFi if (expected_checksum != calculated_checksum) { throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, - "Checksum of extra info doesn't match: corrupted data. Reference: {}{}. Actual: {}{}.", - getHexUIntLowercase(expected_checksum.high64), getHexUIntLowercase(expected_checksum.low64), - getHexUIntLowercase(calculated_checksum.high64), getHexUIntLowercase(calculated_checksum.low64)); + "Checksum of extra info doesn't match: corrupted data. Reference: {}. Actual: {}.", + getHexUIntLowercase(expected_checksum), getHexUIntLowercase(calculated_checksum)); } /// Read the parts of the header. diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index 58ba7acb9ba..6628cd68eaf 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -307,19 +307,7 @@ static void updateHash(SipHash & hash, const std::string & data) /// Hash is the same as MinimalisticDataPartChecksums::hash_of_all_files String MergeTreeDataPartChecksums::getTotalChecksumHex() const { - SipHash hash_of_all_files; - - for (const auto & [name, checksum] : files) - { - updateHash(hash_of_all_files, name); - hash_of_all_files.update(checksum.file_hash); - } - - UInt64 lo; - UInt64 hi; - hash_of_all_files.get128(lo, hi); - - return getHexUIntUppercase(hi) + getHexUIntUppercase(lo); + return getHexUIntUppercase(getTotalChecksumUInt128()); } MergeTreeDataPartChecksums::Checksum::uint128 MergeTreeDataPartChecksums::getTotalChecksumUInt128() const diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp index 324bd4bbaee..bb6462b3058 100644 --- a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp @@ -250,8 +250,8 @@ std::unordered_map PartMetadataManagerWit ErrorCodes::CORRUPTED_DATA, "Checksums doesn't match in part {} for {}. Expected: {}. Found {}.", part->name, file_path, - getHexUIntUppercase(disk_checksum.high64) + getHexUIntUppercase(disk_checksum.low64), - getHexUIntUppercase(cache_checksums[i].high64) + getHexUIntUppercase(cache_checksums[i].low64)); + getHexUIntUppercase(disk_checksum), + getHexUIntUppercase(cache_checksums[i])); disk_checksums.push_back(disk_checksum); continue; @@ -287,8 +287,8 @@ std::unordered_map PartMetadataManagerWit ErrorCodes::CORRUPTED_DATA, "Checksums doesn't match in projection part {} {}. Expected: {}. Found {}.", part->name, proj_name, - getHexUIntUppercase(disk_checksum.high64) + getHexUIntUppercase(disk_checksum.low64), - getHexUIntUppercase(cache_checksums[i].high64) + getHexUIntUppercase(cache_checksums[i].low64)); + getHexUIntUppercase(disk_checksum), + getHexUIntUppercase(cache_checksums[i])); disk_checksums.push_back(disk_checksum); } return results; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b1ba06c77f9..a2138efdf56 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2891,8 +2891,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo desired_checksums = MinimalisticDataPartChecksums::deserializeFrom(desired_checksums_str); } - const auto [lo, hi] = desired_checksums.hash_of_all_files; - log_entry.part_checksum = getHexUIntUppercase(hi) + getHexUIntUppercase(lo); + log_entry.part_checksum = getHexUIntUppercase(desired_checksums.hash_of_all_files); } else { diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index b642f4b5088..7399bd789a7 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -252,17 +252,17 @@ void StorageSystemParts::processNextStorage( if (columns_mask[src_index++]) { auto checksum = helper.hash_of_all_files; - columns[res_index++]->insert(getHexUIntLowercase(checksum.high64) + getHexUIntLowercase(checksum.low64)); + columns[res_index++]->insert(getHexUIntLowercase(checksum)); } if (columns_mask[src_index++]) { auto checksum = helper.hash_of_uncompressed_files; - columns[res_index++]->insert(getHexUIntLowercase(checksum.high64) + getHexUIntLowercase(checksum.low64)); + columns[res_index++]->insert(getHexUIntLowercase(checksum)); } if (columns_mask[src_index++]) { auto checksum = helper.uncompressed_hash_of_compressed_files; - columns[res_index++]->insert(getHexUIntLowercase(checksum.high64) + getHexUIntLowercase(checksum.low64)); + columns[res_index++]->insert(getHexUIntLowercase(checksum)); } } diff --git a/src/Storages/System/StorageSystemProjectionParts.cpp b/src/Storages/System/StorageSystemProjectionParts.cpp index 05c83747c4d..213865a8d61 100644 --- a/src/Storages/System/StorageSystemProjectionParts.cpp +++ b/src/Storages/System/StorageSystemProjectionParts.cpp @@ -221,17 +221,17 @@ void StorageSystemProjectionParts::processNextStorage( if (columns_mask[src_index++]) { auto checksum = helper.hash_of_all_files; - columns[res_index++]->insert(getHexUIntLowercase(checksum.high64) + getHexUIntLowercase(checksum.low64)); + columns[res_index++]->insert(getHexUIntLowercase(checksum)); } if (columns_mask[src_index++]) { auto checksum = helper.hash_of_uncompressed_files; - columns[res_index++]->insert(getHexUIntLowercase(checksum.high64) + getHexUIntLowercase(checksum.low64)); + columns[res_index++]->insert(getHexUIntLowercase(checksum)); } if (columns_mask[src_index++]) { auto checksum = helper.uncompressed_hash_of_compressed_files; - columns[res_index++]->insert(getHexUIntLowercase(checksum.high64) + getHexUIntLowercase(checksum.low64)); + columns[res_index++]->insert(getHexUIntLowercase(checksum)); } } diff --git a/utils/checksum-for-compressed-block/main.cpp b/utils/checksum-for-compressed-block/main.cpp index d30a3798820..4ae06a78ab4 100644 --- a/utils/checksum-for-compressed-block/main.cpp +++ b/utils/checksum-for-compressed-block/main.cpp @@ -45,7 +45,7 @@ int main(int, char **) { auto flipped = flipBit(str, pos); auto checksum = CityHash_v1_0_2::CityHash128(flipped.data(), flipped.size()); - std::cout << getHexUIntLowercase(checksum.high64) << getHexUIntLowercase(checksum.low64) << "\t" << pos / 8 << ", " << pos % 8 << "\n"; + std::cout << getHexUIntLowercase(checksum) << "\t" << pos / 8 << ", " << pos % 8 << "\n"; } return 0; From 5656d18690bd00b0db48c1e12e48e0446e263465 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 29 Jun 2023 13:36:55 +0000 Subject: [PATCH 457/515] Update version_date.tsv and changelogs after v23.5.4.25-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.5.4.25-stable.md | 31 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 5 files changed, 35 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.5.4.25-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 367f6043b90..6d53a6f4c51 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ esac ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release" -ARG VERSION="23.5.3.24" +ARG VERSION="23.5.4.25" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index e7e879fa95f..91b22346f13 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.5.3.24" +ARG VERSION="23.5.4.25" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 42ae81655d2..0ed0e4e1168 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -23,7 +23,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.5.3.24" +ARG VERSION="23.5.4.25" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.5.4.25-stable.md b/docs/changelogs/v23.5.4.25-stable.md new file mode 100644 index 00000000000..53d3a7c9c0a --- /dev/null +++ b/docs/changelogs/v23.5.4.25-stable.md @@ -0,0 +1,31 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.5.4.25-stable (190f962abcf) FIXME as compared to v23.5.3.24-stable (76f54616d3b) + +#### Improvement +* Backported in [#51235](https://github.com/ClickHouse/ClickHouse/issues/51235): Improve the progress bar for file/s3/hdfs/url table functions by using chunk size from source data and using incremental total size counting in each thread. Fix the progress bar for *Cluster functions. This closes [#47250](https://github.com/ClickHouse/ClickHouse/issues/47250). [#51088](https://github.com/ClickHouse/ClickHouse/pull/51088) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#51255](https://github.com/ClickHouse/ClickHouse/issues/51255): Disable cache setting `do_not_evict_index_and_mark_files` (Was enabled in `23.5`). [#51222](https://github.com/ClickHouse/ClickHouse/pull/51222) ([Kseniia Sumarokova](https://github.com/kssenii)). + +#### Build/Testing/Packaging Improvement +* Backported in [#51531](https://github.com/ClickHouse/ClickHouse/issues/51531): Split huge `RUN` in Dockerfile into smaller conditional. Install the necessary tools on demand in the same `RUN` layer, and remove them after that. Upgrade the OS only once at the beginning. Use a modern way to check the signed repository. Downgrade the base repo to ubuntu:20.04 to address the issues on older docker versions. Upgrade golang version to address golang vulnerabilities. [#51504](https://github.com/ClickHouse/ClickHouse/pull/51504) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#51572](https://github.com/ClickHouse/ClickHouse/issues/51572): This a follow-up for [#51504](https://github.com/ClickHouse/ClickHouse/issues/51504), the cleanup was lost during refactoring. [#51564](https://github.com/ClickHouse/ClickHouse/pull/51564) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Query Cache: Try to fix bad cast from ColumnConst to ColumnVector [#50704](https://github.com/ClickHouse/ClickHouse/pull/50704) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix type of LDAP server params hash in cache entry [#50865](https://github.com/ClickHouse/ClickHouse/pull/50865) ([Julian Maicher](https://github.com/jmaicher)). +* Fallback to parsing big integer from String instead of exception in Parquet format [#50873](https://github.com/ClickHouse/ClickHouse/pull/50873) ([Kruglov Pavel](https://github.com/Avogar)). +* Do not apply projection if read-in-order was enabled. [#50923](https://github.com/ClickHouse/ClickHouse/pull/50923) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix race azure blob storage iterator [#50936](https://github.com/ClickHouse/ClickHouse/pull/50936) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix ineffective query cache for SELECTs with subqueries [#51132](https://github.com/ClickHouse/ClickHouse/pull/51132) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix fuzzer failure in ActionsDAG [#51301](https://github.com/ClickHouse/ClickHouse/pull/51301) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Fix ParallelReadBuffer seek [#50820](https://github.com/ClickHouse/ClickHouse/pull/50820) ([Michael Kolupaev](https://github.com/al13n321)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 307ed97068f..5c8dd0d2481 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.5.4.25-stable 2023-06-29 v23.5.3.24-stable 2023-06-17 v23.5.2.7-stable 2023-06-10 v23.5.1.3174-stable 2023-06-09 From 6ec85f9faa548e50ec1a6b4a2d7868c9f7e0079a Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 29 Jun 2023 11:06:28 -0300 Subject: [PATCH 458/515] Update settings.md --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index cff65e049f3..cff13302cdc 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1322,7 +1322,7 @@ Connection pool size for PostgreSQL table engine and database engine. Default value: 16 -## postgresql_connection_pool_size {#postgresql-connection-pool-size} +## postgresql_connection_pool_wait_timeout {#postgresql-connection-pool-wait-timeout} Connection pool push/pop timeout on empty pool for PostgreSQL table engine and database engine. By default it will block on empty pool. From d1bb8f24c0f99ad4af37747306e06691b34c8955 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 29 Jun 2023 08:55:06 +0000 Subject: [PATCH 459/515] Retry chroot creation in ZK --- docker/test/stateless/run.sh | 27 ++++++++++++++++++++++++++- 1 file changed, 26 insertions(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 21cb3168083..914e51a9f66 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -90,6 +90,30 @@ sleep 5 attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01 +function run_with_retry() +{ + set +e + + local total_retries="$1" + shift + + local retry=0 + + until [ "$retry" -ge "$total_retries" ] + do + if "$@"; then + set -e + return + else + retry=$((retry + 1)) + sleep 3 + fi + done + + echo "Command '$*' failed after $total_retries retries, exiting" + exit 1 +} + function run_tests() { set -x @@ -138,7 +162,8 @@ function run_tests() ADDITIONAL_OPTIONS+=('--report-logs-stats') clickhouse-test "00001_select_1" > /dev/null ||: - clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')" ||: + + run_with_retry 5 clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')" set +e clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ From 65ecf79a0ffedf142902f5ece9db6f9d9954496c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 29 Jun 2023 17:52:59 +0200 Subject: [PATCH 460/515] Update getOrCreateDiskFromAST.cpp --- src/Disks/getOrCreateDiskFromAST.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index 691a51d8b48..81d5b7372f3 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -40,8 +40,8 @@ namespace { static constexpr std::string_view custom_disk_prefix = "disk_"; - if (disk_name.size() <= custom_disk_prefix.size() || !disk_name.starts_with(custom_disk_prefix)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid disk name: {}", disk_name); + if (function.name.size() <= custom_disk_prefix.size() || !function.name.starts_with(custom_disk_prefix)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid disk name: {}", function.name); disk_name = function.name.substr(custom_disk_prefix.size()); } From 4ee094cab1a4aed4091679e54e6ac146b2423693 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 29 Jun 2023 12:13:26 -0400 Subject: [PATCH 461/515] review comments --- .../table-engines/mergetree-family/mergetree.md | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 1b7f3263ab9..a1c2fbdbe50 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -758,12 +758,14 @@ If you perform the `SELECT` query between merges, you may get expired data. To a ## Disk types -In addition to local block devices, ClickHouse supports other device types through table engines. These are the types: -- [S3](#table_engine-mergetree-s3) -- GCS (also supported using the [S3 table engine](#table_engine-mergetree-s3)) -- [Azure Blob Storage](#table_engine-mergetree-azure-blob-storage) -- [HDFS](#hdfs-storage) -- [Web (read-only)](#web-storage) +In addition to local block devices, ClickHouse supports these storage types: +- [`s3` for S3 and MinIO](#table_engine-mergetree-s3) +- [`gcs` for GCS](/docs/en/integrations/data-ingestion/gcs/index.md/#creating-a-disk) +- [`blob_storage_disk` for Azure Blob Storage](#table_engine-mergetree-azure-blob-storage) +- [`hdfs` for HDFS](#hdfs-storage) +- [`web` for read-only from web](#web-storage) +- [`cache` for local caching](/docs/en/operations/storing-data.md/#using-local-cache) +- [`s3_plain` for backups to S3](/docs/en/operations/backup#backuprestore-using-an-s3-disk) ## Using Multiple Block Devices for Data Storage {#table_engine-mergetree-multiple-volumes} From a28728b7e537f26faf0c4785fc9fb96218178594 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 29 Jun 2023 17:40:10 +0300 Subject: [PATCH 462/515] Update DatabaseReplicatedWorker.cpp (cherry picked from commit 43fc1af1b3855652536037cc2b5111f5ae1ac983) --- src/Databases/DatabaseReplicatedWorker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 593d0655777..4976f54e417 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -161,7 +161,7 @@ bool DatabaseReplicatedDDLWorker::waitForReplicaToProcessAllEntries(UInt64 timeo LOG_TRACE(log, "Waiting for worker thread to process all entries before {}, current task is {}", max_log, current_task); bool processed = wait_current_task_change.wait_for(lock, std::chrono::milliseconds(timeout_ms), [&]() { - return zookeeper->expired() || current_task == max_log || stop_flag; + return zookeeper->expired() || current_task >= max_log || stop_flag; }); if (!processed) From 7d416cbc60c89ce34e03e7e9de5045e0120af119 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 29 Jun 2023 13:39:39 -0400 Subject: [PATCH 463/515] add exception docs for hasAll --- .../sql-reference/functions/array-functions.md | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index bdd1445c990..7f2b8f3c605 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -230,13 +230,15 @@ hasAll(set, subset) **Arguments** - `set` – Array of any type with a set of elements. -- `subset` – Array of any type with elements that should be tested to be a subset of `set`. +- `subset` – Array of any type that shares a common supertype with `set` containing elements that should be tested to be a subset of `set`. **Return values** - `1`, if `set` contains all of the elements from `subset`. - `0`, otherwise. +Raises an exception `NO_COMMON_TYPE` if the set and subset elements do not share a common supertype. + **Peculiar properties** - An empty array is a subset of any array. @@ -253,7 +255,7 @@ hasAll(set, subset) `SELECT hasAll(['a', 'b'], ['a'])` returns 1. -`SELECT hasAll([1], ['a'])` returns 0. +`SELECT hasAll([1], ['a'])` raises a `NO_COMMON_TYPE` exception. `SELECT hasAll([[1, 2], [3, 4]], [[1, 2], [3, 5]])` returns 0. @@ -268,13 +270,15 @@ hasAny(array1, array2) **Arguments** - `array1` – Array of any type with a set of elements. -- `array2` – Array of any type with a set of elements. +- `array2` – Array of any type that shares a common supertype with `array1`. **Return values** - `1`, if `array1` and `array2` have one similar element at least. - `0`, otherwise. +Raises an exception `NO_COMMON_TYPE` if the array1 and array2 elements do not share a common supertype. + **Peculiar properties** - `Null` processed as a value. @@ -288,7 +292,7 @@ hasAny(array1, array2) `SELECT hasAny([-128, 1., 512], [1])` returns `1`. -`SELECT hasAny([[1, 2], [3, 4]], ['a', 'c'])` returns `0`. +`SELECT hasAny([[1, 2], [3, 4]], ['a', 'c'])` raises a `NO_COMMON_TYPE` exception. `SELECT hasAll([[1, 2], [3, 4]], [[1, 2], [1, 2]])` returns `1`. @@ -318,6 +322,8 @@ For Example: - `1`, if `array1` contains `array2`. - `0`, otherwise. +Raises an exception `NO_COMMON_TYPE` if the array1 and array2 elements do not share a common supertype. + **Peculiar properties** - The function will return `1` if `array2` is empty. @@ -339,6 +345,9 @@ For Example: `SELECT hasSubstr(['a', 'b' , 'c'], ['a', 'c'])` returns 0. `SELECT hasSubstr([[1, 2], [3, 4], [5, 6]], [[1, 2], [3, 4]])` returns 1. +i +`SELECT hasSubstr([1, 2, NULL, 3, 4], ['a'])` raises a `NO_COMMON_TYPE` exception. + ## indexOf(arr, x) From a8172ca5d2557673d370de4f4551f756ff0722a7 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 29 Jun 2023 14:47:23 -0400 Subject: [PATCH 464/515] update spelling list --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 526e674a154..f25d082e5a6 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2264,6 +2264,7 @@ summap summingmergetree sumwithoverflow superaggregates +supertype supremum symlink symlinks From 90129b92c1dd47c0aa86cd5ed7c8107758a74b51 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 29 Jun 2023 21:19:41 +0200 Subject: [PATCH 465/515] Update 23.6 changelog --- CHANGELOG.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a2e7b021081..c1e0dba4465 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -21,7 +21,7 @@ * Added `Overlay` database engine to combine multiple databases into one. Added `Filesystem` database engine to represent a directory in the filesystem as a set of implicitly available tables with auto-detected formats and structures. A new `S3` database engine allows to read-only interact with s3 storage by representing a prefix as a set of tables. A new `HDFS` database engine allows to interact with HDFS storage in the same way. [#48821](https://github.com/ClickHouse/ClickHouse/pull/48821) ([alekseygolub](https://github.com/alekseygolub)). * The function `transform` as well as `CASE` with value matching started to support all data types. This closes [#29730](https://github.com/ClickHouse/ClickHouse/issues/29730). This closes [#32387](https://github.com/ClickHouse/ClickHouse/issues/32387). This closes [#50827](https://github.com/ClickHouse/ClickHouse/issues/50827). This closes [#31336](https://github.com/ClickHouse/ClickHouse/issues/31336). This closes [#40493](https://github.com/ClickHouse/ClickHouse/issues/40493). [#51351](https://github.com/ClickHouse/ClickHouse/pull/51351) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Added option `--rename_files_after_processing `. This closes [#34207](https://github.com/ClickHouse/ClickHouse/issues/34207). [#49626](https://github.com/ClickHouse/ClickHouse/pull/49626) ([alekseygolub](https://github.com/alekseygolub)). -* Add support for `APPEND` modifier in `INTO OUTFILE` clause. Suggest using `APPEND` or `TRUNCATE` for `INTO OUTFILE` when file exists. [#50950](https://github.com/ClickHouse/ClickHouse/pull/50950) ([alekar](https://github.com/alekar)). +* Add support for `TRUNCATE` modifier in `INTO OUTFILE` clause. Suggest using `APPEND` or `TRUNCATE` for `INTO OUTFILE` when file exists. [#50950](https://github.com/ClickHouse/ClickHouse/pull/50950) ([alekar](https://github.com/alekar)). * Add table engine `Redis` and table function `redis`. It allows querying external Redis servers. [#50150](https://github.com/ClickHouse/ClickHouse/pull/50150) ([JackyWoo](https://github.com/JackyWoo)). * Allow to skip empty files in file/s3/url/hdfs table functions using settings `s3_skip_empty_files`, `hdfs_skip_empty_files`, `engine_file_skip_empty_files`, `engine_url_skip_empty_files`. [#50364](https://github.com/ClickHouse/ClickHouse/pull/50364) ([Kruglov Pavel](https://github.com/Avogar)). * Add a new setting named `use_mysql_types_in_show_columns` to alter the `SHOW COLUMNS` SQL statement to display MySQL equivalent types when a client is connected via the MySQL compatibility port. [#49577](https://github.com/ClickHouse/ClickHouse/pull/49577) ([Thomas Panetti](https://github.com/tpanetti)). @@ -40,12 +40,12 @@ * Make multiple list requests to ZooKeeper in parallel to speed up reading from system.zookeeper table. [#51042](https://github.com/ClickHouse/ClickHouse/pull/51042) ([Alexander Gololobov](https://github.com/davenger)). * Speedup initialization of DateTime lookup tables for time zones. This should reduce startup/connect time of clickhouse-client especially in debug build as it is rather heavy. [#51347](https://github.com/ClickHouse/ClickHouse/pull/51347) ([Alexander Gololobov](https://github.com/davenger)). * Fix data lakes slowness because of synchronous head requests. (Related to Iceberg/Deltalake/Hudi being slow with a lot of files). [#50976](https://github.com/ClickHouse/ClickHouse/pull/50976) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Do not replicate `ALTER PARTITION` queries and mutations through `Replicated` database if it has only one shard and the underlying table is `ReplicatedMergeTree`. [#51049](https://github.com/ClickHouse/ClickHouse/pull/51049) ([Alexander Tokmakov](https://github.com/tavplubix)). * Do not read all the columns from right GLOBAL JOIN table. [#50721](https://github.com/ClickHouse/ClickHouse/pull/50721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). #### Experimental Feature * Support parallel replicas with the analyzer. [#50441](https://github.com/ClickHouse/ClickHouse/pull/50441) ([Raúl Marín](https://github.com/Algunenano)). * Add random sleep before large merges/mutations execution to split load more evenly between replicas in case of zero-copy replication. [#51282](https://github.com/ClickHouse/ClickHouse/pull/51282) ([alesapin](https://github.com/alesapin)). +* Do not replicate `ALTER PARTITION` queries and mutations through `Replicated` database if it has only one shard and the underlying table is `ReplicatedMergeTree`. [#51049](https://github.com/ClickHouse/ClickHouse/pull/51049) ([Alexander Tokmakov](https://github.com/tavplubix)). #### Improvement * Relax the thresholds for "too many parts" to be more modern. Return the backpressure during long-running insert queries. [#50856](https://github.com/ClickHouse/ClickHouse/pull/50856) ([Alexey Milovidov](https://github.com/alexey-milovidov)). From f48de18640467e6302c1bd48799e386d4aa39437 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 29 Jun 2023 20:59:01 +0000 Subject: [PATCH 466/515] Update version_date.tsv and changelogs after v23.4.5.22-stable --- docs/changelogs/v23.4.5.22-stable.md | 27 +++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 2 files changed, 29 insertions(+) create mode 100644 docs/changelogs/v23.4.5.22-stable.md diff --git a/docs/changelogs/v23.4.5.22-stable.md b/docs/changelogs/v23.4.5.22-stable.md new file mode 100644 index 00000000000..2d61f5b11cf --- /dev/null +++ b/docs/changelogs/v23.4.5.22-stable.md @@ -0,0 +1,27 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.4.5.22-stable (0ced5d6a8da) FIXME as compared to v23.4.4.16-stable (747ba4fc6a0) + +#### Build/Testing/Packaging Improvement +* Backported in [#51530](https://github.com/ClickHouse/ClickHouse/issues/51530): Split huge `RUN` in Dockerfile into smaller conditional. Install the necessary tools on demand in the same `RUN` layer, and remove them after that. Upgrade the OS only once at the beginning. Use a modern way to check the signed repository. Downgrade the base repo to ubuntu:20.04 to address the issues on older docker versions. Upgrade golang version to address golang vulnerabilities. [#51504](https://github.com/ClickHouse/ClickHouse/pull/51504) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#51570](https://github.com/ClickHouse/ClickHouse/issues/51570): This a follow-up for [#51504](https://github.com/ClickHouse/ClickHouse/issues/51504), the cleanup was lost during refactoring. [#51564](https://github.com/ClickHouse/ClickHouse/pull/51564) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix broken index analysis when binary operator contains a null constant argument [#50177](https://github.com/ClickHouse/ClickHouse/pull/50177) ([Amos Bird](https://github.com/amosbird)). +* Fix reconnecting of HTTPS session when target host IP was changed [#50240](https://github.com/ClickHouse/ClickHouse/pull/50240) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Fix incorrect constant folding [#50536](https://github.com/ClickHouse/ClickHouse/pull/50536) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix type of LDAP server params hash in cache entry [#50865](https://github.com/ClickHouse/ClickHouse/pull/50865) ([Julian Maicher](https://github.com/jmaicher)). +* Fallback to parsing big integer from String instead of exception in Parquet format [#50873](https://github.com/ClickHouse/ClickHouse/pull/50873) ([Kruglov Pavel](https://github.com/Avogar)). +* Do not apply projection if read-in-order was enabled. [#50923](https://github.com/ClickHouse/ClickHouse/pull/50923) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix fuzzer failure in ActionsDAG [#51301](https://github.com/ClickHouse/ClickHouse/pull/51301) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Increase max array size in group bitmap [#50620](https://github.com/ClickHouse/ClickHouse/pull/50620) ([Kruglov Pavel](https://github.com/Avogar)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 5c8dd0d2481..4ca5d1d7497 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -2,10 +2,12 @@ v23.5.4.25-stable 2023-06-29 v23.5.3.24-stable 2023-06-17 v23.5.2.7-stable 2023-06-10 v23.5.1.3174-stable 2023-06-09 +v23.4.5.22-stable 2023-06-29 v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.7.5-lts 2023-06-29 v23.3.6.7-lts 2023-06-28 v23.3.5.9-lts 2023-06-22 v23.3.4.17-lts 2023-06-17 From a0bf1708837667c1f3d00dded295ec5ade7a4ac9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 29 Jun 2023 21:01:19 +0000 Subject: [PATCH 467/515] Update version_date.tsv and changelogs after v23.3.7.5-lts --- docs/changelogs/v23.3.7.5-lts.md | 16 ++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 2 files changed, 18 insertions(+) create mode 100644 docs/changelogs/v23.3.7.5-lts.md diff --git a/docs/changelogs/v23.3.7.5-lts.md b/docs/changelogs/v23.3.7.5-lts.md new file mode 100644 index 00000000000..7a5fd5a19b6 --- /dev/null +++ b/docs/changelogs/v23.3.7.5-lts.md @@ -0,0 +1,16 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.3.7.5-lts (bc683c11c92) FIXME as compared to v23.3.6.7-lts (7e3f0a271b7) + +#### Build/Testing/Packaging Improvement +* Backported in [#51568](https://github.com/ClickHouse/ClickHouse/issues/51568): This a follow-up for [#51504](https://github.com/ClickHouse/ClickHouse/issues/51504), the cleanup was lost during refactoring. [#51564](https://github.com/ClickHouse/ClickHouse/pull/51564) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix fuzzer failure in ActionsDAG [#51301](https://github.com/ClickHouse/ClickHouse/pull/51301) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 5c8dd0d2481..4ca5d1d7497 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -2,10 +2,12 @@ v23.5.4.25-stable 2023-06-29 v23.5.3.24-stable 2023-06-17 v23.5.2.7-stable 2023-06-10 v23.5.1.3174-stable 2023-06-09 +v23.4.5.22-stable 2023-06-29 v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.7.5-lts 2023-06-29 v23.3.6.7-lts 2023-06-28 v23.3.5.9-lts 2023-06-22 v23.3.4.17-lts 2023-06-17 From fd5f6ee2f99989f9ab5c80ecbce1a0b6ab7ae109 Mon Sep 17 00:00:00 2001 From: Ramazan Polat Date: Fri, 30 Jun 2023 04:32:41 +0300 Subject: [PATCH 468/515] Update parts.md Fix misalignment of `active` --- docs/en/operations/system-tables/parts.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/parts.md b/docs/en/operations/system-tables/parts.md index 9159d1e9284..5829e5ad313 100644 --- a/docs/en/operations/system-tables/parts.md +++ b/docs/en/operations/system-tables/parts.md @@ -27,7 +27,7 @@ Columns: Data storing format is controlled by the `min_bytes_for_wide_part` and `min_rows_for_wide_part` settings of the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table. - - `active` ([UInt8](../../sql-reference/data-types/int-uint.md)) – Flag that indicates whether the data part is active. If a data part is active, it’s used in a table. Otherwise, it’s deleted. Inactive data parts remain after merging. +- `active` ([UInt8](../../sql-reference/data-types/int-uint.md)) – Flag that indicates whether the data part is active. If a data part is active, it’s used in a table. Otherwise, it’s deleted. Inactive data parts remain after merging. - `marks` ([UInt64](../../sql-reference/data-types/int-uint.md)) – The number of marks. To get the approximate number of rows in a data part, multiply `marks` by the index granularity (usually 8192) (this hint does not work for adaptive granularity). From 710911f7dc18ecaa5a400d4efe6587a9b656d937 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 30 Jun 2023 11:39:08 +0300 Subject: [PATCH 469/515] Update CHANGELOG.md --- CHANGELOG.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index c1e0dba4465..bf6b309ef2c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -18,7 +18,6 @@ * CGroups metrics related to CPU are replaced with one metric, `CGroupMaxCPU` for better usability. The `Normalized` CPU usage metrics will be normalized to CGroups limits instead of the total number of CPUs when they are set. This closes [#50836](https://github.com/ClickHouse/ClickHouse/issues/50836). [#50835](https://github.com/ClickHouse/ClickHouse/pull/50835) ([Alexey Milovidov](https://github.com/alexey-milovidov)). #### New Feature -* Added `Overlay` database engine to combine multiple databases into one. Added `Filesystem` database engine to represent a directory in the filesystem as a set of implicitly available tables with auto-detected formats and structures. A new `S3` database engine allows to read-only interact with s3 storage by representing a prefix as a set of tables. A new `HDFS` database engine allows to interact with HDFS storage in the same way. [#48821](https://github.com/ClickHouse/ClickHouse/pull/48821) ([alekseygolub](https://github.com/alekseygolub)). * The function `transform` as well as `CASE` with value matching started to support all data types. This closes [#29730](https://github.com/ClickHouse/ClickHouse/issues/29730). This closes [#32387](https://github.com/ClickHouse/ClickHouse/issues/32387). This closes [#50827](https://github.com/ClickHouse/ClickHouse/issues/50827). This closes [#31336](https://github.com/ClickHouse/ClickHouse/issues/31336). This closes [#40493](https://github.com/ClickHouse/ClickHouse/issues/40493). [#51351](https://github.com/ClickHouse/ClickHouse/pull/51351) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Added option `--rename_files_after_processing `. This closes [#34207](https://github.com/ClickHouse/ClickHouse/issues/34207). [#49626](https://github.com/ClickHouse/ClickHouse/pull/49626) ([alekseygolub](https://github.com/alekseygolub)). * Add support for `TRUNCATE` modifier in `INTO OUTFILE` clause. Suggest using `APPEND` or `TRUNCATE` for `INTO OUTFILE` when file exists. [#50950](https://github.com/ClickHouse/ClickHouse/pull/50950) ([alekar](https://github.com/alekar)). @@ -26,7 +25,7 @@ * Allow to skip empty files in file/s3/url/hdfs table functions using settings `s3_skip_empty_files`, `hdfs_skip_empty_files`, `engine_file_skip_empty_files`, `engine_url_skip_empty_files`. [#50364](https://github.com/ClickHouse/ClickHouse/pull/50364) ([Kruglov Pavel](https://github.com/Avogar)). * Add a new setting named `use_mysql_types_in_show_columns` to alter the `SHOW COLUMNS` SQL statement to display MySQL equivalent types when a client is connected via the MySQL compatibility port. [#49577](https://github.com/ClickHouse/ClickHouse/pull/49577) ([Thomas Panetti](https://github.com/tpanetti)). * Clickhouse-client can now be called with a connection string instead of "--host", "--port", "--user" etc. [#50689](https://github.com/ClickHouse/ClickHouse/pull/50689) ([Alexey Gerasimchuck](https://github.com/Demilivor)). -* Add setting `session_timezone`, it is used as default timezone for session when not explicitly specified. [#44149](https://github.com/ClickHouse/ClickHouse/pull/44149) ([Andrey Zvonov](https://github.com/zvonand)). +* Add setting `session_timezone`; it is used as the default timezone for a session when not explicitly specified. [#44149](https://github.com/ClickHouse/ClickHouse/pull/44149) ([Andrey Zvonov](https://github.com/zvonand)). * Codec DEFLATE_QPL is now controlled via server setting "enable_deflate_qpl_codec" (default: false) instead of setting "allow_experimental_codecs". This marks DEFLATE_QPL non-experimental. [#50775](https://github.com/ClickHouse/ClickHouse/pull/50775) ([Robert Schulze](https://github.com/rschu1ze)). #### Performance Improvement From fe93e687de7f2be7796e1034a553fa9f01040e5d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 30 Jun 2023 10:02:24 +0200 Subject: [PATCH 470/515] Add comments. --- base/base/hex.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/base/base/hex.h b/base/base/hex.h index 0780e6159a1..937218fec5a 100644 --- a/base/base/hex.h +++ b/base/base/hex.h @@ -176,7 +176,7 @@ namespace impl }; /// Helper template class to convert a value of any supported type to hexadecimal representation and back. - template + template struct HexConversion; template @@ -185,7 +185,7 @@ namespace impl template struct HexConversion> : public HexConversionUInt> {}; - template + template /// Partial specialization here allows not to include in this header. struct HexConversion>> { static const constexpr size_t num_hex_digits = 32; @@ -208,6 +208,8 @@ namespace impl /// Produces a hexadecimal representation of an integer value with leading zeros (for checksums). /// The function supports native integer types, wide::integer, CityHash_v1_0_2::uint128. +/// It can be used with signed types as well, however they are written as corresponding unsigned numbers +/// using two's complement (i.e. for example "-1" is written as "0xFF", not as "-0x01"). template void writeHexUIntUppercase(const T & value, char * out) { From 2b55734ccf533a4fd9180b57672bc35823ed66d0 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 30 Jun 2023 02:52:03 -0700 Subject: [PATCH 471/515] Fix MergeTreeMarksLoader segfaulting if marks file is longer than expected (#51636) Co-authored-by: Nikita Mikhaylov --- .../MergeTree/MergeTreeMarksLoader.cpp | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index 9a5576f0ad2..5c722eec380 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -135,6 +135,7 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl() if (!index_granularity_info.mark_type.adaptive) { /// Read directly to marks. + chassert(expected_uncompressed_size == plain_marks.size() * sizeof(MarkInCompressedFile)); reader->readStrict(reinterpret_cast(plain_marks.data()), expected_uncompressed_size); if (!reader->eof()) @@ -148,23 +149,25 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl() } else { - size_t i = 0; - size_t granularity; - while (!reader->eof()) + for (size_t i = 0; i < marks_count; ++i) { + if (reader->eof()) + throw Exception( + ErrorCodes::CANNOT_READ_ALL_DATA, + "Cannot read all marks from file {}, marks expected {} (bytes size {}), marks read {} (bytes size {})", + mrk_path, marks_count, expected_uncompressed_size, i, reader->count()); + + size_t granularity; reader->readStrict( reinterpret_cast(plain_marks.data() + i * columns_in_mark), columns_in_mark * sizeof(MarkInCompressedFile)); readIntBinary(granularity, *reader); - ++i; } - if (i * mark_size != expected_uncompressed_size) - { + if (!reader->eof()) throw Exception( ErrorCodes::CANNOT_READ_ALL_DATA, - "Cannot read all marks from file {}, marks expected {} (bytes size {}), marks read {} (bytes size {})", - mrk_path, marks_count, expected_uncompressed_size, i, reader->count()); - } + "Too many marks in file {}, marks expected {} (bytes size {})", + mrk_path, marks_count, expected_uncompressed_size); } auto res = std::make_shared(plain_marks); From 24ae56b5dd383ab4577747a873d9bc4aae9f7f16 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 30 Jun 2023 11:02:16 +0000 Subject: [PATCH 472/515] Correctly apply async deduplication for nonordinary ReplicatedMergeTree tables --- src/Storages/MergeTree/MergeTreeDataWriter.h | 5 ++ .../MergeTree/ReplicatedMergeTreeSink.cpp | 85 ++++++++++++------- 2 files changed, 61 insertions(+), 29 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index ed7151886f5..795453b2afa 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -80,6 +80,11 @@ public: */ TemporaryPart writeTempPart(BlockWithPartition & block, const StorageMetadataPtr & metadata_snapshot, ContextPtr context); + MergeTreeData::MergingParams::Mode getMergingMode() const + { + return data.merging_params.mode; + } + TemporaryPart writeTempPartWithoutPrefix(BlockWithPartition & block, const StorageMetadataPtr & metadata_snapshot, int64_t block_number, ContextPtr context); /// For insertion. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index c93077fb4fb..718e3fc95ce 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -7,6 +7,8 @@ #include #include #include +#include "Storages/MergeTree/MergeAlgorithm.h" +#include "Storages/MergeTree/MergeTreeDataWriter.h" #include #include #include @@ -54,6 +56,9 @@ struct ReplicatedMergeTreeSinkImpl::DelayedChunk UInt64 elapsed_ns; BlockIDsType block_id; BlockWithPartition block_with_partition; + /// Some merging algorithms can mofidy the block which loses the information about the async insert offsets + /// when preprocessing or filtering data for asnyc inserts deduplication we want to use the initial, unmerged block + std::optional unmerged_block_with_partition; std::unordered_map> block_id_to_offset_idx; ProfileEvents::Counters part_counters; @@ -63,12 +68,14 @@ struct ReplicatedMergeTreeSinkImpl::DelayedChunk UInt64 elapsed_ns_, BlockIDsType && block_id_, BlockWithPartition && block_, + std::optional && unmerged_block_with_partition_, ProfileEvents::Counters && part_counters_) : log(log_), temp_part(std::move(temp_part_)), elapsed_ns(elapsed_ns_), block_id(std::move(block_id_)), block_with_partition(std::move(block_)), + unmerged_block_with_partition(std::move(unmerged_block_with_partition_)), part_counters(std::move(part_counters_)) { initBlockIDMap(); @@ -113,6 +120,7 @@ struct ReplicatedMergeTreeSinkImpl::DelayedChunk { if constexpr (async_insert) { + auto * current_block_with_partition = unmerged_block_with_partition.has_value() ? &unmerged_block_with_partition.value() : &block_with_partition; std::vector offset_idx; for (const auto & raw_path : block_paths) { @@ -127,14 +135,14 @@ struct ReplicatedMergeTreeSinkImpl::DelayedChunk } std::sort(offset_idx.begin(), offset_idx.end()); - auto & offsets = block_with_partition.offsets; + auto & offsets = current_block_with_partition->offsets; size_t idx = 0, remove_count = 0; auto it = offset_idx.begin(); std::vector new_offsets; std::vector new_block_ids; /// construct filter - size_t rows = block_with_partition.block.rows(); + size_t rows = current_block_with_partition->block.rows(); auto filter_col = ColumnUInt8::create(rows, 1u); ColumnUInt8::Container & vec = filter_col->getData(); UInt8 * pos = vec.data(); @@ -162,18 +170,21 @@ struct ReplicatedMergeTreeSinkImpl::DelayedChunk LOG_TRACE(log, "New block IDs: {}, new offsets: {}, size: {}", toString(new_block_ids), toString(new_offsets), new_offsets.size()); - block_with_partition.offsets = std::move(new_offsets); + current_block_with_partition->offsets = std::move(new_offsets); block_id = std::move(new_block_ids); - auto cols = block_with_partition.block.getColumns(); + auto cols = current_block_with_partition->block.getColumns(); for (auto & col : cols) { col = col->filter(vec, rows - remove_count); } - block_with_partition.block.setColumns(cols); + current_block_with_partition->block.setColumns(cols); - LOG_TRACE(log, "New block rows {}", block_with_partition.block.rows()); + LOG_TRACE(log, "New block rows {}", current_block_with_partition->block.rows()); initBlockIDMap(); + + if (unmerged_block_with_partition.has_value()) + block_with_partition.block = unmerged_block_with_partition->block; } else { @@ -202,7 +213,7 @@ std::vector testSelfDeduplicate(std::vector data, std::vector::DelayedChunk::Partition part( - &Poco::Logger::get("testSelfDeduplicate"), MergeTreeDataWriter::TemporaryPart(), 0, std::move(hashes), std::move(block1), std::move(profile_counters)); + &Poco::Logger::get("testSelfDeduplicate"), MergeTreeDataWriter::TemporaryPart(), 0, std::move(hashes), std::move(block1), std::nullopt, std::move(profile_counters)); part.filterSelfDuplicate(); @@ -235,8 +246,10 @@ namespace { SipHash hash; for (size_t i = start; i < offset; ++i) + { for (const auto & col : cols) col->updateHashWithValue(i, hash); + } union { char bytes[16]; @@ -432,8 +445,17 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) ProfileEvents::Counters part_counters; auto profile_events_scope = std::make_unique(&part_counters); - /// Write part to the filesystem under temporary name. Calculate a checksum. + /// Some merging algorithms can mofidy the block which loses the information about the async insert offsets + /// when preprocessing or filtering data for asnyc inserts deduplication we want to use the initial, unmerged block + std::optional unmerged_block; + if constexpr (async_insert) + { + if (storage.writer.getMergingMode() != MergeTreeData::MergingParams::Mode::Ordinary) + unmerged_block.emplace(current_block); + } + + /// Write part to the filesystem under temporary name. Calculate a checksum. auto temp_part = storage.writer.writeTempPart(current_block, metadata_snapshot, context); /// If optimize_on_insert setting is true, current_block could become empty after merge @@ -446,31 +468,35 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) if constexpr (async_insert) { /// TODO consider insert_deduplication_token - block_id = getHashesForBlocks(current_block, temp_part.part->info.partition_id); + block_id = getHashesForBlocks(unmerged_block.has_value() ? *unmerged_block : current_block, temp_part.part->info.partition_id); LOG_TRACE(log, "async insert part, part id {}, block id {}, offsets {}, size {}", temp_part.part->info.partition_id, toString(block_id), toString(current_block.offsets), current_block.offsets.size()); } - else if (deduplicate) - { - String block_dedup_token; - - /// We add the hash from the data and partition identifier to deduplication ID. - /// That is, do not insert the same data to the same partition twice. - - const String & dedup_token = settings.insert_deduplication_token; - if (!dedup_token.empty()) - { - /// multiple blocks can be inserted within the same insert query - /// an ordinal number is added to dedup token to generate a distinctive block id for each block - block_dedup_token = fmt::format("{}_{}", dedup_token, chunk_dedup_seqnum); - ++chunk_dedup_seqnum; - } - - block_id = temp_part.part->getZeroLevelPartBlockID(block_dedup_token); - LOG_DEBUG(log, "Wrote block with ID '{}', {} rows{}", block_id, current_block.block.rows(), quorumLogMessage(replicas_num)); - } else { - LOG_DEBUG(log, "Wrote block with {} rows{}", current_block.block.rows(), quorumLogMessage(replicas_num)); + + if (deduplicate) + { + String block_dedup_token; + + /// We add the hash from the data and partition identifier to deduplication ID. + /// That is, do not insert the same data to the same partition twice. + + const String & dedup_token = settings.insert_deduplication_token; + if (!dedup_token.empty()) + { + /// multiple blocks can be inserted within the same insert query + /// an ordinal number is added to dedup token to generate a distinctive block id for each block + block_dedup_token = fmt::format("{}_{}", dedup_token, chunk_dedup_seqnum); + ++chunk_dedup_seqnum; + } + + block_id = temp_part.part->getZeroLevelPartBlockID(block_dedup_token); + LOG_DEBUG(log, "Wrote block with ID '{}', {} rows{}", block_id, current_block.block.rows(), quorumLogMessage(replicas_num)); + } + else + { + LOG_DEBUG(log, "Wrote block with {} rows{}", current_block.block.rows(), quorumLogMessage(replicas_num)); + } } profile_events_scope.reset(); @@ -501,6 +527,7 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) elapsed_ns, std::move(block_id), std::move(current_block), + std::move(unmerged_block), std::move(part_counters) /// profile_events_scope must be reset here. )); } From 0fea8c6d1c9080e5855b2f4532c255aa6d40c5e5 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 30 Jun 2023 13:08:11 +0200 Subject: [PATCH 473/515] Update 02808_custom_disk_with_user_defined_name.sh --- .../0_stateless/02808_custom_disk_with_user_defined_name.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh index 50dee04f6a6..537e117adb9 100755 --- a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh +++ b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh @@ -12,7 +12,7 @@ $CLICKHOUSE_CLIENT -nm --query """ DROP TABLE IF EXISTS test; CREATE TABLE test (a Int32, b String) ENGINE = MergeTree() ORDER BY tuple() -SETTINGS disk = disk_s3disk(type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3disk); +SETTINGS disk = disk_s3disk(type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); """ 2>&1 | grep -q "Disk with name \`s3disk\` already exist" && echo 'OK' || echo 'FAIL' disk_name="${CLICKHOUSE_TEST_UNIQUE_NAME}" From 58f291997176b230395bc906fe642c5dfad4657c Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Jun 2023 12:46:45 +0200 Subject: [PATCH 474/515] Fix --- src/Interpreters/Cache/FileSegment.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index e97d708ba74..5c7fb4c8109 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -648,8 +648,6 @@ void FileSegment::complete() if (segment_kind == FileSegmentKind::Temporary && is_last_holder) { LOG_TEST(log, "Removing temporary file segment: {}", getInfoForLogUnlocked(segment_lock)); - detach(segment_lock, *locked_key); - setDownloadState(State::DETACHED, segment_lock); locked_key->removeFileSegment(offset(), segment_lock); return; } @@ -798,7 +796,6 @@ bool FileSegment::assertCorrectnessUnlocked(const FileSegmentGuard::Lock &) cons } chassert(reserved_size >= downloaded_size); - chassert((reserved_size == 0) || queue_iterator); check_iterator(queue_iterator); } From 96cc02ae0c83581b051ea49fd11a3c1aa97f6c54 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 30 Jun 2023 11:45:10 +0000 Subject: [PATCH 475/515] Add test --- ...sert_dedup_replicated_collapsing.reference | 8 ++++ ...sync_insert_dedup_replicated_collapsing.sh | 39 +++++++++++++++++++ 2 files changed, 47 insertions(+) create mode 100644 tests/queries/0_stateless/02810_async_insert_dedup_replicated_collapsing.reference create mode 100755 tests/queries/0_stateless/02810_async_insert_dedup_replicated_collapsing.sh diff --git a/tests/queries/0_stateless/02810_async_insert_dedup_replicated_collapsing.reference b/tests/queries/0_stateless/02810_async_insert_dedup_replicated_collapsing.reference new file mode 100644 index 00000000000..74624d246de --- /dev/null +++ b/tests/queries/0_stateless/02810_async_insert_dedup_replicated_collapsing.reference @@ -0,0 +1,8 @@ +string1 +------------ +string1 +------------ +string1 +string1 +string2 +------------ diff --git a/tests/queries/0_stateless/02810_async_insert_dedup_replicated_collapsing.sh b/tests/queries/0_stateless/02810_async_insert_dedup_replicated_collapsing.sh new file mode 100755 index 00000000000..69e0e3ecc32 --- /dev/null +++ b/tests/queries/0_stateless/02810_async_insert_dedup_replicated_collapsing.sh @@ -0,0 +1,39 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS 02810_async_insert_dedup_collapsing" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE 02810_async_insert_dedup_collapsing (stringvalue String, sign Int8) ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/02810_async_insert_dedup/' || currentDatabase(), 'r1', sign) ORDER BY stringvalue" + +url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=1&async_insert_busy_timeout_ms=3000&async_insert_deduplicate=1" + +# insert value with same key and sign so it's collapsed on insert +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO 02810_async_insert_dedup_collapsing VALUES ('string1', 1)" & +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO 02810_async_insert_dedup_collapsing VALUES ('string1', 1)" & + +wait + +${CLICKHOUSE_CLIENT} -q "SELECT stringvalue FROM 02810_async_insert_dedup_collapsing ORDER BY stringvalue" +${CLICKHOUSE_CLIENT} -q "SELECT '------------'" + +# trigger same collaps algorithm but also deduplication +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO 02810_async_insert_dedup_collapsing VALUES ('string1', 1)" & +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO 02810_async_insert_dedup_collapsing VALUES ('string1', 1)" & + +wait + +${CLICKHOUSE_CLIENT} -q "SELECT stringvalue FROM 02810_async_insert_dedup_collapsing ORDER BY stringvalue" +${CLICKHOUSE_CLIENT} -q "SELECT '------------'" + +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO 02810_async_insert_dedup_collapsing VALUES ('string2', 1)" & +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO 02810_async_insert_dedup_collapsing VALUES ('string2', 1), ('string1', 1)" & +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO 02810_async_insert_dedup_collapsing VALUES ('string2', 1)" & + +wait + +${CLICKHOUSE_CLIENT} -q "SELECT stringvalue FROM 02810_async_insert_dedup_collapsing ORDER BY stringvalue" +${CLICKHOUSE_CLIENT} -q "SELECT '------------'" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE 02810_async_insert_dedup_collapsing" \ No newline at end of file From 9a79fd6f70b30cd56e37c8a41c8c14bf65ed11ff Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 30 Jun 2023 11:51:20 +0000 Subject: [PATCH 476/515] Better --- src/Processors/Transforms/ExpressionTransform.cpp | 1 - src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 7 ++++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Transforms/ExpressionTransform.cpp b/src/Processors/Transforms/ExpressionTransform.cpp index 49988932947..0d3341b000c 100644 --- a/src/Processors/Transforms/ExpressionTransform.cpp +++ b/src/Processors/Transforms/ExpressionTransform.cpp @@ -39,7 +39,6 @@ void ConvertingTransform::onConsume(Chunk chunk) expression->execute(block, num_rows); chunk.setColumns(block.getColumns(), num_rows); - chunk.setChunkInfo(chunk.getChunkInfo()); cur_chunk = std::move(chunk); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 718e3fc95ce..a13a96ac65e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -7,8 +7,8 @@ #include #include #include -#include "Storages/MergeTree/MergeAlgorithm.h" -#include "Storages/MergeTree/MergeTreeDataWriter.h" +#include +#include #include #include #include @@ -451,8 +451,9 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) if constexpr (async_insert) { + /// we copy everything but offsets which we move because they are only used by async insert if (storage.writer.getMergingMode() != MergeTreeData::MergingParams::Mode::Ordinary) - unmerged_block.emplace(current_block); + unmerged_block.emplace(Block(current_block.block), Row(current_block.partition), std::move(current_block.offsets)); } /// Write part to the filesystem under temporary name. Calculate a checksum. From 042885c35e2e4c82e8594543619a83a0fcd4c0df Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 30 Jun 2023 14:25:24 +0200 Subject: [PATCH 477/515] fix test style --- .../02810_async_insert_dedup_replicated_collapsing.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02810_async_insert_dedup_replicated_collapsing.sh b/tests/queries/0_stateless/02810_async_insert_dedup_replicated_collapsing.sh index 69e0e3ecc32..804cd894ebc 100755 --- a/tests/queries/0_stateless/02810_async_insert_dedup_replicated_collapsing.sh +++ b/tests/queries/0_stateless/02810_async_insert_dedup_replicated_collapsing.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS 02810_async_insert_dedup_collapsing" -${CLICKHOUSE_CLIENT} -q "CREATE TABLE 02810_async_insert_dedup_collapsing (stringvalue String, sign Int8) ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/02810_async_insert_dedup/' || currentDatabase(), 'r1', sign) ORDER BY stringvalue" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE 02810_async_insert_dedup_collapsing (stringvalue String, sign Int8) ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/{database}/02810_async_insert_dedup', 'r1', sign) ORDER BY stringvalue" url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=1&async_insert_busy_timeout_ms=3000&async_insert_deduplicate=1" From 84872517708ad03c764e616fd28d5d1d12593697 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Elena=20Torr=C3=B3?= Date: Fri, 30 Jun 2023 14:46:50 +0200 Subject: [PATCH 478/515] Update File Cache disk configuration settings default values --- docs/en/operations/storing-data.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 5804ad8545b..3f4456d7e12 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -188,9 +188,9 @@ These settings should be defined in the disk configuration section. - `do_not_evict_index_and_mark_files` - do not evict small frequently used files according to cache policy. Default: `false`. This setting was added in version 22.8. If you used filesystem cache before this version, then it will not work on versions starting from 22.8 if this setting is set to `true`. If you want to use this setting, clear old cache created before version 22.8 before upgrading. -- `max_file_segment_size` - a maximum size of a single cache file in bytes or in readable format (`ki, Mi, Gi, etc`, example `10Gi`). Default: `104857600` (`100Mi`). +- `max_file_segment_size` - a maximum size of a single cache file in bytes or in readable format (`ki, Mi, Gi, etc`, example `10Gi`). Default: `8388608` (`8Mi`). -- `max_elements` - a limit for a number of cache files. Default: `1048576`. +- `max_elements` - a limit for a number of cache files. Default: `10000000`. File Cache **query/profile settings**: From bf06e18c348d6d8bd23d4184d4bf3eee89ed4835 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 30 Jun 2023 09:03:25 -0400 Subject: [PATCH 479/515] docs clickhouse-static-files-uploader and demo repo --- .../en/engines/table-engines/mergetree-family/mergetree.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index a1c2fbdbe50..67043ef1062 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -949,7 +949,14 @@ The example uses `type=web`, but any disk type can be configured as dynamic, eve #### Example dynamic web storage +:::tip +A [demo dataset](https://github.com/ClickHouse/web-tables-demo) is hosted in GitHub. To prepare your own tables for web storage see the tool [clickhouse-static-files-uploader](/docs/en/operations/storing-data.md/#storing-data-on-webserver) +::: + +In this `ATTACH TABLE` query the `UUID` provided matches the directory name of the data, and the endpoint is the URL for the raw GitHub content. + ```sql +# highlight-next-line ATTACH TABLE uk_price_paid UUID 'cf712b4f-2ca8-435c-ac23-c4393efe52f7' ( price UInt32, From 904c533a84e5766b4e5ca5e886d7a826cefdd0d4 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 30 Jun 2023 09:32:54 -0400 Subject: [PATCH 480/515] spelling list --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index f25d082e5a6..2802e52c288 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2459,6 +2459,7 @@ unrounded untracked untrusted untuple +uploader uploaders upperUTF uptime From 91579453895b7dc88b0aec78bdde6311a6fc5a39 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Elena=20Torr=C3=B3?= Date: Fri, 30 Jun 2023 15:43:04 +0200 Subject: [PATCH 481/515] Add missing settings --- docs/en/operations/storing-data.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 3f4456d7e12..fd82c955c40 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -184,7 +184,9 @@ These settings should be defined in the disk configuration section. - `enable_filesystem_query_cache_limit` - allow to limit the size of cache which is downloaded within each query (depends on user setting `max_query_cache_size`). Default: `false`. -- `enable_cache_hits_threshold` - number which defines how many times some data needs to be read before it will be cached. Default: `0`, e.g. the data is cached at the first attempt to read it. +- `enable_cache_hits_threshold` - number which defines how many times some data needs to be read before it will be cached, defined by `cache_hits_threshold`. Default: `0`, e.g. the data is cached at the first attempt to read it. + +- `enable_bypass_cache_with_threshold` - allows to skip cache completely in case the requested read range exceeds the threshold defined by `bypass_cache_threashold`. Default: `268435456` (`256Mi`). - `do_not_evict_index_and_mark_files` - do not evict small frequently used files according to cache policy. Default: `false`. This setting was added in version 22.8. If you used filesystem cache before this version, then it will not work on versions starting from 22.8 if this setting is set to `true`. If you want to use this setting, clear old cache created before version 22.8 before upgrading. From 982ded4e8f09891a968f72037637f7f9bfbd6647 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 30 Jun 2023 15:47:31 +0200 Subject: [PATCH 482/515] update timeouts in tests for transactions --- .../0_stateless/01169_alter_partition_isolation_stress.sh | 2 +- .../0_stateless/01171_mv_select_insert_isolation_long.sh | 2 +- tests/queries/0_stateless/01174_select_insert_isolation.sh | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh index 508ad05224c..f2348c29146 100755 --- a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh @@ -240,7 +240,7 @@ kill -TERM $PID_1 kill -TERM $PID_2 wait ||: -wait_for_queries_to_finish +wait_for_queries_to_finish 40 $CLICKHOUSE_CLIENT -q "SELECT type, count(n) = countDistinct(n) FROM merge(currentDatabase(), '') GROUP BY type ORDER BY type" $CLICKHOUSE_CLIENT -q "SELECT DISTINCT arraySort(groupArrayIf(n, type=1)) = arraySort(groupArrayIf(n, type=2)) FROM merge(currentDatabase(), '') GROUP BY _table ORDER BY _table" diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 199c2b5389f..619e212c3ae 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -131,7 +131,7 @@ kill -TERM $PID_6 kill -TERM $PID_7 kill -TERM $PID_8 wait -wait_for_queries_to_finish +wait_for_queries_to_finish 40 $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh index 29ccfbb1ccb..6321f6ff01b 100755 --- a/tests/queries/0_stateless/01174_select_insert_isolation.sh +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -56,7 +56,7 @@ thread_select & PID_4=$! wait $PID_1 && wait $PID_2 && wait $PID_3 kill -TERM $PID_4 wait -wait_for_queries_to_finish +wait_for_queries_to_finish 40 $CLICKHOUSE_CLIENT --multiquery --query " BEGIN TRANSACTION; From 48eb1880651d381b39e2fe64f9ee6f8c64e1b303 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 30 Jun 2023 15:53:37 +0200 Subject: [PATCH 483/515] Remove unused code --- src/Server/TCPHandler.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 629738fa16f..235f634afec 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -83,8 +83,6 @@ struct QueryState NOT_CANCELLED }; - static std::string cancellationStatusToName(CancellationStatus status); - /// Is request cancelled CancellationStatus cancellation_status = CancellationStatus::NOT_CANCELLED; bool is_connection_closed = false; From 9ddf0853bff9b5bbcbf92f7bcc61e7f5b97a4aea Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 30 Jun 2023 16:12:39 +0200 Subject: [PATCH 484/515] Update 02808_custom_disk_with_user_defined_name.sh --- .../0_stateless/02808_custom_disk_with_user_defined_name.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh index 537e117adb9..3fbcde66add 100755 --- a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh +++ b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh @@ -12,8 +12,8 @@ $CLICKHOUSE_CLIENT -nm --query """ DROP TABLE IF EXISTS test; CREATE TABLE test (a Int32, b String) ENGINE = MergeTree() ORDER BY tuple() -SETTINGS disk = disk_s3disk(type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); -""" 2>&1 | grep -q "Disk with name \`s3disk\` already exist" && echo 'OK' || echo 'FAIL' +SETTINGS disk = disk_s3_disk(type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); +""" 2>&1 | grep -q "Disk with name \`s3_disk\` already exist" && echo 'OK' || echo 'FAIL' disk_name="${CLICKHOUSE_TEST_UNIQUE_NAME}" From 0b98406bd345f54c386435a11d126c07234f1aef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Elena=20Torr=C3=B3?= Date: Fri, 30 Jun 2023 16:13:43 +0200 Subject: [PATCH 485/515] Add default values to False --- docs/en/operations/storing-data.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index fd82c955c40..17d62673a8e 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -184,9 +184,9 @@ These settings should be defined in the disk configuration section. - `enable_filesystem_query_cache_limit` - allow to limit the size of cache which is downloaded within each query (depends on user setting `max_query_cache_size`). Default: `false`. -- `enable_cache_hits_threshold` - number which defines how many times some data needs to be read before it will be cached, defined by `cache_hits_threshold`. Default: `0`, e.g. the data is cached at the first attempt to read it. +- `enable_cache_hits_threshold` - number which defines how many times some data needs to be read before it will be cached. Default :`false`. This threshold can be defined by `cache_hits_threshold`. Default: `0`, e.g. the data is cached at the first attempt to read it. -- `enable_bypass_cache_with_threshold` - allows to skip cache completely in case the requested read range exceeds the threshold defined by `bypass_cache_threashold`. Default: `268435456` (`256Mi`). +- `enable_bypass_cache_with_threshold` - allows to skip cache completely in case the requested read range exceeds the threshold. Default :`false`. This threshold can be defined by `bypass_cache_threashold`. Default: `268435456` (`256Mi`). - `do_not_evict_index_and_mark_files` - do not evict small frequently used files according to cache policy. Default: `false`. This setting was added in version 22.8. If you used filesystem cache before this version, then it will not work on versions starting from 22.8 if this setting is set to `true`. If you want to use this setting, clear old cache created before version 22.8 before upgrading. From 689ff6f996585da91cf80eb2916f1d23706c1f30 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 30 Jun 2023 16:15:26 +0200 Subject: [PATCH 486/515] Update storing-data.md --- docs/en/operations/storing-data.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 17d62673a8e..fe6e8e15b0c 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -184,9 +184,9 @@ These settings should be defined in the disk configuration section. - `enable_filesystem_query_cache_limit` - allow to limit the size of cache which is downloaded within each query (depends on user setting `max_query_cache_size`). Default: `false`. -- `enable_cache_hits_threshold` - number which defines how many times some data needs to be read before it will be cached. Default :`false`. This threshold can be defined by `cache_hits_threshold`. Default: `0`, e.g. the data is cached at the first attempt to read it. +- `enable_cache_hits_threshold` - number which defines how many times some data needs to be read before it will be cached. Default: `false`. This threshold can be defined by `cache_hits_threshold`. Default: `0`, e.g. the data is cached at the first attempt to read it. -- `enable_bypass_cache_with_threshold` - allows to skip cache completely in case the requested read range exceeds the threshold. Default :`false`. This threshold can be defined by `bypass_cache_threashold`. Default: `268435456` (`256Mi`). +- `enable_bypass_cache_with_threshold` - allows to skip cache completely in case the requested read range exceeds the threshold. Default: `false`. This threshold can be defined by `bypass_cache_threashold`. Default: `268435456` (`256Mi`). - `do_not_evict_index_and_mark_files` - do not evict small frequently used files according to cache policy. Default: `false`. This setting was added in version 22.8. If you used filesystem cache before this version, then it will not work on versions starting from 22.8 if this setting is set to `true`. If you want to use this setting, clear old cache created before version 22.8 before upgrading. From 542b03bfd1bf8b7f6af69c76c2ef1d7f86d535b0 Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Fri, 30 Jun 2023 10:41:31 -0400 Subject: [PATCH 487/515] SonarCloud: Add C++23 Experimental Flag Signed-off-by: Julio Jimenez --- .github/workflows/nightly.yml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index acf6bbe8f6a..b508758ac7c 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -122,7 +122,7 @@ jobs: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" SonarCloud: # TODO: Remove if: whenever SonarCloud supports c++23 - if: ${{ false }} + # if: ${{ false }} runs-on: [self-hosted, builder] env: SONAR_SCANNER_VERSION: 4.8.0.2856 @@ -178,4 +178,5 @@ jobs: --define sonar.cfamily.build-wrapper-output="${{ env.BUILD_WRAPPER_OUT_DIR }}" \ --define sonar.projectKey="ClickHouse_ClickHouse" \ --define sonar.organization="clickhouse-java" \ - --define sonar.exclusions="**/*.java,**/*.ts,**/*.js,**/*.css,**/*.sql" \ + --define sonar.cfamily.cpp23.enabled=true \ + --define sonar.exclusions="**/*.java,**/*.ts,**/*.js,**/*.css,**/*.sql" From 00aeb407e5f88440ea82c423ccb701678a3e8e3f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 30 Jun 2023 14:41:27 +0000 Subject: [PATCH 488/515] Wait with retries --- docker/test/stateless/run.sh | 31 ++++--------------------------- tests/ci/attach_gdb.lib | 4 +++- tests/ci/utils.lib | 25 +++++++++++++++++++++++++ 3 files changed, 32 insertions(+), 28 deletions(-) create mode 100644 tests/ci/utils.lib diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 914e51a9f66..7ccedb8c0b3 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -18,6 +18,9 @@ ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test # shellcheck disable=SC1091 source /usr/share/clickhouse-test/ci/attach_gdb.lib || true # FIXME: to not break old builds, clean on 2023-09-01 +# shellcheck disable=SC1091 +source /usr/share/clickhouse-test/ci/utils.lib + # install test configs /usr/share/clickhouse-test/config/install.sh @@ -90,30 +93,6 @@ sleep 5 attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01 -function run_with_retry() -{ - set +e - - local total_retries="$1" - shift - - local retry=0 - - until [ "$retry" -ge "$total_retries" ] - do - if "$@"; then - set -e - return - else - retry=$((retry + 1)) - sleep 3 - fi - done - - echo "Command '$*' failed after $total_retries retries, exiting" - exit 1 -} - function run_tests() { set -x @@ -161,9 +140,7 @@ function run_tests() ADDITIONAL_OPTIONS+=('--report-logs-stats') - clickhouse-test "00001_select_1" > /dev/null ||: - - run_with_retry 5 clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')" + run_with_retry 10 clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')" set +e clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ diff --git a/tests/ci/attach_gdb.lib b/tests/ci/attach_gdb.lib index 2df6243f796..e937cf6dba7 100644 --- a/tests/ci/attach_gdb.lib +++ b/tests/ci/attach_gdb.lib @@ -1,5 +1,7 @@ #!/bin/bash +source /usr/share/clickhouse-test/ci/utils.lib + function attach_gdb_to_clickhouse() { # Set follow-fork-mode to parent, because we attach to clickhouse-server, not to watchdog @@ -38,5 +40,5 @@ quit gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" | ts '%Y-%m-%d %H:%M:%S' >> /test_output/gdb.log & sleep 5 # gdb will send SIGSTOP, spend some time loading debug info and then send SIGCONT, wait for it (up to send_timeout, 300s) - time clickhouse-client --query "SELECT 'Connected to clickhouse-server after attaching gdb'" ||: + run_with_retry 60 clickhouse-client --query "SELECT 'Connected to clickhouse-server after attaching gdb'" } diff --git a/tests/ci/utils.lib b/tests/ci/utils.lib new file mode 100644 index 00000000000..95ad50f635b --- /dev/null +++ b/tests/ci/utils.lib @@ -0,0 +1,25 @@ +#!/bin/bash + +function run_with_retry() +{ + set +e + + local total_retries="$1" + shift + + local retry=0 + + until [ "$retry" -ge "$total_retries" ] + do + if "$@"; then + set -e + return + else + retry=$((retry + 1)) + sleep 5 + fi + done + + echo "Command '$*' failed after $total_retries retries, exiting" + exit 1 +} \ No newline at end of file From 185e106c1faff2201db3f33f7497ab4a43d1c7e6 Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Fri, 30 Jun 2023 10:53:04 -0400 Subject: [PATCH 489/515] Please install NASM because NASM compiler cannot be found Signed-off-by: Julio Jimenez --- .github/workflows/nightly.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index b508758ac7c..3f6d9b86fd6 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -159,7 +159,7 @@ jobs: - name: Set Up Build Tools run: | sudo apt-get update - sudo apt-get install -yq git cmake ccache ninja-build python3 yasm + sudo apt-get install -yq git cmake ccache ninja-build python3 yasm nasm sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" - name: Run build-wrapper run: | From 5ee7f47b03d3b741db80dae38dfb169f338d9dcc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 30 Jun 2023 17:09:56 +0200 Subject: [PATCH 490/515] Update autogenerated version to 23.7.1.1 and contributors --- cmake/autogenerated_versions.txt | 10 +++++----- .../System/StorageSystemContributors.generated.cpp | 9 +++++++++ 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 015037b2de6..821b7b46855 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,11 +2,11 @@ # NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54475) +SET(VERSION_REVISION 54476) SET(VERSION_MAJOR 23) -SET(VERSION_MINOR 6) +SET(VERSION_MINOR 7) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH 2fec796e73efda10a538a03af3205ce8ffa1b2de) -SET(VERSION_DESCRIBE v23.6.1.1-testing) -SET(VERSION_STRING 23.6.1.1) +SET(VERSION_GITHASH d1c7e13d08868cb04d3562dcced704dd577cb1df) +SET(VERSION_DESCRIBE v23.7.1.1-testing) +SET(VERSION_STRING 23.7.1.1) # end of autochange diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index f83ee3197fe..f84c554afc0 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -226,6 +226,7 @@ const char * auto_contributors[] { "Carbyn", "Carlos Rodríguez Hernández", "Caspian", + "Chang Chen", "Chao Ma", "Chao Wang", "CheSema", @@ -291,6 +292,7 @@ const char * auto_contributors[] { "Dmitry Belyavtsev", "Dmitry Bilunov", "Dmitry Galuza", + "Dmitry Kardymon", "Dmitry Krylov", "Dmitry Luhtionov", "Dmitry Moskowski", @@ -408,6 +410,7 @@ const char * auto_contributors[] { "HeenaBansal2009", "Hiroaki Nakamura", "Hongbin", + "Hongbin Ma", "Hosun Lee", "HuFuwang", "Hui Wang", @@ -491,6 +494,7 @@ const char * auto_contributors[] { "Josh Taylor", "João Figueiredo", "Julian Gilyadov", + "Julian Maicher", "Julian Zhou", "Julio Jimenez", "Junfu Wu", @@ -917,6 +921,7 @@ const char * auto_contributors[] { "Thom O'Connor", "Thomas Berdy", "Thomas Casteleyn", + "Thomas Panetti", "Tian Xinhui", "Tiaonmmn", "Tigran Khudaverdyan", @@ -1022,6 +1027,7 @@ const char * auto_contributors[] { "Y Lu", "Yakko Majuri", "Yakov Olkhovskiy", + "YalalovSM", "Yangkuan Liu", "Yatian Xu", "Yatsishin Ilya", @@ -1300,6 +1306,7 @@ const char * auto_contributors[] { "kashwy", "keenwolf", "kevin wan", + "kevinyhzou", "kgurjev", "khamadiev", "kigerzhang", @@ -1545,6 +1552,7 @@ const char * auto_contributors[] { "tiger.yan", "tison", "topvisor", + "tpanetti", "turbo jason", "tyrionhuang", "ubuntu", @@ -1673,6 +1681,7 @@ const char * auto_contributors[] { "董海镔", "谢磊", "贾顺名(Jarvis)", + "郭小龙", "陈小玉", "靳阳", "黄朝晖", From 8ed1ec49d181123227192e7dfac6ee0119a2e8d7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 30 Jun 2023 15:21:13 +0000 Subject: [PATCH 491/515] Update version_date.tsv and changelogs after v23.6.1.1524-stable --- SECURITY.md | 1 + docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.6.1.1524-stable.md | 301 +++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 6 files changed, 306 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.6.1.1524-stable.md diff --git a/SECURITY.md b/SECURITY.md index 1864eb6e9e5..4ba5f13d09c 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -13,6 +13,7 @@ The following versions of ClickHouse server are currently being supported with s | Version | Supported | |:-|:-| +| 23.6 | ✔️ | | 23.5 | ✔️ | | 23.4 | ✔️ | | 23.3 | ✔️ | diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 6d53a6f4c51..f13fcdc14d6 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ esac ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release" -ARG VERSION="23.5.4.25" +ARG VERSION="23.6.1.1524" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 91b22346f13..5e5be3f6d73 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.5.4.25" +ARG VERSION="23.6.1.1524" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 0ed0e4e1168..8693193455f 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -23,7 +23,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.5.4.25" +ARG VERSION="23.6.1.1524" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.6.1.1524-stable.md b/docs/changelogs/v23.6.1.1524-stable.md new file mode 100644 index 00000000000..6d295d61ef4 --- /dev/null +++ b/docs/changelogs/v23.6.1.1524-stable.md @@ -0,0 +1,301 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.6.1.1524-stable (d1c7e13d088) FIXME as compared to v23.5.1.3174-stable (2fec796e73e) + +#### Backward Incompatible Change +* Delete feature `do_not_evict_index_and_mark_files` in the fs cache. This feature was only making things worse. [#51253](https://github.com/ClickHouse/ClickHouse/pull/51253) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Remove ALTER support for experimental LIVE VIEW. [#51287](https://github.com/ClickHouse/ClickHouse/pull/51287) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* Add setting `session_timezone`, it is used as default timezone for session when not explicitly specified. [#44149](https://github.com/ClickHouse/ClickHouse/pull/44149) ([Andrey Zvonov](https://github.com/zvonand)). +* Added overlay database engine and representation of a directory as a database This commit adds 4 databases: 1. DatabaseOverlay: Implements the IDatabase interface. Allow to combine multiple databases, such as FileSystem and Memory. Internally, it stores a vector with other database pointers and proxies requests to them in turn until it is executed successfully. 2. DatabaseFilesystem: allows to read-only interact with files stored on the file system. Internally, it uses TableFunctionFile to implicitly load file when a user requests the table. Result of TableFunctionFile call cached inside to provide quick access. 3. DatabaseS3: allows to read-only interact with s3 storage. It uses TableFunctionS3 to implicitly load table from s3 4. DatabaseHDFS: allows to interact with hdfs storage. It uses TableFunctionHDFS to implicitly load table from hdfs. [#48821](https://github.com/ClickHouse/ClickHouse/pull/48821) ([alekseygolub](https://github.com/alekseygolub)). +* Add a new setting named `use_mysql_types_in_show_columns` to alter the `SHOW COLUMNS` SQL statement to display MySQL equivalent types when a client is connected via the MySQL compatibility port. [#49577](https://github.com/ClickHouse/ClickHouse/pull/49577) ([Thomas Panetti](https://github.com/tpanetti)). +* Added option `--rename_files_after_processing `. This closes [#34207](https://github.com/ClickHouse/ClickHouse/issues/34207). [#49626](https://github.com/ClickHouse/ClickHouse/pull/49626) ([alekseygolub](https://github.com/alekseygolub)). +* 1. Add `TableFunctionRedis` 3. Add table engine Redis 4. Add `RedisCommon` which contains Redis related tools and types 5. Support `equals` and `in` filter push down into Redis. [#50150](https://github.com/ClickHouse/ClickHouse/pull/50150) ([JackyWoo](https://github.com/JackyWoo)). +* Allow to skip empty files in file/s3/url/hdfs table functions using settings `s3_skip_empty_files`, `hdfs_skip_empty_files`, `engine_file_skip_empty_files`, `engine_url_skip_empty_files`. [#50364](https://github.com/ClickHouse/ClickHouse/pull/50364) ([Kruglov Pavel](https://github.com/Avogar)). +* Clickhouse-client can now be called with a connection instead of "--host", "--port", "--user" etc. [#50689](https://github.com/ClickHouse/ClickHouse/pull/50689) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Codec DEFLATE_QPL is now controlled via server setting "enable_deflate_qpl_codec" (default: false) instead of setting "allow_experimental_codecs". This marks QPL_DEFLATE non-experimental. [#50775](https://github.com/ClickHouse/ClickHouse/pull/50775) ([Robert Schulze](https://github.com/rschu1ze)). + +#### Performance Improvement +* Improve performance with enabled QueryProfiler using thread-local timer_id instead of global object. [#48778](https://github.com/ClickHouse/ClickHouse/pull/48778) ([Jiebin Sun](https://github.com/jiebinn)). +* Rewrite CapnProto input/output format to improve its performance. Map column names and CapnProto fields case insensitive, fix reading/writing of nested structure fields. [#49752](https://github.com/ClickHouse/ClickHouse/pull/49752) ([Kruglov Pavel](https://github.com/Avogar)). +* Optimize parquet write performance for parallel threads. [#50102](https://github.com/ClickHouse/ClickHouse/pull/50102) ([Hongbin Ma](https://github.com/binmahone)). +* ### Documentation entry for user-facing changes Disable `parallelize_output_from_storages` for processing MATERIALIZED VIEWs and storages with one block only. [#50214](https://github.com/ClickHouse/ClickHouse/pull/50214) ([Azat Khuzhin](https://github.com/azat)). +* Merge PR https://github.com/ClickHouse/ClickHouse/pull/46558 (Avoid processing already sorted data). Avoid block permutation during sort if the block is already sorted. [#50697](https://github.com/ClickHouse/ClickHouse/pull/50697) ([Maksim Kita](https://github.com/kitaisreal)). +* In the earlier PRs ([#50062](https://github.com/ClickHouse/ClickHouse/issues/50062), [#50307](https://github.com/ClickHouse/ClickHouse/issues/50307)), we used to propose an optimization pattern which transforms the predicates with toYear/toYYYYMM into its equivalent but converter-free form. This transformation could bring significant performance impact to some workloads, such as SSB. However, as issue [#50628](https://github.com/ClickHouse/ClickHouse/issues/50628) indicated, these two PRs would introduce some issues which may results in incomplete query results, and as a result, they were reverted by [#50629](https://github.com/ClickHouse/ClickHouse/issues/50629). [#50951](https://github.com/ClickHouse/ClickHouse/pull/50951) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Make multiple list requests to ZooKeeper in parallel to speed up reading from system.zookeeper table. [#51042](https://github.com/ClickHouse/ClickHouse/pull/51042) ([Alexander Gololobov](https://github.com/davenger)). +* Speedup initialization of DateTime lookup tables for time zones. This should reduce startup/connect time of clickhouse client especially in debug build as it is rather heavy. [#51347](https://github.com/ClickHouse/ClickHouse/pull/51347) ([Alexander Gololobov](https://github.com/davenger)). + +#### Improvement +* Allow to cast IPv6 to IPv4 address for CIDR ::ffff:0:0/96 (IPv4-mapped addresses). [#49759](https://github.com/ClickHouse/ClickHouse/pull/49759) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Update MongoDB protocol to support MongoDB 5.1 version and newer. Support for the versions with the old protocol (<3.6) is preserved. Closes [#45621](https://github.com/ClickHouse/ClickHouse/issues/45621), [#49879](https://github.com/ClickHouse/ClickHouse/issues/49879). [#50061](https://github.com/ClickHouse/ClickHouse/pull/50061) ([Nikolay Degterinsky](https://github.com/evillique)). +* Improved scheduling of merge selecting and cleanup tasks in `ReplicatedMergeTree`. The tasks will not be executed too frequently when there's nothing to merge or cleanup. Added settings `max_merge_selecting_sleep_ms`, `merge_selecting_sleep_slowdown_factor`, `max_cleanup_delay_period` and `cleanup_thread_preferred_points_per_iteration`. It should close [#31919](https://github.com/ClickHouse/ClickHouse/issues/31919). [#50107](https://github.com/ClickHouse/ClickHouse/pull/50107) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Support parallel replicas with the analyzer. [#50441](https://github.com/ClickHouse/ClickHouse/pull/50441) ([Raúl Marín](https://github.com/Algunenano)). +* Add setting `input_format_max_bytes_to_read_for_schema_inference` to limit the number of bytes to read in schema inference. Closes [#50577](https://github.com/ClickHouse/ClickHouse/issues/50577). [#50592](https://github.com/ClickHouse/ClickHouse/pull/50592) ([Kruglov Pavel](https://github.com/Avogar)). +* Respect setting input_format_as_default in schema inference. [#50602](https://github.com/ClickHouse/ClickHouse/pull/50602) ([Kruglov Pavel](https://github.com/Avogar)). +* Make filter push down through cross join. [#50605](https://github.com/ClickHouse/ClickHouse/pull/50605) ([Han Fei](https://github.com/hanfei1991)). +* Actual lz4 version is used now. [#50621](https://github.com/ClickHouse/ClickHouse/pull/50621) ([Nikita Taranov](https://github.com/nickitat)). +* Allow to skip trailing empty lines in CSV/TSV/CustomSeparated formats via settings `input_format_csv_skip_trailing_empty_lines`, `input_format_tsv_skip_trailing_empty_lines` and `input_format_custom_skip_trailing_empty_lines` (disabled by default). Closes [#49315](https://github.com/ClickHouse/ClickHouse/issues/49315). [#50635](https://github.com/ClickHouse/ClickHouse/pull/50635) ([Kruglov Pavel](https://github.com/Avogar)). +* Functions "toDateOrDefault|OrNull()" and "accuateCast[OrDefault|OrNull]()" now correctly parse numeric arguments. [#50709](https://github.com/ClickHouse/ClickHouse/pull/50709) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Currently, the csv input format can not parse the csv file with whitespace or \t field delimiter, and these delimiters is supported in spark. [#50712](https://github.com/ClickHouse/ClickHouse/pull/50712) ([KevinyhZou](https://github.com/KevinyhZou)). +* Settings `number_of_mutations_to_delay` and `number_of_mutations_to_throw` are enabled by default now with values 500 and 1000 respectively. [#50726](https://github.com/ClickHouse/ClickHouse/pull/50726) ([Anton Popov](https://github.com/CurtizJ)). +* Keeper improvement: add feature flags for Keeper API. Each feature flag can be disabled or enabled by defining it under `keeper_server.feature_flags` config. E.g. to enable `CheckNotExists` request, `keeper_server.feature_flags.check_not_exists` should be set to `1` on Keeper. [#50796](https://github.com/ClickHouse/ClickHouse/pull/50796) ([Antonio Andelic](https://github.com/antonio2368)). +* The dashboard correctly shows missing values. This closes [#50831](https://github.com/ClickHouse/ClickHouse/issues/50831). [#50832](https://github.com/ClickHouse/ClickHouse/pull/50832) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* CGroups metrics related to CPU are replaced with one metric, `CGroupMaxCPU` for better usability. The `Normalized` CPU usage metrics will be normalized to CGroups limits instead of the total number of CPUs when they are set. This closes [#50836](https://github.com/ClickHouse/ClickHouse/issues/50836). [#50835](https://github.com/ClickHouse/ClickHouse/pull/50835) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Relax the thresholds for "too many parts" to be more modern. Return the backpressure during long-running insert queries. [#50856](https://github.com/ClickHouse/ClickHouse/pull/50856) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Added the possibility to use date and time arguments in syslog timestamp format in functions parseDateTimeBestEffort*() and parseDateTime64BestEffort*(). [#50925](https://github.com/ClickHouse/ClickHouse/pull/50925) ([Victor Krasnov](https://github.com/sirvickr)). +* Suggest using `APPEND` or `TRUNCATE` for `INTO OUTFILE` when file exists. [#50950](https://github.com/ClickHouse/ClickHouse/pull/50950) ([alekar](https://github.com/alekar)). +* Add embedded keeper-client to standalone keeper binary. [#50964](https://github.com/ClickHouse/ClickHouse/pull/50964) ([pufit](https://github.com/pufit)). +* Command line parameter "--password" in clickhouse-client can now be specified only once. [#50966](https://github.com/ClickHouse/ClickHouse/pull/50966) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Fix data lakes slowness because of synchronous head requests. (Related to Iceberg/Deltalake/Hudi being slow with a lot of files). [#50976](https://github.com/ClickHouse/ClickHouse/pull/50976) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Use `hash_of_all_files` from `system.parts` to check identity of parts during on-cluster backups. [#50997](https://github.com/ClickHouse/ClickHouse/pull/50997) ([Vitaly Baranov](https://github.com/vitlibar)). +* The system table zookeeper_connection connected_time identifies the time when the connection is established (standard format), and session_uptime_elapsed_seconds is added, which labels the duration of the established connection session (in seconds). [#51026](https://github.com/ClickHouse/ClickHouse/pull/51026) ([郭小龙](https://github.com/guoxiaolongzte)). +* Show halves of checksums in `system.parts`, `system.projection_parts` and in error messages in the correct order. [#51040](https://github.com/ClickHouse/ClickHouse/pull/51040) ([Vitaly Baranov](https://github.com/vitlibar)). +* Do not replicate `ALTER PARTITION` queries and mutations through `Replicated` database if it has only one shard and the underlying table is `ReplicatedMergeTree`. [#51049](https://github.com/ClickHouse/ClickHouse/pull/51049) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Improve the progress bar for file/s3/hdfs/url table functions by using chunk size from source data and using incremental total size counting in each thread. Fix the progress bar for *Cluster functions. This closes [#47250](https://github.com/ClickHouse/ClickHouse/issues/47250). [#51088](https://github.com/ClickHouse/ClickHouse/pull/51088) ([Kruglov Pavel](https://github.com/Avogar)). +* Add total_bytes_to_read to Progress packet in TCP protocol for better Progress bar. [#51158](https://github.com/ClickHouse/ClickHouse/pull/51158) ([Kruglov Pavel](https://github.com/Avogar)). +* Better checking of data parts on disks with filesystem cache. [#51164](https://github.com/ClickHouse/ClickHouse/pull/51164) ([Anton Popov](https://github.com/CurtizJ)). +* Disable cache setting `do_not_evict_index_and_mark_files` (Was enabled in `23.5`). [#51222](https://github.com/ClickHouse/ClickHouse/pull/51222) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix sometimes not correct current_elements_num in fs cache. [#51242](https://github.com/ClickHouse/ClickHouse/pull/51242) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add random sleep before merges/mutations execution to split load more evenly between replicas in case of zero-copy replication. [#51282](https://github.com/ClickHouse/ClickHouse/pull/51282) ([alesapin](https://github.com/alesapin)). +* The function `transform` as well as `CASE` with value matching started to support all data types. This closes [#29730](https://github.com/ClickHouse/ClickHouse/issues/29730). This closes [#32387](https://github.com/ClickHouse/ClickHouse/issues/32387). This closes [#50827](https://github.com/ClickHouse/ClickHouse/issues/50827). This closes [#31336](https://github.com/ClickHouse/ClickHouse/issues/31336). This closes [#40493](https://github.com/ClickHouse/ClickHouse/issues/40493). [#51351](https://github.com/ClickHouse/ClickHouse/pull/51351) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* We have found a bug in LLVM that makes the usage of `compile_expressions` setting unsafe. It is disabled by default. [#51368](https://github.com/ClickHouse/ClickHouse/pull/51368) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Issue [#50220](https://github.com/ClickHouse/ClickHouse/issues/50220) reports a core in `grace_hash` join. We finally reproduce the exception on local, and found that the issue is related to the failure of creating temporary file. Somehow this is triggered in https://github.com/ClickHouse/ClickHouse/pull/49816 https://github.com/ClickHouse/ClickHouse/pull/49483. [#51382](https://github.com/ClickHouse/ClickHouse/pull/51382) ([lgbo](https://github.com/lgbo-ustc)). + +#### Build/Testing/Packaging Improvement +* Update contrib/re2 to 2023-06-02. [#50949](https://github.com/ClickHouse/ClickHouse/pull/50949) ([Yuriy Chernyshov](https://github.com/georgthegreat)). +* ClickHouse server will print the list of changed settings on fatal errors. This closes [#51137](https://github.com/ClickHouse/ClickHouse/issues/51137). [#51138](https://github.com/ClickHouse/ClickHouse/pull/51138) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* In https://github.com/ClickHouse/ClickHouse/pull/51143 the fasstests failed, but the status wasn't created because of the chown `file not found`. This addresses it. Decrease the default values for `http-max-field-value-size` and `http_max_field_name_size` to 128K. [#51163](https://github.com/ClickHouse/ClickHouse/pull/51163) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update Ubuntu version in docker containers. [#51180](https://github.com/ClickHouse/ClickHouse/pull/51180) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Allow building ClickHouse with clang-17. [#51300](https://github.com/ClickHouse/ClickHouse/pull/51300) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* [SQLancer](https://github.com/sqlancer/sqlancer) check is considered stable as bugs that were triggered by it are fixed. Now failures of SQLancer check will be reported as failed check status. [#51340](https://github.com/ClickHouse/ClickHouse/pull/51340) ([Ilya Yatsishin](https://github.com/qoega)). +* Making our CI even better. [#51494](https://github.com/ClickHouse/ClickHouse/pull/51494) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Split huge `RUN` in Dockerfile into smaller conditional. Install the necessary tools on demand in the same `RUN` layer, and remove them after that. Upgrade the OS only once at the beginning. Use a modern way to check the signed repository. Downgrade the base repo to ubuntu:20.04 to address the issues on older docker versions. Upgrade golang version to address golang vulnerabilities. [#51504](https://github.com/ClickHouse/ClickHouse/pull/51504) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* This a follow-up for [#51504](https://github.com/ClickHouse/ClickHouse/issues/51504), the cleanup was lost during refactoring. [#51564](https://github.com/ClickHouse/ClickHouse/pull/51564) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Report loading status for executable dictionaries correctly [#48775](https://github.com/ClickHouse/ClickHouse/pull/48775) ([Anton Kozlov](https://github.com/tonickkozlov)). +* Proper mutation of skip indices and projections [#50104](https://github.com/ClickHouse/ClickHouse/pull/50104) ([Amos Bird](https://github.com/amosbird)). +* Cleanup moving parts [#50489](https://github.com/ClickHouse/ClickHouse/pull/50489) ([vdimir](https://github.com/vdimir)). +* Fix backward compatibility for IP types hashing in aggregate functions [#50551](https://github.com/ClickHouse/ClickHouse/pull/50551) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix Log family table return wrong rows count after truncate [#50585](https://github.com/ClickHouse/ClickHouse/pull/50585) ([flynn](https://github.com/ucasfl)). +* Fix bug in `uniqExact` parallel merging [#50590](https://github.com/ClickHouse/ClickHouse/pull/50590) ([Nikita Taranov](https://github.com/nickitat)). +* Revert recent grace hash join changes [#50699](https://github.com/ClickHouse/ClickHouse/pull/50699) ([vdimir](https://github.com/vdimir)). +* Query Cache: Try to fix bad cast from ColumnConst to ColumnVector [#50704](https://github.com/ClickHouse/ClickHouse/pull/50704) ([Robert Schulze](https://github.com/rschu1ze)). +* Do not read all the columns from right GLOBAL JOIN table. [#50721](https://github.com/ClickHouse/ClickHouse/pull/50721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Avoid storing logs in Keeper containing unknown operation [#50751](https://github.com/ClickHouse/ClickHouse/pull/50751) ([Antonio Andelic](https://github.com/antonio2368)). +* SummingMergeTree support for DateTime64 [#50797](https://github.com/ClickHouse/ClickHouse/pull/50797) ([Jordi Villar](https://github.com/jrdi)). +* Add compat setting for non-const timezones [#50834](https://github.com/ClickHouse/ClickHouse/pull/50834) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix type of LDAP server params hash in cache entry [#50865](https://github.com/ClickHouse/ClickHouse/pull/50865) ([Julian Maicher](https://github.com/jmaicher)). +* Fallback to parsing big integer from String instead of exception in Parquet format [#50873](https://github.com/ClickHouse/ClickHouse/pull/50873) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix checking the lock file too often while writing a backup [#50889](https://github.com/ClickHouse/ClickHouse/pull/50889) ([Vitaly Baranov](https://github.com/vitlibar)). +* Do not apply projection if read-in-order was enabled. [#50923](https://github.com/ClickHouse/ClickHouse/pull/50923) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix race azure blob storage iterator [#50936](https://github.com/ClickHouse/ClickHouse/pull/50936) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix erroneous `sort_description` propagation in `CreatingSets` [#50955](https://github.com/ClickHouse/ClickHouse/pull/50955) ([Nikita Taranov](https://github.com/nickitat)). +* Fix iceberg V2 optional metadata parsing [#50974](https://github.com/ClickHouse/ClickHouse/pull/50974) ([Kseniia Sumarokova](https://github.com/kssenii)). +* MaterializedMySQL: Keep parentheses for empty table overrides [#50977](https://github.com/ClickHouse/ClickHouse/pull/50977) ([Val Doroshchuk](https://github.com/valbok)). +* Fix crash in BackupCoordinationStageSync::setError() [#51012](https://github.com/ClickHouse/ClickHouse/pull/51012) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix subtly broken copy-on-write of ColumnLowCardinality dictionary [#51064](https://github.com/ClickHouse/ClickHouse/pull/51064) ([Michael Kolupaev](https://github.com/al13n321)). +* Generate safe IVs [#51086](https://github.com/ClickHouse/ClickHouse/pull/51086) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Fix ineffective query cache for SELECTs with subqueries [#51132](https://github.com/ClickHouse/ClickHouse/pull/51132) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix Set index with constant nullable comparison. [#51205](https://github.com/ClickHouse/ClickHouse/pull/51205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix a crash in s3 and s3Cluster functions [#51209](https://github.com/ClickHouse/ClickHouse/pull/51209) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix core dump when compile expression [#51231](https://github.com/ClickHouse/ClickHouse/pull/51231) ([LiuNeng](https://github.com/liuneng1994)). +* Fix use-after-free in StorageURL when switching URLs [#51260](https://github.com/ClickHouse/ClickHouse/pull/51260) ([Michael Kolupaev](https://github.com/al13n321)). +* Updated check for parameterized view [#51272](https://github.com/ClickHouse/ClickHouse/pull/51272) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix multiple writing of same file to backup [#51299](https://github.com/ClickHouse/ClickHouse/pull/51299) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix fuzzer failure in ActionsDAG [#51301](https://github.com/ClickHouse/ClickHouse/pull/51301) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove garbage from function `transform` [#51350](https://github.com/ClickHouse/ClickHouse/pull/51350) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix MSan report in lowerUTF8/upperUTF8 [#51371](https://github.com/ClickHouse/ClickHouse/pull/51371) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* fs cache: fix a bit incorrect use_count after [#44985](https://github.com/ClickHouse/ClickHouse/issues/44985) [#51406](https://github.com/ClickHouse/ClickHouse/pull/51406) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix segfault in MathUnary [#51499](https://github.com/ClickHouse/ClickHouse/pull/51499) ([Ilya Yatsishin](https://github.com/qoega)). +* Fix logical assert in `tupleElement()` with default values [#51534](https://github.com/ClickHouse/ClickHouse/pull/51534) ([Robert Schulze](https://github.com/rschu1ze)). +* fs cache: remove file from opened file cache immediately when evicting file [#51596](https://github.com/ClickHouse/ClickHouse/pull/51596) ([Kseniia Sumarokova](https://github.com/kssenii)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Deprecate delete-on-destroy.txt [#49181](https://github.com/ClickHouse/ClickHouse/pull/49181) ([Alexander Gololobov](https://github.com/davenger)). +* Attempt to increase the general runners' survival rate [#49283](https://github.com/ClickHouse/ClickHouse/pull/49283) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Refactor subqueries for IN [#49570](https://github.com/ClickHouse/ClickHouse/pull/49570) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Test plan optimization analyzer [#50095](https://github.com/ClickHouse/ClickHouse/pull/50095) ([Igor Nikonov](https://github.com/devcrafter)). +* Implement endianness-independent serialization for quantileTiming [#50324](https://github.com/ClickHouse/ClickHouse/pull/50324) ([ltrk2](https://github.com/ltrk2)). +* require `finalize()` call before d-tor for all writes buffers [#50395](https://github.com/ClickHouse/ClickHouse/pull/50395) ([Sema Checherinda](https://github.com/CheSema)). +* Implement big-endian support for the deterministic reservoir sampler [#50405](https://github.com/ClickHouse/ClickHouse/pull/50405) ([ltrk2](https://github.com/ltrk2)). +* Fix compilation error on big-endian platforms [#50406](https://github.com/ClickHouse/ClickHouse/pull/50406) ([ltrk2](https://github.com/ltrk2)). +* Attach gdb in stateless tests [#50487](https://github.com/ClickHouse/ClickHouse/pull/50487) ([Kruglov Pavel](https://github.com/Avogar)). +* JIT infrastructure refactoring [#50531](https://github.com/ClickHouse/ClickHouse/pull/50531) ([Maksim Kita](https://github.com/kitaisreal)). +* Analyzer: Do not apply Query Tree optimizations on shards [#50584](https://github.com/ClickHouse/ClickHouse/pull/50584) ([Dmitry Novik](https://github.com/novikd)). +* Increase max array size in group bitmap [#50620](https://github.com/ClickHouse/ClickHouse/pull/50620) ([Kruglov Pavel](https://github.com/Avogar)). +* Misc Annoy index improvements [#50661](https://github.com/ClickHouse/ClickHouse/pull/50661) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix reading negative decimals in avro format [#50668](https://github.com/ClickHouse/ClickHouse/pull/50668) ([Kruglov Pavel](https://github.com/Avogar)). +* Unify priorities for connection pools [#50675](https://github.com/ClickHouse/ClickHouse/pull/50675) ([Sergei Trifonov](https://github.com/serxa)). +* Prostpone check of outdated parts [#50676](https://github.com/ClickHouse/ClickHouse/pull/50676) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Unify priorities: `IExecutableTask`s [#50677](https://github.com/ClickHouse/ClickHouse/pull/50677) ([Sergei Trifonov](https://github.com/serxa)). +* Disable grace_hash join in stress tests [#50693](https://github.com/ClickHouse/ClickHouse/pull/50693) ([vdimir](https://github.com/vdimir)). +* ReverseTransform small improvement [#50698](https://github.com/ClickHouse/ClickHouse/pull/50698) ([Maksim Kita](https://github.com/kitaisreal)). +* Support OPTIMIZE for temporary tables [#50710](https://github.com/ClickHouse/ClickHouse/pull/50710) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Refactor reading from object storages [#50711](https://github.com/ClickHouse/ClickHouse/pull/50711) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix data race in log message of cached buffer [#50723](https://github.com/ClickHouse/ClickHouse/pull/50723) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add new keywords into projections documentation [#50743](https://github.com/ClickHouse/ClickHouse/pull/50743) ([YalalovSM](https://github.com/YalalovSM)). +* Fix build for aarch64 (temporary disable azure) [#50770](https://github.com/ClickHouse/ClickHouse/pull/50770) ([alesapin](https://github.com/alesapin)). +* Update version after release [#50772](https://github.com/ClickHouse/ClickHouse/pull/50772) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update version_date.tsv and changelogs after v23.5.1.3174-stable [#50774](https://github.com/ClickHouse/ClickHouse/pull/50774) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update CHANGELOG.md [#50788](https://github.com/ClickHouse/ClickHouse/pull/50788) ([Ilya Yatsishin](https://github.com/qoega)). +* Update version_date.tsv and changelogs after v23.2.7.32-stable [#50809](https://github.com/ClickHouse/ClickHouse/pull/50809) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Desctructing --> Destructing [#50810](https://github.com/ClickHouse/ClickHouse/pull/50810) ([Robert Schulze](https://github.com/rschu1ze)). +* Don't mark a part as broken on `Poco::TimeoutException` [#50811](https://github.com/ClickHouse/ClickHouse/pull/50811) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Rename azure_blob_storage to azureBlobStorage [#50812](https://github.com/ClickHouse/ClickHouse/pull/50812) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix ParallelReadBuffer seek [#50820](https://github.com/ClickHouse/ClickHouse/pull/50820) ([Michael Kolupaev](https://github.com/al13n321)). +* [RFC] Print git hash when crashing [#50823](https://github.com/ClickHouse/ClickHouse/pull/50823) ([Michael Kolupaev](https://github.com/al13n321)). +* Add tests for function "transform" [#50833](https://github.com/ClickHouse/ClickHouse/pull/50833) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update version_date.tsv and changelogs after v23.5.2.7-stable [#50844](https://github.com/ClickHouse/ClickHouse/pull/50844) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Updated changelog with azureBlobStorage table function & engine entry [#50850](https://github.com/ClickHouse/ClickHouse/pull/50850) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Update easy_tasks_sorted_ru.md [#50853](https://github.com/ClickHouse/ClickHouse/pull/50853) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Document x86 / ARM prerequisites for Docker image [#50867](https://github.com/ClickHouse/ClickHouse/pull/50867) ([Robert Schulze](https://github.com/rschu1ze)). +* MaterializedMySQL: Add test_named_collections [#50874](https://github.com/ClickHouse/ClickHouse/pull/50874) ([Val Doroshchuk](https://github.com/valbok)). +* Update version_date.tsv and changelogs after v22.8.18.31-lts [#50881](https://github.com/ClickHouse/ClickHouse/pull/50881) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v23.3.3.52-lts [#50882](https://github.com/ClickHouse/ClickHouse/pull/50882) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v23.4.3.48-stable [#50883](https://github.com/ClickHouse/ClickHouse/pull/50883) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* MaterializedMySQL: Add additional test case to insert_with_modify_binlog_checksum [#50884](https://github.com/ClickHouse/ClickHouse/pull/50884) ([Val Doroshchuk](https://github.com/valbok)). +* Update broken tests list [#50886](https://github.com/ClickHouse/ClickHouse/pull/50886) ([Dmitry Novik](https://github.com/novikd)). +* Fix LOGICAL_ERROR in snowflakeToDateTime*() [#50893](https://github.com/ClickHouse/ClickHouse/pull/50893) ([Robert Schulze](https://github.com/rschu1ze)). +* Tests with parallel replicas are no more "always green" [#50896](https://github.com/ClickHouse/ClickHouse/pull/50896) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Slightly more information in error message about cached disk [#50897](https://github.com/ClickHouse/ClickHouse/pull/50897) ([Michael Kolupaev](https://github.com/al13n321)). +* do not call finalize after exception [#50907](https://github.com/ClickHouse/ClickHouse/pull/50907) ([Sema Checherinda](https://github.com/CheSema)). +* Update Annoy docs [#50912](https://github.com/ClickHouse/ClickHouse/pull/50912) ([Robert Schulze](https://github.com/rschu1ze)). +* A bit safer UserDefinedSQLFunctionVisitor [#50913](https://github.com/ClickHouse/ClickHouse/pull/50913) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update contribe/orc in .gitmodules [#50920](https://github.com/ClickHouse/ClickHouse/pull/50920) ([San](https://github.com/santrancisco)). +* MaterializedMySQL: Add missing DROP DATABASE for tests [#50924](https://github.com/ClickHouse/ClickHouse/pull/50924) ([Val Doroshchuk](https://github.com/valbok)). +* Fix 'Illegal column timezone' in stress tests [#50929](https://github.com/ClickHouse/ClickHouse/pull/50929) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix tests sanity checks and avoid dropping system.query_log table [#50934](https://github.com/ClickHouse/ClickHouse/pull/50934) ([Azat Khuzhin](https://github.com/azat)). +* Fix tests for throttling by allowing more margin of error for trottling event [#50935](https://github.com/ClickHouse/ClickHouse/pull/50935) ([Azat Khuzhin](https://github.com/azat)). +* 01746_convert_type_with_default: Temporarily disable flaky test [#50937](https://github.com/ClickHouse/ClickHouse/pull/50937) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix the statless tests image for old commits [#50947](https://github.com/ClickHouse/ClickHouse/pull/50947) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix logic in `AsynchronousBoundedReadBuffer::seek` [#50952](https://github.com/ClickHouse/ClickHouse/pull/50952) ([Nikita Taranov](https://github.com/nickitat)). +* Uncomment flaky test (01746_convert_type_with_default) [#50954](https://github.com/ClickHouse/ClickHouse/pull/50954) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Fix keeper-client help message [#50965](https://github.com/ClickHouse/ClickHouse/pull/50965) ([pufit](https://github.com/pufit)). +* fix build issue on clang 15 [#50967](https://github.com/ClickHouse/ClickHouse/pull/50967) ([Chang chen](https://github.com/baibaichen)). +* Docs: Fix embedded video link [#50972](https://github.com/ClickHouse/ClickHouse/pull/50972) ([Robert Schulze](https://github.com/rschu1ze)). +* Change submodule capnproto to it's fork in ClickHouse [#50987](https://github.com/ClickHouse/ClickHouse/pull/50987) ([Kruglov Pavel](https://github.com/Avogar)). +* Attempt to make 01281_group_by_limit_memory_tracking not flaky [#50995](https://github.com/ClickHouse/ClickHouse/pull/50995) ([Dmitry Novik](https://github.com/novikd)). +* Fix flaky 02561_null_as_default_more_formats [#51001](https://github.com/ClickHouse/ClickHouse/pull/51001) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix flaky test_seekable_formats [#51002](https://github.com/ClickHouse/ClickHouse/pull/51002) ([Kruglov Pavel](https://github.com/Avogar)). +* Follow-up to [#50448](https://github.com/ClickHouse/ClickHouse/issues/50448) [#51006](https://github.com/ClickHouse/ClickHouse/pull/51006) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix a versions' tweak for tagged commits, improve version_helper [#51035](https://github.com/ClickHouse/ClickHouse/pull/51035) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Sqlancer has changed master to main [#51060](https://github.com/ClickHouse/ClickHouse/pull/51060) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Do not spam sqlancer build log [#51061](https://github.com/ClickHouse/ClickHouse/pull/51061) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Refactor IColumn::forEachSubcolumn to make it slightly harder to implement incorrectly [#51072](https://github.com/ClickHouse/ClickHouse/pull/51072) ([Michael Kolupaev](https://github.com/al13n321)). +* MaterializedMySQL: Rename materialize_with_ddl.py -> materialized_with_ddl [#51074](https://github.com/ClickHouse/ClickHouse/pull/51074) ([Val Doroshchuk](https://github.com/valbok)). +* Improve woboq browser report [#51077](https://github.com/ClickHouse/ClickHouse/pull/51077) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix for part_names_mutex used after destruction [#51099](https://github.com/ClickHouse/ClickHouse/pull/51099) ([Alexander Gololobov](https://github.com/davenger)). +* Fix ColumnConst::forEachSubcolumn missing from previous PR [#51102](https://github.com/ClickHouse/ClickHouse/pull/51102) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix the test 02783_parsedatetimebesteffort_syslog flakiness [#51112](https://github.com/ClickHouse/ClickHouse/pull/51112) ([Victor Krasnov](https://github.com/sirvickr)). +* Compatibility with clang-17 [#51114](https://github.com/ClickHouse/ClickHouse/pull/51114) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Make more parallel get requests to ZooKeeper in system.zookeeper [#51118](https://github.com/ClickHouse/ClickHouse/pull/51118) ([Alexander Gololobov](https://github.com/davenger)). +* Fix 02703_max_local_write_bandwidth flakiness [#51120](https://github.com/ClickHouse/ClickHouse/pull/51120) ([Azat Khuzhin](https://github.com/azat)). +* Update version_date.tsv and changelogs after v23.5.3.24-stable [#51121](https://github.com/ClickHouse/ClickHouse/pull/51121) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v23.4.4.16-stable [#51122](https://github.com/ClickHouse/ClickHouse/pull/51122) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v23.3.4.17-lts [#51123](https://github.com/ClickHouse/ClickHouse/pull/51123) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v22.8.19.10-lts [#51124](https://github.com/ClickHouse/ClickHouse/pull/51124) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix typo [#51126](https://github.com/ClickHouse/ClickHouse/pull/51126) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Slightly better diagnostics [#51127](https://github.com/ClickHouse/ClickHouse/pull/51127) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Small fix in `MergeTreePrefetchedReadPool` [#51131](https://github.com/ClickHouse/ClickHouse/pull/51131) ([Nikita Taranov](https://github.com/nickitat)). +* Don't report table function accesses to system.errors [#51147](https://github.com/ClickHouse/ClickHouse/pull/51147) ([Raúl Marín](https://github.com/Algunenano)). +* Fix SQLancer branch name [#51148](https://github.com/ClickHouse/ClickHouse/pull/51148) ([Ilya Yatsishin](https://github.com/qoega)). +* Revert "Added ability to implicitly use file/hdfs/s3 table functions in clickhouse-local" [#51149](https://github.com/ClickHouse/ClickHouse/pull/51149) ([Alexander Tokmakov](https://github.com/tavplubix)). +* More profile events for fs cache [#51161](https://github.com/ClickHouse/ClickHouse/pull/51161) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Unforget to pass callback to readBigAt() in ParallelReadBuffer [#51165](https://github.com/ClickHouse/ClickHouse/pull/51165) ([Michael Kolupaev](https://github.com/al13n321)). +* Update README.md [#51179](https://github.com/ClickHouse/ClickHouse/pull/51179) ([Tyler Hannan](https://github.com/tylerhannan)). +* Update exception message [#51187](https://github.com/ClickHouse/ClickHouse/pull/51187) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Split long test 02149_schema_inference_formats_with_schema into several tests to avoid timeout in debug [#51197](https://github.com/ClickHouse/ClickHouse/pull/51197) ([Kruglov Pavel](https://github.com/Avogar)). +* Avoid initializing DateLUT from emptyArray function registration [#51199](https://github.com/ClickHouse/ClickHouse/pull/51199) ([Alexander Gololobov](https://github.com/davenger)). +* Suppress check for covered parts in ZooKeeper [#51207](https://github.com/ClickHouse/ClickHouse/pull/51207) ([Alexander Tokmakov](https://github.com/tavplubix)). +* One more profile event for fs cache [#51223](https://github.com/ClickHouse/ClickHouse/pull/51223) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Typo: passowrd_sha256_hex --> password_sha256_hex [#51233](https://github.com/ClickHouse/ClickHouse/pull/51233) ([Robert Schulze](https://github.com/rschu1ze)). +* Introduce settings enum field with auto-generated values list [#51237](https://github.com/ClickHouse/ClickHouse/pull/51237) ([Sergei Trifonov](https://github.com/serxa)). +* Drop session if we fail to get Keeper API version [#51238](https://github.com/ClickHouse/ClickHouse/pull/51238) ([Alexander Gololobov](https://github.com/davenger)). +* Revert "Fix a crash in s3 and s3Cluster functions" [#51239](https://github.com/ClickHouse/ClickHouse/pull/51239) ([Alexander Tokmakov](https://github.com/tavplubix)). +* fix flaky `AsyncLoader` destructor [#51245](https://github.com/ClickHouse/ClickHouse/pull/51245) ([Sergei Trifonov](https://github.com/serxa)). +* Docs: little cleanup of configuration-files.md [#51249](https://github.com/ClickHouse/ClickHouse/pull/51249) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix a stupid bug on Replicated database recovery [#51252](https://github.com/ClickHouse/ClickHouse/pull/51252) ([Alexander Tokmakov](https://github.com/tavplubix)). +* FileCache: tryReserve() slight improvement [#51259](https://github.com/ClickHouse/ClickHouse/pull/51259) ([Igor Nikonov](https://github.com/devcrafter)). +* Ugly hotfix for "terminate on uncaught exception" in WriteBufferFromOStream [#51265](https://github.com/ClickHouse/ClickHouse/pull/51265) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Avoid too many calls to Poco::Logger::get [#51266](https://github.com/ClickHouse/ClickHouse/pull/51266) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Update version_date.tsv and changelogs after v23.3.5.9-lts [#51269](https://github.com/ClickHouse/ClickHouse/pull/51269) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Better reporting of broken parts [#51270](https://github.com/ClickHouse/ClickHouse/pull/51270) ([Anton Popov](https://github.com/CurtizJ)). +* Update ext-dict-functions.md [#51283](https://github.com/ClickHouse/ClickHouse/pull/51283) ([Mike Kot](https://github.com/myrrc)). +* Disable table structure check for secondary queries from Replicated db [#51284](https://github.com/ClickHouse/ClickHouse/pull/51284) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Define Thrift version for parquet and use correct arrow version [#51285](https://github.com/ClickHouse/ClickHouse/pull/51285) ([Kruglov Pavel](https://github.com/Avogar)). +* Restore Azure build on ARM [#51288](https://github.com/ClickHouse/ClickHouse/pull/51288) ([Robert Schulze](https://github.com/rschu1ze)). +* Query Cache: Un-comment settings in server cfg [#51294](https://github.com/ClickHouse/ClickHouse/pull/51294) ([Robert Schulze](https://github.com/rschu1ze)). +* Require more checks [#51295](https://github.com/ClickHouse/ClickHouse/pull/51295) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix metadata loading test [#51297](https://github.com/ClickHouse/ClickHouse/pull/51297) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Scratch the strange Python code [#51302](https://github.com/ClickHouse/ClickHouse/pull/51302) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#47865](https://github.com/ClickHouse/ClickHouse/issues/47865) [#51306](https://github.com/ClickHouse/ClickHouse/pull/51306) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#48894](https://github.com/ClickHouse/ClickHouse/issues/48894) [#51307](https://github.com/ClickHouse/ClickHouse/pull/51307) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#48676](https://github.com/ClickHouse/ClickHouse/issues/48676) [#51308](https://github.com/ClickHouse/ClickHouse/pull/51308) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix long test `functions_bad_arguments` [#51310](https://github.com/ClickHouse/ClickHouse/pull/51310) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Unify merge predicate [#51344](https://github.com/ClickHouse/ClickHouse/pull/51344) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix using locks in ProcessList [#51348](https://github.com/ClickHouse/ClickHouse/pull/51348) ([Vitaly Baranov](https://github.com/vitlibar)). +* Add a test for [#42631](https://github.com/ClickHouse/ClickHouse/issues/42631) [#51353](https://github.com/ClickHouse/ClickHouse/pull/51353) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix performance tests due to warnings from jemalloc about Per-CPU arena disabled [#51362](https://github.com/ClickHouse/ClickHouse/pull/51362) ([Azat Khuzhin](https://github.com/azat)). +* Fix "merge_truncate_long" test [#51369](https://github.com/ClickHouse/ClickHouse/pull/51369) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Increase timeout of Fast Test [#51372](https://github.com/ClickHouse/ClickHouse/pull/51372) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad tests for DNS [#51374](https://github.com/ClickHouse/ClickHouse/pull/51374) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Attempt to fix the `relax_too_many_parts` test [#51375](https://github.com/ClickHouse/ClickHouse/pull/51375) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix MySQL test in Debug mode [#51376](https://github.com/ClickHouse/ClickHouse/pull/51376) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad test `01018_Distributed__shard_num` [#51377](https://github.com/ClickHouse/ClickHouse/pull/51377) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix "logical error" in addressToLineWithInlines [#51379](https://github.com/ClickHouse/ClickHouse/pull/51379) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test 01280_ttl_where_group_by [#51380](https://github.com/ClickHouse/ClickHouse/pull/51380) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Attempt to fix `test_ssl_cert_authentication` [#51384](https://github.com/ClickHouse/ClickHouse/pull/51384) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Revert "Merge pull request [#50951](https://github.com/ClickHouse/ClickHouse/issues/50951) from ZhiguoZh/20230607-toyear-fix" [#51390](https://github.com/ClickHouse/ClickHouse/pull/51390) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Two tests are twice longer in average with Analyzer and sometimes failing [#51391](https://github.com/ClickHouse/ClickHouse/pull/51391) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix 00899_long_attach_memory_limit [#51395](https://github.com/ClickHouse/ClickHouse/pull/51395) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test 01293_optimize_final_force [#51396](https://github.com/ClickHouse/ClickHouse/pull/51396) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test 02481_parquet_list_monotonically_increasing_offsets [#51397](https://github.com/ClickHouse/ClickHouse/pull/51397) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test 02497_trace_events_stress_long [#51398](https://github.com/ClickHouse/ClickHouse/pull/51398) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix broken labeling for `manual approve` [#51405](https://github.com/ClickHouse/ClickHouse/pull/51405) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix parts lifetime in `MergeTreeTransaction` [#51407](https://github.com/ClickHouse/ClickHouse/pull/51407) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix flaky test test_skip_empty_files [#51409](https://github.com/ClickHouse/ClickHouse/pull/51409) ([Kruglov Pavel](https://github.com/Avogar)). +* fix flacky test test_profile_events_s3 [#51412](https://github.com/ClickHouse/ClickHouse/pull/51412) ([Sema Checherinda](https://github.com/CheSema)). +* Update README.md [#51413](https://github.com/ClickHouse/ClickHouse/pull/51413) ([Tyler Hannan](https://github.com/tylerhannan)). +* Replace try/catch logic in hasTokenOrNull() by something more lightweight [#51425](https://github.com/ClickHouse/ClickHouse/pull/51425) ([Robert Schulze](https://github.com/rschu1ze)). +* Add retries to `tlsv1_3` tests [#51434](https://github.com/ClickHouse/ClickHouse/pull/51434) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Update exception message [#51440](https://github.com/ClickHouse/ClickHouse/pull/51440) ([Kseniia Sumarokova](https://github.com/kssenii)). +* fs cache: add check for intersecting ranges [#51444](https://github.com/ClickHouse/ClickHouse/pull/51444) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Slightly better code around packets for parallel replicas [#51451](https://github.com/ClickHouse/ClickHouse/pull/51451) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Update system_warnings test [#51453](https://github.com/ClickHouse/ClickHouse/pull/51453) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Many fixes [#51455](https://github.com/ClickHouse/ClickHouse/pull/51455) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test 01605_adaptive_granularity_block_borders [#51457](https://github.com/ClickHouse/ClickHouse/pull/51457) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Try fix flaky 02497_storage_file_reader_selection [#51468](https://github.com/ClickHouse/ClickHouse/pull/51468) ([Kruglov Pavel](https://github.com/Avogar)). +* Try making Keeper in `DatabaseReplicated` tests more stable [#51473](https://github.com/ClickHouse/ClickHouse/pull/51473) ([Antonio Andelic](https://github.com/antonio2368)). +* Convert 02003_memory_limit_in_client from expect to sh test (to fix flakiness) [#51475](https://github.com/ClickHouse/ClickHouse/pull/51475) ([Azat Khuzhin](https://github.com/azat)). +* Fix test_disk_over_web_server [#51476](https://github.com/ClickHouse/ClickHouse/pull/51476) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Delay shutdown of system and temporary databases [#51479](https://github.com/ClickHouse/ClickHouse/pull/51479) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix memory leakage in CompressionCodecDeflateQpl [#51480](https://github.com/ClickHouse/ClickHouse/pull/51480) ([Vitaly Baranov](https://github.com/vitlibar)). +* Increase retries in test_multiple_disks/test.py::test_start_stop_moves [#51482](https://github.com/ClickHouse/ClickHouse/pull/51482) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix race in BoundedReadBuffer [#51484](https://github.com/ClickHouse/ClickHouse/pull/51484) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky unit test [#51485](https://github.com/ClickHouse/ClickHouse/pull/51485) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test `test_host_regexp_multiple_ptr_records` [#51506](https://github.com/ClickHouse/ClickHouse/pull/51506) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add a comment [#51517](https://github.com/ClickHouse/ClickHouse/pull/51517) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Make `test_ssl_cert_authentication` similar to `test_tlvs1_3` [#51520](https://github.com/ClickHouse/ClickHouse/pull/51520) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix duplicate storage set logical error. [#51521](https://github.com/ClickHouse/ClickHouse/pull/51521) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Update test_storage_postgresql/test.py::test_concurrent_queries [#51523](https://github.com/ClickHouse/ClickHouse/pull/51523) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix FATAL: query context is not detached from thread group [#51540](https://github.com/ClickHouse/ClickHouse/pull/51540) ([Igor Nikonov](https://github.com/devcrafter)). +* Update version_date.tsv and changelogs after v23.3.6.7-lts [#51548](https://github.com/ClickHouse/ClickHouse/pull/51548) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Decoupled commits from [#51180](https://github.com/ClickHouse/ClickHouse/issues/51180) for backports [#51561](https://github.com/ClickHouse/ClickHouse/pull/51561) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Try to fix deadlock in ZooKeeper client [#51563](https://github.com/ClickHouse/ClickHouse/pull/51563) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Retry chroot creation in ZK before stateless tests [#51585](https://github.com/ClickHouse/ClickHouse/pull/51585) ([Antonio Andelic](https://github.com/antonio2368)). +* use timeout instead trap in 01443_merge_truncate_long.sh [#51593](https://github.com/ClickHouse/ClickHouse/pull/51593) ([Sema Checherinda](https://github.com/CheSema)). +* Update version_date.tsv and changelogs after v23.5.4.25-stable [#51604](https://github.com/ClickHouse/ClickHouse/pull/51604) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix MergeTreeMarksLoader segfaulting if marks file is longer than expected [#51636](https://github.com/ClickHouse/ClickHouse/pull/51636) ([Michael Kolupaev](https://github.com/al13n321)). +* Update version_date.tsv and changelogs after v23.4.5.22-stable [#51638](https://github.com/ClickHouse/ClickHouse/pull/51638) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v23.3.7.5-lts [#51639](https://github.com/ClickHouse/ClickHouse/pull/51639) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update parts.md [#51643](https://github.com/ClickHouse/ClickHouse/pull/51643) ([Ramazan Polat](https://github.com/ramazanpolat)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 4ca5d1d7497..2a098d8c1da 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.6.1.1524-stable 2023-06-30 v23.5.4.25-stable 2023-06-29 v23.5.3.24-stable 2023-06-17 v23.5.2.7-stable 2023-06-10 From 32aa66922a5bd26b4ced763130a8a401516c365f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 30 Jun 2023 18:31:46 +0200 Subject: [PATCH 492/515] Update 02808_custom_disk_with_user_defined_name.sh --- .../0_stateless/02808_custom_disk_with_user_defined_name.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh index 3fbcde66add..a43cd6deb9e 100755 --- a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh +++ b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh @@ -25,7 +25,7 @@ $CLICKHOUSE_CLIENT -nm --query """ DROP TABLE IF EXISTS test; CREATE TABLE test (a Int32, b String) ENGINE = MergeTree() ORDER BY tuple() -SETTINGS disk = disk_$disk_name(type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3disk); +SETTINGS disk = disk_$disk_name(type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); """ $CLICKHOUSE_CLIENT -nm --query """ From fe17a9b589a5bae6e2608fe101558fdcbd761107 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 30 Jun 2023 19:36:32 +0200 Subject: [PATCH 493/515] do not log aborted for pending mutate/merge when shutdown --- src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index b569fa73a73..d4f8d1140a2 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -182,7 +182,6 @@ void MergeTreeBackgroundExecutor::removeTasksCorrespondingToStorage(Stora } catch (...) { - printExceptionWithRespectToAbort(log); pending.remove(id); } From 4511213a1f61ae7d67c8c44b6bab0ff3fe6ebaaa Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Fri, 30 Jun 2023 13:52:22 -0400 Subject: [PATCH 494/515] job successful Signed-off-by: Julio Jimenez --- .github/workflows/nightly.yml | 2 -- 1 file changed, 2 deletions(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 3f6d9b86fd6..cf61012f2bc 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -121,8 +121,6 @@ jobs: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" SonarCloud: - # TODO: Remove if: whenever SonarCloud supports c++23 - # if: ${{ false }} runs-on: [self-hosted, builder] env: SONAR_SCANNER_VERSION: 4.8.0.2856 From 5d652c4fd5befcaabfdf0fd6c7f1beec160708b3 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 30 Jun 2023 20:30:50 +0000 Subject: [PATCH 495/515] Finalize ColumnObject with only default values --- src/Columns/ColumnObject.cpp | 14 ++------------ src/Processors/QueryPlan/AggregatingStep.cpp | 2 ++ ...02789_object_type_invalid_num_of_rows.reference | 1 + 3 files changed, 5 insertions(+), 12 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 4fa0c3ee41a..07872774559 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -626,9 +626,6 @@ ColumnObject::ColumnObject(Subcolumns && subcolumns_, bool is_nullable_) void ColumnObject::checkConsistency() const { - if (num_rows && subcolumns.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "ColumnObject is inconsistent: it has no subcolumns, but has {} rows", num_rows); - if (subcolumns.empty()) return; @@ -645,9 +642,9 @@ void ColumnObject::checkConsistency() const size_t ColumnObject::size() const { -// #ifndef NDEBUG +#ifndef NDEBUG checkConsistency(); -// #endif +#endif return num_rows; } @@ -714,8 +711,6 @@ void ColumnObject::insert(const Field & field) } ++num_rows; - - checkConsistency(); } void ColumnObject::insertDefault() @@ -724,8 +719,6 @@ void ColumnObject::insertDefault() entry->data.insertDefault(); ++num_rows; - - checkConsistency(); } Field ColumnObject::operator[](size_t n) const @@ -783,8 +776,6 @@ void ColumnObject::insertRangeFrom(const IColumn & src, size_t start, size_t len num_rows += length; finalize(); - - checkConsistency(); } void ColumnObject::popBack(size_t length) @@ -793,7 +784,6 @@ void ColumnObject::popBack(size_t length) entry->data.popBack(length); num_rows -= length; - checkConsistency(); } template diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 4ac972e2a79..eebbfc04304 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -319,6 +319,8 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B { auto column_with_default = col.column->cloneEmpty(); col.type->insertDefaultInto(*column_with_default); + column_with_default->finalize(); + auto column = ColumnConst::create(std::move(column_with_default), 0); const auto * node = &dag->addColumn({ColumnPtr(std::move(column)), col.type, col.name}); node = &dag->materializeNode(*node); diff --git a/tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.reference b/tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.reference index e69de29bb2d..7dec35f7acb 100644 --- a/tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.reference +++ b/tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.reference @@ -0,0 +1 @@ +0.02 From 5e1cfb05318b8d3d511208146db020ae560e2ff7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sat, 1 Jul 2023 08:53:23 +0000 Subject: [PATCH 496/515] Avoid copying block if optimize_on_insert is false --- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index a13a96ac65e..adb162b9fe7 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -452,7 +452,7 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) if constexpr (async_insert) { /// we copy everything but offsets which we move because they are only used by async insert - if (storage.writer.getMergingMode() != MergeTreeData::MergingParams::Mode::Ordinary) + if (settings.optimize_on_insert && storage.writer.getMergingMode() != MergeTreeData::MergingParams::Mode::Ordinary) unmerged_block.emplace(Block(current_block.block), Row(current_block.partition), std::move(current_block.offsets)); } From a830ba399bd6979ab93646cd4fba6c7daf060b12 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 29 Jun 2023 16:20:54 +0200 Subject: [PATCH 497/515] Repro test --- .../02809_prewhere_and_in.reference | 8 +++ .../0_stateless/02809_prewhere_and_in.sql | 56 +++++++++++++++++++ 2 files changed, 64 insertions(+) create mode 100644 tests/queries/0_stateless/02809_prewhere_and_in.reference create mode 100644 tests/queries/0_stateless/02809_prewhere_and_in.sql diff --git a/tests/queries/0_stateless/02809_prewhere_and_in.reference b/tests/queries/0_stateless/02809_prewhere_and_in.reference new file mode 100644 index 00000000000..fb713be9991 --- /dev/null +++ b/tests/queries/0_stateless/02809_prewhere_and_in.reference @@ -0,0 +1,8 @@ +PREWHERE a IN (( +PREWHERE a IN (1, 2, 3) +PREWHERE a IN (t_02809_set) +PREWHERE a IN (t_02809_aux) +PREWHERE b NOT IN (( +PREWHERE b NOT IN (1, 2, 3) +PREWHERE b NOT IN (t_02809_set) +PREWHERE b NOT IN (t_02809_aux) diff --git a/tests/queries/0_stateless/02809_prewhere_and_in.sql b/tests/queries/0_stateless/02809_prewhere_and_in.sql new file mode 100644 index 00000000000..d2816cd5b52 --- /dev/null +++ b/tests/queries/0_stateless/02809_prewhere_and_in.sql @@ -0,0 +1,56 @@ +DROP TABLE IF EXISTS t_02809; + +CREATE TABLE t_02809(a Int64, b Int64, s String) +ENGINE=MergeTree order by tuple() +AS SELECT number, number%10, toString(arrayMap(i-> cityHash64(i*number), range(50))) FROM numbers(10000); + +CREATE TABLE t_02809_set(c Int64) +ENGINE=Set() +AS SELECT * FROM numbers(10); + +CREATE TABLE t_02809_aux(c Int64) +ENGINE=Memory() +AS SELECT * FROM numbers(10); + + +SET optimize_move_to_prewhere=1; + +-- Queries with 'IN' +SELECT * FROM (EXPLAIN SYNTAX + SELECT * FROM t_02809 WHERE a IN (SELECT * FROM system.one) +) WHERE explain LIKE '%WHERE%'; + +SELECT * FROM (EXPLAIN SYNTAX + SELECT * FROM t_02809 WHERE a IN (1,2,3) +) WHERE explain LIKE '%WHERE%'; + +SELECT * FROM (EXPLAIN SYNTAX + SELECT * FROM t_02809 WHERE a IN t_02809_set +) WHERE explain LIKE '%WHERE%'; + +SELECT * FROM (EXPLAIN SYNTAX + SELECT * FROM t_02809 WHERE a IN t_02809_aux +) WHERE explain LIKE '%WHERE%'; + + +-- Queries with 'NOT IN' +SELECT * FROM (EXPLAIN SYNTAX + SELECT * FROM t_02809 WHERE b NOT IN (SELECT * FROM system.one) +) WHERE explain LIKE '%WHERE%'; + +SELECT * FROM (EXPLAIN SYNTAX + SELECT * FROM t_02809 WHERE b NOT IN (1,2,3) +) WHERE explain LIKE '%WHERE%'; + +SELECT * FROM (EXPLAIN SYNTAX + SELECT * FROM t_02809 WHERE b NOT IN t_02809_set +) WHERE explain LIKE '%WHERE%'; + +SELECT * FROM (EXPLAIN SYNTAX + SELECT * FROM t_02809 WHERE b NOT IN t_02809_aux +) WHERE explain LIKE '%WHERE%'; + + +DROP TABLE t_02809; +DROP TABLE t_02809_set; +DROP TABLE t_02809_aux; From 05d399c0b34cd7a2e13cabbc1fd396f2b251fedb Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 29 Jun 2023 16:25:52 +0200 Subject: [PATCH 498/515] Handle subqueries and sets when collecting table columns --- .../MergeTree/MergeTreeWhereOptimizer.cpp | 3 +++ src/Storages/MergeTree/RPNBuilder.cpp | 15 +++++++++++++++ src/Storages/MergeTree/RPNBuilder.h | 2 ++ 3 files changed, 20 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 25a4579c73e..1620ba98d58 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -110,6 +110,9 @@ static void collectColumns(const RPNBuilderTreeNode & node, const NameSet & colu if (node.isConstant()) return; + if (node.isSubqueryOrSet()) + return; + if (!node.isFunction()) { auto column_name = node.getColumnName(); diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index 0b42111be45..a0c96c13d59 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -181,6 +181,21 @@ bool RPNBuilderTreeNode::isConstant() const } } +bool RPNBuilderTreeNode::isSubqueryOrSet() const +{ + if (ast_node) + { + return + typeid_cast(ast_node) || + typeid_cast(ast_node); + } + else + { + const auto * node_without_alias = getNodeWithoutAlias(dag_node); + return node_without_alias->result_type->getTypeId() == TypeIndex::Set; + } +} + ColumnWithTypeAndName RPNBuilderTreeNode::getConstantColumn() const { if (!isConstant()) diff --git a/src/Storages/MergeTree/RPNBuilder.h b/src/Storages/MergeTree/RPNBuilder.h index 6f624d93cd6..9eeb6deefd5 100644 --- a/src/Storages/MergeTree/RPNBuilder.h +++ b/src/Storages/MergeTree/RPNBuilder.h @@ -98,6 +98,8 @@ public: /// Is node constant bool isConstant() const; + bool isSubqueryOrSet() const; + /** Get constant as constant column. * Node must be constant before calling these method, otherwise logical exception is thrown. */ From f36f981f15f0a7aed86762c32ee706ccaa415df9 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 30 Jun 2023 18:22:47 +0200 Subject: [PATCH 499/515] Move adding CreateSets to the end of optimizations, after applying key conditions and indexes --- .../QueryPlan/Optimizations/Optimizations.h | 3 ++ .../optimizePrimaryKeyCondition.cpp | 2 -- .../QueryPlan/Optimizations/optimizeTree.cpp | 31 ++++++++++++++++++- src/Processors/QueryPlan/QueryPlan.cpp | 1 + .../QueryPlan/ReadFromMergeTree.cpp | 9 ++---- src/Processors/QueryPlan/ReadFromMergeTree.h | 2 +- .../QueryPlan/SourceStepWithFilter.h | 3 +- .../System/StorageSystemZooKeeper.cpp | 4 +-- 8 files changed, 42 insertions(+), 13 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 2b934ec440b..bc47413cbb5 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -14,6 +14,9 @@ namespace QueryPlanOptimizations void optimizeTreeFirstPass(const QueryPlanOptimizationSettings & settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes); /// Second pass is used to apply read-in-order and attach a predicate to PK. void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes); +/// Third pass is used to apply filters such as key conditions and skip indexes to the storages that support them. +/// After that it add CreateSetsStep for the subqueries that has not be used in the filters. +void optimizeTreeThirdPass(QueryPlan::Node & root, QueryPlan::Nodes & nodes); /// Optimization (first pass) is a function applied to QueryPlan::Node. /// It can read and update subtree of specified node. diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index 5ef786ff975..e98386a6ee9 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -28,8 +28,6 @@ void optimizePrimaryKeyCondition(const Stack & stack) else break; } - - source_step_with_filter->onAddFilterFinish(); } } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index d620170e29c..e788918703e 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -167,7 +167,6 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s optimizePrewhere(stack, nodes); optimizePrimaryKeyCondition(stack); enableMemoryBoundMerging(*stack.back().node, nodes); - addPlansForSets(*stack.back().node, nodes); stack.pop_back(); } @@ -178,5 +177,35 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s "No projection is used when optimize_use_projections = 1 and force_optimize_projection = 1"); } +void optimizeTreeThirdPass(QueryPlan::Node & root, QueryPlan::Nodes & nodes) +{ + Stack stack; + stack.push_back({.node = &root}); + + while (!stack.empty()) + { + /// NOTE: frame cannot be safely used after stack was modified. + auto & frame = stack.back(); + + /// Traverse all children first. + if (frame.next_child < frame.node->children.size()) + { + auto next_frame = Frame{.node = frame.node->children[frame.next_child]}; + ++frame.next_child; + stack.push_back(next_frame); + continue; + } + + if (auto * source_step_with_filter = dynamic_cast(frame.node->step.get())) + { + source_step_with_filter->applyFilters(); + } + + addPlansForSets(*frame.node, nodes); + + stack.pop_back(); + } +} + } } diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index cb732e58855..687260441ff 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -482,6 +482,7 @@ void QueryPlan::optimize(const QueryPlanOptimizationSettings & optimization_sett QueryPlanOptimizations::optimizeTreeFirstPass(optimization_settings, *root, nodes); QueryPlanOptimizations::optimizeTreeSecondPass(optimization_settings, *root, nodes); + QueryPlanOptimizations::optimizeTreeThirdPass(*root, nodes); updateDataStreams(*root); } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 1e21d13e2b1..e7bf1ce2edf 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1299,13 +1299,10 @@ static void buildIndexes( indexes->skip_indexes = std::move(skip_indexes); } -void ReadFromMergeTree::onAddFilterFinish() +void ReadFromMergeTree::applyFilters() { - if (!filter_nodes.nodes.empty()) - { - auto filter_actions_dag = buildFilterDAG(context, prewhere_info, filter_nodes, query_info); - buildIndexes(indexes, filter_actions_dag, data, context, query_info, metadata_for_reading); - } + auto filter_actions_dag = buildFilterDAG(context, prewhere_info, filter_nodes, query_info); + buildIndexes(indexes, filter_actions_dag, data, context, query_info, metadata_for_reading); } MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 06021cb1c5b..3e3edd4dc5c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -226,7 +226,7 @@ public: size_t getNumStreams() const { return requested_num_streams; } bool isParallelReadingEnabled() const { return read_task_callback != std::nullopt; } - void onAddFilterFinish() override; + void applyFilters() override; private: static MergeTreeDataSelectAnalysisResultPtr selectRangesToReadImpl( diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.h b/src/Processors/QueryPlan/SourceStepWithFilter.h index 34b6e3c6a7b..dbaff06fde6 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.h +++ b/src/Processors/QueryPlan/SourceStepWithFilter.h @@ -37,7 +37,8 @@ public: filter_dags.push_back(std::move(filter_dag)); } - virtual void onAddFilterFinish() {} + /// Apply filters that can optimize reading from storage. + virtual void applyFilters() {} protected: std::vector filter_dags; diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index d361a4173c1..4d7f59b8ccd 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -184,7 +184,7 @@ public: void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; - void onAddFilterFinish() override; + void applyFilters() override; private: void fillData(MutableColumns & res_columns); @@ -421,7 +421,7 @@ static Paths extractPath(const ActionsDAG::NodeRawConstPtrs & filter_nodes, Cont } -void ReadFromSystemZooKeeper::onAddFilterFinish() +void ReadFromSystemZooKeeper::applyFilters() { paths = extractPath(getFilterNodes().nodes, context, context->getSettingsRef().allow_unrestricted_reads_from_keeper); } From 9f53c48636f511d00187e1c80547b0ff236bcd39 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sat, 1 Jul 2023 09:19:54 +0200 Subject: [PATCH 500/515] Fix test with analyzer --- .../0_stateless/02809_prewhere_and_in.reference | 16 ++++++++-------- .../0_stateless/02809_prewhere_and_in.sql | 16 ++++++++-------- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/02809_prewhere_and_in.reference b/tests/queries/0_stateless/02809_prewhere_and_in.reference index fb713be9991..3080ae862bb 100644 --- a/tests/queries/0_stateless/02809_prewhere_and_in.reference +++ b/tests/queries/0_stateless/02809_prewhere_and_in.reference @@ -1,8 +1,8 @@ -PREWHERE a IN (( -PREWHERE a IN (1, 2, 3) -PREWHERE a IN (t_02809_set) -PREWHERE a IN (t_02809_aux) -PREWHERE b NOT IN (( -PREWHERE b NOT IN (1, 2, 3) -PREWHERE b NOT IN (t_02809_set) -PREWHERE b NOT IN (t_02809_aux) +PREWHERE a IN +PREWHERE a IN +PREWHERE a IN +PREWHERE a IN +PREWHERE b NOT IN +PREWHERE b NOT IN +PREWHERE b NOT IN +PREWHERE b NOT IN diff --git a/tests/queries/0_stateless/02809_prewhere_and_in.sql b/tests/queries/0_stateless/02809_prewhere_and_in.sql index d2816cd5b52..345577d6c7c 100644 --- a/tests/queries/0_stateless/02809_prewhere_and_in.sql +++ b/tests/queries/0_stateless/02809_prewhere_and_in.sql @@ -16,37 +16,37 @@ AS SELECT * FROM numbers(10); SET optimize_move_to_prewhere=1; -- Queries with 'IN' -SELECT * FROM (EXPLAIN SYNTAX +SELECT substring(explain, 1, 13) FROM (EXPLAIN SYNTAX SELECT * FROM t_02809 WHERE a IN (SELECT * FROM system.one) ) WHERE explain LIKE '%WHERE%'; -SELECT * FROM (EXPLAIN SYNTAX +SELECT substring(explain, 1, 13) FROM (EXPLAIN SYNTAX SELECT * FROM t_02809 WHERE a IN (1,2,3) ) WHERE explain LIKE '%WHERE%'; -SELECT * FROM (EXPLAIN SYNTAX +SELECT substring(explain, 1, 13) FROM (EXPLAIN SYNTAX SELECT * FROM t_02809 WHERE a IN t_02809_set ) WHERE explain LIKE '%WHERE%'; -SELECT * FROM (EXPLAIN SYNTAX +SELECT substring(explain, 1, 13) FROM (EXPLAIN SYNTAX SELECT * FROM t_02809 WHERE a IN t_02809_aux ) WHERE explain LIKE '%WHERE%'; -- Queries with 'NOT IN' -SELECT * FROM (EXPLAIN SYNTAX +SELECT substring(explain, 1, 17) FROM (EXPLAIN SYNTAX SELECT * FROM t_02809 WHERE b NOT IN (SELECT * FROM system.one) ) WHERE explain LIKE '%WHERE%'; -SELECT * FROM (EXPLAIN SYNTAX +SELECT substring(explain, 1, 17) FROM (EXPLAIN SYNTAX SELECT * FROM t_02809 WHERE b NOT IN (1,2,3) ) WHERE explain LIKE '%WHERE%'; -SELECT * FROM (EXPLAIN SYNTAX +SELECT substring(explain, 1, 17) FROM (EXPLAIN SYNTAX SELECT * FROM t_02809 WHERE b NOT IN t_02809_set ) WHERE explain LIKE '%WHERE%'; -SELECT * FROM (EXPLAIN SYNTAX +SELECT substring(explain, 1, 17) FROM (EXPLAIN SYNTAX SELECT * FROM t_02809 WHERE b NOT IN t_02809_aux ) WHERE explain LIKE '%WHERE%'; From c06afc53b4a37c526eb8b628b4102785baedee11 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sun, 2 Jul 2023 09:52:50 +0000 Subject: [PATCH 501/515] Make scripts backwards compatible --- docker/test/stateless/run.sh | 20 ++++++++++++++++++-- tests/ci/utils.lib | 4 ++++ 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 7ccedb8c0b3..fe53925ecc8 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -19,7 +19,7 @@ ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test source /usr/share/clickhouse-test/ci/attach_gdb.lib || true # FIXME: to not break old builds, clean on 2023-09-01 # shellcheck disable=SC1091 -source /usr/share/clickhouse-test/ci/utils.lib +source /usr/share/clickhouse-test/ci/utils.lib || true # FIXME: to not break old builds, clean on 2023-09-01 # install test configs /usr/share/clickhouse-test/config/install.sh @@ -93,6 +93,22 @@ sleep 5 attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01 +function fn_exists() { + declare -F "$1" > /dev/null; +} + +# FIXME: to not break old builds, clean on 2023-09-01 +function try_run_with_retry() { + local total_retries="$1" + shift + + if fn_exists run_with_retry; then + run_with_retry "$total_retries" "$@" + else + "$@" + fi +} + function run_tests() { set -x @@ -140,7 +156,7 @@ function run_tests() ADDITIONAL_OPTIONS+=('--report-logs-stats') - run_with_retry 10 clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')" + try_run_with_retry 10 clickhouse-client -q "insert into system.zookeeper (name, path, value) values ('auxiliary_zookeeper2', '/test/chroot/', '')" set +e clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ diff --git a/tests/ci/utils.lib b/tests/ci/utils.lib index 95ad50f635b..b5ce4ae0d78 100644 --- a/tests/ci/utils.lib +++ b/tests/ci/utils.lib @@ -22,4 +22,8 @@ function run_with_retry() echo "Command '$*' failed after $total_retries retries, exiting" exit 1 +} + +function fn_exists() { + declare -F "$1" > /dev/null; } \ No newline at end of file From d85f5cc4cf46aed7419feb82dffa085b392f6bff Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 2 Jul 2023 18:02:14 +0200 Subject: [PATCH 502/515] Fix 02116_tuple_element with Analyzer (#51669) * Fix 02116_tuple_element with Analyzer * Use alternative errors in test --------- Co-authored-by: Alexander Tokmakov --- tests/queries/0_stateless/02116_tuple_element.sql | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02116_tuple_element.sql b/tests/queries/0_stateless/02116_tuple_element.sql index bedfedd7c2d..97f6c049705 100644 --- a/tests/queries/0_stateless/02116_tuple_element.sql +++ b/tests/queries/0_stateless/02116_tuple_element.sql @@ -17,8 +17,8 @@ EXPLAIN SYNTAX SELECT tupleElement(t1, 'a') FROM t_tuple_element; SELECT tupleElement(number, 1) FROM numbers(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT tupleElement(t1) FROM t_tuple_element; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT tupleElement(t1, 'b') FROM t_tuple_element; -- { serverError NOT_FOUND_COLUMN_IN_BLOCK, UNKNOWN_IDENTIFIER } -SELECT tupleElement(t1, 0) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX } -SELECT tupleElement(t1, 3) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX } +SELECT tupleElement(t1, 0) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX, NOT_FOUND_COLUMN_IN_BLOCK } +SELECT tupleElement(t1, 3) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX, NOT_FOUND_COLUMN_IN_BLOCK } SELECT tupleElement(t1, materialize('a')) FROM t_tuple_element; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT t2.1 FROM t_tuple_element; @@ -29,8 +29,8 @@ EXPLAIN SYNTAX SELECT tupleElement(t2, 1) FROM t_tuple_element; SELECT tupleElement(t2) FROM t_tuple_element; -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } SELECT tupleElement(t2, 'a') FROM t_tuple_element; -- { serverError NOT_FOUND_COLUMN_IN_BLOCK, UNKNOWN_IDENTIFIER } -SELECT tupleElement(t2, 0) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX } -SELECT tupleElement(t2, 3) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX } +SELECT tupleElement(t2, 0) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX, NOT_FOUND_COLUMN_IN_BLOCK } +SELECT tupleElement(t2, 3) FROM t_tuple_element; -- { serverError ILLEGAL_INDEX, NOT_FOUND_COLUMN_IN_BLOCK } SELECT tupleElement(t2, materialize(1)) FROM t_tuple_element; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } DROP TABLE t_tuple_element; From 392b70d6f852a3e3912f4d2638da6f9a3fb7ddeb Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sun, 2 Jul 2023 19:30:18 +0300 Subject: [PATCH 503/515] fix merge_selecting_task scheduling (#51591) Co-authored-by: Alexey Milovidov --- src/Storages/MergeTree/MergeFromLogEntryTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 5 +++-- src/Storages/MergeTree/MutateFromLogEntryTask.cpp | 2 +- .../MergeTree/ReplicatedMergeMutateTaskBase.h | 11 ++++++++--- 4 files changed, 13 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index f29d37312f9..17582e7df98 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -392,7 +392,7 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite /** With `ZSESSIONEXPIRED` or `ZOPERATIONTIMEOUT`, we can inadvertently roll back local changes to the parts. * This is not a problem, because in this case the merge will remain in the queue, and we will try again. */ - storage.merge_selecting_task->schedule(); + finish_callback = [storage_ptr = &storage]() { storage_ptr->merge_selecting_task->schedule(); }; ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges); write_part_log({}); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 29a1574b66e..c0acc401506 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -239,8 +239,9 @@ MergeTreeDataMergerMutator::PartitionIdsHint MergeTreeDataMergerMutator::getPart if (!best_partition_id_to_optimize.empty()) res.emplace(std::move(best_partition_id_to_optimize)); - LOG_TRACE(log, "Checked {} partitions, found {} partitions with parts that may be merged: {}", - all_partition_ids.size(), res.size(), fmt::join(res, ", ")); + LOG_TRACE(log, "Checked {} partitions, found {} partitions with parts that may be merged: [{}]" + "(max_total_size_to_merge={}, merge_with_ttl_allowed{})", + all_partition_ids.size(), res.size(), fmt::join(res, ", "), max_total_size_to_merge, merge_with_ttl_allowed); return res; } diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 4c2def34e8a..ba55fb400ca 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -267,7 +267,7 @@ bool MutateFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrit /** With `ZSESSIONEXPIRED` or `ZOPERATIONTIMEOUT`, we can inadvertently roll back local changes to the parts. * This is not a problem, because in this case the entry will remain in the queue, and we will try again. */ - storage.merge_selecting_task->schedule(); + finish_callback = [storage_ptr = &storage]() { storage_ptr->merge_selecting_task->schedule(); }; ProfileEvents::increment(ProfileEvents::ReplicatedPartMutations); write_part_log({}); diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h index a7bf1290274..1e7f9834245 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.h @@ -21,10 +21,10 @@ public: StorageReplicatedMergeTree & storage_, ReplicatedMergeTreeQueue::SelectedEntryPtr & selected_entry_, IExecutableTask::TaskResultCallback & task_result_callback_) - : selected_entry(selected_entry_) + : storage(storage_) + , selected_entry(selected_entry_) , entry(*selected_entry->log_entry) , log(log_) - , storage(storage_) /// This is needed to ask an asssignee to assign a new merge/mutate operation /// It takes bool argument and true means that current task is successfully executed. , task_result_callback(task_result_callback_) @@ -52,13 +52,18 @@ protected: /// Will execute a part of inner MergeTask or MutateTask virtual bool executeInnerTask() = 0; + StorageReplicatedMergeTree & storage; + + /// A callback to reschedule merge_selecting_task after destroying merge_mutate_entry + /// The order is important, because merge_selecting_task may rely on the number of entries in MergeList + scope_guard finish_callback; + /// This is important not to execute the same mutation in parallel /// selected_entry is a RAII class, so the time of living must be the same as for the whole task ReplicatedMergeTreeQueue::SelectedEntryPtr selected_entry; ReplicatedMergeTreeLogEntry & entry; MergeList::EntryPtr merge_mutate_entry{nullptr}; Poco::Logger * log; - StorageReplicatedMergeTree & storage; /// ProfileEvents for current part will be stored here ProfileEvents::Counters profile_counters; ContextMutablePtr task_context; From aeea3dc8247fecc8f4be75c3330cce696bd054ba Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 3 Jul 2023 07:40:05 +0000 Subject: [PATCH 504/515] Add odbcinst dep to sqllogic docker --- docker/test/sqllogic/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/sqllogic/Dockerfile b/docker/test/sqllogic/Dockerfile index 83dcf7e1f56..5cf71e4d3f8 100644 --- a/docker/test/sqllogic/Dockerfile +++ b/docker/test/sqllogic/Dockerfile @@ -13,6 +13,7 @@ RUN apt-get update --yes \ sqlite3 \ unixodbc \ unixodbc-dev \ + odbcinst \ sudo \ && apt-get clean From f1709b3cf4360bbd57e7740a305d2a888849197b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 3 Jul 2023 12:58:41 +0300 Subject: [PATCH 505/515] better diagnostics for 01193_metadata_loading (#51414) --- tests/queries/0_stateless/01193_metadata_loading.sh | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01193_metadata_loading.sh b/tests/queries/0_stateless/01193_metadata_loading.sh index 63fcf8eb93b..50425eae018 100755 --- a/tests/queries/0_stateless/01193_metadata_loading.sh +++ b/tests/queries/0_stateless/01193_metadata_loading.sh @@ -12,7 +12,7 @@ db="test_01193_$RANDOM" tables=1000 threads=10 count_multiplier=1 -max_time_ms=5000 +max_time_ms=1500 debug_or_sanitizer_build=$($CLICKHOUSE_CLIENT -q "WITH ((SELECT value FROM system.build_options WHERE name='BUILD_TYPE') AS build, (SELECT value FROM system.build_options WHERE name='CXX_FLAGS') as flags) SELECT build='Debug' OR flags LIKE '%fsanitize%' OR hasThreadFuzzer()") @@ -42,13 +42,15 @@ wait $CLICKHOUSE_CLIENT -q "CREATE TABLE $db.table_merge (i UInt64, d Date, s String, n Nested(i UInt8, f Float32)) ENGINE=Merge('$db', '^table_')" $CLICKHOUSE_CLIENT -q "SELECT count() * $count_multiplier, i, d, s, n.i, n.f FROM merge('$db', '^table_9') GROUP BY i, d, s, n.i, n.f ORDER BY i" -for i in {1..10}; do +for i in {1..50}; do $CLICKHOUSE_CLIENT -q "DETACH DATABASE $db" - $CLICKHOUSE_CLIENT -q "ATTACH DATABASE $db" --query_id="$db-$i"; + $CLICKHOUSE_CLIENT --query_profiler_real_time_period_ns=100000000 --query_profiler_cpu_time_period_ns=100000000 -q "ATTACH DATABASE $db" --query_id="$db-$i"; done $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" -$CLICKHOUSE_CLIENT -q "SELECT if(min(query_duration_ms) < $max_time_ms, 'ok', toString(groupArray(query_duration_ms))) FROM system.query_log WHERE current_database = currentDatabase() AND query_id LIKE '$db-%' AND type=2" +durations=$($CLICKHOUSE_CLIENT -q "SELECT groupArray(query_duration_ms) FROM system.query_log WHERE current_database = currentDatabase() AND query_id LIKE '$db-%' AND type=2") +$CLICKHOUSE_CLIENT -q "SELECT 'durations', '$db', $durations FORMAT Null" +$CLICKHOUSE_CLIENT -q "SELECT if(quantile(0.5)(arrayJoin($durations)) < $max_time_ms, 'ok', toString($durations))" $CLICKHOUSE_CLIENT -q "SELECT count() * $count_multiplier, i, d, s, n.i, n.f FROM $db.table_merge GROUP BY i, d, s, n.i, n.f ORDER BY i" From d0c38474274ce0c64478b33230b49db574e06e11 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 3 Jul 2023 12:29:12 +0200 Subject: [PATCH 506/515] Fix source image for sqllogic --- docker/images.json | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/docker/images.json b/docker/images.json index b4f3e755bd1..e8fc329a640 100644 --- a/docker/images.json +++ b/docker/images.json @@ -120,11 +120,12 @@ "docker/test/base": { "name": "clickhouse/test-base", "dependent": [ - "docker/test/stateless", - "docker/test/integration/base", "docker/test/fuzzer", + "docker/test/integration/base", "docker/test/keeper-jepsen", - "docker/test/server-jepsen" + "docker/test/server-jepsen", + "docker/test/sqllogic", + "docker/test/stateless" ] }, "docker/test/integration/kerberized_hadoop": { From 22e44ced8a30763ff147174633255fab91f01d70 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 30 Jun 2023 15:51:41 +0200 Subject: [PATCH 507/515] Add column modification time into system.parts_columns This can be useful to obtain at least some time for the part after mutations, since mutations will change the modification time of all parts. Signed-off-by: Azat Khuzhin --- .../MergeTree/DataPartStorageOnDiskFull.cpp | 5 ++++ .../MergeTree/DataPartStorageOnDiskFull.h | 1 + src/Storages/MergeTree/IDataPartStorage.h | 1 + src/Storages/MergeTree/IMergeTreeDataPart.h | 2 ++ .../MergeTree/MergeTreeDataPartCompact.cpp | 5 ++++ .../MergeTree/MergeTreeDataPartCompact.h | 2 ++ .../MergeTree/MergeTreeDataPartInMemory.h | 1 + .../MergeTree/MergeTreeDataPartWide.cpp | 12 ++++++++ .../MergeTree/MergeTreeDataPartWide.h | 2 ++ .../System/StorageSystemPartsColumns.cpp | 10 +++++++ .../StorageSystemProjectionPartsColumns.cpp | 11 ++++++- .../02117_show_create_table_system.reference | 2 ++ ..._parts_columns_modification_time.reference | 6 ++++ ...tem_parts_columns_modification_time.sql.j2 | 30 +++++++++++++++++++ 14 files changed, 89 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02806_system_parts_columns_modification_time.reference create mode 100644 tests/queries/0_stateless/02806_system_parts_columns_modification_time.sql.j2 diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskFull.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskFull.cpp index e62e5a5d6b2..20b6c5a919e 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskFull.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskFull.cpp @@ -80,6 +80,11 @@ DataPartStorageIteratorPtr DataPartStorageOnDiskFull::iterate() const volume->getDisk()->iterateDirectory(fs::path(root_path) / part_dir)); } +Poco::Timestamp DataPartStorageOnDiskFull::getFileLastModified(const String & file_name) const +{ + return volume->getDisk()->getLastModified(fs::path(root_path) / part_dir / file_name); +} + size_t DataPartStorageOnDiskFull::getFileSize(const String & file_name) const { return volume->getDisk()->getFileSize(fs::path(root_path) / part_dir / file_name); diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskFull.h b/src/Storages/MergeTree/DataPartStorageOnDiskFull.h index 2ceb392e80f..5d70404fcfa 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskFull.h +++ b/src/Storages/MergeTree/DataPartStorageOnDiskFull.h @@ -20,6 +20,7 @@ public: bool isDirectory(const std::string & name) const override; DataPartStorageIteratorPtr iterate() const override; + Poco::Timestamp getFileLastModified(const String & file_name) const override; size_t getFileSize(const std::string & file_name) const override; UInt32 getRefCount(const std::string & file_name) const override; std::string getRemotePath(const std::string & file_name) const override; diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index 9d6c5d0dcba..8c9fd6d0426 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -108,6 +108,7 @@ public: virtual DataPartStorageIteratorPtr iterate() const = 0; /// Get metadata for a file inside path dir. + virtual Poco::Timestamp getFileLastModified(const std::string & file_name) const = 0; virtual size_t getFileSize(const std::string & file_name) const = 0; virtual UInt32 getRefCount(const std::string & file_name) const = 0; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index fd73d802579..797de39b020 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -116,6 +116,8 @@ public: /// Otherwise return information about column size on disk. ColumnSize getColumnSize(const String & column_name) const; + virtual std::optional getColumnModificationTime(const String & column_name) const = 0; + /// NOTE: Returns zeros if secondary indexes are not found in checksums. /// Otherwise return information about secondary index size on disk. IndexSize getSecondaryIndexSize(const String & secondary_index_name) const; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 07e20f16a9f..9c47608e364 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -144,6 +144,11 @@ bool MergeTreeDataPartCompact::hasColumnFiles(const NameAndTypePair & column) co return (bin_checksum != checksums.files.end() && mrk_checksum != checksums.files.end()); } +std::optional MergeTreeDataPartCompact::getColumnModificationTime(const String & /* column_name */) const +{ + return getDataPartStorage().getFileLastModified(DATA_FILE_NAME_WITH_EXTENSION).epochTime(); +} + void MergeTreeDataPartCompact::checkConsistency(bool require_part_metadata) const { checkConsistencyBase(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index b115692a7cf..08764eedb43 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -55,6 +55,8 @@ public: bool hasColumnFiles(const NameAndTypePair & column) const override; + std::optional getColumnModificationTime(const String & column_name) const override; + String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return DATA_FILE_NAME; } ~MergeTreeDataPartCompact() override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index db7244d8e99..2698b69b38e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -43,6 +43,7 @@ public: String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return ""; } void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) override; DataPartStoragePtr makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const override; + std::optional getColumnModificationTime(const String & /* column_name */) const override { return {}; } MutableDataPartStoragePtr flushToDisk(const String & new_relative_path, const StorageMetadataPtr & metadata_snapshot) const; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index f44cbdd8628..2d886e2058b 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -260,6 +260,18 @@ bool MergeTreeDataPartWide::hasColumnFiles(const NameAndTypePair & column) const return res; } +std::optional MergeTreeDataPartWide::getColumnModificationTime(const String & column_name) const +{ + try + { + return getDataPartStorage().getFileLastModified(column_name + DATA_FILE_EXTENSION).epochTime(); + } + catch (const fs::filesystem_error &) + { + return {}; + } +} + String MergeTreeDataPartWide::getFileNameForColumn(const NameAndTypePair & column) const { String filename; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 5ee497b9b21..0b2ffeb4b18 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -54,6 +54,8 @@ public: bool hasColumnFiles(const NameAndTypePair & column) const override; + std::optional getColumnModificationTime(const String & column_name) const override; + protected: static void loadIndexGranularityImpl( MergeTreeIndexGranularity & index_granularity_, MergeTreeIndexGranularityInfo & index_granularity_info_, diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index 00b958b015f..67c8d06e432 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -62,6 +63,8 @@ StorageSystemPartsColumns::StorageSystemPartsColumns(const StorageID & table_id_ {"column_data_compressed_bytes", std::make_shared()}, {"column_data_uncompressed_bytes", std::make_shared()}, {"column_marks_bytes", std::make_shared()}, + {"column_modification_time", std::make_shared(std::make_shared())}, + {"serialization_kind", std::make_shared()}, {"subcolumns.names", std::make_shared(std::make_shared())}, {"subcolumns.types", std::make_shared(std::make_shared())}, @@ -235,6 +238,13 @@ void StorageSystemPartsColumns::processNextStorage( columns[res_index++]->insert(column_size.data_uncompressed); if (columns_mask[src_index++]) columns[res_index++]->insert(column_size.marks); + if (columns_mask[src_index++]) + { + if (auto column_modification_time = part->getColumnModificationTime(column.name)) + columns[res_index++]->insert(UInt64(column_modification_time.value())); + else + columns[res_index++]->insertDefault(); + } auto serialization = part->getSerialization(column.name); if (columns_mask[src_index++]) diff --git a/src/Storages/System/StorageSystemProjectionPartsColumns.cpp b/src/Storages/System/StorageSystemProjectionPartsColumns.cpp index a5968597885..06becc6d91c 100644 --- a/src/Storages/System/StorageSystemProjectionPartsColumns.cpp +++ b/src/Storages/System/StorageSystemProjectionPartsColumns.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -66,7 +67,8 @@ StorageSystemProjectionPartsColumns::StorageSystemProjectionPartsColumns(const S {"column_bytes_on_disk", std::make_shared()}, {"column_data_compressed_bytes", std::make_shared()}, {"column_data_uncompressed_bytes", std::make_shared()}, - {"column_marks_bytes", std::make_shared()} + {"column_marks_bytes", std::make_shared()}, + {"column_modification_time", std::make_shared(std::make_shared())}, } ) { @@ -247,6 +249,13 @@ void StorageSystemProjectionPartsColumns::processNextStorage( columns[res_index++]->insert(column_size.data_uncompressed); if (columns_mask[src_index++]) columns[res_index++]->insert(column_size.marks); + if (columns_mask[src_index++]) + { + if (auto column_modification_time = part->getColumnModificationTime(column.name)) + columns[res_index++]->insert(UInt64(column_modification_time.value())); + else + columns[res_index++]->insertDefault(); + } if (has_state_column) columns[res_index++]->insert(part->stateString()); diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 703972279e7..3834b05601f 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -565,6 +565,7 @@ CREATE TABLE system.parts_columns `column_data_compressed_bytes` UInt64, `column_data_uncompressed_bytes` UInt64, `column_marks_bytes` UInt64, + `column_modification_time` Nullable(DateTime), `serialization_kind` String, `subcolumns.names` Array(String), `subcolumns.types` Array(String), @@ -750,6 +751,7 @@ CREATE TABLE system.projection_parts_columns `column_data_compressed_bytes` UInt64, `column_data_uncompressed_bytes` UInt64, `column_marks_bytes` UInt64, + `column_modification_time` Nullable(DateTime), `bytes` UInt64, `marks_size` UInt64, `part_name` String diff --git a/tests/queries/0_stateless/02806_system_parts_columns_modification_time.reference b/tests/queries/0_stateless/02806_system_parts_columns_modification_time.reference new file mode 100644 index 00000000000..f1be11980bb --- /dev/null +++ b/tests/queries/0_stateless/02806_system_parts_columns_modification_time.reference @@ -0,0 +1,6 @@ +Wide key 1 1 +Wide key 1 1 +Wide value 1 0 +Compact key 1 1 +Compact key 1 1 +Compact value 1 1 diff --git a/tests/queries/0_stateless/02806_system_parts_columns_modification_time.sql.j2 b/tests/queries/0_stateless/02806_system_parts_columns_modification_time.sql.j2 new file mode 100644 index 00000000000..eee236ff681 --- /dev/null +++ b/tests/queries/0_stateless/02806_system_parts_columns_modification_time.sql.j2 @@ -0,0 +1,30 @@ +-- Tags: no-s3-storage +-- Tag: no-s3-storage because S3 updates metadata for the virtual link file on metadata disk (see CreateHardlinkOperation::execute() for details) + +set mutations_sync=1; + +{# modification time of the part folder and column files not always equal, this is how much seconds of difference is allowed #} +{% set mtime_diff_in_seconds = 5 %} + +{% for id, settings, file_per_column in [ + ("wide", "min_bytes_for_wide_part=0, min_rows_for_wide_part=0", true), + ("compact", "min_bytes_for_wide_part=1000, min_rows_for_wide_part=100", false) +] +%} + +drop table if exists data_{{ id }}; +create table data_{{ id }} (key Int) engine=MergeTree() order by tuple() settings {{ settings }}; +insert into data_{{ id }} values (1); +select sleep(3) format Null; +select part_type, column, now()-modification_time < 10, modification_time - column_modification_time < {{ mtime_diff_in_seconds }} from system.parts_columns where database = currentDatabase() and table = 'data_{{ id }}'; +alter table data_{{ id }} add column value Int default 0; +alter table data_{{ id }} materialize column value; +select part_type, column, now()-modification_time < 10, +{% if file_per_column %} +modification_time - column_modification_time >= 3 +{% else %} +modification_time - column_modification_time < {{ mtime_diff_in_seconds }} +{% endif %} +from system.parts_columns where active and database = currentDatabase() and table = 'data_{{ id }}' order by column; + +{% endfor %} From 138e28cf6b98697297c35430f877782c437ddbe0 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Mon, 3 Jul 2023 13:45:39 +0200 Subject: [PATCH 508/515] Update README.md (#51726) --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index eac036c2d9b..80a8d0e331f 100644 --- a/README.md +++ b/README.md @@ -23,11 +23,11 @@ curl https://clickhouse.com/ | sh ## Upcoming Events * [**v23.6 Release Webinar**](https://clickhouse.com/company/events/v23-6-release-call?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-06) - Jun 29 - 23.6 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. -* [**ClickHouse Meetup in Paris**](https://www.meetup.com/clickhouse-france-user-group/events/294283460) - Jul 4 * [**ClickHouse Meetup in Boston**](https://www.meetup.com/clickhouse-boston-user-group/events/293913596) - Jul 18 * [**ClickHouse Meetup in NYC**](https://www.meetup.com/clickhouse-new-york-user-group/events/293913441) - Jul 19 * [**ClickHouse Meetup in Toronto**](https://www.meetup.com/clickhouse-toronto-user-group/events/294183127) - Jul 20 * [**ClickHouse Meetup in Singapore**](https://www.meetup.com/clickhouse-singapore-meetup-group/events/294428050/) - Jul 27 +* [**ClickHouse Meetup in Paris**](https://www.meetup.com/clickhouse-france-user-group/events/294283460) - Sep 12 Also, keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler clickhouse com. From 509a0c6f30543b364f805deaca5ba8a025ab8538 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Mon, 3 Jul 2023 13:50:40 -0400 Subject: [PATCH 509/515] add missing doc templates back --- docs/_description_templates/template-data-type.md | 1 + docs/_description_templates/template-engine.md | 1 + docs/_description_templates/template-function.md | 1 + docs/_description_templates/template-server-setting.md | 1 + docs/_description_templates/template-setting.md | 1 + docs/_description_templates/template-statement.md | 1 + docs/_description_templates/template-system-table.md | 1 + 7 files changed, 7 insertions(+) create mode 100644 docs/_description_templates/template-data-type.md create mode 100644 docs/_description_templates/template-engine.md create mode 100644 docs/_description_templates/template-function.md create mode 100644 docs/_description_templates/template-server-setting.md create mode 100644 docs/_description_templates/template-setting.md create mode 100644 docs/_description_templates/template-statement.md create mode 100644 docs/_description_templates/template-system-table.md diff --git a/docs/_description_templates/template-data-type.md b/docs/_description_templates/template-data-type.md new file mode 100644 index 00000000000..b66d77c7a77 --- /dev/null +++ b/docs/_description_templates/template-data-type.md @@ -0,0 +1 @@ +{"payload":{"allShortcutsEnabled":true,"fileTree":{"docs/_description_templates":{"items":[{"name":"template-data-type.md","path":"docs/_description_templates/template-data-type.md","contentType":"file"},{"name":"template-engine.md","path":"docs/_description_templates/template-engine.md","contentType":"file"},{"name":"template-function.md","path":"docs/_description_templates/template-function.md","contentType":"file"},{"name":"template-server-setting.md","path":"docs/_description_templates/template-server-setting.md","contentType":"file"},{"name":"template-setting.md","path":"docs/_description_templates/template-setting.md","contentType":"file"},{"name":"template-statement.md","path":"docs/_description_templates/template-statement.md","contentType":"file"},{"name":"template-system-table.md","path":"docs/_description_templates/template-system-table.md","contentType":"file"}],"totalCount":7},"docs":{"items":[{"name":"_description_templates","path":"docs/_description_templates","contentType":"directory"},{"name":"_includes","path":"docs/_includes","contentType":"directory"},{"name":"changelogs","path":"docs/changelogs","contentType":"directory"},{"name":"en","path":"docs/en","contentType":"directory"},{"name":"ru","path":"docs/ru","contentType":"directory"},{"name":"tools","path":"docs/tools","contentType":"directory"},{"name":"zh","path":"docs/zh","contentType":"directory"},{"name":".gitignore","path":"docs/.gitignore","contentType":"file"},{"name":"README.md","path":"docs/README.md","contentType":"file"},{"name":"clean","path":"docs/clean","contentType":"file"},{"name":"mkdocs.yml","path":"docs/mkdocs.yml","contentType":"file"},{"name":"redirects.txt","path":"docs/redirects.txt","contentType":"file"}],"totalCount":12},"":{"items":[{"name":".github","path":".github","contentType":"directory"},{"name":"base","path":"base","contentType":"directory"},{"name":"benchmark","path":"benchmark","contentType":"directory"},{"name":"cmake","path":"cmake","contentType":"directory"},{"name":"contrib","path":"contrib","contentType":"directory"},{"name":"docker","path":"docker","contentType":"directory"},{"name":"docs","path":"docs","contentType":"directory"},{"name":"packages","path":"packages","contentType":"directory"},{"name":"programs","path":"programs","contentType":"directory"},{"name":"src","path":"src","contentType":"directory"},{"name":"tests","path":"tests","contentType":"directory"},{"name":"utils","path":"utils","contentType":"directory"},{"name":"website","path":"website","contentType":"directory"},{"name":".clang-format","path":".clang-format","contentType":"file"},{"name":".clang-tidy","path":".clang-tidy","contentType":"file"},{"name":".editorconfig","path":".editorconfig","contentType":"file"},{"name":".gitattributes","path":".gitattributes","contentType":"file"},{"name":".gitignore","path":".gitignore","contentType":"file"},{"name":".gitmodules","path":".gitmodules","contentType":"file"},{"name":".pylintrc","path":".pylintrc","contentType":"file"},{"name":".vimrc","path":".vimrc","contentType":"file"},{"name":".yamllint","path":".yamllint","contentType":"file"},{"name":"AUTHORS","path":"AUTHORS","contentType":"file"},{"name":"CHANGELOG.md","path":"CHANGELOG.md","contentType":"file"},{"name":"CMakeLists.txt","path":"CMakeLists.txt","contentType":"file"},{"name":"CODE_OF_CONDUCT.md","path":"CODE_OF_CONDUCT.md","contentType":"file"},{"name":"CONTRIBUTING.md","path":"CONTRIBUTING.md","contentType":"file"},{"name":"LICENSE","path":"LICENSE","contentType":"file"},{"name":"PreLoad.cmake","path":"PreLoad.cmake","contentType":"file"},{"name":"README.md","path":"README.md","contentType":"file"},{"name":"SECURITY.md","path":"SECURITY.md","contentType":"file"},{"name":"format_sources","path":"format_sources","contentType":"file"}],"totalCount":32}},"fileTreeProcessingTime":14.564540000000001,"foldersToFetch":[],"reducedMotionEnabled":"system","repo":{"id":60246359,"defaultBranch":"master","name":"ClickHouse","ownerLogin":"ClickHouse","currentUserCanPush":true,"isFork":false,"isEmpty":false,"createdAt":"2016-06-02T04:28:18.000-04:00","ownerAvatar":"https://avatars.githubusercontent.com/u/54801242?v=4","public":true,"private":false,"isOrgOwned":true},"refInfo":{"name":"5f18640215159ed1ad50be1efce2cb996a49fd73","listCacheKey":"v0:1688403108.0","canEdit":false,"refType":"tree","currentOid":"5f18640215159ed1ad50be1efce2cb996a49fd73"},"path":"docs/_description_templates/template-data-type.md","currentUser":{"id":25182304,"login":"DanRoscigno","userEmail":"dan@roscigno.com"},"blob":{"rawBlob":null,"colorizedLines":null,"stylingDirectives":null,"csv":null,"csvError":null,"dependabotInfo":{"showConfigurationBanner":null,"configFilePath":null,"networkDependabotPath":"/ClickHouse/ClickHouse/network/updates","dismissConfigurationNoticePath":"/settings/dismiss-notice/dependabot_configuration_notice","configurationNoticeDismissed":false,"repoAlertsPath":"/ClickHouse/ClickHouse/security/dependabot","repoSecurityAndAnalysisPath":"/ClickHouse/ClickHouse/settings/security_analysis","repoOwnerIsOrg":true,"currentUserCanAdminRepo":false},"displayName":"template-data-type.md","displayUrl":"https://github.com/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-data-type.md?raw=true","headerInfo":{"blobSize":"540 Bytes","deleteInfo":{"deletePath":null,"deleteTooltip":"You must be on a branch to make or propose changes to this file"},"editInfo":{"editTooltip":"You must be on a branch to make or propose changes to this file"},"ghDesktopPath":null,"gitLfsPath":null,"onBranch":false,"shortPath":"239edb2","siteNavLoginPath":"/login?return_to=https%3A%2F%2Fgithub.com%2FClickHouse%2FClickHouse%2Fblob%2F5f18640215159ed1ad50be1efce2cb996a49fd73%2Fdocs%2F_description_templates%2Ftemplate-data-type.md","isCSV":false,"isRichtext":true,"toc":[{"level":1,"text":"data_type_name {#data_type-name}","anchor":"data_type_name-data_type-name","htmlText":"data_type_name {#data_type-name}"},{"level":2,"text":"Additional Info {#additional-info} (Optional)","anchor":"additional-info-additional-info-optional","htmlText":"Additional Info {#additional-info} (Optional)"}],"lineInfo":{"truncatedLoc":"29","truncatedSloc":"17"},"mode":"file"},"image":false,"isCodeownersFile":null,"isValidLegacyIssueTemplate":false,"issueTemplateHelpUrl":"https://docs.github.com/articles/about-issue-and-pull-request-templates","issueTemplate":null,"discussionTemplate":null,"language":"Markdown","large":false,"loggedIn":true,"newDiscussionPath":"/ClickHouse/ClickHouse/discussions/new","newIssuePath":"/ClickHouse/ClickHouse/issues/new","planSupportInfo":{"repoIsFork":null,"repoOwnedByCurrentUser":null,"requestFullPath":"/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-data-type.md","showFreeOrgGatedFeatureMessage":null,"showPlanSupportBanner":null,"upgradeDataAttributes":null,"upgradePath":null},"publishBannersInfo":{"dismissActionNoticePath":"/settings/dismiss-notice/publish_action_from_dockerfile","dismissStackNoticePath":"/settings/dismiss-notice/publish_stack_from_file","releasePath":"/ClickHouse/ClickHouse/releases/new?marketplace=true","showPublishActionBanner":false,"showPublishStackBanner":false},"renderImageOrRaw":false,"richText":"
\n \n \n \n \n \n \n \n \n
toc_prioritytoc_title
\n\n

data_type_name {#data_type-name}

\n

Description.

\n

Parameters (Optional)

\n\n

Examples

\n
\n

Additional Info {#additional-info} (Optional)

\n

The name of an additional section can be any, for example, Usage.

\n

See Also (Optional)

\n\n

Original article

\n
","renderedFileInfo":null,"tabSize":8,"topBannersInfo":{"overridingGlobalFundingFile":false,"globalPreferredFundingPath":null,"repoOwner":"ClickHouse","repoName":"ClickHouse","showInvalidCitationWarning":false,"citationHelpUrl":"https://docs.github.com/en/github/creating-cloning-and-archiving-repositories/creating-a-repository-on-github/about-citation-files","showDependabotConfigurationBanner":null,"actionsOnboardingTip":null},"truncated":false,"viewable":true,"workflowRedirectUrl":null,"symbols":{"timedOut":false,"notAnalyzed":true,"symbols":[]}},"csrf_tokens":{"/ClickHouse/ClickHouse/branches":{"post":"N-EHr0Rgr9I38z0iGu5sqtyb4o4AwtOW1kjllryDXwFKHQnulJDSASf5e_zz3Uts_O0aS5zN9jEL88ie9T5sww"}}},"title":"ClickHouse/docs/_description_templates/template-data-type.md at 5f18640215159ed1ad50be1efce2cb996a49fd73 · ClickHouse/ClickHouse","locale":"en"} \ No newline at end of file diff --git a/docs/_description_templates/template-engine.md b/docs/_description_templates/template-engine.md new file mode 100644 index 00000000000..c5bb5feb85d --- /dev/null +++ b/docs/_description_templates/template-engine.md @@ -0,0 +1 @@ +{"payload":{"allShortcutsEnabled":true,"fileTree":{"docs/_description_templates":{"items":[{"name":"template-data-type.md","path":"docs/_description_templates/template-data-type.md","contentType":"file"},{"name":"template-engine.md","path":"docs/_description_templates/template-engine.md","contentType":"file"},{"name":"template-function.md","path":"docs/_description_templates/template-function.md","contentType":"file"},{"name":"template-server-setting.md","path":"docs/_description_templates/template-server-setting.md","contentType":"file"},{"name":"template-setting.md","path":"docs/_description_templates/template-setting.md","contentType":"file"},{"name":"template-statement.md","path":"docs/_description_templates/template-statement.md","contentType":"file"},{"name":"template-system-table.md","path":"docs/_description_templates/template-system-table.md","contentType":"file"}],"totalCount":7},"docs":{"items":[{"name":"_description_templates","path":"docs/_description_templates","contentType":"directory"},{"name":"_includes","path":"docs/_includes","contentType":"directory"},{"name":"changelogs","path":"docs/changelogs","contentType":"directory"},{"name":"en","path":"docs/en","contentType":"directory"},{"name":"ru","path":"docs/ru","contentType":"directory"},{"name":"tools","path":"docs/tools","contentType":"directory"},{"name":"zh","path":"docs/zh","contentType":"directory"},{"name":".gitignore","path":"docs/.gitignore","contentType":"file"},{"name":"README.md","path":"docs/README.md","contentType":"file"},{"name":"clean","path":"docs/clean","contentType":"file"},{"name":"mkdocs.yml","path":"docs/mkdocs.yml","contentType":"file"},{"name":"redirects.txt","path":"docs/redirects.txt","contentType":"file"}],"totalCount":12},"":{"items":[{"name":".github","path":".github","contentType":"directory"},{"name":"base","path":"base","contentType":"directory"},{"name":"benchmark","path":"benchmark","contentType":"directory"},{"name":"cmake","path":"cmake","contentType":"directory"},{"name":"contrib","path":"contrib","contentType":"directory"},{"name":"docker","path":"docker","contentType":"directory"},{"name":"docs","path":"docs","contentType":"directory"},{"name":"packages","path":"packages","contentType":"directory"},{"name":"programs","path":"programs","contentType":"directory"},{"name":"src","path":"src","contentType":"directory"},{"name":"tests","path":"tests","contentType":"directory"},{"name":"utils","path":"utils","contentType":"directory"},{"name":"website","path":"website","contentType":"directory"},{"name":".clang-format","path":".clang-format","contentType":"file"},{"name":".clang-tidy","path":".clang-tidy","contentType":"file"},{"name":".editorconfig","path":".editorconfig","contentType":"file"},{"name":".gitattributes","path":".gitattributes","contentType":"file"},{"name":".gitignore","path":".gitignore","contentType":"file"},{"name":".gitmodules","path":".gitmodules","contentType":"file"},{"name":".pylintrc","path":".pylintrc","contentType":"file"},{"name":".vimrc","path":".vimrc","contentType":"file"},{"name":".yamllint","path":".yamllint","contentType":"file"},{"name":"AUTHORS","path":"AUTHORS","contentType":"file"},{"name":"CHANGELOG.md","path":"CHANGELOG.md","contentType":"file"},{"name":"CMakeLists.txt","path":"CMakeLists.txt","contentType":"file"},{"name":"CODE_OF_CONDUCT.md","path":"CODE_OF_CONDUCT.md","contentType":"file"},{"name":"CONTRIBUTING.md","path":"CONTRIBUTING.md","contentType":"file"},{"name":"LICENSE","path":"LICENSE","contentType":"file"},{"name":"PreLoad.cmake","path":"PreLoad.cmake","contentType":"file"},{"name":"README.md","path":"README.md","contentType":"file"},{"name":"SECURITY.md","path":"SECURITY.md","contentType":"file"},{"name":"format_sources","path":"format_sources","contentType":"file"}],"totalCount":32}},"fileTreeProcessingTime":10.334819,"foldersToFetch":[],"reducedMotionEnabled":"system","repo":{"id":60246359,"defaultBranch":"master","name":"ClickHouse","ownerLogin":"ClickHouse","currentUserCanPush":true,"isFork":false,"isEmpty":false,"createdAt":"2016-06-02T04:28:18.000-04:00","ownerAvatar":"https://avatars.githubusercontent.com/u/54801242?v=4","public":true,"private":false,"isOrgOwned":true},"refInfo":{"name":"5f18640215159ed1ad50be1efce2cb996a49fd73","listCacheKey":"v0:1688403108.0","canEdit":false,"refType":"tree","currentOid":"5f18640215159ed1ad50be1efce2cb996a49fd73"},"path":"docs/_description_templates/template-engine.md","currentUser":{"id":25182304,"login":"DanRoscigno","userEmail":"dan@roscigno.com"},"blob":{"rawBlob":null,"colorizedLines":null,"stylingDirectives":null,"csv":null,"csvError":null,"dependabotInfo":{"showConfigurationBanner":null,"configFilePath":null,"networkDependabotPath":"/ClickHouse/ClickHouse/network/updates","dismissConfigurationNoticePath":"/settings/dismiss-notice/dependabot_configuration_notice","configurationNoticeDismissed":false,"repoAlertsPath":"/ClickHouse/ClickHouse/security/dependabot","repoSecurityAndAnalysisPath":"/ClickHouse/ClickHouse/settings/security_analysis","repoOwnerIsOrg":true,"currentUserCanAdminRepo":false},"displayName":"template-engine.md","displayUrl":"https://github.com/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-engine.md?raw=true","headerInfo":{"blobSize":"1.19 KB","deleteInfo":{"deletePath":null,"deleteTooltip":"You must be on a branch to make or propose changes to this file"},"editInfo":{"editTooltip":"You must be on a branch to make or propose changes to this file"},"ghDesktopPath":null,"gitLfsPath":null,"onBranch":false,"shortPath":"392bc59","siteNavLoginPath":"/login?return_to=https%3A%2F%2Fgithub.com%2FClickHouse%2FClickHouse%2Fblob%2F5f18640215159ed1ad50be1efce2cb996a49fd73%2Fdocs%2F_description_templates%2Ftemplate-engine.md","isCSV":false,"isRichtext":true,"toc":[{"level":1,"text":"EngineName {#enginename}","anchor":"enginename-enginename","htmlText":"EngineName {#enginename}"},{"level":2,"text":"Creating a Database {#creating-a-database}","anchor":"creating-a-database-creating-a-database","htmlText":"Creating a Database {#creating-a-database}"},{"level":2,"text":"Creating a Table {#creating-a-table}","anchor":"creating-a-table-creating-a-table","htmlText":"Creating a Table {#creating-a-table}"},{"level":2,"text":"Virtual columns {#virtual-columns} (for Table engines only)","anchor":"virtual-columns-virtual-columns-for-table-engines-only","htmlText":"Virtual columns {#virtual-columns} (for Table engines only)"},{"level":2,"text":"Data Types Support {#data_types-support} (for Database engines only)","anchor":"data-types-support-data_types-support-for-database-engines-only","htmlText":"Data Types Support {#data_types-support} (for Database engines only)"},{"level":2,"text":"Specifics and recommendations {#specifics-and-recommendations}","anchor":"specifics-and-recommendations-specifics-and-recommendations","htmlText":"Specifics and recommendations {#specifics-and-recommendations}"},{"level":2,"text":"Usage Example {#usage-example}","anchor":"usage-example-usage-example","htmlText":"Usage Example {#usage-example}"}],"lineInfo":{"truncatedLoc":"63","truncatedSloc":"40"},"mode":"file"},"image":false,"isCodeownersFile":null,"isValidLegacyIssueTemplate":false,"issueTemplateHelpUrl":"https://docs.github.com/articles/about-issue-and-pull-request-templates","issueTemplate":null,"discussionTemplate":null,"language":"Markdown","large":false,"loggedIn":true,"newDiscussionPath":"/ClickHouse/ClickHouse/discussions/new","newIssuePath":"/ClickHouse/ClickHouse/issues/new","planSupportInfo":{"repoIsFork":null,"repoOwnedByCurrentUser":null,"requestFullPath":"/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-engine.md","showFreeOrgGatedFeatureMessage":null,"showPlanSupportBanner":null,"upgradeDataAttributes":null,"upgradePath":null},"publishBannersInfo":{"dismissActionNoticePath":"/settings/dismiss-notice/publish_action_from_dockerfile","dismissStackNoticePath":"/settings/dismiss-notice/publish_stack_from_file","releasePath":"/ClickHouse/ClickHouse/releases/new?marketplace=true","showPublishActionBanner":false,"showPublishStackBanner":false},"renderImageOrRaw":false,"richText":"

EngineName {#enginename}

\n
    \n
  • What the Database/Table engine does.
  • \n
  • Relations with other engines if they exist.
  • \n
\n

Creating a Database {#creating-a-database}

\n
    CREATE DATABASE ...
\n

or

\n

Creating a Table {#creating-a-table}

\n
    CREATE TABLE ...
\n

Engine Parameters

\n

Query Clauses (for Table engines only)

\n

Virtual columns {#virtual-columns} (for Table engines only)

\n

List and virtual columns with description, if they exist.

\n

Data Types Support {#data_types-support} (for Database engines only)

\n\n\n\n\n\n\n\n\n\n\n\n\n\n
EngineNameClickHouse
NativeDataTypeNameClickHouseDataTypeName
\n

Specifics and recommendations {#specifics-and-recommendations}

\n

Algorithms\nSpecifics of read and write processes\nExamples of tasks\nRecommendations for usage\nSpecifics of data storage

\n

Usage Example {#usage-example}

\n

The example must show usage and use cases. The following text contains the recommended parts of this section.

\n

Input table:

\n
\n

Query:

\n
\n

Result:

\n
\n

Follow up with any text to clarify the example.

\n

See Also

\n\n
","renderedFileInfo":null,"tabSize":8,"topBannersInfo":{"overridingGlobalFundingFile":false,"globalPreferredFundingPath":null,"repoOwner":"ClickHouse","repoName":"ClickHouse","showInvalidCitationWarning":false,"citationHelpUrl":"https://docs.github.com/en/github/creating-cloning-and-archiving-repositories/creating-a-repository-on-github/about-citation-files","showDependabotConfigurationBanner":null,"actionsOnboardingTip":null},"truncated":false,"viewable":true,"workflowRedirectUrl":null,"symbols":{"timedOut":false,"notAnalyzed":true,"symbols":[]}},"csrf_tokens":{"/ClickHouse/ClickHouse/branches":{"post":"mbr-hnliBQCB5MoHGpQf6OTZtY7x7nHuM4vvq5uev5jkRvDHqZJ405HujNnzpzguxK9NS23hVEnuMMKj0iOMWg"}}},"title":"ClickHouse/docs/_description_templates/template-engine.md at 5f18640215159ed1ad50be1efce2cb996a49fd73 · ClickHouse/ClickHouse","locale":"en"} \ No newline at end of file diff --git a/docs/_description_templates/template-function.md b/docs/_description_templates/template-function.md new file mode 100644 index 00000000000..0891e5d872d --- /dev/null +++ b/docs/_description_templates/template-function.md @@ -0,0 +1 @@ +{"payload":{"allShortcutsEnabled":true,"fileTree":{"docs/_description_templates":{"items":[{"name":"template-data-type.md","path":"docs/_description_templates/template-data-type.md","contentType":"file"},{"name":"template-engine.md","path":"docs/_description_templates/template-engine.md","contentType":"file"},{"name":"template-function.md","path":"docs/_description_templates/template-function.md","contentType":"file"},{"name":"template-server-setting.md","path":"docs/_description_templates/template-server-setting.md","contentType":"file"},{"name":"template-setting.md","path":"docs/_description_templates/template-setting.md","contentType":"file"},{"name":"template-statement.md","path":"docs/_description_templates/template-statement.md","contentType":"file"},{"name":"template-system-table.md","path":"docs/_description_templates/template-system-table.md","contentType":"file"}],"totalCount":7},"docs":{"items":[{"name":"_description_templates","path":"docs/_description_templates","contentType":"directory"},{"name":"_includes","path":"docs/_includes","contentType":"directory"},{"name":"changelogs","path":"docs/changelogs","contentType":"directory"},{"name":"en","path":"docs/en","contentType":"directory"},{"name":"ru","path":"docs/ru","contentType":"directory"},{"name":"tools","path":"docs/tools","contentType":"directory"},{"name":"zh","path":"docs/zh","contentType":"directory"},{"name":".gitignore","path":"docs/.gitignore","contentType":"file"},{"name":"README.md","path":"docs/README.md","contentType":"file"},{"name":"clean","path":"docs/clean","contentType":"file"},{"name":"mkdocs.yml","path":"docs/mkdocs.yml","contentType":"file"},{"name":"redirects.txt","path":"docs/redirects.txt","contentType":"file"}],"totalCount":12},"":{"items":[{"name":".github","path":".github","contentType":"directory"},{"name":"base","path":"base","contentType":"directory"},{"name":"benchmark","path":"benchmark","contentType":"directory"},{"name":"cmake","path":"cmake","contentType":"directory"},{"name":"contrib","path":"contrib","contentType":"directory"},{"name":"docker","path":"docker","contentType":"directory"},{"name":"docs","path":"docs","contentType":"directory"},{"name":"packages","path":"packages","contentType":"directory"},{"name":"programs","path":"programs","contentType":"directory"},{"name":"src","path":"src","contentType":"directory"},{"name":"tests","path":"tests","contentType":"directory"},{"name":"utils","path":"utils","contentType":"directory"},{"name":"website","path":"website","contentType":"directory"},{"name":".clang-format","path":".clang-format","contentType":"file"},{"name":".clang-tidy","path":".clang-tidy","contentType":"file"},{"name":".editorconfig","path":".editorconfig","contentType":"file"},{"name":".gitattributes","path":".gitattributes","contentType":"file"},{"name":".gitignore","path":".gitignore","contentType":"file"},{"name":".gitmodules","path":".gitmodules","contentType":"file"},{"name":".pylintrc","path":".pylintrc","contentType":"file"},{"name":".vimrc","path":".vimrc","contentType":"file"},{"name":".yamllint","path":".yamllint","contentType":"file"},{"name":"AUTHORS","path":"AUTHORS","contentType":"file"},{"name":"CHANGELOG.md","path":"CHANGELOG.md","contentType":"file"},{"name":"CMakeLists.txt","path":"CMakeLists.txt","contentType":"file"},{"name":"CODE_OF_CONDUCT.md","path":"CODE_OF_CONDUCT.md","contentType":"file"},{"name":"CONTRIBUTING.md","path":"CONTRIBUTING.md","contentType":"file"},{"name":"LICENSE","path":"LICENSE","contentType":"file"},{"name":"PreLoad.cmake","path":"PreLoad.cmake","contentType":"file"},{"name":"README.md","path":"README.md","contentType":"file"},{"name":"SECURITY.md","path":"SECURITY.md","contentType":"file"},{"name":"format_sources","path":"format_sources","contentType":"file"}],"totalCount":32}},"fileTreeProcessingTime":6.875799,"foldersToFetch":[],"reducedMotionEnabled":"system","repo":{"id":60246359,"defaultBranch":"master","name":"ClickHouse","ownerLogin":"ClickHouse","currentUserCanPush":true,"isFork":false,"isEmpty":false,"createdAt":"2016-06-02T04:28:18.000-04:00","ownerAvatar":"https://avatars.githubusercontent.com/u/54801242?v=4","public":true,"private":false,"isOrgOwned":true},"refInfo":{"name":"5f18640215159ed1ad50be1efce2cb996a49fd73","listCacheKey":"v0:1688403108.0","canEdit":false,"refType":"tree","currentOid":"5f18640215159ed1ad50be1efce2cb996a49fd73"},"path":"docs/_description_templates/template-function.md","currentUser":{"id":25182304,"login":"DanRoscigno","userEmail":"dan@roscigno.com"},"blob":{"rawBlob":null,"colorizedLines":null,"stylingDirectives":null,"csv":null,"csvError":null,"dependabotInfo":{"showConfigurationBanner":null,"configFilePath":null,"networkDependabotPath":"/ClickHouse/ClickHouse/network/updates","dismissConfigurationNoticePath":"/settings/dismiss-notice/dependabot_configuration_notice","configurationNoticeDismissed":false,"repoAlertsPath":"/ClickHouse/ClickHouse/security/dependabot","repoSecurityAndAnalysisPath":"/ClickHouse/ClickHouse/settings/security_analysis","repoOwnerIsOrg":true,"currentUserCanAdminRepo":false},"displayName":"template-function.md","displayUrl":"https://github.com/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-function.md?raw=true","headerInfo":{"blobSize":"1.12 KB","deleteInfo":{"deletePath":null,"deleteTooltip":"You must be on a branch to make or propose changes to this file"},"editInfo":{"editTooltip":"You must be on a branch to make or propose changes to this file"},"ghDesktopPath":null,"gitLfsPath":null,"onBranch":false,"shortPath":"6bdc764","siteNavLoginPath":"/login?return_to=https%3A%2F%2Fgithub.com%2FClickHouse%2FClickHouse%2Fblob%2F5f18640215159ed1ad50be1efce2cb996a49fd73%2Fdocs%2F_description_templates%2Ftemplate-function.md","isCSV":false,"isRichtext":true,"toc":[{"level":2,"text":"functionName {#functionname-in-lower-case}","anchor":"functionname-functionname-in-lower-case","htmlText":"functionName {#functionname-in-lower-case}"}],"lineInfo":{"truncatedLoc":"51","truncatedSloc":"29"},"mode":"file"},"image":false,"isCodeownersFile":null,"isValidLegacyIssueTemplate":false,"issueTemplateHelpUrl":"https://docs.github.com/articles/about-issue-and-pull-request-templates","issueTemplate":null,"discussionTemplate":null,"language":"Markdown","large":false,"loggedIn":true,"newDiscussionPath":"/ClickHouse/ClickHouse/discussions/new","newIssuePath":"/ClickHouse/ClickHouse/issues/new","planSupportInfo":{"repoIsFork":null,"repoOwnedByCurrentUser":null,"requestFullPath":"/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-function.md","showFreeOrgGatedFeatureMessage":null,"showPlanSupportBanner":null,"upgradeDataAttributes":null,"upgradePath":null},"publishBannersInfo":{"dismissActionNoticePath":"/settings/dismiss-notice/publish_action_from_dockerfile","dismissStackNoticePath":"/settings/dismiss-notice/publish_stack_from_file","releasePath":"/ClickHouse/ClickHouse/releases/new?marketplace=true","showPublishActionBanner":false,"showPublishStackBanner":false},"renderImageOrRaw":false,"richText":"

functionName {#functionname-in-lower-case}

\n

Short description.

\n

Syntax (without SELECT)

\n
<function syntax>
\n

Alias: <alias name>. (Optional)

\n

More text (Optional).

\n

Arguments (Optional)

\n
    \n
  • x — Description. Optional (only for optional arguments). Possible values: . Default value: . Type name.
  • \n
  • y — Description. Optional (only for optional arguments). Possible values: .Default value: . Type name.
  • \n
\n

Parameters (Optional, only for parametric aggregate functions)

\n
    \n
  • z — Description. Optional (only for optional parameters). Possible values: . Default value: . Type name.
  • \n
\n

Returned value(s)

\n
    \n
  • Returned values list.
  • \n
\n

Type: Type name.

\n

Example

\n

The example must show usage and/or a use cases. The following text contains recommended parts of an example.

\n

Input table (Optional):

\n
\n

Query:

\n
\n

Result:

\n
\n

See Also (Optional)

\n\n
","renderedFileInfo":null,"tabSize":8,"topBannersInfo":{"overridingGlobalFundingFile":false,"globalPreferredFundingPath":null,"repoOwner":"ClickHouse","repoName":"ClickHouse","showInvalidCitationWarning":false,"citationHelpUrl":"https://docs.github.com/en/github/creating-cloning-and-archiving-repositories/creating-a-repository-on-github/about-citation-files","showDependabotConfigurationBanner":null,"actionsOnboardingTip":null},"truncated":false,"viewable":true,"workflowRedirectUrl":null,"symbols":{"timedOut":false,"notAnalyzed":true,"symbols":[]}},"csrf_tokens":{"/ClickHouse/ClickHouse/branches":{"post":"LBpdgwbUgHW-H_hinQl7e32v5vocOJKA0kZyCn6oG8tR5lPC1iT9pq4Vvrx0Oly9XdkeP4A3tycP_V8CNxUoCQ"}}},"title":"ClickHouse/docs/_description_templates/template-function.md at 5f18640215159ed1ad50be1efce2cb996a49fd73 · ClickHouse/ClickHouse","locale":"en"} \ No newline at end of file diff --git a/docs/_description_templates/template-server-setting.md b/docs/_description_templates/template-server-setting.md new file mode 100644 index 00000000000..fc474059f05 --- /dev/null +++ b/docs/_description_templates/template-server-setting.md @@ -0,0 +1 @@ +{"payload":{"allShortcutsEnabled":true,"fileTree":{"docs/_description_templates":{"items":[{"name":"template-data-type.md","path":"docs/_description_templates/template-data-type.md","contentType":"file"},{"name":"template-engine.md","path":"docs/_description_templates/template-engine.md","contentType":"file"},{"name":"template-function.md","path":"docs/_description_templates/template-function.md","contentType":"file"},{"name":"template-server-setting.md","path":"docs/_description_templates/template-server-setting.md","contentType":"file"},{"name":"template-setting.md","path":"docs/_description_templates/template-setting.md","contentType":"file"},{"name":"template-statement.md","path":"docs/_description_templates/template-statement.md","contentType":"file"},{"name":"template-system-table.md","path":"docs/_description_templates/template-system-table.md","contentType":"file"}],"totalCount":7},"docs":{"items":[{"name":"_description_templates","path":"docs/_description_templates","contentType":"directory"},{"name":"_includes","path":"docs/_includes","contentType":"directory"},{"name":"changelogs","path":"docs/changelogs","contentType":"directory"},{"name":"en","path":"docs/en","contentType":"directory"},{"name":"ru","path":"docs/ru","contentType":"directory"},{"name":"tools","path":"docs/tools","contentType":"directory"},{"name":"zh","path":"docs/zh","contentType":"directory"},{"name":".gitignore","path":"docs/.gitignore","contentType":"file"},{"name":"README.md","path":"docs/README.md","contentType":"file"},{"name":"clean","path":"docs/clean","contentType":"file"},{"name":"mkdocs.yml","path":"docs/mkdocs.yml","contentType":"file"},{"name":"redirects.txt","path":"docs/redirects.txt","contentType":"file"}],"totalCount":12},"":{"items":[{"name":".github","path":".github","contentType":"directory"},{"name":"base","path":"base","contentType":"directory"},{"name":"benchmark","path":"benchmark","contentType":"directory"},{"name":"cmake","path":"cmake","contentType":"directory"},{"name":"contrib","path":"contrib","contentType":"directory"},{"name":"docker","path":"docker","contentType":"directory"},{"name":"docs","path":"docs","contentType":"directory"},{"name":"packages","path":"packages","contentType":"directory"},{"name":"programs","path":"programs","contentType":"directory"},{"name":"src","path":"src","contentType":"directory"},{"name":"tests","path":"tests","contentType":"directory"},{"name":"utils","path":"utils","contentType":"directory"},{"name":"website","path":"website","contentType":"directory"},{"name":".clang-format","path":".clang-format","contentType":"file"},{"name":".clang-tidy","path":".clang-tidy","contentType":"file"},{"name":".editorconfig","path":".editorconfig","contentType":"file"},{"name":".gitattributes","path":".gitattributes","contentType":"file"},{"name":".gitignore","path":".gitignore","contentType":"file"},{"name":".gitmodules","path":".gitmodules","contentType":"file"},{"name":".pylintrc","path":".pylintrc","contentType":"file"},{"name":".vimrc","path":".vimrc","contentType":"file"},{"name":".yamllint","path":".yamllint","contentType":"file"},{"name":"AUTHORS","path":"AUTHORS","contentType":"file"},{"name":"CHANGELOG.md","path":"CHANGELOG.md","contentType":"file"},{"name":"CMakeLists.txt","path":"CMakeLists.txt","contentType":"file"},{"name":"CODE_OF_CONDUCT.md","path":"CODE_OF_CONDUCT.md","contentType":"file"},{"name":"CONTRIBUTING.md","path":"CONTRIBUTING.md","contentType":"file"},{"name":"LICENSE","path":"LICENSE","contentType":"file"},{"name":"PreLoad.cmake","path":"PreLoad.cmake","contentType":"file"},{"name":"README.md","path":"README.md","contentType":"file"},{"name":"SECURITY.md","path":"SECURITY.md","contentType":"file"},{"name":"format_sources","path":"format_sources","contentType":"file"}],"totalCount":32}},"fileTreeProcessingTime":7.7234929999999995,"foldersToFetch":[],"reducedMotionEnabled":"system","repo":{"id":60246359,"defaultBranch":"master","name":"ClickHouse","ownerLogin":"ClickHouse","currentUserCanPush":true,"isFork":false,"isEmpty":false,"createdAt":"2016-06-02T04:28:18.000-04:00","ownerAvatar":"https://avatars.githubusercontent.com/u/54801242?v=4","public":true,"private":false,"isOrgOwned":true},"refInfo":{"name":"5f18640215159ed1ad50be1efce2cb996a49fd73","listCacheKey":"v0:1688403108.0","canEdit":false,"refType":"tree","currentOid":"5f18640215159ed1ad50be1efce2cb996a49fd73"},"path":"docs/_description_templates/template-server-setting.md","currentUser":{"id":25182304,"login":"DanRoscigno","userEmail":"dan@roscigno.com"},"blob":{"rawBlob":null,"colorizedLines":null,"stylingDirectives":null,"csv":null,"csvError":null,"dependabotInfo":{"showConfigurationBanner":null,"configFilePath":null,"networkDependabotPath":"/ClickHouse/ClickHouse/network/updates","dismissConfigurationNoticePath":"/settings/dismiss-notice/dependabot_configuration_notice","configurationNoticeDismissed":false,"repoAlertsPath":"/ClickHouse/ClickHouse/security/dependabot","repoSecurityAndAnalysisPath":"/ClickHouse/ClickHouse/settings/security_analysis","repoOwnerIsOrg":true,"currentUserCanAdminRepo":false},"displayName":"template-server-setting.md","displayUrl":"https://github.com/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-server-setting.md?raw=true","headerInfo":{"blobSize":"629 Bytes","deleteInfo":{"deletePath":null,"deleteTooltip":"You must be on a branch to make or propose changes to this file"},"editInfo":{"editTooltip":"You must be on a branch to make or propose changes to this file"},"ghDesktopPath":null,"gitLfsPath":null,"onBranch":false,"shortPath":"0b37d46","siteNavLoginPath":"/login?return_to=https%3A%2F%2Fgithub.com%2FClickHouse%2FClickHouse%2Fblob%2F5f18640215159ed1ad50be1efce2cb996a49fd73%2Fdocs%2F_description_templates%2Ftemplate-server-setting.md","isCSV":false,"isRichtext":true,"toc":[{"level":2,"text":"server_setting_name {#server_setting_name}","anchor":"server_setting_name-server_setting_name","htmlText":"server_setting_name {#server_setting_name}"}],"lineInfo":{"truncatedLoc":"33","truncatedSloc":"20"},"mode":"file"},"image":false,"isCodeownersFile":null,"isValidLegacyIssueTemplate":false,"issueTemplateHelpUrl":"https://docs.github.com/articles/about-issue-and-pull-request-templates","issueTemplate":null,"discussionTemplate":null,"language":"Markdown","large":false,"loggedIn":true,"newDiscussionPath":"/ClickHouse/ClickHouse/discussions/new","newIssuePath":"/ClickHouse/ClickHouse/issues/new","planSupportInfo":{"repoIsFork":null,"repoOwnedByCurrentUser":null,"requestFullPath":"/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-server-setting.md","showFreeOrgGatedFeatureMessage":null,"showPlanSupportBanner":null,"upgradeDataAttributes":null,"upgradePath":null},"publishBannersInfo":{"dismissActionNoticePath":"/settings/dismiss-notice/publish_action_from_dockerfile","dismissStackNoticePath":"/settings/dismiss-notice/publish_stack_from_file","releasePath":"/ClickHouse/ClickHouse/releases/new?marketplace=true","showPublishActionBanner":false,"showPublishStackBanner":false},"renderImageOrRaw":false,"richText":"

server_setting_name {#server_setting_name}

\n

Description.

\n

Describe what is configured in this section of settings.

\n

Possible value: ...

\n

Default value: ...

\n

Settings (Optional)

\n

If the section contains several settings, list them here. Specify possible values and default values:

\n
    \n
  • setting_1 — Description.
  • \n
  • setting_2 — Description.
  • \n
\n

Example

\n
<server_setting_name>\n    <setting_1> ... </setting_1>\n    <setting_2> ... </setting_2>\n</server_setting_name>
\n

Additional Info (Optional)

\n

The name of an additional section can be any, for example, Usage.

\n

See Also (Optional)

\n\n
","renderedFileInfo":null,"tabSize":8,"topBannersInfo":{"overridingGlobalFundingFile":false,"globalPreferredFundingPath":null,"repoOwner":"ClickHouse","repoName":"ClickHouse","showInvalidCitationWarning":false,"citationHelpUrl":"https://docs.github.com/en/github/creating-cloning-and-archiving-repositories/creating-a-repository-on-github/about-citation-files","showDependabotConfigurationBanner":null,"actionsOnboardingTip":null},"truncated":false,"viewable":true,"workflowRedirectUrl":null,"symbols":{"timedOut":false,"notAnalyzed":true,"symbols":[]}},"csrf_tokens":{"/ClickHouse/ClickHouse/branches":{"post":"a6lijUe_lOiTRf_mzmSRAi4tMYD_1HdAhwqytaKk2q8WVWzMl0_pO4NPuTgnV7bEDlvJRWPbUudasZ-96xnpbQ"}}},"title":"ClickHouse/docs/_description_templates/template-server-setting.md at 5f18640215159ed1ad50be1efce2cb996a49fd73 · ClickHouse/ClickHouse","locale":"en"} \ No newline at end of file diff --git a/docs/_description_templates/template-setting.md b/docs/_description_templates/template-setting.md new file mode 100644 index 00000000000..df51a7e8241 --- /dev/null +++ b/docs/_description_templates/template-setting.md @@ -0,0 +1 @@ +{"payload":{"allShortcutsEnabled":true,"fileTree":{"docs/_description_templates":{"items":[{"name":"template-data-type.md","path":"docs/_description_templates/template-data-type.md","contentType":"file"},{"name":"template-engine.md","path":"docs/_description_templates/template-engine.md","contentType":"file"},{"name":"template-function.md","path":"docs/_description_templates/template-function.md","contentType":"file"},{"name":"template-server-setting.md","path":"docs/_description_templates/template-server-setting.md","contentType":"file"},{"name":"template-setting.md","path":"docs/_description_templates/template-setting.md","contentType":"file"},{"name":"template-statement.md","path":"docs/_description_templates/template-statement.md","contentType":"file"},{"name":"template-system-table.md","path":"docs/_description_templates/template-system-table.md","contentType":"file"}],"totalCount":7},"docs":{"items":[{"name":"_description_templates","path":"docs/_description_templates","contentType":"directory"},{"name":"_includes","path":"docs/_includes","contentType":"directory"},{"name":"changelogs","path":"docs/changelogs","contentType":"directory"},{"name":"en","path":"docs/en","contentType":"directory"},{"name":"ru","path":"docs/ru","contentType":"directory"},{"name":"tools","path":"docs/tools","contentType":"directory"},{"name":"zh","path":"docs/zh","contentType":"directory"},{"name":".gitignore","path":"docs/.gitignore","contentType":"file"},{"name":"README.md","path":"docs/README.md","contentType":"file"},{"name":"clean","path":"docs/clean","contentType":"file"},{"name":"mkdocs.yml","path":"docs/mkdocs.yml","contentType":"file"},{"name":"redirects.txt","path":"docs/redirects.txt","contentType":"file"}],"totalCount":12},"":{"items":[{"name":".github","path":".github","contentType":"directory"},{"name":"base","path":"base","contentType":"directory"},{"name":"benchmark","path":"benchmark","contentType":"directory"},{"name":"cmake","path":"cmake","contentType":"directory"},{"name":"contrib","path":"contrib","contentType":"directory"},{"name":"docker","path":"docker","contentType":"directory"},{"name":"docs","path":"docs","contentType":"directory"},{"name":"packages","path":"packages","contentType":"directory"},{"name":"programs","path":"programs","contentType":"directory"},{"name":"src","path":"src","contentType":"directory"},{"name":"tests","path":"tests","contentType":"directory"},{"name":"utils","path":"utils","contentType":"directory"},{"name":"website","path":"website","contentType":"directory"},{"name":".clang-format","path":".clang-format","contentType":"file"},{"name":".clang-tidy","path":".clang-tidy","contentType":"file"},{"name":".editorconfig","path":".editorconfig","contentType":"file"},{"name":".gitattributes","path":".gitattributes","contentType":"file"},{"name":".gitignore","path":".gitignore","contentType":"file"},{"name":".gitmodules","path":".gitmodules","contentType":"file"},{"name":".pylintrc","path":".pylintrc","contentType":"file"},{"name":".vimrc","path":".vimrc","contentType":"file"},{"name":".yamllint","path":".yamllint","contentType":"file"},{"name":"AUTHORS","path":"AUTHORS","contentType":"file"},{"name":"CHANGELOG.md","path":"CHANGELOG.md","contentType":"file"},{"name":"CMakeLists.txt","path":"CMakeLists.txt","contentType":"file"},{"name":"CODE_OF_CONDUCT.md","path":"CODE_OF_CONDUCT.md","contentType":"file"},{"name":"CONTRIBUTING.md","path":"CONTRIBUTING.md","contentType":"file"},{"name":"LICENSE","path":"LICENSE","contentType":"file"},{"name":"PreLoad.cmake","path":"PreLoad.cmake","contentType":"file"},{"name":"README.md","path":"README.md","contentType":"file"},{"name":"SECURITY.md","path":"SECURITY.md","contentType":"file"},{"name":"format_sources","path":"format_sources","contentType":"file"}],"totalCount":32}},"fileTreeProcessingTime":9.96412,"foldersToFetch":[],"reducedMotionEnabled":"system","repo":{"id":60246359,"defaultBranch":"master","name":"ClickHouse","ownerLogin":"ClickHouse","currentUserCanPush":true,"isFork":false,"isEmpty":false,"createdAt":"2016-06-02T04:28:18.000-04:00","ownerAvatar":"https://avatars.githubusercontent.com/u/54801242?v=4","public":true,"private":false,"isOrgOwned":true},"refInfo":{"name":"5f18640215159ed1ad50be1efce2cb996a49fd73","listCacheKey":"v0:1688403108.0","canEdit":false,"refType":"tree","currentOid":"5f18640215159ed1ad50be1efce2cb996a49fd73"},"path":"docs/_description_templates/template-setting.md","currentUser":{"id":25182304,"login":"DanRoscigno","userEmail":"dan@roscigno.com"},"blob":{"rawBlob":null,"colorizedLines":null,"stylingDirectives":null,"csv":null,"csvError":null,"dependabotInfo":{"showConfigurationBanner":null,"configFilePath":null,"networkDependabotPath":"/ClickHouse/ClickHouse/network/updates","dismissConfigurationNoticePath":"/settings/dismiss-notice/dependabot_configuration_notice","configurationNoticeDismissed":false,"repoAlertsPath":"/ClickHouse/ClickHouse/security/dependabot","repoSecurityAndAnalysisPath":"/ClickHouse/ClickHouse/settings/security_analysis","repoOwnerIsOrg":true,"currentUserCanAdminRepo":false},"displayName":"template-setting.md","displayUrl":"https://github.com/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-setting.md?raw=true","headerInfo":{"blobSize":"503 Bytes","deleteInfo":{"deletePath":null,"deleteTooltip":"You must be on a branch to make or propose changes to this file"},"editInfo":{"editTooltip":"You must be on a branch to make or propose changes to this file"},"ghDesktopPath":null,"gitLfsPath":null,"onBranch":false,"shortPath":"fc912ab","siteNavLoginPath":"/login?return_to=https%3A%2F%2Fgithub.com%2FClickHouse%2FClickHouse%2Fblob%2F5f18640215159ed1ad50be1efce2cb996a49fd73%2Fdocs%2F_description_templates%2Ftemplate-setting.md","isCSV":false,"isRichtext":true,"toc":[{"level":2,"text":"setting_name {#setting_name}","anchor":"setting_name-setting_name","htmlText":"setting_name {#setting_name}"}],"lineInfo":{"truncatedLoc":"27","truncatedSloc":"15"},"mode":"file"},"image":false,"isCodeownersFile":null,"isValidLegacyIssueTemplate":false,"issueTemplateHelpUrl":"https://docs.github.com/articles/about-issue-and-pull-request-templates","issueTemplate":null,"discussionTemplate":null,"language":"Markdown","large":false,"loggedIn":true,"newDiscussionPath":"/ClickHouse/ClickHouse/discussions/new","newIssuePath":"/ClickHouse/ClickHouse/issues/new","planSupportInfo":{"repoIsFork":null,"repoOwnedByCurrentUser":null,"requestFullPath":"/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-setting.md","showFreeOrgGatedFeatureMessage":null,"showPlanSupportBanner":null,"upgradeDataAttributes":null,"upgradePath":null},"publishBannersInfo":{"dismissActionNoticePath":"/settings/dismiss-notice/publish_action_from_dockerfile","dismissStackNoticePath":"/settings/dismiss-notice/publish_stack_from_file","releasePath":"/ClickHouse/ClickHouse/releases/new?marketplace=true","showPublishActionBanner":false,"showPublishStackBanner":false},"renderImageOrRaw":false,"richText":"

setting_name {#setting_name}

\n

Description.

\n

For the switch setting, use the typical phrase: “Enables or disables something …”.

\n

Possible values:

\n

For switcher setting:

\n
    \n
  • 0 — Disabled.
  • \n
  • 1 — Enabled.
  • \n
\n

For another setting (typical phrases):

\n
    \n
  • Positive integer.
  • \n
  • 0 — Disabled or unlimited or something else.
  • \n
\n

Default value: value.

\n

Additional Info (Optional)

\n

The name of an additional section can be any, for example, Usage.

\n

See Also (Optional)

\n\n
","renderedFileInfo":null,"tabSize":8,"topBannersInfo":{"overridingGlobalFundingFile":false,"globalPreferredFundingPath":null,"repoOwner":"ClickHouse","repoName":"ClickHouse","showInvalidCitationWarning":false,"citationHelpUrl":"https://docs.github.com/en/github/creating-cloning-and-archiving-repositories/creating-a-repository-on-github/about-citation-files","showDependabotConfigurationBanner":null,"actionsOnboardingTip":null},"truncated":false,"viewable":true,"workflowRedirectUrl":null,"symbols":{"timedOut":false,"notAnalyzed":true,"symbols":[]}},"csrf_tokens":{"/ClickHouse/ClickHouse/branches":{"post":"gFv6-keA6Z1eYZATbMVq8_sVPc-l2b8pTaH03K1R_zf9p_S7l3CUTk5r1s2F9k0122PFCjnWmo6QGtnU5OzM9Q"}}},"title":"ClickHouse/docs/_description_templates/template-setting.md at 5f18640215159ed1ad50be1efce2cb996a49fd73 · ClickHouse/ClickHouse","locale":"en"} \ No newline at end of file diff --git a/docs/_description_templates/template-statement.md b/docs/_description_templates/template-statement.md new file mode 100644 index 00000000000..b5ae0d9b26b --- /dev/null +++ b/docs/_description_templates/template-statement.md @@ -0,0 +1 @@ +{"payload":{"allShortcutsEnabled":true,"fileTree":{"docs/_description_templates":{"items":[{"name":"template-data-type.md","path":"docs/_description_templates/template-data-type.md","contentType":"file"},{"name":"template-engine.md","path":"docs/_description_templates/template-engine.md","contentType":"file"},{"name":"template-function.md","path":"docs/_description_templates/template-function.md","contentType":"file"},{"name":"template-server-setting.md","path":"docs/_description_templates/template-server-setting.md","contentType":"file"},{"name":"template-setting.md","path":"docs/_description_templates/template-setting.md","contentType":"file"},{"name":"template-statement.md","path":"docs/_description_templates/template-statement.md","contentType":"file"},{"name":"template-system-table.md","path":"docs/_description_templates/template-system-table.md","contentType":"file"}],"totalCount":7},"docs":{"items":[{"name":"_description_templates","path":"docs/_description_templates","contentType":"directory"},{"name":"_includes","path":"docs/_includes","contentType":"directory"},{"name":"changelogs","path":"docs/changelogs","contentType":"directory"},{"name":"en","path":"docs/en","contentType":"directory"},{"name":"ru","path":"docs/ru","contentType":"directory"},{"name":"tools","path":"docs/tools","contentType":"directory"},{"name":"zh","path":"docs/zh","contentType":"directory"},{"name":".gitignore","path":"docs/.gitignore","contentType":"file"},{"name":"README.md","path":"docs/README.md","contentType":"file"},{"name":"clean","path":"docs/clean","contentType":"file"},{"name":"mkdocs.yml","path":"docs/mkdocs.yml","contentType":"file"},{"name":"redirects.txt","path":"docs/redirects.txt","contentType":"file"}],"totalCount":12},"":{"items":[{"name":".github","path":".github","contentType":"directory"},{"name":"base","path":"base","contentType":"directory"},{"name":"benchmark","path":"benchmark","contentType":"directory"},{"name":"cmake","path":"cmake","contentType":"directory"},{"name":"contrib","path":"contrib","contentType":"directory"},{"name":"docker","path":"docker","contentType":"directory"},{"name":"docs","path":"docs","contentType":"directory"},{"name":"packages","path":"packages","contentType":"directory"},{"name":"programs","path":"programs","contentType":"directory"},{"name":"src","path":"src","contentType":"directory"},{"name":"tests","path":"tests","contentType":"directory"},{"name":"utils","path":"utils","contentType":"directory"},{"name":"website","path":"website","contentType":"directory"},{"name":".clang-format","path":".clang-format","contentType":"file"},{"name":".clang-tidy","path":".clang-tidy","contentType":"file"},{"name":".editorconfig","path":".editorconfig","contentType":"file"},{"name":".gitattributes","path":".gitattributes","contentType":"file"},{"name":".gitignore","path":".gitignore","contentType":"file"},{"name":".gitmodules","path":".gitmodules","contentType":"file"},{"name":".pylintrc","path":".pylintrc","contentType":"file"},{"name":".vimrc","path":".vimrc","contentType":"file"},{"name":".yamllint","path":".yamllint","contentType":"file"},{"name":"AUTHORS","path":"AUTHORS","contentType":"file"},{"name":"CHANGELOG.md","path":"CHANGELOG.md","contentType":"file"},{"name":"CMakeLists.txt","path":"CMakeLists.txt","contentType":"file"},{"name":"CODE_OF_CONDUCT.md","path":"CODE_OF_CONDUCT.md","contentType":"file"},{"name":"CONTRIBUTING.md","path":"CONTRIBUTING.md","contentType":"file"},{"name":"LICENSE","path":"LICENSE","contentType":"file"},{"name":"PreLoad.cmake","path":"PreLoad.cmake","contentType":"file"},{"name":"README.md","path":"README.md","contentType":"file"},{"name":"SECURITY.md","path":"SECURITY.md","contentType":"file"},{"name":"format_sources","path":"format_sources","contentType":"file"}],"totalCount":32}},"fileTreeProcessingTime":9.954742000000001,"foldersToFetch":[],"reducedMotionEnabled":"system","repo":{"id":60246359,"defaultBranch":"master","name":"ClickHouse","ownerLogin":"ClickHouse","currentUserCanPush":true,"isFork":false,"isEmpty":false,"createdAt":"2016-06-02T04:28:18.000-04:00","ownerAvatar":"https://avatars.githubusercontent.com/u/54801242?v=4","public":true,"private":false,"isOrgOwned":true},"refInfo":{"name":"5f18640215159ed1ad50be1efce2cb996a49fd73","listCacheKey":"v0:1688403108.0","canEdit":false,"refType":"tree","currentOid":"5f18640215159ed1ad50be1efce2cb996a49fd73"},"path":"docs/_description_templates/template-statement.md","currentUser":{"id":25182304,"login":"DanRoscigno","userEmail":"dan@roscigno.com"},"blob":{"rawBlob":null,"colorizedLines":null,"stylingDirectives":null,"csv":null,"csvError":null,"dependabotInfo":{"showConfigurationBanner":null,"configFilePath":null,"networkDependabotPath":"/ClickHouse/ClickHouse/network/updates","dismissConfigurationNoticePath":"/settings/dismiss-notice/dependabot_configuration_notice","configurationNoticeDismissed":false,"repoAlertsPath":"/ClickHouse/ClickHouse/security/dependabot","repoSecurityAndAnalysisPath":"/ClickHouse/ClickHouse/settings/security_analysis","repoOwnerIsOrg":true,"currentUserCanAdminRepo":false},"displayName":"template-statement.md","displayUrl":"https://github.com/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-statement.md?raw=true","headerInfo":{"blobSize":"575 Bytes","deleteInfo":{"deletePath":null,"deleteTooltip":"You must be on a branch to make or propose changes to this file"},"editInfo":{"editTooltip":"You must be on a branch to make or propose changes to this file"},"ghDesktopPath":null,"gitLfsPath":null,"onBranch":false,"shortPath":"238570c","siteNavLoginPath":"/login?return_to=https%3A%2F%2Fgithub.com%2FClickHouse%2FClickHouse%2Fblob%2F5f18640215159ed1ad50be1efce2cb996a49fd73%2Fdocs%2F_description_templates%2Ftemplate-statement.md","isCSV":false,"isRichtext":true,"toc":[{"level":1,"text":"Statement name (for example, SHOW USER) {#statement-name-in-lower-case}","anchor":"statement-name-for-example-show-user-statement-name-in-lower-case","htmlText":"Statement name (for example, SHOW USER) {#statement-name-in-lower-case}"},{"level":2,"text":"Other necessary sections of the description (Optional) {#anchor}","anchor":"other-necessary-sections-of-the-description-optional-anchor","htmlText":"Other necessary sections of the description (Optional) {#anchor}"}],"lineInfo":{"truncatedLoc":"24","truncatedSloc":"14"},"mode":"file"},"image":false,"isCodeownersFile":null,"isValidLegacyIssueTemplate":false,"issueTemplateHelpUrl":"https://docs.github.com/articles/about-issue-and-pull-request-templates","issueTemplate":null,"discussionTemplate":null,"language":"Markdown","large":false,"loggedIn":true,"newDiscussionPath":"/ClickHouse/ClickHouse/discussions/new","newIssuePath":"/ClickHouse/ClickHouse/issues/new","planSupportInfo":{"repoIsFork":null,"repoOwnedByCurrentUser":null,"requestFullPath":"/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-statement.md","showFreeOrgGatedFeatureMessage":null,"showPlanSupportBanner":null,"upgradeDataAttributes":null,"upgradePath":null},"publishBannersInfo":{"dismissActionNoticePath":"/settings/dismiss-notice/publish_action_from_dockerfile","dismissStackNoticePath":"/settings/dismiss-notice/publish_stack_from_file","releasePath":"/ClickHouse/ClickHouse/releases/new?marketplace=true","showPublishActionBanner":false,"showPublishStackBanner":false},"renderImageOrRaw":false,"richText":"

Statement name (for example, SHOW USER) {#statement-name-in-lower-case}

\n

Brief description of what the statement does.

\n

Syntax

\n
Syntax of the statement.
\n

Other necessary sections of the description (Optional) {#anchor}

\n

Examples of descriptions with a complicated structure:

\n\n

See Also (Optional)

\n

Links to related topics as a list.

\n\n
","renderedFileInfo":null,"tabSize":8,"topBannersInfo":{"overridingGlobalFundingFile":false,"globalPreferredFundingPath":null,"repoOwner":"ClickHouse","repoName":"ClickHouse","showInvalidCitationWarning":false,"citationHelpUrl":"https://docs.github.com/en/github/creating-cloning-and-archiving-repositories/creating-a-repository-on-github/about-citation-files","showDependabotConfigurationBanner":null,"actionsOnboardingTip":null},"truncated":false,"viewable":true,"workflowRedirectUrl":null,"symbols":{"timedOut":false,"notAnalyzed":true,"symbols":[]}},"csrf_tokens":{"/ClickHouse/ClickHouse/branches":{"post":"xohFK8TmjwjnIXPER5IKigBaxPXoFkjI0tOrZrMZen-7dEtqFBby2_crNRquoS1MICw8MHQZbW8PaIZu-qRJvQ"}}},"title":"ClickHouse/docs/_description_templates/template-statement.md at 5f18640215159ed1ad50be1efce2cb996a49fd73 · ClickHouse/ClickHouse","locale":"en"} \ No newline at end of file diff --git a/docs/_description_templates/template-system-table.md b/docs/_description_templates/template-system-table.md new file mode 100644 index 00000000000..02d622a52cf --- /dev/null +++ b/docs/_description_templates/template-system-table.md @@ -0,0 +1 @@ +{"payload":{"allShortcutsEnabled":true,"fileTree":{"docs/_description_templates":{"items":[{"name":"template-data-type.md","path":"docs/_description_templates/template-data-type.md","contentType":"file"},{"name":"template-engine.md","path":"docs/_description_templates/template-engine.md","contentType":"file"},{"name":"template-function.md","path":"docs/_description_templates/template-function.md","contentType":"file"},{"name":"template-server-setting.md","path":"docs/_description_templates/template-server-setting.md","contentType":"file"},{"name":"template-setting.md","path":"docs/_description_templates/template-setting.md","contentType":"file"},{"name":"template-statement.md","path":"docs/_description_templates/template-statement.md","contentType":"file"},{"name":"template-system-table.md","path":"docs/_description_templates/template-system-table.md","contentType":"file"}],"totalCount":7},"docs":{"items":[{"name":"_description_templates","path":"docs/_description_templates","contentType":"directory"},{"name":"_includes","path":"docs/_includes","contentType":"directory"},{"name":"changelogs","path":"docs/changelogs","contentType":"directory"},{"name":"en","path":"docs/en","contentType":"directory"},{"name":"ru","path":"docs/ru","contentType":"directory"},{"name":"tools","path":"docs/tools","contentType":"directory"},{"name":"zh","path":"docs/zh","contentType":"directory"},{"name":".gitignore","path":"docs/.gitignore","contentType":"file"},{"name":"README.md","path":"docs/README.md","contentType":"file"},{"name":"clean","path":"docs/clean","contentType":"file"},{"name":"mkdocs.yml","path":"docs/mkdocs.yml","contentType":"file"},{"name":"redirects.txt","path":"docs/redirects.txt","contentType":"file"}],"totalCount":12},"":{"items":[{"name":".github","path":".github","contentType":"directory"},{"name":"base","path":"base","contentType":"directory"},{"name":"benchmark","path":"benchmark","contentType":"directory"},{"name":"cmake","path":"cmake","contentType":"directory"},{"name":"contrib","path":"contrib","contentType":"directory"},{"name":"docker","path":"docker","contentType":"directory"},{"name":"docs","path":"docs","contentType":"directory"},{"name":"packages","path":"packages","contentType":"directory"},{"name":"programs","path":"programs","contentType":"directory"},{"name":"src","path":"src","contentType":"directory"},{"name":"tests","path":"tests","contentType":"directory"},{"name":"utils","path":"utils","contentType":"directory"},{"name":"website","path":"website","contentType":"directory"},{"name":".clang-format","path":".clang-format","contentType":"file"},{"name":".clang-tidy","path":".clang-tidy","contentType":"file"},{"name":".editorconfig","path":".editorconfig","contentType":"file"},{"name":".gitattributes","path":".gitattributes","contentType":"file"},{"name":".gitignore","path":".gitignore","contentType":"file"},{"name":".gitmodules","path":".gitmodules","contentType":"file"},{"name":".pylintrc","path":".pylintrc","contentType":"file"},{"name":".vimrc","path":".vimrc","contentType":"file"},{"name":".yamllint","path":".yamllint","contentType":"file"},{"name":"AUTHORS","path":"AUTHORS","contentType":"file"},{"name":"CHANGELOG.md","path":"CHANGELOG.md","contentType":"file"},{"name":"CMakeLists.txt","path":"CMakeLists.txt","contentType":"file"},{"name":"CODE_OF_CONDUCT.md","path":"CODE_OF_CONDUCT.md","contentType":"file"},{"name":"CONTRIBUTING.md","path":"CONTRIBUTING.md","contentType":"file"},{"name":"LICENSE","path":"LICENSE","contentType":"file"},{"name":"PreLoad.cmake","path":"PreLoad.cmake","contentType":"file"},{"name":"README.md","path":"README.md","contentType":"file"},{"name":"SECURITY.md","path":"SECURITY.md","contentType":"file"},{"name":"format_sources","path":"format_sources","contentType":"file"}],"totalCount":32}},"fileTreeProcessingTime":8.697185,"foldersToFetch":[],"reducedMotionEnabled":"system","repo":{"id":60246359,"defaultBranch":"master","name":"ClickHouse","ownerLogin":"ClickHouse","currentUserCanPush":true,"isFork":false,"isEmpty":false,"createdAt":"2016-06-02T04:28:18.000-04:00","ownerAvatar":"https://avatars.githubusercontent.com/u/54801242?v=4","public":true,"private":false,"isOrgOwned":true},"refInfo":{"name":"5f18640215159ed1ad50be1efce2cb996a49fd73","listCacheKey":"v0:1688403108.0","canEdit":false,"refType":"tree","currentOid":"5f18640215159ed1ad50be1efce2cb996a49fd73"},"path":"docs/_description_templates/template-system-table.md","currentUser":{"id":25182304,"login":"DanRoscigno","userEmail":"dan@roscigno.com"},"blob":{"rawBlob":null,"colorizedLines":null,"stylingDirectives":null,"csv":null,"csvError":null,"dependabotInfo":{"showConfigurationBanner":null,"configFilePath":null,"networkDependabotPath":"/ClickHouse/ClickHouse/network/updates","dismissConfigurationNoticePath":"/settings/dismiss-notice/dependabot_configuration_notice","configurationNoticeDismissed":false,"repoAlertsPath":"/ClickHouse/ClickHouse/security/dependabot","repoSecurityAndAnalysisPath":"/ClickHouse/ClickHouse/settings/security_analysis","repoOwnerIsOrg":true,"currentUserCanAdminRepo":false},"displayName":"template-system-table.md","displayUrl":"https://github.com/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-system-table.md?raw=true","headerInfo":{"blobSize":"381 Bytes","deleteInfo":{"deletePath":null,"deleteTooltip":"You must be on a branch to make or propose changes to this file"},"editInfo":{"editTooltip":"You must be on a branch to make or propose changes to this file"},"ghDesktopPath":null,"gitLfsPath":null,"onBranch":false,"shortPath":"f2decc4","siteNavLoginPath":"/login?return_to=https%3A%2F%2Fgithub.com%2FClickHouse%2FClickHouse%2Fblob%2F5f18640215159ed1ad50be1efce2cb996a49fd73%2Fdocs%2F_description_templates%2Ftemplate-system-table.md","isCSV":false,"isRichtext":true,"toc":[{"level":1,"text":"system.table_name {#system-tables_table-name}","anchor":"systemtable_name-system-tables_table-name","htmlText":"system.table_name {#system-tables_table-name}"}],"lineInfo":{"truncatedLoc":"25","truncatedSloc":"15"},"mode":"file"},"image":false,"isCodeownersFile":null,"isValidLegacyIssueTemplate":false,"issueTemplateHelpUrl":"https://docs.github.com/articles/about-issue-and-pull-request-templates","issueTemplate":null,"discussionTemplate":null,"language":"Markdown","large":false,"loggedIn":true,"newDiscussionPath":"/ClickHouse/ClickHouse/discussions/new","newIssuePath":"/ClickHouse/ClickHouse/issues/new","planSupportInfo":{"repoIsFork":null,"repoOwnedByCurrentUser":null,"requestFullPath":"/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-system-table.md","showFreeOrgGatedFeatureMessage":null,"showPlanSupportBanner":null,"upgradeDataAttributes":null,"upgradePath":null},"publishBannersInfo":{"dismissActionNoticePath":"/settings/dismiss-notice/publish_action_from_dockerfile","dismissStackNoticePath":"/settings/dismiss-notice/publish_stack_from_file","releasePath":"/ClickHouse/ClickHouse/releases/new?marketplace=true","showPublishActionBanner":false,"showPublishStackBanner":false},"renderImageOrRaw":false,"richText":"

system.table_name {#system-tables_table-name}

\n

Description.

\n

Columns:

\n\n

Example

\n

Query:

\n
SELECT * FROM system.table_name
\n

Result:

\n
Some output. It shouldn't be too long.\n
\n

See Also

\n
    \n
  • Article name — Some words about referenced information.
  • \n
\n
","renderedFileInfo":null,"tabSize":8,"topBannersInfo":{"overridingGlobalFundingFile":false,"globalPreferredFundingPath":null,"repoOwner":"ClickHouse","repoName":"ClickHouse","showInvalidCitationWarning":false,"citationHelpUrl":"https://docs.github.com/en/github/creating-cloning-and-archiving-repositories/creating-a-repository-on-github/about-citation-files","showDependabotConfigurationBanner":null,"actionsOnboardingTip":null},"truncated":false,"viewable":true,"workflowRedirectUrl":null,"symbols":{"timedOut":false,"notAnalyzed":true,"symbols":[]}},"csrf_tokens":{"/ClickHouse/ClickHouse/branches":{"post":"9K5xHq3WERnrVDYaCTfS2yzVWybDSHIH0WqtxpxdRGaJUn9ffSZsyvtecMTgBPUdDKOj419HV6AM0YDO1eB3pA"}}},"title":"ClickHouse/docs/_description_templates/template-system-table.md at 5f18640215159ed1ad50be1efce2cb996a49fd73 · ClickHouse/ClickHouse","locale":"en"} \ No newline at end of file From 584b46c5ca358d077db30777a0050cf690166a08 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 30 Jun 2023 14:38:59 +0200 Subject: [PATCH 510/515] Add documentation for building in docker --- docs/en/development/build.md | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index 83a4550df88..ab47ee8aac5 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -13,6 +13,20 @@ Supported platforms: - AArch64 - Power9 (experimental) +## Building in docker +We use the docker image `clickhouse/binary-builder` for our CI builds. It contains everything necessary to build the binary and packages. There is a script `docker/packager/packager` to ease the image usage: + +```bash +# define a directory for the output artifacts +output_dir="build_results" +# a simplest build +./docker/packager/packager --package-type=binary --output-dir "$output_dir" +# build debian packages +./docker/packager/packager --package-type=deb --output-dir "$output_dir" +# by default, debian packages use thin LTO, so we can override it to speed up the build +CMAKE_FLAGS='-DENABLE_THINLTO=' ./docker/packager/packager --package-type=deb --output-dir "$output_dir" +``` + ## Building on Ubuntu The following tutorial is based on Ubuntu Linux. From c715ee5cef3646e9ce71a26c46d72fb91ff3d551 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 3 Jul 2023 17:28:48 +0200 Subject: [PATCH 511/515] Replace `--build-type=debug` by `--debug-build` --- docker/packager/README.md | 2 +- docker/packager/packager | 16 ++++++++-------- tests/ci/build_check.py | 6 +++--- tests/ci/build_report_check.py | 4 ++-- tests/ci/ci_config.py | 34 +++++++++++++++++----------------- tests/ci/report.py | 6 +++--- 6 files changed, 34 insertions(+), 34 deletions(-) diff --git a/docker/packager/README.md b/docker/packager/README.md index a78feb8d7fc..3a91f9a63f0 100644 --- a/docker/packager/README.md +++ b/docker/packager/README.md @@ -6,7 +6,7 @@ Usage: Build deb package with `clang-14` in `debug` mode: ``` $ mkdir deb/test_output -$ ./packager --output-dir deb/test_output/ --package-type deb --compiler=clang-14 --build-type=debug +$ ./packager --output-dir deb/test_output/ --package-type deb --compiler=clang-14 --debug-build $ ls -l deb/test_output -rw-r--r-- 1 root root 3730 clickhouse-client_22.2.2+debug_all.deb -rw-r--r-- 1 root root 84221888 clickhouse-common-static_22.2.2+debug_amd64.deb diff --git a/docker/packager/packager b/docker/packager/packager index 1b3df858cd2..3c3304165b3 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -112,12 +112,12 @@ def run_docker_image_with_env( subprocess.check_call(cmd, shell=True) -def is_release_build(build_type: str, package_type: str, sanitizer: str) -> bool: - return build_type == "" and package_type == "deb" and sanitizer == "" +def is_release_build(debug_build: bool, package_type: str, sanitizer: str) -> bool: + return not debug_build and package_type == "deb" and sanitizer == "" def parse_env_variables( - build_type: str, + debug_build: bool, compiler: str, sanitizer: str, package_type: str, @@ -233,7 +233,7 @@ def parse_env_variables( build_target = ( f"{build_target} clickhouse-odbc-bridge clickhouse-library-bridge" ) - if is_release_build(build_type, package_type, sanitizer): + if is_release_build(debug_build, package_type, sanitizer): cmake_flags.append("-DSPLIT_DEBUG_SYMBOLS=ON") result.append("WITH_PERFORMANCE=1") if is_cross_arm: @@ -253,8 +253,8 @@ def parse_env_variables( if sanitizer: result.append(f"SANITIZER={sanitizer}") - if build_type: - result.append(f"BUILD_TYPE={build_type.capitalize()}") + if debug_build: + result.append("BUILD_TYPE=DEBUG") else: result.append("BUILD_TYPE=None") @@ -359,7 +359,7 @@ def parse_args() -> argparse.Namespace: help="ClickHouse git repository", ) parser.add_argument("--output-dir", type=dir_name, required=True) - parser.add_argument("--build-type", choices=("debug", ""), default="") + parser.add_argument("--debug-build", action="store_true") parser.add_argument( "--compiler", @@ -464,7 +464,7 @@ def main(): build_image(image_with_version, dockerfile) env_prepared = parse_env_variables( - args.build_type, + args.debug_build, args.compiler, args.sanitizer, args.package_type, diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 35b98a7c3bb..2a636faf967 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -45,7 +45,7 @@ def _can_export_binaries(build_config: BuildConfig) -> bool: return False if build_config["sanitizer"] != "": return True - if build_config["build_type"] != "": + if build_config["debug_build"]: return True return False @@ -66,8 +66,8 @@ def get_packager_cmd( f"--package-type={package_type} --compiler={comp}" ) - if build_config["build_type"]: - cmd += f" --build-type={build_config['build_type']}" + if build_config["debug_build"]: + cmd += " --debug-build" if build_config["sanitizer"]: cmd += f" --sanitizer={build_config['sanitizer']}" if build_config["tidy"] == "enable": diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 1362f3c8934..295b6cf9740 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -70,7 +70,7 @@ def get_failed_report( message = f"{job_name} failed" build_result = BuildResult( compiler="unknown", - build_type="unknown", + debug_build=False, sanitizer="unknown", status=message, elapsed_seconds=0, @@ -85,7 +85,7 @@ def process_report( build_config = build_report["build_config"] build_result = BuildResult( compiler=build_config["compiler"], - build_type=build_config["build_type"], + debug_build=build_config["debug_build"], sanitizer=build_config["sanitizer"], status="success" if build_report["status"] else "failure", elapsed_seconds=build_report["elapsed_seconds"], diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index c680b5810fc..875c5a3c8bd 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -10,7 +10,7 @@ CI_CONFIG = { "build_config": { "package_release": { "compiler": "clang-16", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "deb", "static_binary_name": "amd64", @@ -21,7 +21,7 @@ CI_CONFIG = { }, "coverity": { "compiler": "clang-16", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "coverity", "tidy": "disable", @@ -31,7 +31,7 @@ CI_CONFIG = { }, "package_aarch64": { "compiler": "clang-16-aarch64", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "deb", "static_binary_name": "aarch64", @@ -42,7 +42,7 @@ CI_CONFIG = { }, "package_asan": { "compiler": "clang-16", - "build_type": "", + "debug_build": False, "sanitizer": "address", "package_type": "deb", "tidy": "disable", @@ -51,7 +51,7 @@ CI_CONFIG = { }, "package_ubsan": { "compiler": "clang-16", - "build_type": "", + "debug_build": False, "sanitizer": "undefined", "package_type": "deb", "tidy": "disable", @@ -60,7 +60,7 @@ CI_CONFIG = { }, "package_tsan": { "compiler": "clang-16", - "build_type": "", + "debug_build": False, "sanitizer": "thread", "package_type": "deb", "tidy": "disable", @@ -69,7 +69,7 @@ CI_CONFIG = { }, "package_msan": { "compiler": "clang-16", - "build_type": "", + "debug_build": False, "sanitizer": "memory", "package_type": "deb", "tidy": "disable", @@ -78,7 +78,7 @@ CI_CONFIG = { }, "package_debug": { "compiler": "clang-16", - "build_type": "debug", + "debug_build": True, "sanitizer": "", "package_type": "deb", "tidy": "disable", @@ -87,7 +87,7 @@ CI_CONFIG = { }, "binary_release": { "compiler": "clang-16", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "binary", "tidy": "disable", @@ -96,7 +96,7 @@ CI_CONFIG = { }, "binary_tidy": { "compiler": "clang-16", - "build_type": "debug", + "debug_build": True, "sanitizer": "", "package_type": "binary", "static_binary_name": "debug-amd64", @@ -106,7 +106,7 @@ CI_CONFIG = { }, "binary_darwin": { "compiler": "clang-16-darwin", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "binary", "static_binary_name": "macos", @@ -116,7 +116,7 @@ CI_CONFIG = { }, "binary_aarch64": { "compiler": "clang-16-aarch64", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "binary", "tidy": "disable", @@ -125,7 +125,7 @@ CI_CONFIG = { }, "binary_aarch64_v80compat": { "compiler": "clang-16-aarch64-v80compat", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "binary", "static_binary_name": "aarch64v80compat", @@ -135,7 +135,7 @@ CI_CONFIG = { }, "binary_freebsd": { "compiler": "clang-16-freebsd", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "binary", "static_binary_name": "freebsd", @@ -145,7 +145,7 @@ CI_CONFIG = { }, "binary_darwin_aarch64": { "compiler": "clang-16-darwin-aarch64", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "binary", "static_binary_name": "macos-aarch64", @@ -155,7 +155,7 @@ CI_CONFIG = { }, "binary_ppc64le": { "compiler": "clang-16-ppc64le", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "binary", "static_binary_name": "powerpc64le", @@ -165,7 +165,7 @@ CI_CONFIG = { }, "binary_amd64_compat": { "compiler": "clang-16-amd64-compat", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "binary", "static_binary_name": "amd64compat", diff --git a/tests/ci/report.py b/tests/ci/report.py index a9014acec12..0f84fbcaeb2 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -239,7 +239,7 @@ def read_test_results(results_path: Path, with_raw_logs: bool = True) -> TestRes @dataclass class BuildResult: compiler: str - build_type: str + debug_build: bool sanitizer: str status: str elapsed_seconds: int @@ -484,8 +484,8 @@ def create_build_html_report( ): row = "" row += f"{build_result.compiler}" - if build_result.build_type: - row += f"{build_result.build_type}" + if build_result.debug_build: + row += "debug" else: row += "relwithdebuginfo" if build_result.sanitizer: From 4191e3eb952bdcc136da29213f8a74d895a12ee6 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Mon, 3 Jul 2023 14:15:20 -0400 Subject: [PATCH 512/515] add missing doc templates back --- .../template-data-type.md | 30 ++++++++- .../_description_templates/template-engine.md | 64 ++++++++++++++++++- .../template-function.md | 52 ++++++++++++++- .../template-server-setting.md | 34 +++++++++- .../template-setting.md | 28 +++++++- .../template-statement.md | 25 +++++++- .../template-system-table.md | 26 +++++++- 7 files changed, 252 insertions(+), 7 deletions(-) diff --git a/docs/_description_templates/template-data-type.md b/docs/_description_templates/template-data-type.md index b66d77c7a77..239edb2808b 100644 --- a/docs/_description_templates/template-data-type.md +++ b/docs/_description_templates/template-data-type.md @@ -1 +1,29 @@ -{"payload":{"allShortcutsEnabled":true,"fileTree":{"docs/_description_templates":{"items":[{"name":"template-data-type.md","path":"docs/_description_templates/template-data-type.md","contentType":"file"},{"name":"template-engine.md","path":"docs/_description_templates/template-engine.md","contentType":"file"},{"name":"template-function.md","path":"docs/_description_templates/template-function.md","contentType":"file"},{"name":"template-server-setting.md","path":"docs/_description_templates/template-server-setting.md","contentType":"file"},{"name":"template-setting.md","path":"docs/_description_templates/template-setting.md","contentType":"file"},{"name":"template-statement.md","path":"docs/_description_templates/template-statement.md","contentType":"file"},{"name":"template-system-table.md","path":"docs/_description_templates/template-system-table.md","contentType":"file"}],"totalCount":7},"docs":{"items":[{"name":"_description_templates","path":"docs/_description_templates","contentType":"directory"},{"name":"_includes","path":"docs/_includes","contentType":"directory"},{"name":"changelogs","path":"docs/changelogs","contentType":"directory"},{"name":"en","path":"docs/en","contentType":"directory"},{"name":"ru","path":"docs/ru","contentType":"directory"},{"name":"tools","path":"docs/tools","contentType":"directory"},{"name":"zh","path":"docs/zh","contentType":"directory"},{"name":".gitignore","path":"docs/.gitignore","contentType":"file"},{"name":"README.md","path":"docs/README.md","contentType":"file"},{"name":"clean","path":"docs/clean","contentType":"file"},{"name":"mkdocs.yml","path":"docs/mkdocs.yml","contentType":"file"},{"name":"redirects.txt","path":"docs/redirects.txt","contentType":"file"}],"totalCount":12},"":{"items":[{"name":".github","path":".github","contentType":"directory"},{"name":"base","path":"base","contentType":"directory"},{"name":"benchmark","path":"benchmark","contentType":"directory"},{"name":"cmake","path":"cmake","contentType":"directory"},{"name":"contrib","path":"contrib","contentType":"directory"},{"name":"docker","path":"docker","contentType":"directory"},{"name":"docs","path":"docs","contentType":"directory"},{"name":"packages","path":"packages","contentType":"directory"},{"name":"programs","path":"programs","contentType":"directory"},{"name":"src","path":"src","contentType":"directory"},{"name":"tests","path":"tests","contentType":"directory"},{"name":"utils","path":"utils","contentType":"directory"},{"name":"website","path":"website","contentType":"directory"},{"name":".clang-format","path":".clang-format","contentType":"file"},{"name":".clang-tidy","path":".clang-tidy","contentType":"file"},{"name":".editorconfig","path":".editorconfig","contentType":"file"},{"name":".gitattributes","path":".gitattributes","contentType":"file"},{"name":".gitignore","path":".gitignore","contentType":"file"},{"name":".gitmodules","path":".gitmodules","contentType":"file"},{"name":".pylintrc","path":".pylintrc","contentType":"file"},{"name":".vimrc","path":".vimrc","contentType":"file"},{"name":".yamllint","path":".yamllint","contentType":"file"},{"name":"AUTHORS","path":"AUTHORS","contentType":"file"},{"name":"CHANGELOG.md","path":"CHANGELOG.md","contentType":"file"},{"name":"CMakeLists.txt","path":"CMakeLists.txt","contentType":"file"},{"name":"CODE_OF_CONDUCT.md","path":"CODE_OF_CONDUCT.md","contentType":"file"},{"name":"CONTRIBUTING.md","path":"CONTRIBUTING.md","contentType":"file"},{"name":"LICENSE","path":"LICENSE","contentType":"file"},{"name":"PreLoad.cmake","path":"PreLoad.cmake","contentType":"file"},{"name":"README.md","path":"README.md","contentType":"file"},{"name":"SECURITY.md","path":"SECURITY.md","contentType":"file"},{"name":"format_sources","path":"format_sources","contentType":"file"}],"totalCount":32}},"fileTreeProcessingTime":14.564540000000001,"foldersToFetch":[],"reducedMotionEnabled":"system","repo":{"id":60246359,"defaultBranch":"master","name":"ClickHouse","ownerLogin":"ClickHouse","currentUserCanPush":true,"isFork":false,"isEmpty":false,"createdAt":"2016-06-02T04:28:18.000-04:00","ownerAvatar":"https://avatars.githubusercontent.com/u/54801242?v=4","public":true,"private":false,"isOrgOwned":true},"refInfo":{"name":"5f18640215159ed1ad50be1efce2cb996a49fd73","listCacheKey":"v0:1688403108.0","canEdit":false,"refType":"tree","currentOid":"5f18640215159ed1ad50be1efce2cb996a49fd73"},"path":"docs/_description_templates/template-data-type.md","currentUser":{"id":25182304,"login":"DanRoscigno","userEmail":"dan@roscigno.com"},"blob":{"rawBlob":null,"colorizedLines":null,"stylingDirectives":null,"csv":null,"csvError":null,"dependabotInfo":{"showConfigurationBanner":null,"configFilePath":null,"networkDependabotPath":"/ClickHouse/ClickHouse/network/updates","dismissConfigurationNoticePath":"/settings/dismiss-notice/dependabot_configuration_notice","configurationNoticeDismissed":false,"repoAlertsPath":"/ClickHouse/ClickHouse/security/dependabot","repoSecurityAndAnalysisPath":"/ClickHouse/ClickHouse/settings/security_analysis","repoOwnerIsOrg":true,"currentUserCanAdminRepo":false},"displayName":"template-data-type.md","displayUrl":"https://github.com/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-data-type.md?raw=true","headerInfo":{"blobSize":"540 Bytes","deleteInfo":{"deletePath":null,"deleteTooltip":"You must be on a branch to make or propose changes to this file"},"editInfo":{"editTooltip":"You must be on a branch to make or propose changes to this file"},"ghDesktopPath":null,"gitLfsPath":null,"onBranch":false,"shortPath":"239edb2","siteNavLoginPath":"/login?return_to=https%3A%2F%2Fgithub.com%2FClickHouse%2FClickHouse%2Fblob%2F5f18640215159ed1ad50be1efce2cb996a49fd73%2Fdocs%2F_description_templates%2Ftemplate-data-type.md","isCSV":false,"isRichtext":true,"toc":[{"level":1,"text":"data_type_name {#data_type-name}","anchor":"data_type_name-data_type-name","htmlText":"data_type_name {#data_type-name}"},{"level":2,"text":"Additional Info {#additional-info} (Optional)","anchor":"additional-info-additional-info-optional","htmlText":"Additional Info {#additional-info} (Optional)"}],"lineInfo":{"truncatedLoc":"29","truncatedSloc":"17"},"mode":"file"},"image":false,"isCodeownersFile":null,"isValidLegacyIssueTemplate":false,"issueTemplateHelpUrl":"https://docs.github.com/articles/about-issue-and-pull-request-templates","issueTemplate":null,"discussionTemplate":null,"language":"Markdown","large":false,"loggedIn":true,"newDiscussionPath":"/ClickHouse/ClickHouse/discussions/new","newIssuePath":"/ClickHouse/ClickHouse/issues/new","planSupportInfo":{"repoIsFork":null,"repoOwnedByCurrentUser":null,"requestFullPath":"/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-data-type.md","showFreeOrgGatedFeatureMessage":null,"showPlanSupportBanner":null,"upgradeDataAttributes":null,"upgradePath":null},"publishBannersInfo":{"dismissActionNoticePath":"/settings/dismiss-notice/publish_action_from_dockerfile","dismissStackNoticePath":"/settings/dismiss-notice/publish_stack_from_file","releasePath":"/ClickHouse/ClickHouse/releases/new?marketplace=true","showPublishActionBanner":false,"showPublishStackBanner":false},"renderImageOrRaw":false,"richText":"
\n \n \n \n \n \n \n \n \n
toc_prioritytoc_title
\n\n

data_type_name {#data_type-name}

\n

Description.

\n

Parameters (Optional)

\n\n

Examples

\n
\n

Additional Info {#additional-info} (Optional)

\n

The name of an additional section can be any, for example, Usage.

\n

See Also (Optional)

\n\n

Original article

\n
","renderedFileInfo":null,"tabSize":8,"topBannersInfo":{"overridingGlobalFundingFile":false,"globalPreferredFundingPath":null,"repoOwner":"ClickHouse","repoName":"ClickHouse","showInvalidCitationWarning":false,"citationHelpUrl":"https://docs.github.com/en/github/creating-cloning-and-archiving-repositories/creating-a-repository-on-github/about-citation-files","showDependabotConfigurationBanner":null,"actionsOnboardingTip":null},"truncated":false,"viewable":true,"workflowRedirectUrl":null,"symbols":{"timedOut":false,"notAnalyzed":true,"symbols":[]}},"csrf_tokens":{"/ClickHouse/ClickHouse/branches":{"post":"N-EHr0Rgr9I38z0iGu5sqtyb4o4AwtOW1kjllryDXwFKHQnulJDSASf5e_zz3Uts_O0aS5zN9jEL88ie9T5sww"}}},"title":"ClickHouse/docs/_description_templates/template-data-type.md at 5f18640215159ed1ad50be1efce2cb996a49fd73 · ClickHouse/ClickHouse","locale":"en"} \ No newline at end of file +--- +toc_priority: +toc_title: +--- + +# data_type_name {#data_type-name} + +Description. + +**Parameters** (Optional) + +- `x` — Description. [Type name](relative/path/to/type/dscr.md#type). +- `y` — Description. [Type name](relative/path/to/type/dscr.md#type). + +**Examples** + +```sql + +``` + +## Additional Info {#additional-info} (Optional) + +The name of an additional section can be any, for example, **Usage**. + +**See Also** (Optional) + +- [link](#) + +[Original article](https://clickhouse.com/docs/en/data-types//) diff --git a/docs/_description_templates/template-engine.md b/docs/_description_templates/template-engine.md index c5bb5feb85d..392bc59ed33 100644 --- a/docs/_description_templates/template-engine.md +++ b/docs/_description_templates/template-engine.md @@ -1 +1,63 @@ -{"payload":{"allShortcutsEnabled":true,"fileTree":{"docs/_description_templates":{"items":[{"name":"template-data-type.md","path":"docs/_description_templates/template-data-type.md","contentType":"file"},{"name":"template-engine.md","path":"docs/_description_templates/template-engine.md","contentType":"file"},{"name":"template-function.md","path":"docs/_description_templates/template-function.md","contentType":"file"},{"name":"template-server-setting.md","path":"docs/_description_templates/template-server-setting.md","contentType":"file"},{"name":"template-setting.md","path":"docs/_description_templates/template-setting.md","contentType":"file"},{"name":"template-statement.md","path":"docs/_description_templates/template-statement.md","contentType":"file"},{"name":"template-system-table.md","path":"docs/_description_templates/template-system-table.md","contentType":"file"}],"totalCount":7},"docs":{"items":[{"name":"_description_templates","path":"docs/_description_templates","contentType":"directory"},{"name":"_includes","path":"docs/_includes","contentType":"directory"},{"name":"changelogs","path":"docs/changelogs","contentType":"directory"},{"name":"en","path":"docs/en","contentType":"directory"},{"name":"ru","path":"docs/ru","contentType":"directory"},{"name":"tools","path":"docs/tools","contentType":"directory"},{"name":"zh","path":"docs/zh","contentType":"directory"},{"name":".gitignore","path":"docs/.gitignore","contentType":"file"},{"name":"README.md","path":"docs/README.md","contentType":"file"},{"name":"clean","path":"docs/clean","contentType":"file"},{"name":"mkdocs.yml","path":"docs/mkdocs.yml","contentType":"file"},{"name":"redirects.txt","path":"docs/redirects.txt","contentType":"file"}],"totalCount":12},"":{"items":[{"name":".github","path":".github","contentType":"directory"},{"name":"base","path":"base","contentType":"directory"},{"name":"benchmark","path":"benchmark","contentType":"directory"},{"name":"cmake","path":"cmake","contentType":"directory"},{"name":"contrib","path":"contrib","contentType":"directory"},{"name":"docker","path":"docker","contentType":"directory"},{"name":"docs","path":"docs","contentType":"directory"},{"name":"packages","path":"packages","contentType":"directory"},{"name":"programs","path":"programs","contentType":"directory"},{"name":"src","path":"src","contentType":"directory"},{"name":"tests","path":"tests","contentType":"directory"},{"name":"utils","path":"utils","contentType":"directory"},{"name":"website","path":"website","contentType":"directory"},{"name":".clang-format","path":".clang-format","contentType":"file"},{"name":".clang-tidy","path":".clang-tidy","contentType":"file"},{"name":".editorconfig","path":".editorconfig","contentType":"file"},{"name":".gitattributes","path":".gitattributes","contentType":"file"},{"name":".gitignore","path":".gitignore","contentType":"file"},{"name":".gitmodules","path":".gitmodules","contentType":"file"},{"name":".pylintrc","path":".pylintrc","contentType":"file"},{"name":".vimrc","path":".vimrc","contentType":"file"},{"name":".yamllint","path":".yamllint","contentType":"file"},{"name":"AUTHORS","path":"AUTHORS","contentType":"file"},{"name":"CHANGELOG.md","path":"CHANGELOG.md","contentType":"file"},{"name":"CMakeLists.txt","path":"CMakeLists.txt","contentType":"file"},{"name":"CODE_OF_CONDUCT.md","path":"CODE_OF_CONDUCT.md","contentType":"file"},{"name":"CONTRIBUTING.md","path":"CONTRIBUTING.md","contentType":"file"},{"name":"LICENSE","path":"LICENSE","contentType":"file"},{"name":"PreLoad.cmake","path":"PreLoad.cmake","contentType":"file"},{"name":"README.md","path":"README.md","contentType":"file"},{"name":"SECURITY.md","path":"SECURITY.md","contentType":"file"},{"name":"format_sources","path":"format_sources","contentType":"file"}],"totalCount":32}},"fileTreeProcessingTime":10.334819,"foldersToFetch":[],"reducedMotionEnabled":"system","repo":{"id":60246359,"defaultBranch":"master","name":"ClickHouse","ownerLogin":"ClickHouse","currentUserCanPush":true,"isFork":false,"isEmpty":false,"createdAt":"2016-06-02T04:28:18.000-04:00","ownerAvatar":"https://avatars.githubusercontent.com/u/54801242?v=4","public":true,"private":false,"isOrgOwned":true},"refInfo":{"name":"5f18640215159ed1ad50be1efce2cb996a49fd73","listCacheKey":"v0:1688403108.0","canEdit":false,"refType":"tree","currentOid":"5f18640215159ed1ad50be1efce2cb996a49fd73"},"path":"docs/_description_templates/template-engine.md","currentUser":{"id":25182304,"login":"DanRoscigno","userEmail":"dan@roscigno.com"},"blob":{"rawBlob":null,"colorizedLines":null,"stylingDirectives":null,"csv":null,"csvError":null,"dependabotInfo":{"showConfigurationBanner":null,"configFilePath":null,"networkDependabotPath":"/ClickHouse/ClickHouse/network/updates","dismissConfigurationNoticePath":"/settings/dismiss-notice/dependabot_configuration_notice","configurationNoticeDismissed":false,"repoAlertsPath":"/ClickHouse/ClickHouse/security/dependabot","repoSecurityAndAnalysisPath":"/ClickHouse/ClickHouse/settings/security_analysis","repoOwnerIsOrg":true,"currentUserCanAdminRepo":false},"displayName":"template-engine.md","displayUrl":"https://github.com/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-engine.md?raw=true","headerInfo":{"blobSize":"1.19 KB","deleteInfo":{"deletePath":null,"deleteTooltip":"You must be on a branch to make or propose changes to this file"},"editInfo":{"editTooltip":"You must be on a branch to make or propose changes to this file"},"ghDesktopPath":null,"gitLfsPath":null,"onBranch":false,"shortPath":"392bc59","siteNavLoginPath":"/login?return_to=https%3A%2F%2Fgithub.com%2FClickHouse%2FClickHouse%2Fblob%2F5f18640215159ed1ad50be1efce2cb996a49fd73%2Fdocs%2F_description_templates%2Ftemplate-engine.md","isCSV":false,"isRichtext":true,"toc":[{"level":1,"text":"EngineName {#enginename}","anchor":"enginename-enginename","htmlText":"EngineName {#enginename}"},{"level":2,"text":"Creating a Database {#creating-a-database}","anchor":"creating-a-database-creating-a-database","htmlText":"Creating a Database {#creating-a-database}"},{"level":2,"text":"Creating a Table {#creating-a-table}","anchor":"creating-a-table-creating-a-table","htmlText":"Creating a Table {#creating-a-table}"},{"level":2,"text":"Virtual columns {#virtual-columns} (for Table engines only)","anchor":"virtual-columns-virtual-columns-for-table-engines-only","htmlText":"Virtual columns {#virtual-columns} (for Table engines only)"},{"level":2,"text":"Data Types Support {#data_types-support} (for Database engines only)","anchor":"data-types-support-data_types-support-for-database-engines-only","htmlText":"Data Types Support {#data_types-support} (for Database engines only)"},{"level":2,"text":"Specifics and recommendations {#specifics-and-recommendations}","anchor":"specifics-and-recommendations-specifics-and-recommendations","htmlText":"Specifics and recommendations {#specifics-and-recommendations}"},{"level":2,"text":"Usage Example {#usage-example}","anchor":"usage-example-usage-example","htmlText":"Usage Example {#usage-example}"}],"lineInfo":{"truncatedLoc":"63","truncatedSloc":"40"},"mode":"file"},"image":false,"isCodeownersFile":null,"isValidLegacyIssueTemplate":false,"issueTemplateHelpUrl":"https://docs.github.com/articles/about-issue-and-pull-request-templates","issueTemplate":null,"discussionTemplate":null,"language":"Markdown","large":false,"loggedIn":true,"newDiscussionPath":"/ClickHouse/ClickHouse/discussions/new","newIssuePath":"/ClickHouse/ClickHouse/issues/new","planSupportInfo":{"repoIsFork":null,"repoOwnedByCurrentUser":null,"requestFullPath":"/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-engine.md","showFreeOrgGatedFeatureMessage":null,"showPlanSupportBanner":null,"upgradeDataAttributes":null,"upgradePath":null},"publishBannersInfo":{"dismissActionNoticePath":"/settings/dismiss-notice/publish_action_from_dockerfile","dismissStackNoticePath":"/settings/dismiss-notice/publish_stack_from_file","releasePath":"/ClickHouse/ClickHouse/releases/new?marketplace=true","showPublishActionBanner":false,"showPublishStackBanner":false},"renderImageOrRaw":false,"richText":"

EngineName {#enginename}

\n
    \n
  • What the Database/Table engine does.
  • \n
  • Relations with other engines if they exist.
  • \n
\n

Creating a Database {#creating-a-database}

\n
    CREATE DATABASE ...
\n

or

\n

Creating a Table {#creating-a-table}

\n
    CREATE TABLE ...
\n

Engine Parameters

\n

Query Clauses (for Table engines only)

\n

Virtual columns {#virtual-columns} (for Table engines only)

\n

List and virtual columns with description, if they exist.

\n

Data Types Support {#data_types-support} (for Database engines only)

\n\n\n\n\n\n\n\n\n\n\n\n\n\n
EngineNameClickHouse
NativeDataTypeNameClickHouseDataTypeName
\n

Specifics and recommendations {#specifics-and-recommendations}

\n

Algorithms\nSpecifics of read and write processes\nExamples of tasks\nRecommendations for usage\nSpecifics of data storage

\n

Usage Example {#usage-example}

\n

The example must show usage and use cases. The following text contains the recommended parts of this section.

\n

Input table:

\n
\n

Query:

\n
\n

Result:

\n
\n

Follow up with any text to clarify the example.

\n

See Also

\n\n
","renderedFileInfo":null,"tabSize":8,"topBannersInfo":{"overridingGlobalFundingFile":false,"globalPreferredFundingPath":null,"repoOwner":"ClickHouse","repoName":"ClickHouse","showInvalidCitationWarning":false,"citationHelpUrl":"https://docs.github.com/en/github/creating-cloning-and-archiving-repositories/creating-a-repository-on-github/about-citation-files","showDependabotConfigurationBanner":null,"actionsOnboardingTip":null},"truncated":false,"viewable":true,"workflowRedirectUrl":null,"symbols":{"timedOut":false,"notAnalyzed":true,"symbols":[]}},"csrf_tokens":{"/ClickHouse/ClickHouse/branches":{"post":"mbr-hnliBQCB5MoHGpQf6OTZtY7x7nHuM4vvq5uev5jkRvDHqZJ405HujNnzpzguxK9NS23hVEnuMMKj0iOMWg"}}},"title":"ClickHouse/docs/_description_templates/template-engine.md at 5f18640215159ed1ad50be1efce2cb996a49fd73 · ClickHouse/ClickHouse","locale":"en"} \ No newline at end of file +# EngineName {#enginename} + +- What the Database/Table engine does. +- Relations with other engines if they exist. + +## Creating a Database {#creating-a-database} +``` sql + CREATE DATABASE ... +``` +or + +## Creating a Table {#creating-a-table} +``` sql + CREATE TABLE ... +``` + +**Engine Parameters** + +**Query Clauses** (for Table engines only) + +## Virtual columns {#virtual-columns} (for Table engines only) + +List and virtual columns with description, if they exist. + +## Data Types Support {#data_types-support} (for Database engines only) + +| EngineName | ClickHouse | +|-----------------------|------------------------------------| +| NativeDataTypeName | [ClickHouseDataTypeName](link#) | + + +## Specifics and recommendations {#specifics-and-recommendations} + +Algorithms +Specifics of read and write processes +Examples of tasks +Recommendations for usage +Specifics of data storage + +## Usage Example {#usage-example} + +The example must show usage and use cases. The following text contains the recommended parts of this section. + +Input table: + +``` text +``` + +Query: + +``` sql +``` + +Result: + +``` text +``` + +Follow up with any text to clarify the example. + +**See Also** + +- [link](#) diff --git a/docs/_description_templates/template-function.md b/docs/_description_templates/template-function.md index 0891e5d872d..6bdc764c449 100644 --- a/docs/_description_templates/template-function.md +++ b/docs/_description_templates/template-function.md @@ -1 +1,51 @@ -{"payload":{"allShortcutsEnabled":true,"fileTree":{"docs/_description_templates":{"items":[{"name":"template-data-type.md","path":"docs/_description_templates/template-data-type.md","contentType":"file"},{"name":"template-engine.md","path":"docs/_description_templates/template-engine.md","contentType":"file"},{"name":"template-function.md","path":"docs/_description_templates/template-function.md","contentType":"file"},{"name":"template-server-setting.md","path":"docs/_description_templates/template-server-setting.md","contentType":"file"},{"name":"template-setting.md","path":"docs/_description_templates/template-setting.md","contentType":"file"},{"name":"template-statement.md","path":"docs/_description_templates/template-statement.md","contentType":"file"},{"name":"template-system-table.md","path":"docs/_description_templates/template-system-table.md","contentType":"file"}],"totalCount":7},"docs":{"items":[{"name":"_description_templates","path":"docs/_description_templates","contentType":"directory"},{"name":"_includes","path":"docs/_includes","contentType":"directory"},{"name":"changelogs","path":"docs/changelogs","contentType":"directory"},{"name":"en","path":"docs/en","contentType":"directory"},{"name":"ru","path":"docs/ru","contentType":"directory"},{"name":"tools","path":"docs/tools","contentType":"directory"},{"name":"zh","path":"docs/zh","contentType":"directory"},{"name":".gitignore","path":"docs/.gitignore","contentType":"file"},{"name":"README.md","path":"docs/README.md","contentType":"file"},{"name":"clean","path":"docs/clean","contentType":"file"},{"name":"mkdocs.yml","path":"docs/mkdocs.yml","contentType":"file"},{"name":"redirects.txt","path":"docs/redirects.txt","contentType":"file"}],"totalCount":12},"":{"items":[{"name":".github","path":".github","contentType":"directory"},{"name":"base","path":"base","contentType":"directory"},{"name":"benchmark","path":"benchmark","contentType":"directory"},{"name":"cmake","path":"cmake","contentType":"directory"},{"name":"contrib","path":"contrib","contentType":"directory"},{"name":"docker","path":"docker","contentType":"directory"},{"name":"docs","path":"docs","contentType":"directory"},{"name":"packages","path":"packages","contentType":"directory"},{"name":"programs","path":"programs","contentType":"directory"},{"name":"src","path":"src","contentType":"directory"},{"name":"tests","path":"tests","contentType":"directory"},{"name":"utils","path":"utils","contentType":"directory"},{"name":"website","path":"website","contentType":"directory"},{"name":".clang-format","path":".clang-format","contentType":"file"},{"name":".clang-tidy","path":".clang-tidy","contentType":"file"},{"name":".editorconfig","path":".editorconfig","contentType":"file"},{"name":".gitattributes","path":".gitattributes","contentType":"file"},{"name":".gitignore","path":".gitignore","contentType":"file"},{"name":".gitmodules","path":".gitmodules","contentType":"file"},{"name":".pylintrc","path":".pylintrc","contentType":"file"},{"name":".vimrc","path":".vimrc","contentType":"file"},{"name":".yamllint","path":".yamllint","contentType":"file"},{"name":"AUTHORS","path":"AUTHORS","contentType":"file"},{"name":"CHANGELOG.md","path":"CHANGELOG.md","contentType":"file"},{"name":"CMakeLists.txt","path":"CMakeLists.txt","contentType":"file"},{"name":"CODE_OF_CONDUCT.md","path":"CODE_OF_CONDUCT.md","contentType":"file"},{"name":"CONTRIBUTING.md","path":"CONTRIBUTING.md","contentType":"file"},{"name":"LICENSE","path":"LICENSE","contentType":"file"},{"name":"PreLoad.cmake","path":"PreLoad.cmake","contentType":"file"},{"name":"README.md","path":"README.md","contentType":"file"},{"name":"SECURITY.md","path":"SECURITY.md","contentType":"file"},{"name":"format_sources","path":"format_sources","contentType":"file"}],"totalCount":32}},"fileTreeProcessingTime":6.875799,"foldersToFetch":[],"reducedMotionEnabled":"system","repo":{"id":60246359,"defaultBranch":"master","name":"ClickHouse","ownerLogin":"ClickHouse","currentUserCanPush":true,"isFork":false,"isEmpty":false,"createdAt":"2016-06-02T04:28:18.000-04:00","ownerAvatar":"https://avatars.githubusercontent.com/u/54801242?v=4","public":true,"private":false,"isOrgOwned":true},"refInfo":{"name":"5f18640215159ed1ad50be1efce2cb996a49fd73","listCacheKey":"v0:1688403108.0","canEdit":false,"refType":"tree","currentOid":"5f18640215159ed1ad50be1efce2cb996a49fd73"},"path":"docs/_description_templates/template-function.md","currentUser":{"id":25182304,"login":"DanRoscigno","userEmail":"dan@roscigno.com"},"blob":{"rawBlob":null,"colorizedLines":null,"stylingDirectives":null,"csv":null,"csvError":null,"dependabotInfo":{"showConfigurationBanner":null,"configFilePath":null,"networkDependabotPath":"/ClickHouse/ClickHouse/network/updates","dismissConfigurationNoticePath":"/settings/dismiss-notice/dependabot_configuration_notice","configurationNoticeDismissed":false,"repoAlertsPath":"/ClickHouse/ClickHouse/security/dependabot","repoSecurityAndAnalysisPath":"/ClickHouse/ClickHouse/settings/security_analysis","repoOwnerIsOrg":true,"currentUserCanAdminRepo":false},"displayName":"template-function.md","displayUrl":"https://github.com/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-function.md?raw=true","headerInfo":{"blobSize":"1.12 KB","deleteInfo":{"deletePath":null,"deleteTooltip":"You must be on a branch to make or propose changes to this file"},"editInfo":{"editTooltip":"You must be on a branch to make or propose changes to this file"},"ghDesktopPath":null,"gitLfsPath":null,"onBranch":false,"shortPath":"6bdc764","siteNavLoginPath":"/login?return_to=https%3A%2F%2Fgithub.com%2FClickHouse%2FClickHouse%2Fblob%2F5f18640215159ed1ad50be1efce2cb996a49fd73%2Fdocs%2F_description_templates%2Ftemplate-function.md","isCSV":false,"isRichtext":true,"toc":[{"level":2,"text":"functionName {#functionname-in-lower-case}","anchor":"functionname-functionname-in-lower-case","htmlText":"functionName {#functionname-in-lower-case}"}],"lineInfo":{"truncatedLoc":"51","truncatedSloc":"29"},"mode":"file"},"image":false,"isCodeownersFile":null,"isValidLegacyIssueTemplate":false,"issueTemplateHelpUrl":"https://docs.github.com/articles/about-issue-and-pull-request-templates","issueTemplate":null,"discussionTemplate":null,"language":"Markdown","large":false,"loggedIn":true,"newDiscussionPath":"/ClickHouse/ClickHouse/discussions/new","newIssuePath":"/ClickHouse/ClickHouse/issues/new","planSupportInfo":{"repoIsFork":null,"repoOwnedByCurrentUser":null,"requestFullPath":"/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-function.md","showFreeOrgGatedFeatureMessage":null,"showPlanSupportBanner":null,"upgradeDataAttributes":null,"upgradePath":null},"publishBannersInfo":{"dismissActionNoticePath":"/settings/dismiss-notice/publish_action_from_dockerfile","dismissStackNoticePath":"/settings/dismiss-notice/publish_stack_from_file","releasePath":"/ClickHouse/ClickHouse/releases/new?marketplace=true","showPublishActionBanner":false,"showPublishStackBanner":false},"renderImageOrRaw":false,"richText":"

functionName {#functionname-in-lower-case}

\n

Short description.

\n

Syntax (without SELECT)

\n
<function syntax>
\n

Alias: <alias name>. (Optional)

\n

More text (Optional).

\n

Arguments (Optional)

\n
    \n
  • x — Description. Optional (only for optional arguments). Possible values: . Default value: . Type name.
  • \n
  • y — Description. Optional (only for optional arguments). Possible values: .Default value: . Type name.
  • \n
\n

Parameters (Optional, only for parametric aggregate functions)

\n
    \n
  • z — Description. Optional (only for optional parameters). Possible values: . Default value: . Type name.
  • \n
\n

Returned value(s)

\n
    \n
  • Returned values list.
  • \n
\n

Type: Type name.

\n

Example

\n

The example must show usage and/or a use cases. The following text contains recommended parts of an example.

\n

Input table (Optional):

\n
\n

Query:

\n
\n

Result:

\n
\n

See Also (Optional)

\n\n
","renderedFileInfo":null,"tabSize":8,"topBannersInfo":{"overridingGlobalFundingFile":false,"globalPreferredFundingPath":null,"repoOwner":"ClickHouse","repoName":"ClickHouse","showInvalidCitationWarning":false,"citationHelpUrl":"https://docs.github.com/en/github/creating-cloning-and-archiving-repositories/creating-a-repository-on-github/about-citation-files","showDependabotConfigurationBanner":null,"actionsOnboardingTip":null},"truncated":false,"viewable":true,"workflowRedirectUrl":null,"symbols":{"timedOut":false,"notAnalyzed":true,"symbols":[]}},"csrf_tokens":{"/ClickHouse/ClickHouse/branches":{"post":"LBpdgwbUgHW-H_hinQl7e32v5vocOJKA0kZyCn6oG8tR5lPC1iT9pq4Vvrx0Oly9XdkeP4A3tycP_V8CNxUoCQ"}}},"title":"ClickHouse/docs/_description_templates/template-function.md at 5f18640215159ed1ad50be1efce2cb996a49fd73 · ClickHouse/ClickHouse","locale":"en"} \ No newline at end of file +## functionName {#functionname-in-lower-case} + +Short description. + +**Syntax** (without SELECT) + +``` sql + +``` + +Alias: ``. (Optional) + +More text (Optional). + +**Arguments** (Optional) + +- `x` — Description. Optional (only for optional arguments). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). +- `y` — Description. Optional (only for optional arguments). Possible values: .Default value: . [Type name](relative/path/to/type/dscr.md#type). + +**Parameters** (Optional, only for parametric aggregate functions) + +- `z` — Description. Optional (only for optional parameters). Possible values: . Default value: . [Type name](relative/path/to/type/dscr.md#type). + +**Returned value(s)** + +- Returned values list. + +Type: [Type name](relative/path/to/type/dscr.md#type). + +**Example** + +The example must show usage and/or a use cases. The following text contains recommended parts of an example. + +Input table (Optional): + +``` text +``` + +Query: + +``` sql +``` + +Result: + +``` text +``` + +**See Also** (Optional) + +- [link](#) diff --git a/docs/_description_templates/template-server-setting.md b/docs/_description_templates/template-server-setting.md index fc474059f05..0b37d46cf41 100644 --- a/docs/_description_templates/template-server-setting.md +++ b/docs/_description_templates/template-server-setting.md @@ -1 +1,33 @@ -{"payload":{"allShortcutsEnabled":true,"fileTree":{"docs/_description_templates":{"items":[{"name":"template-data-type.md","path":"docs/_description_templates/template-data-type.md","contentType":"file"},{"name":"template-engine.md","path":"docs/_description_templates/template-engine.md","contentType":"file"},{"name":"template-function.md","path":"docs/_description_templates/template-function.md","contentType":"file"},{"name":"template-server-setting.md","path":"docs/_description_templates/template-server-setting.md","contentType":"file"},{"name":"template-setting.md","path":"docs/_description_templates/template-setting.md","contentType":"file"},{"name":"template-statement.md","path":"docs/_description_templates/template-statement.md","contentType":"file"},{"name":"template-system-table.md","path":"docs/_description_templates/template-system-table.md","contentType":"file"}],"totalCount":7},"docs":{"items":[{"name":"_description_templates","path":"docs/_description_templates","contentType":"directory"},{"name":"_includes","path":"docs/_includes","contentType":"directory"},{"name":"changelogs","path":"docs/changelogs","contentType":"directory"},{"name":"en","path":"docs/en","contentType":"directory"},{"name":"ru","path":"docs/ru","contentType":"directory"},{"name":"tools","path":"docs/tools","contentType":"directory"},{"name":"zh","path":"docs/zh","contentType":"directory"},{"name":".gitignore","path":"docs/.gitignore","contentType":"file"},{"name":"README.md","path":"docs/README.md","contentType":"file"},{"name":"clean","path":"docs/clean","contentType":"file"},{"name":"mkdocs.yml","path":"docs/mkdocs.yml","contentType":"file"},{"name":"redirects.txt","path":"docs/redirects.txt","contentType":"file"}],"totalCount":12},"":{"items":[{"name":".github","path":".github","contentType":"directory"},{"name":"base","path":"base","contentType":"directory"},{"name":"benchmark","path":"benchmark","contentType":"directory"},{"name":"cmake","path":"cmake","contentType":"directory"},{"name":"contrib","path":"contrib","contentType":"directory"},{"name":"docker","path":"docker","contentType":"directory"},{"name":"docs","path":"docs","contentType":"directory"},{"name":"packages","path":"packages","contentType":"directory"},{"name":"programs","path":"programs","contentType":"directory"},{"name":"src","path":"src","contentType":"directory"},{"name":"tests","path":"tests","contentType":"directory"},{"name":"utils","path":"utils","contentType":"directory"},{"name":"website","path":"website","contentType":"directory"},{"name":".clang-format","path":".clang-format","contentType":"file"},{"name":".clang-tidy","path":".clang-tidy","contentType":"file"},{"name":".editorconfig","path":".editorconfig","contentType":"file"},{"name":".gitattributes","path":".gitattributes","contentType":"file"},{"name":".gitignore","path":".gitignore","contentType":"file"},{"name":".gitmodules","path":".gitmodules","contentType":"file"},{"name":".pylintrc","path":".pylintrc","contentType":"file"},{"name":".vimrc","path":".vimrc","contentType":"file"},{"name":".yamllint","path":".yamllint","contentType":"file"},{"name":"AUTHORS","path":"AUTHORS","contentType":"file"},{"name":"CHANGELOG.md","path":"CHANGELOG.md","contentType":"file"},{"name":"CMakeLists.txt","path":"CMakeLists.txt","contentType":"file"},{"name":"CODE_OF_CONDUCT.md","path":"CODE_OF_CONDUCT.md","contentType":"file"},{"name":"CONTRIBUTING.md","path":"CONTRIBUTING.md","contentType":"file"},{"name":"LICENSE","path":"LICENSE","contentType":"file"},{"name":"PreLoad.cmake","path":"PreLoad.cmake","contentType":"file"},{"name":"README.md","path":"README.md","contentType":"file"},{"name":"SECURITY.md","path":"SECURITY.md","contentType":"file"},{"name":"format_sources","path":"format_sources","contentType":"file"}],"totalCount":32}},"fileTreeProcessingTime":7.7234929999999995,"foldersToFetch":[],"reducedMotionEnabled":"system","repo":{"id":60246359,"defaultBranch":"master","name":"ClickHouse","ownerLogin":"ClickHouse","currentUserCanPush":true,"isFork":false,"isEmpty":false,"createdAt":"2016-06-02T04:28:18.000-04:00","ownerAvatar":"https://avatars.githubusercontent.com/u/54801242?v=4","public":true,"private":false,"isOrgOwned":true},"refInfo":{"name":"5f18640215159ed1ad50be1efce2cb996a49fd73","listCacheKey":"v0:1688403108.0","canEdit":false,"refType":"tree","currentOid":"5f18640215159ed1ad50be1efce2cb996a49fd73"},"path":"docs/_description_templates/template-server-setting.md","currentUser":{"id":25182304,"login":"DanRoscigno","userEmail":"dan@roscigno.com"},"blob":{"rawBlob":null,"colorizedLines":null,"stylingDirectives":null,"csv":null,"csvError":null,"dependabotInfo":{"showConfigurationBanner":null,"configFilePath":null,"networkDependabotPath":"/ClickHouse/ClickHouse/network/updates","dismissConfigurationNoticePath":"/settings/dismiss-notice/dependabot_configuration_notice","configurationNoticeDismissed":false,"repoAlertsPath":"/ClickHouse/ClickHouse/security/dependabot","repoSecurityAndAnalysisPath":"/ClickHouse/ClickHouse/settings/security_analysis","repoOwnerIsOrg":true,"currentUserCanAdminRepo":false},"displayName":"template-server-setting.md","displayUrl":"https://github.com/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-server-setting.md?raw=true","headerInfo":{"blobSize":"629 Bytes","deleteInfo":{"deletePath":null,"deleteTooltip":"You must be on a branch to make or propose changes to this file"},"editInfo":{"editTooltip":"You must be on a branch to make or propose changes to this file"},"ghDesktopPath":null,"gitLfsPath":null,"onBranch":false,"shortPath":"0b37d46","siteNavLoginPath":"/login?return_to=https%3A%2F%2Fgithub.com%2FClickHouse%2FClickHouse%2Fblob%2F5f18640215159ed1ad50be1efce2cb996a49fd73%2Fdocs%2F_description_templates%2Ftemplate-server-setting.md","isCSV":false,"isRichtext":true,"toc":[{"level":2,"text":"server_setting_name {#server_setting_name}","anchor":"server_setting_name-server_setting_name","htmlText":"server_setting_name {#server_setting_name}"}],"lineInfo":{"truncatedLoc":"33","truncatedSloc":"20"},"mode":"file"},"image":false,"isCodeownersFile":null,"isValidLegacyIssueTemplate":false,"issueTemplateHelpUrl":"https://docs.github.com/articles/about-issue-and-pull-request-templates","issueTemplate":null,"discussionTemplate":null,"language":"Markdown","large":false,"loggedIn":true,"newDiscussionPath":"/ClickHouse/ClickHouse/discussions/new","newIssuePath":"/ClickHouse/ClickHouse/issues/new","planSupportInfo":{"repoIsFork":null,"repoOwnedByCurrentUser":null,"requestFullPath":"/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-server-setting.md","showFreeOrgGatedFeatureMessage":null,"showPlanSupportBanner":null,"upgradeDataAttributes":null,"upgradePath":null},"publishBannersInfo":{"dismissActionNoticePath":"/settings/dismiss-notice/publish_action_from_dockerfile","dismissStackNoticePath":"/settings/dismiss-notice/publish_stack_from_file","releasePath":"/ClickHouse/ClickHouse/releases/new?marketplace=true","showPublishActionBanner":false,"showPublishStackBanner":false},"renderImageOrRaw":false,"richText":"

server_setting_name {#server_setting_name}

\n

Description.

\n

Describe what is configured in this section of settings.

\n

Possible value: ...

\n

Default value: ...

\n

Settings (Optional)

\n

If the section contains several settings, list them here. Specify possible values and default values:

\n
    \n
  • setting_1 — Description.
  • \n
  • setting_2 — Description.
  • \n
\n

Example

\n
<server_setting_name>\n    <setting_1> ... </setting_1>\n    <setting_2> ... </setting_2>\n</server_setting_name>
\n

Additional Info (Optional)

\n

The name of an additional section can be any, for example, Usage.

\n

See Also (Optional)

\n\n
","renderedFileInfo":null,"tabSize":8,"topBannersInfo":{"overridingGlobalFundingFile":false,"globalPreferredFundingPath":null,"repoOwner":"ClickHouse","repoName":"ClickHouse","showInvalidCitationWarning":false,"citationHelpUrl":"https://docs.github.com/en/github/creating-cloning-and-archiving-repositories/creating-a-repository-on-github/about-citation-files","showDependabotConfigurationBanner":null,"actionsOnboardingTip":null},"truncated":false,"viewable":true,"workflowRedirectUrl":null,"symbols":{"timedOut":false,"notAnalyzed":true,"symbols":[]}},"csrf_tokens":{"/ClickHouse/ClickHouse/branches":{"post":"a6lijUe_lOiTRf_mzmSRAi4tMYD_1HdAhwqytaKk2q8WVWzMl0_pO4NPuTgnV7bEDlvJRWPbUudasZ-96xnpbQ"}}},"title":"ClickHouse/docs/_description_templates/template-server-setting.md at 5f18640215159ed1ad50be1efce2cb996a49fd73 · ClickHouse/ClickHouse","locale":"en"} \ No newline at end of file +## server_setting_name {#server_setting_name} + +Description. + +Describe what is configured in this section of settings. + +Possible value: ... + +Default value: ... + +**Settings** (Optional) + +If the section contains several settings, list them here. Specify possible values and default values: + +- setting_1 — Description. +- setting_2 — Description. + +**Example** + +```xml + + ... + ... + +``` + +**Additional Info** (Optional) + +The name of an additional section can be any, for example, **Usage**. + +**See Also** (Optional) + +- [link](#) diff --git a/docs/_description_templates/template-setting.md b/docs/_description_templates/template-setting.md index df51a7e8241..fc912aba3e1 100644 --- a/docs/_description_templates/template-setting.md +++ b/docs/_description_templates/template-setting.md @@ -1 +1,27 @@ -{"payload":{"allShortcutsEnabled":true,"fileTree":{"docs/_description_templates":{"items":[{"name":"template-data-type.md","path":"docs/_description_templates/template-data-type.md","contentType":"file"},{"name":"template-engine.md","path":"docs/_description_templates/template-engine.md","contentType":"file"},{"name":"template-function.md","path":"docs/_description_templates/template-function.md","contentType":"file"},{"name":"template-server-setting.md","path":"docs/_description_templates/template-server-setting.md","contentType":"file"},{"name":"template-setting.md","path":"docs/_description_templates/template-setting.md","contentType":"file"},{"name":"template-statement.md","path":"docs/_description_templates/template-statement.md","contentType":"file"},{"name":"template-system-table.md","path":"docs/_description_templates/template-system-table.md","contentType":"file"}],"totalCount":7},"docs":{"items":[{"name":"_description_templates","path":"docs/_description_templates","contentType":"directory"},{"name":"_includes","path":"docs/_includes","contentType":"directory"},{"name":"changelogs","path":"docs/changelogs","contentType":"directory"},{"name":"en","path":"docs/en","contentType":"directory"},{"name":"ru","path":"docs/ru","contentType":"directory"},{"name":"tools","path":"docs/tools","contentType":"directory"},{"name":"zh","path":"docs/zh","contentType":"directory"},{"name":".gitignore","path":"docs/.gitignore","contentType":"file"},{"name":"README.md","path":"docs/README.md","contentType":"file"},{"name":"clean","path":"docs/clean","contentType":"file"},{"name":"mkdocs.yml","path":"docs/mkdocs.yml","contentType":"file"},{"name":"redirects.txt","path":"docs/redirects.txt","contentType":"file"}],"totalCount":12},"":{"items":[{"name":".github","path":".github","contentType":"directory"},{"name":"base","path":"base","contentType":"directory"},{"name":"benchmark","path":"benchmark","contentType":"directory"},{"name":"cmake","path":"cmake","contentType":"directory"},{"name":"contrib","path":"contrib","contentType":"directory"},{"name":"docker","path":"docker","contentType":"directory"},{"name":"docs","path":"docs","contentType":"directory"},{"name":"packages","path":"packages","contentType":"directory"},{"name":"programs","path":"programs","contentType":"directory"},{"name":"src","path":"src","contentType":"directory"},{"name":"tests","path":"tests","contentType":"directory"},{"name":"utils","path":"utils","contentType":"directory"},{"name":"website","path":"website","contentType":"directory"},{"name":".clang-format","path":".clang-format","contentType":"file"},{"name":".clang-tidy","path":".clang-tidy","contentType":"file"},{"name":".editorconfig","path":".editorconfig","contentType":"file"},{"name":".gitattributes","path":".gitattributes","contentType":"file"},{"name":".gitignore","path":".gitignore","contentType":"file"},{"name":".gitmodules","path":".gitmodules","contentType":"file"},{"name":".pylintrc","path":".pylintrc","contentType":"file"},{"name":".vimrc","path":".vimrc","contentType":"file"},{"name":".yamllint","path":".yamllint","contentType":"file"},{"name":"AUTHORS","path":"AUTHORS","contentType":"file"},{"name":"CHANGELOG.md","path":"CHANGELOG.md","contentType":"file"},{"name":"CMakeLists.txt","path":"CMakeLists.txt","contentType":"file"},{"name":"CODE_OF_CONDUCT.md","path":"CODE_OF_CONDUCT.md","contentType":"file"},{"name":"CONTRIBUTING.md","path":"CONTRIBUTING.md","contentType":"file"},{"name":"LICENSE","path":"LICENSE","contentType":"file"},{"name":"PreLoad.cmake","path":"PreLoad.cmake","contentType":"file"},{"name":"README.md","path":"README.md","contentType":"file"},{"name":"SECURITY.md","path":"SECURITY.md","contentType":"file"},{"name":"format_sources","path":"format_sources","contentType":"file"}],"totalCount":32}},"fileTreeProcessingTime":9.96412,"foldersToFetch":[],"reducedMotionEnabled":"system","repo":{"id":60246359,"defaultBranch":"master","name":"ClickHouse","ownerLogin":"ClickHouse","currentUserCanPush":true,"isFork":false,"isEmpty":false,"createdAt":"2016-06-02T04:28:18.000-04:00","ownerAvatar":"https://avatars.githubusercontent.com/u/54801242?v=4","public":true,"private":false,"isOrgOwned":true},"refInfo":{"name":"5f18640215159ed1ad50be1efce2cb996a49fd73","listCacheKey":"v0:1688403108.0","canEdit":false,"refType":"tree","currentOid":"5f18640215159ed1ad50be1efce2cb996a49fd73"},"path":"docs/_description_templates/template-setting.md","currentUser":{"id":25182304,"login":"DanRoscigno","userEmail":"dan@roscigno.com"},"blob":{"rawBlob":null,"colorizedLines":null,"stylingDirectives":null,"csv":null,"csvError":null,"dependabotInfo":{"showConfigurationBanner":null,"configFilePath":null,"networkDependabotPath":"/ClickHouse/ClickHouse/network/updates","dismissConfigurationNoticePath":"/settings/dismiss-notice/dependabot_configuration_notice","configurationNoticeDismissed":false,"repoAlertsPath":"/ClickHouse/ClickHouse/security/dependabot","repoSecurityAndAnalysisPath":"/ClickHouse/ClickHouse/settings/security_analysis","repoOwnerIsOrg":true,"currentUserCanAdminRepo":false},"displayName":"template-setting.md","displayUrl":"https://github.com/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-setting.md?raw=true","headerInfo":{"blobSize":"503 Bytes","deleteInfo":{"deletePath":null,"deleteTooltip":"You must be on a branch to make or propose changes to this file"},"editInfo":{"editTooltip":"You must be on a branch to make or propose changes to this file"},"ghDesktopPath":null,"gitLfsPath":null,"onBranch":false,"shortPath":"fc912ab","siteNavLoginPath":"/login?return_to=https%3A%2F%2Fgithub.com%2FClickHouse%2FClickHouse%2Fblob%2F5f18640215159ed1ad50be1efce2cb996a49fd73%2Fdocs%2F_description_templates%2Ftemplate-setting.md","isCSV":false,"isRichtext":true,"toc":[{"level":2,"text":"setting_name {#setting_name}","anchor":"setting_name-setting_name","htmlText":"setting_name {#setting_name}"}],"lineInfo":{"truncatedLoc":"27","truncatedSloc":"15"},"mode":"file"},"image":false,"isCodeownersFile":null,"isValidLegacyIssueTemplate":false,"issueTemplateHelpUrl":"https://docs.github.com/articles/about-issue-and-pull-request-templates","issueTemplate":null,"discussionTemplate":null,"language":"Markdown","large":false,"loggedIn":true,"newDiscussionPath":"/ClickHouse/ClickHouse/discussions/new","newIssuePath":"/ClickHouse/ClickHouse/issues/new","planSupportInfo":{"repoIsFork":null,"repoOwnedByCurrentUser":null,"requestFullPath":"/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-setting.md","showFreeOrgGatedFeatureMessage":null,"showPlanSupportBanner":null,"upgradeDataAttributes":null,"upgradePath":null},"publishBannersInfo":{"dismissActionNoticePath":"/settings/dismiss-notice/publish_action_from_dockerfile","dismissStackNoticePath":"/settings/dismiss-notice/publish_stack_from_file","releasePath":"/ClickHouse/ClickHouse/releases/new?marketplace=true","showPublishActionBanner":false,"showPublishStackBanner":false},"renderImageOrRaw":false,"richText":"

setting_name {#setting_name}

\n

Description.

\n

For the switch setting, use the typical phrase: “Enables or disables something …”.

\n

Possible values:

\n

For switcher setting:

\n
    \n
  • 0 — Disabled.
  • \n
  • 1 — Enabled.
  • \n
\n

For another setting (typical phrases):

\n
    \n
  • Positive integer.
  • \n
  • 0 — Disabled or unlimited or something else.
  • \n
\n

Default value: value.

\n

Additional Info (Optional)

\n

The name of an additional section can be any, for example, Usage.

\n

See Also (Optional)

\n\n
","renderedFileInfo":null,"tabSize":8,"topBannersInfo":{"overridingGlobalFundingFile":false,"globalPreferredFundingPath":null,"repoOwner":"ClickHouse","repoName":"ClickHouse","showInvalidCitationWarning":false,"citationHelpUrl":"https://docs.github.com/en/github/creating-cloning-and-archiving-repositories/creating-a-repository-on-github/about-citation-files","showDependabotConfigurationBanner":null,"actionsOnboardingTip":null},"truncated":false,"viewable":true,"workflowRedirectUrl":null,"symbols":{"timedOut":false,"notAnalyzed":true,"symbols":[]}},"csrf_tokens":{"/ClickHouse/ClickHouse/branches":{"post":"gFv6-keA6Z1eYZATbMVq8_sVPc-l2b8pTaH03K1R_zf9p_S7l3CUTk5r1s2F9k0122PFCjnWmo6QGtnU5OzM9Q"}}},"title":"ClickHouse/docs/_description_templates/template-setting.md at 5f18640215159ed1ad50be1efce2cb996a49fd73 · ClickHouse/ClickHouse","locale":"en"} \ No newline at end of file +## setting_name {#setting_name} + +Description. + +For the switch setting, use the typical phrase: “Enables or disables something …”. + +Possible values: + +*For switcher setting:* + +- 0 — Disabled. +- 1 — Enabled. + +*For another setting (typical phrases):* + +- Positive integer. +- 0 — Disabled or unlimited or something else. + +Default value: `value`. + +**Additional Info** (Optional) + +The name of an additional section can be any, for example, **Usage**. + +**See Also** (Optional) + +- [link](#) diff --git a/docs/_description_templates/template-statement.md b/docs/_description_templates/template-statement.md index b5ae0d9b26b..238570c2217 100644 --- a/docs/_description_templates/template-statement.md +++ b/docs/_description_templates/template-statement.md @@ -1 +1,24 @@ -{"payload":{"allShortcutsEnabled":true,"fileTree":{"docs/_description_templates":{"items":[{"name":"template-data-type.md","path":"docs/_description_templates/template-data-type.md","contentType":"file"},{"name":"template-engine.md","path":"docs/_description_templates/template-engine.md","contentType":"file"},{"name":"template-function.md","path":"docs/_description_templates/template-function.md","contentType":"file"},{"name":"template-server-setting.md","path":"docs/_description_templates/template-server-setting.md","contentType":"file"},{"name":"template-setting.md","path":"docs/_description_templates/template-setting.md","contentType":"file"},{"name":"template-statement.md","path":"docs/_description_templates/template-statement.md","contentType":"file"},{"name":"template-system-table.md","path":"docs/_description_templates/template-system-table.md","contentType":"file"}],"totalCount":7},"docs":{"items":[{"name":"_description_templates","path":"docs/_description_templates","contentType":"directory"},{"name":"_includes","path":"docs/_includes","contentType":"directory"},{"name":"changelogs","path":"docs/changelogs","contentType":"directory"},{"name":"en","path":"docs/en","contentType":"directory"},{"name":"ru","path":"docs/ru","contentType":"directory"},{"name":"tools","path":"docs/tools","contentType":"directory"},{"name":"zh","path":"docs/zh","contentType":"directory"},{"name":".gitignore","path":"docs/.gitignore","contentType":"file"},{"name":"README.md","path":"docs/README.md","contentType":"file"},{"name":"clean","path":"docs/clean","contentType":"file"},{"name":"mkdocs.yml","path":"docs/mkdocs.yml","contentType":"file"},{"name":"redirects.txt","path":"docs/redirects.txt","contentType":"file"}],"totalCount":12},"":{"items":[{"name":".github","path":".github","contentType":"directory"},{"name":"base","path":"base","contentType":"directory"},{"name":"benchmark","path":"benchmark","contentType":"directory"},{"name":"cmake","path":"cmake","contentType":"directory"},{"name":"contrib","path":"contrib","contentType":"directory"},{"name":"docker","path":"docker","contentType":"directory"},{"name":"docs","path":"docs","contentType":"directory"},{"name":"packages","path":"packages","contentType":"directory"},{"name":"programs","path":"programs","contentType":"directory"},{"name":"src","path":"src","contentType":"directory"},{"name":"tests","path":"tests","contentType":"directory"},{"name":"utils","path":"utils","contentType":"directory"},{"name":"website","path":"website","contentType":"directory"},{"name":".clang-format","path":".clang-format","contentType":"file"},{"name":".clang-tidy","path":".clang-tidy","contentType":"file"},{"name":".editorconfig","path":".editorconfig","contentType":"file"},{"name":".gitattributes","path":".gitattributes","contentType":"file"},{"name":".gitignore","path":".gitignore","contentType":"file"},{"name":".gitmodules","path":".gitmodules","contentType":"file"},{"name":".pylintrc","path":".pylintrc","contentType":"file"},{"name":".vimrc","path":".vimrc","contentType":"file"},{"name":".yamllint","path":".yamllint","contentType":"file"},{"name":"AUTHORS","path":"AUTHORS","contentType":"file"},{"name":"CHANGELOG.md","path":"CHANGELOG.md","contentType":"file"},{"name":"CMakeLists.txt","path":"CMakeLists.txt","contentType":"file"},{"name":"CODE_OF_CONDUCT.md","path":"CODE_OF_CONDUCT.md","contentType":"file"},{"name":"CONTRIBUTING.md","path":"CONTRIBUTING.md","contentType":"file"},{"name":"LICENSE","path":"LICENSE","contentType":"file"},{"name":"PreLoad.cmake","path":"PreLoad.cmake","contentType":"file"},{"name":"README.md","path":"README.md","contentType":"file"},{"name":"SECURITY.md","path":"SECURITY.md","contentType":"file"},{"name":"format_sources","path":"format_sources","contentType":"file"}],"totalCount":32}},"fileTreeProcessingTime":9.954742000000001,"foldersToFetch":[],"reducedMotionEnabled":"system","repo":{"id":60246359,"defaultBranch":"master","name":"ClickHouse","ownerLogin":"ClickHouse","currentUserCanPush":true,"isFork":false,"isEmpty":false,"createdAt":"2016-06-02T04:28:18.000-04:00","ownerAvatar":"https://avatars.githubusercontent.com/u/54801242?v=4","public":true,"private":false,"isOrgOwned":true},"refInfo":{"name":"5f18640215159ed1ad50be1efce2cb996a49fd73","listCacheKey":"v0:1688403108.0","canEdit":false,"refType":"tree","currentOid":"5f18640215159ed1ad50be1efce2cb996a49fd73"},"path":"docs/_description_templates/template-statement.md","currentUser":{"id":25182304,"login":"DanRoscigno","userEmail":"dan@roscigno.com"},"blob":{"rawBlob":null,"colorizedLines":null,"stylingDirectives":null,"csv":null,"csvError":null,"dependabotInfo":{"showConfigurationBanner":null,"configFilePath":null,"networkDependabotPath":"/ClickHouse/ClickHouse/network/updates","dismissConfigurationNoticePath":"/settings/dismiss-notice/dependabot_configuration_notice","configurationNoticeDismissed":false,"repoAlertsPath":"/ClickHouse/ClickHouse/security/dependabot","repoSecurityAndAnalysisPath":"/ClickHouse/ClickHouse/settings/security_analysis","repoOwnerIsOrg":true,"currentUserCanAdminRepo":false},"displayName":"template-statement.md","displayUrl":"https://github.com/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-statement.md?raw=true","headerInfo":{"blobSize":"575 Bytes","deleteInfo":{"deletePath":null,"deleteTooltip":"You must be on a branch to make or propose changes to this file"},"editInfo":{"editTooltip":"You must be on a branch to make or propose changes to this file"},"ghDesktopPath":null,"gitLfsPath":null,"onBranch":false,"shortPath":"238570c","siteNavLoginPath":"/login?return_to=https%3A%2F%2Fgithub.com%2FClickHouse%2FClickHouse%2Fblob%2F5f18640215159ed1ad50be1efce2cb996a49fd73%2Fdocs%2F_description_templates%2Ftemplate-statement.md","isCSV":false,"isRichtext":true,"toc":[{"level":1,"text":"Statement name (for example, SHOW USER) {#statement-name-in-lower-case}","anchor":"statement-name-for-example-show-user-statement-name-in-lower-case","htmlText":"Statement name (for example, SHOW USER) {#statement-name-in-lower-case}"},{"level":2,"text":"Other necessary sections of the description (Optional) {#anchor}","anchor":"other-necessary-sections-of-the-description-optional-anchor","htmlText":"Other necessary sections of the description (Optional) {#anchor}"}],"lineInfo":{"truncatedLoc":"24","truncatedSloc":"14"},"mode":"file"},"image":false,"isCodeownersFile":null,"isValidLegacyIssueTemplate":false,"issueTemplateHelpUrl":"https://docs.github.com/articles/about-issue-and-pull-request-templates","issueTemplate":null,"discussionTemplate":null,"language":"Markdown","large":false,"loggedIn":true,"newDiscussionPath":"/ClickHouse/ClickHouse/discussions/new","newIssuePath":"/ClickHouse/ClickHouse/issues/new","planSupportInfo":{"repoIsFork":null,"repoOwnedByCurrentUser":null,"requestFullPath":"/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-statement.md","showFreeOrgGatedFeatureMessage":null,"showPlanSupportBanner":null,"upgradeDataAttributes":null,"upgradePath":null},"publishBannersInfo":{"dismissActionNoticePath":"/settings/dismiss-notice/publish_action_from_dockerfile","dismissStackNoticePath":"/settings/dismiss-notice/publish_stack_from_file","releasePath":"/ClickHouse/ClickHouse/releases/new?marketplace=true","showPublishActionBanner":false,"showPublishStackBanner":false},"renderImageOrRaw":false,"richText":"

Statement name (for example, SHOW USER) {#statement-name-in-lower-case}

\n

Brief description of what the statement does.

\n

Syntax

\n
Syntax of the statement.
\n

Other necessary sections of the description (Optional) {#anchor}

\n

Examples of descriptions with a complicated structure:

\n\n

See Also (Optional)

\n

Links to related topics as a list.

\n\n
","renderedFileInfo":null,"tabSize":8,"topBannersInfo":{"overridingGlobalFundingFile":false,"globalPreferredFundingPath":null,"repoOwner":"ClickHouse","repoName":"ClickHouse","showInvalidCitationWarning":false,"citationHelpUrl":"https://docs.github.com/en/github/creating-cloning-and-archiving-repositories/creating-a-repository-on-github/about-citation-files","showDependabotConfigurationBanner":null,"actionsOnboardingTip":null},"truncated":false,"viewable":true,"workflowRedirectUrl":null,"symbols":{"timedOut":false,"notAnalyzed":true,"symbols":[]}},"csrf_tokens":{"/ClickHouse/ClickHouse/branches":{"post":"xohFK8TmjwjnIXPER5IKigBaxPXoFkjI0tOrZrMZen-7dEtqFBby2_crNRquoS1MICw8MHQZbW8PaIZu-qRJvQ"}}},"title":"ClickHouse/docs/_description_templates/template-statement.md at 5f18640215159ed1ad50be1efce2cb996a49fd73 · ClickHouse/ClickHouse","locale":"en"} \ No newline at end of file +# Statement name (for example, SHOW USER) {#statement-name-in-lower-case} + +Brief description of what the statement does. + +**Syntax** + +```sql +Syntax of the statement. +``` + +## Other necessary sections of the description (Optional) {#anchor} + +Examples of descriptions with a complicated structure: + +- https://clickhouse.com/docs/en/sql-reference/statements/grant/ +- https://clickhouse.com/docs/en/sql-reference/statements/revoke/ +- https://clickhouse.com/docs/en/sql-reference/statements/select/join/ + + +**See Also** (Optional) + +Links to related topics as a list. + +- [link](#) diff --git a/docs/_description_templates/template-system-table.md b/docs/_description_templates/template-system-table.md index 02d622a52cf..f2decc4bb6d 100644 --- a/docs/_description_templates/template-system-table.md +++ b/docs/_description_templates/template-system-table.md @@ -1 +1,25 @@ -{"payload":{"allShortcutsEnabled":true,"fileTree":{"docs/_description_templates":{"items":[{"name":"template-data-type.md","path":"docs/_description_templates/template-data-type.md","contentType":"file"},{"name":"template-engine.md","path":"docs/_description_templates/template-engine.md","contentType":"file"},{"name":"template-function.md","path":"docs/_description_templates/template-function.md","contentType":"file"},{"name":"template-server-setting.md","path":"docs/_description_templates/template-server-setting.md","contentType":"file"},{"name":"template-setting.md","path":"docs/_description_templates/template-setting.md","contentType":"file"},{"name":"template-statement.md","path":"docs/_description_templates/template-statement.md","contentType":"file"},{"name":"template-system-table.md","path":"docs/_description_templates/template-system-table.md","contentType":"file"}],"totalCount":7},"docs":{"items":[{"name":"_description_templates","path":"docs/_description_templates","contentType":"directory"},{"name":"_includes","path":"docs/_includes","contentType":"directory"},{"name":"changelogs","path":"docs/changelogs","contentType":"directory"},{"name":"en","path":"docs/en","contentType":"directory"},{"name":"ru","path":"docs/ru","contentType":"directory"},{"name":"tools","path":"docs/tools","contentType":"directory"},{"name":"zh","path":"docs/zh","contentType":"directory"},{"name":".gitignore","path":"docs/.gitignore","contentType":"file"},{"name":"README.md","path":"docs/README.md","contentType":"file"},{"name":"clean","path":"docs/clean","contentType":"file"},{"name":"mkdocs.yml","path":"docs/mkdocs.yml","contentType":"file"},{"name":"redirects.txt","path":"docs/redirects.txt","contentType":"file"}],"totalCount":12},"":{"items":[{"name":".github","path":".github","contentType":"directory"},{"name":"base","path":"base","contentType":"directory"},{"name":"benchmark","path":"benchmark","contentType":"directory"},{"name":"cmake","path":"cmake","contentType":"directory"},{"name":"contrib","path":"contrib","contentType":"directory"},{"name":"docker","path":"docker","contentType":"directory"},{"name":"docs","path":"docs","contentType":"directory"},{"name":"packages","path":"packages","contentType":"directory"},{"name":"programs","path":"programs","contentType":"directory"},{"name":"src","path":"src","contentType":"directory"},{"name":"tests","path":"tests","contentType":"directory"},{"name":"utils","path":"utils","contentType":"directory"},{"name":"website","path":"website","contentType":"directory"},{"name":".clang-format","path":".clang-format","contentType":"file"},{"name":".clang-tidy","path":".clang-tidy","contentType":"file"},{"name":".editorconfig","path":".editorconfig","contentType":"file"},{"name":".gitattributes","path":".gitattributes","contentType":"file"},{"name":".gitignore","path":".gitignore","contentType":"file"},{"name":".gitmodules","path":".gitmodules","contentType":"file"},{"name":".pylintrc","path":".pylintrc","contentType":"file"},{"name":".vimrc","path":".vimrc","contentType":"file"},{"name":".yamllint","path":".yamllint","contentType":"file"},{"name":"AUTHORS","path":"AUTHORS","contentType":"file"},{"name":"CHANGELOG.md","path":"CHANGELOG.md","contentType":"file"},{"name":"CMakeLists.txt","path":"CMakeLists.txt","contentType":"file"},{"name":"CODE_OF_CONDUCT.md","path":"CODE_OF_CONDUCT.md","contentType":"file"},{"name":"CONTRIBUTING.md","path":"CONTRIBUTING.md","contentType":"file"},{"name":"LICENSE","path":"LICENSE","contentType":"file"},{"name":"PreLoad.cmake","path":"PreLoad.cmake","contentType":"file"},{"name":"README.md","path":"README.md","contentType":"file"},{"name":"SECURITY.md","path":"SECURITY.md","contentType":"file"},{"name":"format_sources","path":"format_sources","contentType":"file"}],"totalCount":32}},"fileTreeProcessingTime":8.697185,"foldersToFetch":[],"reducedMotionEnabled":"system","repo":{"id":60246359,"defaultBranch":"master","name":"ClickHouse","ownerLogin":"ClickHouse","currentUserCanPush":true,"isFork":false,"isEmpty":false,"createdAt":"2016-06-02T04:28:18.000-04:00","ownerAvatar":"https://avatars.githubusercontent.com/u/54801242?v=4","public":true,"private":false,"isOrgOwned":true},"refInfo":{"name":"5f18640215159ed1ad50be1efce2cb996a49fd73","listCacheKey":"v0:1688403108.0","canEdit":false,"refType":"tree","currentOid":"5f18640215159ed1ad50be1efce2cb996a49fd73"},"path":"docs/_description_templates/template-system-table.md","currentUser":{"id":25182304,"login":"DanRoscigno","userEmail":"dan@roscigno.com"},"blob":{"rawBlob":null,"colorizedLines":null,"stylingDirectives":null,"csv":null,"csvError":null,"dependabotInfo":{"showConfigurationBanner":null,"configFilePath":null,"networkDependabotPath":"/ClickHouse/ClickHouse/network/updates","dismissConfigurationNoticePath":"/settings/dismiss-notice/dependabot_configuration_notice","configurationNoticeDismissed":false,"repoAlertsPath":"/ClickHouse/ClickHouse/security/dependabot","repoSecurityAndAnalysisPath":"/ClickHouse/ClickHouse/settings/security_analysis","repoOwnerIsOrg":true,"currentUserCanAdminRepo":false},"displayName":"template-system-table.md","displayUrl":"https://github.com/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-system-table.md?raw=true","headerInfo":{"blobSize":"381 Bytes","deleteInfo":{"deletePath":null,"deleteTooltip":"You must be on a branch to make or propose changes to this file"},"editInfo":{"editTooltip":"You must be on a branch to make or propose changes to this file"},"ghDesktopPath":null,"gitLfsPath":null,"onBranch":false,"shortPath":"f2decc4","siteNavLoginPath":"/login?return_to=https%3A%2F%2Fgithub.com%2FClickHouse%2FClickHouse%2Fblob%2F5f18640215159ed1ad50be1efce2cb996a49fd73%2Fdocs%2F_description_templates%2Ftemplate-system-table.md","isCSV":false,"isRichtext":true,"toc":[{"level":1,"text":"system.table_name {#system-tables_table-name}","anchor":"systemtable_name-system-tables_table-name","htmlText":"system.table_name {#system-tables_table-name}"}],"lineInfo":{"truncatedLoc":"25","truncatedSloc":"15"},"mode":"file"},"image":false,"isCodeownersFile":null,"isValidLegacyIssueTemplate":false,"issueTemplateHelpUrl":"https://docs.github.com/articles/about-issue-and-pull-request-templates","issueTemplate":null,"discussionTemplate":null,"language":"Markdown","large":false,"loggedIn":true,"newDiscussionPath":"/ClickHouse/ClickHouse/discussions/new","newIssuePath":"/ClickHouse/ClickHouse/issues/new","planSupportInfo":{"repoIsFork":null,"repoOwnedByCurrentUser":null,"requestFullPath":"/ClickHouse/ClickHouse/blob/5f18640215159ed1ad50be1efce2cb996a49fd73/docs/_description_templates/template-system-table.md","showFreeOrgGatedFeatureMessage":null,"showPlanSupportBanner":null,"upgradeDataAttributes":null,"upgradePath":null},"publishBannersInfo":{"dismissActionNoticePath":"/settings/dismiss-notice/publish_action_from_dockerfile","dismissStackNoticePath":"/settings/dismiss-notice/publish_stack_from_file","releasePath":"/ClickHouse/ClickHouse/releases/new?marketplace=true","showPublishActionBanner":false,"showPublishStackBanner":false},"renderImageOrRaw":false,"richText":"

system.table_name {#system-tables_table-name}

\n

Description.

\n

Columns:

\n\n

Example

\n

Query:

\n
SELECT * FROM system.table_name
\n

Result:

\n
Some output. It shouldn't be too long.\n
\n

See Also

\n
    \n
  • Article name — Some words about referenced information.
  • \n
\n
","renderedFileInfo":null,"tabSize":8,"topBannersInfo":{"overridingGlobalFundingFile":false,"globalPreferredFundingPath":null,"repoOwner":"ClickHouse","repoName":"ClickHouse","showInvalidCitationWarning":false,"citationHelpUrl":"https://docs.github.com/en/github/creating-cloning-and-archiving-repositories/creating-a-repository-on-github/about-citation-files","showDependabotConfigurationBanner":null,"actionsOnboardingTip":null},"truncated":false,"viewable":true,"workflowRedirectUrl":null,"symbols":{"timedOut":false,"notAnalyzed":true,"symbols":[]}},"csrf_tokens":{"/ClickHouse/ClickHouse/branches":{"post":"9K5xHq3WERnrVDYaCTfS2yzVWybDSHIH0WqtxpxdRGaJUn9ffSZsyvtecMTgBPUdDKOj419HV6AM0YDO1eB3pA"}}},"title":"ClickHouse/docs/_description_templates/template-system-table.md at 5f18640215159ed1ad50be1efce2cb996a49fd73 · ClickHouse/ClickHouse","locale":"en"} \ No newline at end of file +# system.table_name {#system-tables_table-name} + +Description. + +Columns: + +- `column_name` ([data_type_name](path/to/data_type.md)) — Description. + +**Example** + +Query: + +``` sql +SELECT * FROM system.table_name +``` + +Result: + +``` text +Some output. It shouldn't be too long. +``` + +**See Also** + +- [Article name](path/to/article_name.md) — Some words about referenced information. From ec5c9a013744baee5f3e501eadbb98cd596b73e0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 3 Jul 2023 23:43:38 +0300 Subject: [PATCH 513/515] Revert "Fix: Invalid number of rows in Chunk column Object" --- src/Processors/QueryPlan/AggregatingStep.cpp | 2 -- .../0_stateless/02789_object_type_invalid_num_of_rows.reference | 1 - .../0_stateless/02789_object_type_invalid_num_of_rows.sql | 2 -- 3 files changed, 5 deletions(-) delete mode 100644 tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.reference delete mode 100644 tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.sql diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index eebbfc04304..4ac972e2a79 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -319,8 +319,6 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B { auto column_with_default = col.column->cloneEmpty(); col.type->insertDefaultInto(*column_with_default); - column_with_default->finalize(); - auto column = ColumnConst::create(std::move(column_with_default), 0); const auto * node = &dag->addColumn({ColumnPtr(std::move(column)), col.type, col.name}); node = &dag->materializeNode(*node); diff --git a/tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.reference b/tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.reference deleted file mode 100644 index 7dec35f7acb..00000000000 --- a/tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.reference +++ /dev/null @@ -1 +0,0 @@ -0.02 diff --git a/tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.sql b/tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.sql deleted file mode 100644 index a9c8a844aa0..00000000000 --- a/tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.sql +++ /dev/null @@ -1,2 +0,0 @@ -set allow_experimental_object_type=1; -SELECT '0.02' GROUP BY GROUPING SETS (('6553.6'), (CAST('{"x" : 1}', 'Object(\'json\')'))) settings max_threads=1; -- { serverError NOT_IMPLEMENTED } From 29200341addefeeece3a437a740cacf249f35a61 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 3 Jul 2023 21:05:13 +0000 Subject: [PATCH 514/515] Add SonarCloud to README --- README.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index eac036c2d9b..a1147f81077 100644 --- a/README.md +++ b/README.md @@ -16,8 +16,9 @@ curl https://clickhouse.com/ | sh * [YouTube channel](https://www.youtube.com/c/ClickHouseDB) has a lot of content about ClickHouse in video format. * [Slack](https://clickhouse.com/slack) and [Telegram](https://telegram.me/clickhouse_en) allow chatting with ClickHouse users in real-time. * [Blog](https://clickhouse.com/blog/) contains various ClickHouse-related articles, as well as announcements and reports about events. -* [Code Browser (Woboq)](https://clickhouse.com/codebrowser/ClickHouse/index.html) with syntax highlight and navigation. -* [Code Browser (github.dev)](https://github.dev/ClickHouse/ClickHouse) with syntax highlight, powered by github.dev. +* [Code Browser (Woboq)](https://clickhouse.com/codebrowser/ClickHouse/index.html) with syntax highlighting and navigation. +* [Code Browser (github.dev)](https://github.dev/ClickHouse/ClickHouse) with syntax highlighting, powered by github.dev. +* [Static Analysis (SonarCloud)](https://sonarcloud.io/project/issues?resolved=false&id=ClickHouse_ClickHouse) proposes C++ quality improvements. * [Contacts](https://clickhouse.com/company/contact) can help to get your questions answered if there are any. ## Upcoming Events From d263b6bf1739f4cd3431c469e60643e29dd10fe7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 3 Jul 2023 21:17:13 +0000 Subject: [PATCH 515/515] Docs: Mention homebrew as an alternative and non-production install method --- docs/en/getting-started/install.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index d44dc861888..d2e7ab30478 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -378,6 +378,10 @@ request](https://github.com/ClickHouse/ClickHouse/commits/master) and find CI ch https://s3.amazonaws.com/clickhouse/builds/PRs/.../.../binary_aarch64_v80compat/clickhouse". You can then click the link to download the build. +### macOS-only: Install with Homebrew + +To install ClickHouse using the popular `brew` package manager, follow the instructions listed in the [ClickHouse Homebrew tap](https://github.com/ClickHouse/homebrew-clickhouse). + ## Launch {#launch} To start the server as a daemon, run: