From f29700bd2fe1fc79cb6fd5bf2b31129119e6978c Mon Sep 17 00:00:00 2001 From: LiuYangkuan Date: Wed, 1 Feb 2023 18:50:43 +0800 Subject: [PATCH 001/228] 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 14cb7ed773f671ef4f6fa3f41aa6d05ce4705228 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Fri, 10 Mar 2023 10:05:06 -0500 Subject: [PATCH 002/228] Udate system tables doc closes #47030 --- docs/en/operations/system-tables/index.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/system-tables/index.md b/docs/en/operations/system-tables/index.md index 284ba866cc8..65efe8b01b3 100644 --- a/docs/en/operations/system-tables/index.md +++ b/docs/en/operations/system-tables/index.md @@ -13,6 +13,7 @@ System tables provide information about: - Server states, processes, and environment. - Server’s internal processes. +- Options used when the ClickHouse binary was built. System tables: From bf6afd27a655bb159583af2020113e101ca82e00 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 17 May 2023 13:32:51 +0000 Subject: [PATCH 003/228] 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 bde2cf96b135235908b71cc0bf071a175321dc4d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 22 May 2023 12:24:16 +0000 Subject: [PATCH 004/228] 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 161afea266bc8f3a13dacfb99f7333ab98f2ac46 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 May 2023 07:36:39 +0000 Subject: [PATCH 005/228] 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 006/228] 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 007/228] 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 7e68f61df1671b0be0b30c8cef33f34c8c971d3b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 May 2023 08:43:11 +0000 Subject: [PATCH 008/228] 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 009/228] 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 010/228] 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 011/228] 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 6a8a21e09a49303919d2357f0b7f9ca38c79684b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 May 2023 16:01:40 +0000 Subject: [PATCH 012/228] 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 0740bfbe4bfb3d186c8803e15aaffd2042c7f1fa Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 26 May 2023 14:31:09 +0000 Subject: [PATCH 013/228] 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 21aba94909e2ab7fe357c30d694af8674f81dbd0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 30 May 2023 13:22:40 +0000 Subject: [PATCH 014/228] 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 0b4d29ff98916d3ab1f2ea24f4254897b18351cf Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 1 Jun 2023 12:19:00 +0000 Subject: [PATCH 015/228] 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 016/228] 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 017/228] 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 018/228] 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 019/228] 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 47966c4e8129c3777679e42966cfec9c7a2aa383 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Jun 2023 14:10:52 +0000 Subject: [PATCH 020/228] 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 ece96f54e96b526693e58b859c9c835f17eff5f4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 5 Jun 2023 09:47:58 +0000 Subject: [PATCH 021/228] 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 ad2b926248a4fd464f6f278045c9103a75d92ca7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 6 Jun 2023 07:17:30 +0000 Subject: [PATCH 022/228] 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 023/228] 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 07582d56f32d72a3f13d9e7303310ff1753c97f5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Jun 2023 13:18:13 +0000 Subject: [PATCH 024/228] 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 025/228] 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 b634012feb40445079145639d23c44967c00547e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 12 Jun 2023 10:57:03 +0000 Subject: [PATCH 026/228] 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 708a99a6ea63409ce33f83d450592eaa42411ebb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 12 Jun 2023 11:19:38 +0000 Subject: [PATCH 027/228] 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 028/228] 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 029/228] 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 6f1c4865372b408d0f773a8dd9b1db8e63b5e4bb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 13 Jun 2023 10:25:53 +0200 Subject: [PATCH 030/228] 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 4ac090b12212567f7b9d30cb27132f566ae438c4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 13 Jun 2023 11:01:44 +0000 Subject: [PATCH 031/228] 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 9a5c5c7e931c41f406e316cb20cc7659c40fccc5 Mon Sep 17 00:00:00 2001 From: Yuriy Chernyshov Date: Tue, 13 Jun 2023 19:29:41 +0300 Subject: [PATCH 032/228] 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 f9f1e870c8468b829b4aa449c8a9b3736b733056 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 14 Jun 2023 10:09:01 +0000 Subject: [PATCH 033/228] 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 739dc6739f085e097cb39dbabf6e24588722d8cb Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 14 Jun 2023 13:31:02 +0200 Subject: [PATCH 034/228] Progress --- src/Interpreters/Cache/FileCache.cpp | 22 +++- src/Interpreters/Cache/FileCache.h | 5 +- src/Interpreters/Cache/FileCacheSettings.cpp | 8 +- src/Interpreters/Cache/FileCache_fwd.h | 3 +- src/Interpreters/Cache/FileSegment.cpp | 67 +++++----- src/Interpreters/Cache/Metadata.cpp | 128 ++++++++++++++++++- src/Interpreters/Cache/Metadata.h | 13 ++ 7 files changed, 201 insertions(+), 45 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index e6bc8b1f79b..65f8ecf7e89 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include @@ -49,9 +50,9 @@ FileCache::FileCache(const FileCacheSettings & settings) , 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) + , boundary_alignment(settings.boundary_alignment) , log(&Poco::Logger::get("FileCache")) , metadata(settings.base_path) - , boundary_alignment(settings.boundary_alignment) { main_priority = std::make_unique(settings.max_size, settings.max_elements); @@ -124,7 +125,12 @@ void FileCache::initialize() is_initialized = true; - cleanup_task = Context::getGlobalContextInstance()->getSchedulePool().createTask("FileCacheCleanup", [this]{ cleanupThreadFunc(); }); + size_t num_threads=2; + for (size_t i = 0; i < num_threads; ++i) + download_threads.emplace_back([this] { metadata.downloadThreadFunc(); }); + + auto & schedule_pool = Context::getGlobalContextInstance()->getSchedulePool(); + cleanup_task = schedule_pool.createTask("FileCacheCleanup", [this]{ cleanupThreadFunc(); }); cleanup_task->activate(); cleanup_task->scheduleAfter(delayed_cleanup_interval_ms); } @@ -412,7 +418,12 @@ FileSegmentsHolderPtr FileCache::set( } FileSegmentsHolderPtr -FileCache::getOrSet(const Key & key, size_t offset, size_t size, size_t file_size, const CreateFileSegmentSettings & settings) +FileCache::getOrSet( + const Key & key, + size_t offset, + size_t size, + size_t file_size, + const CreateFileSegmentSettings & settings) { assertInitialized(); @@ -979,6 +990,10 @@ void FileCache::deactivateBackgroundOperations() { if (cleanup_task) cleanup_task->deactivate(); + + metadata.cancelDownload(); + for (auto & thread : download_threads) + thread.join(); } void FileCache::cleanup() @@ -1017,6 +1032,7 @@ FileSegmentsHolderPtr FileCache::getSnapshot() { for (const auto & [_, file_segment_metadata] : locked_key) file_segments.push_back(FileSegment::getSnapshot(file_segment_metadata->file_segment)); + return true; }); return std::make_unique(std::move(file_segments), /* complete_on_dtor */false); } diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 71fc1722844..ed693b475ec 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -134,6 +134,7 @@ private: const bool allow_persistent_files; const size_t bypass_cache_threshold = 0; const size_t delayed_cleanup_interval_ms; + const size_t boundary_alignment; Poco::Logger * log; @@ -178,9 +179,9 @@ private: */ BackgroundSchedulePool::TaskHolder cleanup_task; - void assertInitialized() const; + std::vector download_threads; - size_t boundary_alignment; + void assertInitialized() const; void assertCacheCorrectness(); diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index 1fe51bf5f3e..d41aa8ffcb9 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -31,10 +31,9 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & throw Exception(ErrorCodes::BAD_ARGUMENTS, "Disk Cache requires non-empty `path` field (cache base path) in config"); max_elements = config.getUInt64(config_prefix + ".max_elements", FILECACHE_DEFAULT_MAX_ELEMENTS); + if (config.has(config_prefix + ".max_file_segment_size")) max_file_segment_size = parseWithSizeSuffix(config.getString(config_prefix + ".max_file_segment_size")); - else - max_file_segment_size = FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE; cache_on_write_operations = config.getUInt64(config_prefix + ".cache_on_write_operations", false); enable_filesystem_query_cache_limit = config.getUInt64(config_prefix + ".enable_filesystem_query_cache_limit", false); @@ -44,12 +43,11 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & if (config.has(config_prefix + ".bypass_cache_threashold")) bypass_cache_threashold = parseWithSizeSuffix(config.getString(config_prefix + ".bypass_cache_threashold")); - 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); + if (config.has(config_prefix + ".boundary_alignment")) + boundary_alignment = parseWithSizeSuffix(config.getString(config_prefix + ".boundary_alignment")); delayed_cleanup_interval_ms = config.getUInt64(config_prefix + ".delayed_cleanup_interval_ms", FILECACHE_DELAYED_CLEANUP_INTERVAL_MS); } diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index 01f518d0c4e..c791d08ed4f 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -4,7 +4,8 @@ namespace DB { -static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 8 * 1024 * 1024; +static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 32 * 1024 * 1024; /// 32Mi +static constexpr int FILECACHE_DEFAULT_MIN_FILE_SEGMENT_SIZE = 4 * 1024 * 1024; /// 4Mi static constexpr int FILECACHE_DEFAULT_MAX_ELEMENTS = 10000000; static constexpr int FILECACHE_DEFAULT_HITS_THRESHOLD = 0; static constexpr size_t FILECACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024; diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 7b82c58080c..f95379ba07f 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -214,8 +214,10 @@ void FileSegment::resetDownloadingStateUnlocked(const FileSegmentGuard::Lock & l /// range().size() can equal 0 in case of write-though cache. if (!is_unbound && current_downloaded_size != 0 && current_downloaded_size == range().size()) setDownloadedUnlocked(lock); - else + else if (current_downloaded_size) setDownloadState(State::PARTIALLY_DOWNLOADED, lock); + else + setDownloadState(State::EMPTY, lock); } void FileSegment::resetDownloader() @@ -280,22 +282,9 @@ void FileSegment::resetRemoteFileReader() FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader() { - auto locked_key = lockKeyMetadata(false); - if (!locked_key) - { - assert(isDetached()); + if (download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION) return std::move(remote_file_reader); - } - - auto segment_lock = segment_guard.lock(); - - assert(download_state != State::DETACHED); - - bool is_last_holder = locked_key->isLastOwnerOfFileSegment(offset()); - if (!downloader_id.empty() || !is_last_holder) - return nullptr; - - return std::move(remote_file_reader); + return nullptr; } void FileSegment::setRemoteFileReader(RemoteFileReaderPtr remote_file_reader_) @@ -607,22 +596,17 @@ void FileSegment::complete() resetDownloaderUnlocked(segment_lock); } - if (is_downloader || is_last_holder) + auto remove_from_cache = [&, this]() { - if (cache_writer) - { - cache_writer->finalize(); - cache_writer.reset(); - } - remote_file_reader.reset(); - } + LOG_TEST(log, "Remove file segment {} (nothing downloaded)", range().toString()); + locked_key->removeFileSegment(offset(), segment_lock); + setDetachedState(segment_lock); + }; 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); + remove_from_cache(); return; } @@ -633,6 +617,7 @@ void FileSegment::complete() chassert(current_downloaded_size == range().size()); chassert(current_downloaded_size == fs::file_size(getPathInLocalCache())); chassert(!cache_writer); + chassert(!remote_file_reader); break; } case State::DOWNLOADING: @@ -640,8 +625,24 @@ void FileSegment::complete() chassert(!is_last_holder); break; } - case State::EMPTY: case State::PARTIALLY_DOWNLOADED: + { + if (is_last_holder) + { + LOG_TEST( + log, "Submitted file segment for background download " + "(having {}/{})", downloaded_size, range().size()); + + locked_key->addToDownloadQueue(offset(), segment_lock); /// Finish download in background. + } + break; + } + case State::EMPTY: + { + if (is_last_holder) + remove_from_cache(); + break; + } case State::PARTIALLY_DOWNLOADED_NO_CONTINUATION: { chassert(current_downloaded_size != range().size()); @@ -650,10 +651,9 @@ void FileSegment::complete() { if (current_downloaded_size == 0) { - LOG_TEST(log, "Remove file segment {} (nothing downloaded)", range().toString()); - locked_key->removeFileSegment(offset(), segment_lock); + remove_from_cache(); } - else + else if (download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION) { LOG_TEST(log, "Resize file segment {} to downloaded: {}", range().toString(), current_downloaded_size); @@ -672,9 +672,8 @@ void FileSegment::complete() /// We mark current file segment with state DETACHED, even though the data is still in cache /// (but a separate file segment) because is_last_holder is satisfied, so it does not matter. + setDetachedState(segment_lock); } - - setDetachedState(segment_lock); } break; } @@ -844,6 +843,8 @@ void FileSegment::setDetachedState(const FileSegmentGuard::Lock & lock) setDownloadState(State::DETACHED, lock); key_metadata.reset(); cache = nullptr; + cache_writer.reset(); + remote_file_reader.reset(); } void FileSegment::detach(const FileSegmentGuard::Lock & lock, const LockedKey &) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index c9a23d1c785..2a3803eb3d7 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -46,10 +46,12 @@ KeyMetadata::KeyMetadata( const Key & key_, const std::string & key_path_, CleanupQueue & cleanup_queue_, + DownloadQueue & download_queue_, bool created_base_directory_) : key(key_) , key_path(key_path_) , cleanup_queue(cleanup_queue_) + , download_queue(download_queue_) , created_base_directory(created_base_directory_) { if (created_base_directory) @@ -123,6 +125,7 @@ private: CacheMetadata::CacheMetadata(const std::string & path_) : path(path_) , cleanup_queue(std::make_unique()) + , download_queue(std::make_unique()) , log(&Poco::Logger::get("CacheMetadata")) { } @@ -175,7 +178,7 @@ LockedKeyPtr CacheMetadata::lockKeyMetadata( it = emplace( key, std::make_shared( - key, getPathForKey(key), *cleanup_queue, is_initial_load)).first; + key, getPathForKey(key), *cleanup_queue, *download_queue, is_initial_load)).first; } key_metadata = it->second; @@ -293,6 +296,121 @@ void CacheMetadata::doCleanup() } } +class DownloadQueue +{ +friend struct CacheMetadata; +public: + void add(std::weak_ptr file_segment) + { + { + std::lock_guard lock(mutex); + queue.push(file_segment); + } + cv.notify_one(); + } + +private: + void cancel() + { + std::lock_guard lock(mutex); + cancelled = true; + } + + std::mutex mutex; + std::condition_variable cv; + std::queue> queue; + bool cancelled = false; +}; + +void CacheMetadata::downloadThreadFunc() +{ + std::optional> memory; + while (true) + { + std::weak_ptr file_segment_weak; + { + std::unique_lock lock(download_queue->mutex); + + if (download_queue->cancelled) + return; + + if (download_queue->queue.empty()) + { + download_queue->cv.wait(lock); + continue; + } + + file_segment_weak = download_queue->queue.front(); + download_queue->queue.pop(); + } + + FileSegmentsHolderPtr holder; + { + auto file_segment = file_segment_weak.lock(); + if (!file_segment + || file_segment->state() != FileSegment::State::PARTIALLY_DOWNLOADED) + continue; + + auto lock = lockKeyMetadata(file_segment->key(), KeyNotFoundPolicy::RETURN_NULL); + if (!lock) + continue; + + holder = std::make_unique(FileSegments{file_segment}); + } + + auto & file_segment = holder->front(); + chassert(file_segment.assertCorrectness()); + + if (file_segment.getOrSetDownloader() != FileSegment::getCallerId()) + continue; + + LOG_TRACE(log, "Downloading file segment: {}", file_segment.getInfoForLog()); + + auto reader = file_segment.getRemoteFileReader(); + + /// If remote_fs_read_method == 'threadpool', + /// reader iteself does not allocate the buffer, but uses the buffer passed to it. + /// So will need to allocate a buffer here as well. + if (reader->buffer().begin() == nullptr) + { + if (!memory) + memory.emplace(DBMS_DEFAULT_BUFFER_SIZE); + reader->set(memory->data(), memory->size()); + } + + size_t offset = file_segment.getCurrentWriteOffset(false); + while (!reader->eof()) + { + auto size = reader->available(); + + if (!file_segment.reserve(size)) + return; + + try + { + file_segment.write(reader->position(), size, offset); + offset += size; + } + catch (ErrnoException & e) + { + int code = e.getErrno(); + if (code == /* No space left on device */28 || code == /* Quota exceeded */122) + { + LOG_INFO(log, "Insert into cache is skipped due to insufficient disk space. ({})", e.displayText()); + continue; + } + throw; + } + } + } +} + +void CacheMetadata::cancelDownload() +{ + download_queue->cancel(); + download_queue->cv.notify_all(); +} + LockedKey::LockedKey(std::shared_ptr key_metadata_) : key_metadata(key_metadata_) , lock(key_metadata->guard.lock()) @@ -426,6 +544,14 @@ void LockedKey::shrinkFileSegmentToDownloadedSize( chassert(file_segment->assertCorrectnessUnlocked(segment_lock)); } +void LockedKey::addToDownloadQueue(size_t offset, const FileSegmentGuard::Lock &) +{ + auto it = key_metadata->find(offset); + if (it == key_metadata->end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is not offset {}", offset); + key_metadata->download_queue.add(it->second->file_segment); +} + 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 3fd6176f201..f96243c3f1f 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -8,8 +8,12 @@ namespace DB { + class CleanupQueue; using CleanupQueuePtr = std::shared_ptr; +class DownloadQueue; +using DownloadQueuePtr = std::shared_ptr; +using FileSegmentsHolderPtr = std::unique_ptr; struct FileSegmentMetadata : private boost::noncopyable @@ -44,6 +48,7 @@ struct KeyMetadata : public std::map, const Key & key_, const std::string & key_path_, CleanupQueue & cleanup_queue_, + DownloadQueue & download_queue_, bool created_base_directory_ = false); enum class KeyState @@ -69,6 +74,7 @@ private: KeyState key_state = KeyState::ACTIVE; KeyGuard guard; CleanupQueue & cleanup_queue; + DownloadQueue & download_queue; std::atomic created_base_directory = false; }; @@ -109,10 +115,15 @@ public: void doCleanup(); + void downloadThreadFunc(); + + void cancelDownload(); + private: const std::string path; /// Cache base path CacheMetadataGuard guard; const CleanupQueuePtr cleanup_queue; + const DownloadQueuePtr download_queue; Poco::Logger * log; }; @@ -159,6 +170,8 @@ struct LockedKey : private boost::noncopyable void shrinkFileSegmentToDownloadedSize(size_t offset, const FileSegmentGuard::Lock &); + void addToDownloadQueue(size_t offset, const FileSegmentGuard::Lock &); + bool isLastOwnerOfFileSegment(size_t offset) const; void removeFromCleanupQueue(); From 4096c082b9f7bd49664e6341e4862c1aed3d7a73 Mon Sep 17 00:00:00 2001 From: Yuriy Chernyshov Date: Wed, 14 Jun 2023 15:09:17 +0300 Subject: [PATCH 035/228] 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 96fb7f04cbc94c625229afd2958cea056c563920 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 14 Jun 2023 15:11:43 +0200 Subject: [PATCH 036/228] Finish download of partially downloaded file segments in the background instead of resizing --- src/Common/CurrentMetrics.cpp | 1 + src/Interpreters/Cache/FileCache.cpp | 38 ++--- src/Interpreters/Cache/FileCache.h | 1 + src/Interpreters/Cache/FileCacheSettings.cpp | 3 + src/Interpreters/Cache/FileCacheSettings.h | 3 +- src/Interpreters/Cache/FileCache_fwd.h | 1 + src/Interpreters/Cache/FileSegment.cpp | 20 ++- src/Interpreters/Cache/Metadata.cpp | 154 ++++++++++++------ src/Interpreters/Cache/Metadata.h | 2 + ...02789_filesystem_cache_alignment.reference | 7 + .../02789_filesystem_cache_alignment.sh | 125 ++++++++++++++ 11 files changed, 264 insertions(+), 91 deletions(-) create mode 100644 tests/queries/0_stateless/02789_filesystem_cache_alignment.reference create mode 100755 tests/queries/0_stateless/02789_filesystem_cache_alignment.sh diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 61725d079bf..b479eac3021 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -188,6 +188,7 @@ M(CacheDetachedFileSegments, "Number of existing detached cache file segments") \ M(FilesystemCacheSize, "Filesystem cache size in bytes") \ M(FilesystemCacheElements, "Filesystem cache elements (file segments)") \ + M(FilesystemCacheDownloadQueueElements, "Filesystem cache elements in download queue") \ M(AsyncInsertCacheSize, "Number of async insert hash id in cache") \ M(S3Requests, "S3 requests") \ M(KeeperAliveConnections, "Number of alive connections") \ diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 65f8ecf7e89..c090d3accd4 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -51,6 +51,7 @@ FileCache::FileCache(const FileCacheSettings & settings) , bypass_cache_threshold(settings.enable_bypass_cache_with_threashold ? settings.bypass_cache_threashold : 0) , delayed_cleanup_interval_ms(settings.delayed_cleanup_interval_ms) , boundary_alignment(settings.boundary_alignment) + , background_download_threads(settings.background_download_threads) , log(&Poco::Logger::get("FileCache")) , metadata(settings.base_path) { @@ -125,12 +126,10 @@ void FileCache::initialize() is_initialized = true; - size_t num_threads=2; - for (size_t i = 0; i < num_threads; ++i) + for (size_t i = 0; i < background_download_threads; ++i) download_threads.emplace_back([this] { metadata.downloadThreadFunc(); }); - auto & schedule_pool = Context::getGlobalContextInstance()->getSchedulePool(); - cleanup_task = schedule_pool.createTask("FileCacheCleanup", [this]{ cleanupThreadFunc(); }); + cleanup_task = Context::getGlobalContextInstance()->getSchedulePool().createTask("FileCacheCleanup", [this]{ cleanupThreadFunc(); }); cleanup_task->activate(); cleanup_task->scheduleAfter(delayed_cleanup_interval_ms); } @@ -644,27 +643,14 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) if (releasable) { - auto segment = segment_metadata->file_segment; - if (segment->state() == FileSegment::State::DOWNLOADED) - { - const auto & key = segment->key(); + const auto & key = segment_metadata->file_segment->key(); + auto it = to_delete.find(key); + if (it == to_delete.end()) + it = to_delete.emplace(key, locked_key.getKeyMetadata()).first; + it->second.add(segment_metadata); - auto it = to_delete.find(key); - if (it == to_delete.end()) - it = to_delete.emplace(key, locked_key.getKeyMetadata()).first; - it->second.add(segment_metadata); - - freeable_space += segment_metadata->size(); - freeable_count += 1; - - return PriorityIterationResult::CONTINUE; - } - - ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); - ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->getDownloadedSize(false)); - - locked_key.removeFileSegment(segment->offset(), segment->lock()); - return PriorityIterationResult::REMOVE_AND_CONTINUE; + freeable_space += segment_metadata->size(); + freeable_count += 1; } return PriorityIterationResult::CONTINUE; }; @@ -1005,10 +991,6 @@ void FileCache::cleanupThreadFunc() { try { -#ifdef ABORT_ON_LOGICAL_ERROR - assertCacheCorrectness(); -#endif - cleanup(); } catch (...) diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index ed693b475ec..931cecf607b 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -135,6 +135,7 @@ private: const size_t bypass_cache_threshold = 0; const size_t delayed_cleanup_interval_ms; const size_t boundary_alignment; + const size_t background_download_threads; Poco::Logger * log; diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index d41aa8ffcb9..bc6e641c869 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -49,6 +49,9 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & if (config.has(config_prefix + ".boundary_alignment")) boundary_alignment = parseWithSizeSuffix(config.getString(config_prefix + ".boundary_alignment")); + if (config.has(config_prefix + ".background_download_threads")) + background_download_threads = config.getUInt(config_prefix + ".background_download_threads"); + 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..fcc5c02c52e 100644 --- a/src/Interpreters/Cache/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -28,7 +28,8 @@ struct FileCacheSettings size_t bypass_cache_threashold = FILECACHE_BYPASS_THRESHOLD; size_t delayed_cleanup_interval_ms = FILECACHE_DELAYED_CLEANUP_INTERVAL_MS; - size_t boundary_alignment = DBMS_DEFAULT_BUFFER_SIZE; + size_t boundary_alignment = FILECACHE_DEFAULT_MIN_FILE_SEGMENT_SIZE; + size_t background_download_threads = FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_THREADS; void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); }; diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index c791d08ed4f..902a6ff42d0 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -6,6 +6,7 @@ namespace DB static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 32 * 1024 * 1024; /// 32Mi static constexpr int FILECACHE_DEFAULT_MIN_FILE_SEGMENT_SIZE = 4 * 1024 * 1024; /// 4Mi +static constexpr int FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_THREADS = 2; static constexpr int FILECACHE_DEFAULT_MAX_ELEMENTS = 10000000; static constexpr int FILECACHE_DEFAULT_HITS_THRESHOLD = 0; static constexpr size_t FILECACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024; diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index f95379ba07f..5dd35a720c1 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -282,8 +282,11 @@ void FileSegment::resetRemoteFileReader() FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader() { - if (download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION) + if (isCompleted(false) + || download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION) + { return std::move(remote_file_reader); + } return nullptr; } @@ -598,7 +601,6 @@ void FileSegment::complete() auto remove_from_cache = [&, this]() { - LOG_TEST(log, "Remove file segment {} (nothing downloaded)", range().toString()); locked_key->removeFileSegment(offset(), segment_lock); setDetachedState(segment_lock); }; @@ -625,6 +627,12 @@ void FileSegment::complete() chassert(!is_last_holder); break; } + case State::EMPTY: + { + if (is_last_holder) + remove_from_cache(); + break; + } case State::PARTIALLY_DOWNLOADED: { if (is_last_holder) @@ -637,12 +645,6 @@ void FileSegment::complete() } break; } - case State::EMPTY: - { - if (is_last_holder) - remove_from_cache(); - break; - } case State::PARTIALLY_DOWNLOADED_NO_CONTINUATION: { chassert(current_downloaded_size != range().size()); @@ -653,7 +655,7 @@ void FileSegment::complete() { remove_from_cache(); } - else if (download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION) + else { LOG_TEST(log, "Resize file segment {} to downloaded: {}", range().toString(), current_downloaded_size); diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 2a3803eb3d7..c3f428c3e08 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -1,11 +1,17 @@ #include #include #include +#include "Common/Exception.h" #include #include namespace fs = std::filesystem; +namespace CurrentMetrics +{ + extern const Metric FilesystemCacheDownloadQueueElements; +} + namespace DB { @@ -256,6 +262,7 @@ void CacheMetadata::doCleanup() continue; } + chassert(it->second->empty()); locked_metadata->markAsRemoved(); erase(it); LOG_DEBUG(log, "Key {} is removed from metadata", cleanup_key); @@ -306,6 +313,8 @@ public: std::lock_guard lock(mutex); queue.push(file_segment); } + + CurrentMetrics::add(CurrentMetrics::FilesystemCacheDownloadQueueElements); cv.notify_one(); } @@ -324,10 +333,10 @@ private: void CacheMetadata::downloadThreadFunc() { - std::optional> memory; while (true) { std::weak_ptr file_segment_weak; + { std::unique_lock lock(download_queue->mutex); @@ -344,67 +353,106 @@ void CacheMetadata::downloadThreadFunc() download_queue->queue.pop(); } + CurrentMetrics::sub(CurrentMetrics::FilesystemCacheDownloadQueueElements); + FileSegmentsHolderPtr holder; + try { - auto file_segment = file_segment_weak.lock(); - if (!file_segment - || file_segment->state() != FileSegment::State::PARTIALLY_DOWNLOADED) - continue; - - auto lock = lockKeyMetadata(file_segment->key(), KeyNotFoundPolicy::RETURN_NULL); - if (!lock) - continue; - - holder = std::make_unique(FileSegments{file_segment}); - } - - auto & file_segment = holder->front(); - chassert(file_segment.assertCorrectness()); - - if (file_segment.getOrSetDownloader() != FileSegment::getCallerId()) - continue; - - LOG_TRACE(log, "Downloading file segment: {}", file_segment.getInfoForLog()); - - auto reader = file_segment.getRemoteFileReader(); - - /// If remote_fs_read_method == 'threadpool', - /// reader iteself does not allocate the buffer, but uses the buffer passed to it. - /// So will need to allocate a buffer here as well. - if (reader->buffer().begin() == nullptr) - { - if (!memory) - memory.emplace(DBMS_DEFAULT_BUFFER_SIZE); - reader->set(memory->data(), memory->size()); - } - - size_t offset = file_segment.getCurrentWriteOffset(false); - while (!reader->eof()) - { - auto size = reader->available(); - - if (!file_segment.reserve(size)) - return; - - try { - file_segment.write(reader->position(), size, offset); - offset += size; - } - catch (ErrnoException & e) - { - int code = e.getErrno(); - if (code == /* No space left on device */28 || code == /* Quota exceeded */122) - { - LOG_INFO(log, "Insert into cache is skipped due to insufficient disk space. ({})", e.displayText()); + auto file_segment = file_segment_weak.lock(); + if (!file_segment + || file_segment->state() != FileSegment::State::PARTIALLY_DOWNLOADED) continue; - } - throw; + + auto lock = lockKeyMetadata(file_segment->key(), KeyNotFoundPolicy::RETURN_NULL); + if (!lock) + continue; + + holder = std::make_unique(FileSegments{file_segment}); + } + + downloadImpl(holder->front()); + } + catch (...) + { + if (holder) + { + const auto & file_segment = holder->front(); + LOG_ERROR( + log, "Error during background download of {}:{} ({}): {}", + file_segment.key(), file_segment.offset(), + file_segment.getInfoForLog(), getCurrentExceptionMessage(true)); + } + else + { + tryLogCurrentException(__PRETTY_FUNCTION__); + chassert(false); } } } } +void CacheMetadata::downloadImpl(FileSegment & file_segment) +{ + chassert(file_segment.assertCorrectness()); + + if (file_segment.getOrSetDownloader() != FileSegment::getCallerId()) + return; + + LOG_TEST( + log, "Downloading {} bytes for file segment {}", + file_segment.range().size() - file_segment.getDownloadedSize(false), file_segment.getInfoForLog()); + + auto reader = file_segment.getRemoteFileReader(); + + /// If remote_fs_read_method == 'threadpool', + /// reader iteself bever owns/allocates the buffer. + std::optional> memory; + if (reader->internalBuffer().empty()) + { + memory.emplace(DBMS_DEFAULT_BUFFER_SIZE); + reader->set(memory->data(), memory->size()); + } + + size_t offset = file_segment.getCurrentWriteOffset(false); + if (offset != static_cast(reader->getPosition())) + reader->seek(offset, SEEK_SET); + + while (!reader->eof()) + { + auto size = reader->available(); + + if (!file_segment.reserve(size)) + { + LOG_TEST( + log, "Failed to reserve space during background download " + "for {}:{} (downloaded size: {}/{})", + file_segment.key(), file_segment.offset(), + file_segment.getDownloadedSize(false), file_segment.range().size()); + return; + } + + try + { + file_segment.write(reader->position(), size, offset); + offset += size; + reader->position() += size; + } + catch (ErrnoException & e) + { + int code = e.getErrno(); + if (code == /* No space left on device */28 || code == /* Quota exceeded */122) + { + LOG_INFO(log, "Insert into cache is skipped due to insufficient disk space. ({})", e.displayText()); + return; + } + throw; + } + } + + LOG_TEST(log, "Downloaded file segment: {}", file_segment.getInfoForLog()); +} + void CacheMetadata::cancelDownload() { download_queue->cancel(); diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index f96243c3f1f..7e8fdef6641 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -125,6 +125,8 @@ private: const CleanupQueuePtr cleanup_queue; const DownloadQueuePtr download_queue; Poco::Logger * log; + + void downloadImpl(FileSegment & file_segment); }; diff --git a/tests/queries/0_stateless/02789_filesystem_cache_alignment.reference b/tests/queries/0_stateless/02789_filesystem_cache_alignment.reference new file mode 100644 index 00000000000..70e1fca6a65 --- /dev/null +++ b/tests/queries/0_stateless/02789_filesystem_cache_alignment.reference @@ -0,0 +1,7 @@ +0 +OK +OK +0 +0 +OK +OK diff --git a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh new file mode 100755 index 00000000000..a8bb4440a4c --- /dev/null +++ b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh @@ -0,0 +1,125 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -nm -q " +DROP TABLE IF EXISTS test; +CREATE TABLE test (a Int32, b String) +ENGINE = MergeTree() +ORDER BY a +SETTINGS disk = disk(type = cache, + max_size = '1Gi', + max_file_segment_size = '40Mi', + boundary_alignment = '20Mi', + path = '$CLICKHOUSE_TEST_UNIQUE_NAME', + disk = 's3disk'); + +INSERT INTO test SELECT number, randomString(100) FROM numbers(1000000); +" + +QUERY_ID=$RANDOM +$CLICKHOUSE_CLIENT --query_id "$QUERY_ID" -nm -q " +SET enable_filesystem_cache_log = 1; +SYSTEM DROP FILESYSTEM CACHE; +SELECT * FROM test WHERE NOT ignore() LIMIT 1 FORMAT Null; +SYSTEM FLUSH LOGS; +" + +query=" +SELECT cache_path, file_size, + tupleElement(file_segment_range, 2) - tupleElement(file_segment_range, 1) + 1 as file_segment_size, + formatReadableSize(file_size) as formatted_file_size, + formatReadableSize(file_segment_size) as formatted_file_segment_size, + tupleElement(file_segment_range, 2) as end_offset +FROM ( + SELECT arrayJoin(cache_paths) AS cache_path, + local_path, + remote_path, + size as file_size + FROM system.remote_data_paths + WHERE endsWith(local_path, '.bin') +) AS data_paths +INNER JOIN system.filesystem_cache_log AS cache_log +ON data_paths.remote_path = cache_log.source_file_path +WHERE query_id = '$QUERY_ID' " + +# File segments cannot be less that 20Mi, +# except for last file segment in a file or if file size is less. +$CLICKHOUSE_CLIENT -nm -q " +SELECT count() FROM ($query) +WHERE file_segment_size < file_size +AND end_offset + 1 != file_size +AND file_segment_size < 20 * 1024 * 1024; +" + +all=$($CLICKHOUSE_CLIENT -nm -q " +SELECT count() FROM ($query) +WHERE file_segment_size < file_size AND end_offset + 1 != file_size; +") +#echo $all + +if [ "$all" -gt "10" ]; then + echo "OK" +else + echo "FAIL" +fi + +count=$($CLICKHOUSE_CLIENT -nm -q " +SELECT count() FROM ($query) +WHERE file_segment_size < file_size +AND end_offset + 1 != file_size +AND formatted_file_segment_size in ('20.00 MiB', '40.00 MiB') +") + +if [ "$count" = "$all" ]; then + echo "OK" +else + echo "FAIL" +fi + +query2=" +SELECT * +FROM (SELECT * FROM ($query)) AS cache_log +INNER JOIN system.filesystem_cache AS cache +ON cache_log.cache_path = cache.cache_path " + +$CLICKHOUSE_CLIENT -nm -q " +SELECT count() FROM ($query2) +WHERE file_segment_range_begin - file_segment_range_end + 1 < file_size +AND file_segment_range_end + 1 != file_size +AND downloaded_size < 20 * 1024 * 1024; +" + +$CLICKHOUSE_CLIENT -nm -q " +SELECT count() FROM ($query2) +WHERE file_segment_range_begin - file_segment_range_end + 1 < file_size +AND file_segment_range_end + 1 != file_size +AND formatReadableSize(downloaded_size) not in ('20.00 MiB', '40.00 MiB'); +" + +all=$($CLICKHOUSE_CLIENT -nm -q " +SELECT count() FROM ($query2) +WHERE file_segment_size < file_size AND file_segment_range_end + 1 != file_size; +") + +if [ "$all" -gt "10" ]; then + echo "OK" +else + echo "FAIL" +fi + +count2=$($CLICKHOUSE_CLIENT -nm -q " +SELECT count() FROM ($query2) +WHERE file_segment_range_begin - file_segment_range_end + 1 < file_size +AND file_segment_range_end + 1 != file_size +AND formatReadableSize(downloaded_size) in ('20.00 MiB', '40.00 MiB'); +") + +if [ "$count2" = "$all" ]; then + echo "OK" +else + echo "FAIL" +fi From 0aa363f5254455ea3a351c9fcc9adc7c99701c53 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 14 Jun 2023 23:13:09 +0200 Subject: [PATCH 037/228] Fix style check --- src/Interpreters/Cache/Metadata.cpp | 2 +- tests/queries/0_stateless/02789_filesystem_cache_alignment.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index c3f428c3e08..b0009c6abc1 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -406,7 +406,7 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment) auto reader = file_segment.getRemoteFileReader(); /// If remote_fs_read_method == 'threadpool', - /// reader iteself bever owns/allocates the buffer. + /// reader itself never owns/allocates the buffer. std::optional> memory; if (reader->internalBuffer().empty()) { diff --git a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh index a8bb4440a4c..ca459153c03 100755 --- a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh +++ b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh @@ -15,7 +15,7 @@ SETTINGS disk = disk(type = cache, max_file_segment_size = '40Mi', boundary_alignment = '20Mi', path = '$CLICKHOUSE_TEST_UNIQUE_NAME', - disk = 's3disk'); + disk = 's3_disk'); INSERT INTO test SELECT number, randomString(100) FROM numbers(1000000); " From 8c610275c2e271cd3a506d207fd9cef938fcd258 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 14 Jun 2023 23:30:21 +0200 Subject: [PATCH 038/228] Review fixes --- src/Interpreters/Cache/FileCache.cpp | 1 - src/Interpreters/Cache/FileCacheSettings.h | 2 +- src/Interpreters/Cache/FileCache_fwd.h | 2 +- src/Interpreters/Cache/Metadata.cpp | 17 ++++++++++------- src/Interpreters/Cache/Metadata.h | 2 +- 5 files changed, 13 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index c090d3accd4..3b5ae9acb1b 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1014,7 +1014,6 @@ FileSegmentsHolderPtr FileCache::getSnapshot() { for (const auto & [_, file_segment_metadata] : locked_key) file_segments.push_back(FileSegment::getSnapshot(file_segment_metadata->file_segment)); - return true; }); return std::make_unique(std::move(file_segments), /* complete_on_dtor */false); } diff --git a/src/Interpreters/Cache/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h index fcc5c02c52e..6820aff61e3 100644 --- a/src/Interpreters/Cache/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -28,7 +28,7 @@ struct FileCacheSettings size_t bypass_cache_threashold = FILECACHE_BYPASS_THRESHOLD; size_t delayed_cleanup_interval_ms = FILECACHE_DELAYED_CLEANUP_INTERVAL_MS; - size_t boundary_alignment = FILECACHE_DEFAULT_MIN_FILE_SEGMENT_SIZE; + size_t boundary_alignment = FILECACHE_DEFAULT_FILE_SEGMENT_ALIGNMENT; size_t background_download_threads = FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_THREADS; void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index 902a6ff42d0..4d4a1c1429c 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -5,7 +5,7 @@ namespace DB { static constexpr int FILECACHE_DEFAULT_MAX_FILE_SEGMENT_SIZE = 32 * 1024 * 1024; /// 32Mi -static constexpr int FILECACHE_DEFAULT_MIN_FILE_SEGMENT_SIZE = 4 * 1024 * 1024; /// 4Mi +static constexpr int FILECACHE_DEFAULT_FILE_SEGMENT_ALIGNMENT = 4 * 1024 * 1024; /// 4Mi static constexpr int FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_THREADS = 2; static constexpr int FILECACHE_DEFAULT_MAX_ELEMENTS = 10000000; static constexpr int FILECACHE_DEFAULT_HITS_THRESHOLD = 0; diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index b0009c6abc1..f94b8ae4db2 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -321,8 +321,11 @@ public: private: void cancel() { - std::lock_guard lock(mutex); - cancelled = true; + { + std::lock_guard lock(mutex); + cancelled = true; + } + cv.notify_all(); } std::mutex mutex; @@ -333,6 +336,7 @@ private: void CacheMetadata::downloadThreadFunc() { + std::optional> memory; while (true) { std::weak_ptr file_segment_weak; @@ -371,7 +375,7 @@ void CacheMetadata::downloadThreadFunc() holder = std::make_unique(FileSegments{file_segment}); } - downloadImpl(holder->front()); + downloadImpl(holder->front(), memory); } catch (...) { @@ -392,7 +396,7 @@ void CacheMetadata::downloadThreadFunc() } } -void CacheMetadata::downloadImpl(FileSegment & file_segment) +void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optional> & memory) { chassert(file_segment.assertCorrectness()); @@ -407,10 +411,10 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment) /// If remote_fs_read_method == 'threadpool', /// reader itself never owns/allocates the buffer. - std::optional> memory; if (reader->internalBuffer().empty()) { - memory.emplace(DBMS_DEFAULT_BUFFER_SIZE); + if (!memory) + memory.emplace(DBMS_DEFAULT_BUFFER_SIZE); reader->set(memory->data(), memory->size()); } @@ -456,7 +460,6 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment) void CacheMetadata::cancelDownload() { download_queue->cancel(); - download_queue->cv.notify_all(); } LockedKey::LockedKey(std::shared_ptr key_metadata_) diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index 7e8fdef6641..f9f7dfccaa5 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -126,7 +126,7 @@ private: const DownloadQueuePtr download_queue; Poco::Logger * log; - void downloadImpl(FileSegment & file_segment); + void downloadImpl(FileSegment & file_segment, std::optional> & memory); }; From 0023bf69cbaf2ec756bca883e3bb6e9338e574de Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 15 Jun 2023 12:22:50 +0200 Subject: [PATCH 039/228] Fix --- src/Interpreters/Cache/FileSegment.cpp | 18 ++++++++++--- src/Interpreters/Cache/Metadata.cpp | 3 +++ .../InterpreterDescribeCacheQuery.cpp | 27 ++++++++++++------- .../02344_describe_cache.reference | 4 +-- 4 files changed, 37 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 5dd35a720c1..4cc5e4307ad 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -635,13 +635,23 @@ void FileSegment::complete() } case State::PARTIALLY_DOWNLOADED: { + chassert(current_downloaded_size > 0); + if (is_last_holder) { - LOG_TEST( - log, "Submitted file segment for background download " - "(having {}/{})", downloaded_size, range().size()); + if (remote_file_reader) + { + LOG_TEST( + log, "Submitting file segment for background download " + "(having {}/{})", downloaded_size, range().size()); - locked_key->addToDownloadQueue(offset(), segment_lock); /// Finish download in background. + locked_key->addToDownloadQueue(offset(), segment_lock); /// Finish download in background. + } + else + { + locked_key->shrinkFileSegmentToDownloadedSize(offset(), segment_lock); + setDetachedState(segment_lock); /// See comment below. + } } break; } diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index f94b8ae4db2..fd590315ab1 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -403,6 +403,9 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optional(), "max_size"}, ColumnWithTypeAndName{std::make_shared(), "max_elements"}, ColumnWithTypeAndName{std::make_shared(), "max_file_segment_size"}, + ColumnWithTypeAndName{std::make_shared(), "boundary_alignment"}, ColumnWithTypeAndName{std::make_shared>(), "cache_on_write_operations"}, ColumnWithTypeAndName{std::make_shared>(), "cache_hits_threshold"}, 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"}, + ColumnWithTypeAndName{std::make_shared>(), "delayed_cleanup_interval_ms"}, + ColumnWithTypeAndName{std::make_shared>(), "background_download_threads"}, + ColumnWithTypeAndName{std::make_shared>(), "enable_bypass_cache_with_threshold"}, }; return Block(columns); } @@ -41,15 +45,20 @@ BlockIO InterpreterDescribeCacheQuery::execute() const auto & settings = cache_data.settings; const auto & cache = cache_data.cache; - res_columns[0]->insert(settings.max_size); - res_columns[1]->insert(settings.max_elements); - res_columns[2]->insert(settings.max_file_segment_size); - res_columns[3]->insert(settings.cache_on_write_operations); - res_columns[4]->insert(settings.cache_hits_threshold); - 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); + size_t i = 0; + res_columns[i++]->insert(settings.max_size); + res_columns[i++]->insert(settings.max_elements); + res_columns[i++]->insert(settings.max_file_segment_size); + res_columns[i++]->insert(settings.boundary_alignment); + res_columns[i++]->insert(settings.cache_on_write_operations); + res_columns[i++]->insert(settings.cache_hits_threshold); + res_columns[i++]->insert(cache->getUsedCacheSize()); + res_columns[i++]->insert(cache->getFileSegmentsNum()); + res_columns[i++]->insert(cache->getBasePath()); + res_columns[i++]->insert(settings.do_not_evict_index_and_mark_files); + res_columns[i++]->insert(settings.delayed_cleanup_interval_ms); + res_columns[i++]->insert(settings.background_download_threads); + res_columns[i++]->insert(settings.enable_bypass_cache_with_threashold); BlockIO res; size_t num_rows = res_columns[0]->size(); diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index d8a2ffab1fa..7496f754937 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 33554432 4194304 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 0 60000 2 0 +134217728 10000000 104857600 4194304 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 0 60000 2 0 From b8a4a784debcfb4e739b2368bc5611885ffa2a51 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 15 Jun 2023 14:22:13 +0200 Subject: [PATCH 040/228] Fix unit test, add check --- src/Interpreters/Cache/Metadata.cpp | 8 ++++++++ src/Interpreters/tests/gtest_lru_file_cache.cpp | 4 ++-- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index fd590315ab1..987361c0537 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -412,6 +412,14 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optionalinternalBuffer().empty()) diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 8c8e715ce92..d8baf1c0b45 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -533,8 +533,8 @@ TEST_F(FileCacheTest, get) cv.notify_one(); file_segment2.wait(file_segment2.range().left); - ASSERT_TRUE(file_segment2.state() == DB::FileSegment::State::PARTIALLY_DOWNLOADED); - ASSERT_TRUE(file_segment2.getOrSetDownloader() == DB::FileSegment::getCallerId()); + ASSERT_EQ(file_segment2.state(), DB::FileSegment::State::EMPTY); + ASSERT_EQ(file_segment2.getOrSetDownloader(), DB::FileSegment::getCallerId()); download(file_segment2); }); From ddc95796d3da99e0de3cf8b665c319f5305f39fe Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 16 Jun 2023 13:45:29 +0200 Subject: [PATCH 041/228] Fix tests --- tests/queries/0_stateless/02344_describe_cache.reference | 4 ++-- tests/queries/0_stateless/02789_filesystem_cache_alignment.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index 7496f754937..97bcf240a16 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 33554432 4194304 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 0 60000 2 0 -134217728 10000000 104857600 4194304 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 0 60000 2 0 +134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 0 100 2 0 +134217728 10000000 104857600 4194304 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 0 100 2 0 diff --git a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh index ca459153c03..b9f9548a7b3 100755 --- a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh +++ b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel +# Tags: no-fasttest, no-parallel, no-random-settings CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 12f979dd2bcc66ba396f4a3eed6e03d280661c49 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 17 Jun 2023 02:08:11 +0200 Subject: [PATCH 042/228] Update .reference --- tests/queries/0_stateless/02344_describe_cache.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index 97bcf240a16..6f445dc13e7 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 33554432 4194304 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 0 100 2 0 -134217728 10000000 104857600 4194304 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 0 100 2 0 +134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 100 2 0 +134217728 10000000 104857600 4194304 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 100 2 0 From 7df815d2806a9154a4f7010590952a647be97edd Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 20 Jun 2023 13:33:16 +0200 Subject: [PATCH 043/228] Fix --- src/Interpreters/Cache/Metadata.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index d659a58fdaf..ed2174213e1 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -400,6 +400,10 @@ void CacheMetadata::downloadThreadFunc() if (!lock) continue; + auto file_segment_metadata = lock->tryGetByOffset(file_segment->offset()); + if (!file_segment_metadata || file_segment_metadata->evicting()) + continue; + holder = std::make_unique(FileSegments{file_segment}); } From 44377b01809dd4e85ca66d03c0b83a6f9da14ef9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 20 Jun 2023 13:36:11 +0200 Subject: [PATCH 044/228] Rename --- src/Interpreters/Cache/Metadata.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index ed2174213e1..76dd5172929 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -396,11 +396,11 @@ void CacheMetadata::downloadThreadFunc() || file_segment->state() != FileSegment::State::PARTIALLY_DOWNLOADED) continue; - auto lock = lockKeyMetadata(file_segment->key(), KeyNotFoundPolicy::RETURN_NULL); - if (!lock) + auto locked_key = lockKeyMetadata(file_segment->key(), KeyNotFoundPolicy::RETURN_NULL); + if (!locked_key) continue; - auto file_segment_metadata = lock->tryGetByOffset(file_segment->offset()); + auto file_segment_metadata = locked_key->tryGetByOffset(file_segment->offset()); if (!file_segment_metadata || file_segment_metadata->evicting()) continue; From b8085dfa0e63b88132abcaecee5084936515a6d6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 21 Jun 2023 00:20:40 +0200 Subject: [PATCH 045/228] Fix bad merge --- src/Interpreters/Cache/FileSegment.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index de8ede69848..daae9b447a4 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -305,8 +305,9 @@ void FileSegment::resetRemoteFileReader() FileSegment::RemoteFileReaderPtr FileSegment::extractRemoteFileReader() { - if (isCompleted(false) - || download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION) + auto lock = lockFileSegment(); + if (remote_file_reader && (download_state == State::DOWNLOADED + || download_state == State::PARTIALLY_DOWNLOADED_NO_CONTINUATION)) { return std::move(remote_file_reader); } From 26ba3d8f13a078d7b41be00ca9c687e59f04774a Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 22 Jun 2023 10:01:22 -0300 Subject: [PATCH 046/228] test for #36894 --- ...02302_lc_nullable_string_insert_as_number.reference | 1 + .../02302_lc_nullable_string_insert_as_number.sql | 10 ++++++++++ 2 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/02302_lc_nullable_string_insert_as_number.reference create mode 100644 tests/queries/0_stateless/02302_lc_nullable_string_insert_as_number.sql diff --git a/tests/queries/0_stateless/02302_lc_nullable_string_insert_as_number.reference b/tests/queries/0_stateless/02302_lc_nullable_string_insert_as_number.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02302_lc_nullable_string_insert_as_number.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02302_lc_nullable_string_insert_as_number.sql b/tests/queries/0_stateless/02302_lc_nullable_string_insert_as_number.sql new file mode 100644 index 00000000000..89b31e2b2b0 --- /dev/null +++ b/tests/queries/0_stateless/02302_lc_nullable_string_insert_as_number.sql @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS lc_nullable_string; + +CREATE TABLE lc_nullable_string(`c1` LowCardinality(Nullable(String)) DEFAULT CAST(NULL, 'LowCardinality(Nullable(String))')) +ENGINE = Memory; + +INSERT INTO lc_nullable_string (c1) FORMAT Values (0); + +SELECT * FROM lc_nullable_string; + +DROP TABLE lc_nullable_string; From b1d37fbc6bbeac4613c4e58555a56b1092f4018b Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 22 Jun 2023 10:23:32 -0300 Subject: [PATCH 047/228] test for #36894 --- .../02302_lc_nullable_string_insert_as_number.reference | 1 + .../0_stateless/02302_lc_nullable_string_insert_as_number.sql | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02302_lc_nullable_string_insert_as_number.reference b/tests/queries/0_stateless/02302_lc_nullable_string_insert_as_number.reference index 573541ac970..0d66ea1aee9 100644 --- a/tests/queries/0_stateless/02302_lc_nullable_string_insert_as_number.reference +++ b/tests/queries/0_stateless/02302_lc_nullable_string_insert_as_number.reference @@ -1 +1,2 @@ 0 +1 diff --git a/tests/queries/0_stateless/02302_lc_nullable_string_insert_as_number.sql b/tests/queries/0_stateless/02302_lc_nullable_string_insert_as_number.sql index 89b31e2b2b0..9859c1559d5 100644 --- a/tests/queries/0_stateless/02302_lc_nullable_string_insert_as_number.sql +++ b/tests/queries/0_stateless/02302_lc_nullable_string_insert_as_number.sql @@ -4,7 +4,8 @@ CREATE TABLE lc_nullable_string(`c1` LowCardinality(Nullable(String)) DEFAULT CA ENGINE = Memory; INSERT INTO lc_nullable_string (c1) FORMAT Values (0); +INSERT INTO lc_nullable_string (c1) Values (1); -SELECT * FROM lc_nullable_string; +SELECT * FROM lc_nullable_string ORDER BY c1; DROP TABLE lc_nullable_string; From 2b8238786e4e8f649e3bdffcf4f373dcbe54617e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 22 Jun 2023 21:36:45 +0000 Subject: [PATCH 048/228] Added test --- .../0_stateless/02789_object_type_invalid_num_of_rows.reference | 0 .../0_stateless/02789_object_type_invalid_num_of_rows.sql | 2 ++ 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.reference create mode 100644 tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.sql 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 new file mode 100644 index 00000000000..e69de29bb2d 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 new file mode 100644 index 00000000000..53d391d69ac --- /dev/null +++ b/tests/queries/0_stateless/02789_object_type_invalid_num_of_rows.sql @@ -0,0 +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\')'))) -- { serverError NOT_IMPLEMENTED } From 003acb4fbe1f4a1704688cf4edf44183c09c65db Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 22 Jun 2023 18:31:01 +0200 Subject: [PATCH 049/228] add absl as include dir --- contrib/abseil-cpp-cmake/CMakeLists.txt | 14 ++++++++++++++ contrib/re2-cmake/CMakeLists.txt | 15 ++++----------- 2 files changed, 18 insertions(+), 11 deletions(-) diff --git a/contrib/abseil-cpp-cmake/CMakeLists.txt b/contrib/abseil-cpp-cmake/CMakeLists.txt index 4c31ecfc599..d64b6036611 100644 --- a/contrib/abseil-cpp-cmake/CMakeLists.txt +++ b/contrib/abseil-cpp-cmake/CMakeLists.txt @@ -17,3 +17,17 @@ get_target_property(FLAT_HASH_SET_INCLUDE_DIR absl::flat_hash_set INTERFACE_INCL target_include_directories (_abseil_swiss_tables SYSTEM BEFORE INTERFACE ${FLAT_HASH_SET_INCLUDE_DIR}) add_library(ch_contrib::abseil_swiss_tables ALIAS _abseil_swiss_tables) + +set(ABSL_FORMAT_SRC + ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/arg.cc + ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/bind.cc + ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/extension.cc + ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/float_conversion.cc + ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/output.cc + ${ABSL_ROOT_DIR}/absl/strings/internal/str_format/parser.cc +) + +add_library(_abseil_str_format ${ABSL_FORMAT_SRC}) +target_include_directories(_abseil_str_format PUBLIC ${ABSL_ROOT_DIR}) + +add_library(ch_contrib::abseil_str_format ALIAS _abseil_str_format) diff --git a/contrib/re2-cmake/CMakeLists.txt b/contrib/re2-cmake/CMakeLists.txt index a081f92bc94..305c2400c77 100644 --- a/contrib/re2-cmake/CMakeLists.txt +++ b/contrib/re2-cmake/CMakeLists.txt @@ -12,6 +12,7 @@ endif() set(SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/re2") set(RE2_SOURCES + ${SRC_DIR}/re2/bitmap256.cc ${SRC_DIR}/re2/bitstate.cc ${SRC_DIR}/re2/compile.cc ${SRC_DIR}/re2/dfa.cc @@ -31,11 +32,13 @@ set(RE2_SOURCES ${SRC_DIR}/re2/tostring.cc ${SRC_DIR}/re2/unicode_casefold.cc ${SRC_DIR}/re2/unicode_groups.cc + ${SRC_DIR}/util/pcre.cc ${SRC_DIR}/util/rune.cc ${SRC_DIR}/util/strutil.cc ) add_library(re2 ${RE2_SOURCES}) target_include_directories(re2 PUBLIC "${SRC_DIR}") +target_link_libraries(re2 ch_contrib::abseil_str_format) # Building re2 which is thread-safe and re2_st which is not. # re2 changes its state during matching of regular expression, e.g. creates temporary DFA. @@ -47,6 +50,7 @@ target_compile_definitions (re2_st PRIVATE NDEBUG NO_THREADS re2=re2_st) target_include_directories (re2_st PRIVATE .) target_include_directories (re2_st SYSTEM PUBLIC ${CMAKE_CURRENT_BINARY_DIR}) target_include_directories (re2_st SYSTEM BEFORE PUBLIC ${SRC_DIR}) +target_link_libraries (re2_st ch_contrib::abseil_str_format) file (MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/re2_st) foreach (FILENAME filtered_re2.h re2.h set.h stringpiece.h) @@ -59,17 +63,6 @@ foreach (FILENAME filtered_re2.h re2.h set.h stringpiece.h) add_dependencies (re2_st transform_${FILENAME}) endforeach () -file (MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/util) -foreach (FILENAME mutex.h) - add_custom_command (OUTPUT "${CMAKE_CURRENT_BINARY_DIR}/util/${FILENAME}" - COMMAND ${CMAKE_COMMAND} -DSOURCE_FILENAME="${SRC_DIR}/util/${FILENAME}" - -DTARGET_FILENAME="${CMAKE_CURRENT_BINARY_DIR}/util/${FILENAME}" - -P "${CMAKE_CURRENT_SOURCE_DIR}/re2_transform.cmake" - COMMENT "Creating ${FILENAME} for re2_st library.") - add_custom_target (transform_${FILENAME} DEPENDS "${CMAKE_CURRENT_BINARY_DIR}/util/${FILENAME}") - add_dependencies (re2_st transform_${FILENAME}) -endforeach () - # NOTE: you should not change name of library here, since it is used to generate required header (see above) add_library(ch_contrib::re2 ALIAS re2) add_library(ch_contrib::re2_st ALIAS re2_st) From fb7d23f245c3cd44f4f096f362fcb4ef948b1580 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 22 Jun 2023 18:31:12 +0200 Subject: [PATCH 050/228] fix build --- src/Dictionaries/RegExpTreeDictionary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index 3852cca6928..074b179c48e 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -479,7 +479,7 @@ std::pair processBackRefs(const String & data, const re2_st::RE2 & for (const auto & item : pieces) { if (item.ref_num >= 0 && item.ref_num < 10) - result += matches[item.ref_num].ToString(); + result += String{matches[item.ref_num]}; else result += item.literal; } From 7d00b26a884aaf09b5170e56311caba17a6ba7ce Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 22 Jun 2023 22:39:38 +0000 Subject: [PATCH 051/228] always reprodusible with max_threads=1 --- .../0_stateless/02789_object_type_invalid_num_of_rows.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 53d391d69ac..8ced133c8eb 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\')'))) -- { serverError NOT_IMPLEMENTED } +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 } From be17a817929ea7fef4a68ac108f89a65f0e21827 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 24 Jun 2023 00:12:31 +0000 Subject: [PATCH 052/228] recursively collect call stacks on job scheduling and add all stack traces to exceptions printout --- src/Common/Exception.cpp | 19 ++++++++++++++++++- src/Common/Exception.h | 12 +++++++++++- src/Common/ThreadPool.cpp | 9 ++++++++- src/Common/ThreadPool.h | 14 ++++++++++++-- 4 files changed, 49 insertions(+), 5 deletions(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 20206b76225..a43335ed8de 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -1,5 +1,6 @@ #include "Exception.h" +#include #include #include #include @@ -83,6 +84,7 @@ Exception::Exception(const MessageMasked & msg_masked, int code, bool remote_) : Poco::Exception(msg_masked.msg, code) , remote(remote_) { + capture_thread_frame_pointers = thread_frame_pointers; handle_error_code(msg_masked.msg, code, remote, getStackFramePointers()); } @@ -90,12 +92,14 @@ Exception::Exception(MessageMasked && msg_masked, int code, bool remote_) : Poco::Exception(msg_masked.msg, code) , remote(remote_) { + capture_thread_frame_pointers = thread_frame_pointers; handle_error_code(message(), code, remote, getStackFramePointers()); } Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc) : Poco::Exception(exc.displayText(), ErrorCodes::POCO_EXCEPTION) { + capture_thread_frame_pointers = thread_frame_pointers; #ifdef STD_EXCEPTION_HAS_STACK_TRACE auto * stack_trace_frames = exc.get_stack_trace_frames(); auto stack_trace_size = exc.get_stack_trace_size(); @@ -107,6 +111,7 @@ Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc) Exception::Exception(CreateFromSTDTag, const std::exception & exc) : Poco::Exception(demangle(typeid(exc).name()) + ": " + String(exc.what()), ErrorCodes::STD_EXCEPTION) { + capture_thread_frame_pointers = thread_frame_pointers; #ifdef STD_EXCEPTION_HAS_STACK_TRACE auto * stack_trace_frames = exc.get_stack_trace_frames(); auto stack_trace_size = exc.get_stack_trace_size(); @@ -153,7 +158,17 @@ std::string Exception::getStackTraceString() const auto * stack_trace_frames = get_stack_trace_frames(); auto stack_trace_size = get_stack_trace_size(); __msan_unpoison(stack_trace_frames, stack_trace_size * sizeof(stack_trace_frames[0])); - return StackTrace::toString(stack_trace_frames, 0, stack_trace_size); + String thread_stack_trace; + std::for_each(capture_thread_frame_pointers.rbegin(), capture_thread_frame_pointers.rend(), + [&thread_stack_trace](StackTrace::FramePointers & frame_pointers) + { + thread_stack_trace += + "\nJob's origin stack trace:\n" + + StackTrace::toString(&frame_pointers[0], 0, std::ranges::find(frame_pointers, nullptr) - frame_pointers.begin()); + } + ); + + return StackTrace::toString(stack_trace_frames, 0, stack_trace_size) + thread_stack_trace; #else return trace.toString(); #endif @@ -185,6 +200,8 @@ Exception::FramePointers Exception::getStackFramePointers() const return frame_pointers; } +thread_local std::vector Exception::thread_frame_pointers = {}; + void throwFromErrno(const std::string & s, int code, int the_errno) { diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 170e0d32b3c..c1bbeada506 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -25,18 +25,25 @@ class Exception : public Poco::Exception public: using FramePointers = std::vector; - Exception() = default; + Exception() + { + capture_thread_frame_pointers = thread_frame_pointers; + } Exception(const PreformattedMessage & msg, int code): Exception(msg.text, code) { + capture_thread_frame_pointers = thread_frame_pointers; message_format_string = msg.format_string; } Exception(PreformattedMessage && msg, int code): Exception(std::move(msg.text), code) { + capture_thread_frame_pointers = thread_frame_pointers; message_format_string = msg.format_string; } + static thread_local std::vector thread_frame_pointers; + protected: // used to remove the sensitive information from exceptions if query_masking_rules is configured struct MessageMasked @@ -66,6 +73,7 @@ public: Exception(int code, T && message) : Exception(message, code) { + capture_thread_frame_pointers = thread_frame_pointers; message_format_string = tryGetStaticFormatString(message); } @@ -80,6 +88,7 @@ public: Exception(int code, FormatStringHelper fmt, Args &&... args) : Exception(fmt::format(fmt.fmt_str, std::forward(args)...), code) { + capture_thread_frame_pointers = thread_frame_pointers; message_format_string = fmt.message_format_string; } @@ -131,6 +140,7 @@ private: protected: std::string_view message_format_string; + mutable std::vector capture_thread_frame_pointers; }; diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index f91849ead66..0bd4fcb0455 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -189,7 +189,9 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, Priority priority, std: jobs.emplace(std::move(job), priority, /// 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()); + propagate_opentelemetry_tracing_context ? DB::OpenTelemetry::CurrentContext() : DB::OpenTelemetry::TracingContextOnThread(), + /// capture_frame_pointers + true); ++scheduled_jobs; } @@ -348,6 +350,8 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ /// A copy of parent trace context DB::OpenTelemetry::TracingContextOnThread parent_thread_trace_context; + std::vector thread_frame_pointers; + /// Get a job from the queue. Job job; @@ -393,6 +397,7 @@ 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)); jobs.pop(); /// We don't run jobs after `shutdown` is set, but we have to properly dequeue all jobs and finish them. @@ -411,6 +416,8 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ /// Run the job. try { + DB::Exception::thread_frame_pointers = std::move(thread_frame_pointers); + CurrentMetrics::Increment metric_active_pool_threads(metric_active_threads); job(); diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 70053ff7558..17731de9540 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -19,6 +19,8 @@ #include #include #include +#include +#include #include /** Very simple thread pool similar to boost::threadpool. @@ -127,8 +129,16 @@ private: Priority priority; DB::OpenTelemetry::TracingContextOnThread thread_trace_context; - JobWithPriority(Job job_, Priority priority_, const DB::OpenTelemetry::TracingContextOnThread & thread_trace_context_) - : job(job_), priority(priority_), thread_trace_context(thread_trace_context_) {} + std::vector frame_pointers; + + 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_) + { + if (!capture_frame_pointers) + return; + frame_pointers = DB::Exception::thread_frame_pointers; + frame_pointers.push_back(StackTrace().getFramePointers()); + } bool operator<(const JobWithPriority & rhs) const { From 7aa089902081922cb08e3e149f359a85add7785d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Jun 2023 07:55:38 +0200 Subject: [PATCH 053/228] Add a test for #42631 --- .../02803_remote_cannot_clone_block.reference | 1 + .../02803_remote_cannot_clone_block.sql | 19 +++++++++++++++++++ 2 files changed, 20 insertions(+) create mode 100644 tests/queries/0_stateless/02803_remote_cannot_clone_block.reference create mode 100644 tests/queries/0_stateless/02803_remote_cannot_clone_block.sql diff --git a/tests/queries/0_stateless/02803_remote_cannot_clone_block.reference b/tests/queries/0_stateless/02803_remote_cannot_clone_block.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02803_remote_cannot_clone_block.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql b/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql new file mode 100644 index 00000000000..3e14aa6be96 --- /dev/null +++ b/tests/queries/0_stateless/02803_remote_cannot_clone_block.sql @@ -0,0 +1,19 @@ +DROP TABLE IF EXISTS numbers_10_00223; + +CREATE TABLE numbers_10_00223 +ENGINE = Log AS +SELECT * +FROM system.numbers +LIMIT 10000; + +SELECT * +FROM +( + SELECT 1 + FROM remote('127.0.0.{2,3}', currentDatabase(), numbers_10_00223) + WITH TOTALS +) +WHERE 1 +GROUP BY 1; + +DROP TABLE numbers_10_00223; From 3711430d9f97401d80dff14eaab68008bb03132c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 15 Jun 2023 16:14:56 +0200 Subject: [PATCH 054/228] Rename member fields of CityHash_v1_0_2::uint128: "first" -> "low64", "second" -> "high64". --- base/base/wide_integer_impl.h | 16 ++++++++++++++++ contrib/cityhash102/include/city.h | 19 ++++++++++++++++--- src/Compression/CompressedReadBufferBase.cpp | 16 ++++++++-------- src/Compression/CompressedWriteBuffer.cpp | 8 ++++---- .../DistributedAsyncInsertHeader.cpp | 4 ++-- .../MergeTree/DataPartStorageOnDiskBase.cpp | 2 +- .../MergeTree/MergeTreeDataPartChecksum.cpp | 8 ++++---- .../PartMetadataManagerWithCache.cpp | 8 ++++---- src/Storages/System/StorageSystemParts.cpp | 6 +++--- .../System/StorageSystemProjectionParts.cpp | 6 +++--- 10 files changed, 61 insertions(+), 32 deletions(-) diff --git a/base/base/wide_integer_impl.h b/base/base/wide_integer_impl.h index 4a80c176829..dc6a49694ae 100644 --- a/base/base/wide_integer_impl.h +++ b/base/base/wide_integer_impl.h @@ -15,6 +15,8 @@ #include #include +#include + // NOLINTBEGIN(*) /// Use same extended double for all platforms @@ -281,6 +283,14 @@ struct integer::_impl } } + constexpr static void wide_integer_from_cityhash_uint128(integer & self, const CityHash_v1_0_2::uint128 & value) noexcept + { + if constexpr (std::endian::native == std::endian::little) + wide_integer_from_tuple_like(self, std::make_pair(value.low64, value.high64)); + else + wide_integer_from_tuple_like(self, std::make_pair(value.high64, value.low64)); + } + /** * N.B. t is constructed from double, so max(t) = max(double) ~ 2^310 * the recursive call happens when t / 2^64 > 2^64, so there won't be more than 5 of them. @@ -1036,6 +1046,8 @@ constexpr integer::integer(T rhs) noexcept _impl::wide_integer_from_wide_integer(*this, rhs); else if constexpr (IsTupleLike::value) _impl::wide_integer_from_tuple_like(*this, rhs); + else if constexpr (std::is_same_v, CityHash_v1_0_2::uint128>) + _impl::wide_integer_from_cityhash_uint128(*this, rhs); else _impl::wide_integer_from_builtin(*this, rhs); } @@ -1051,6 +1063,8 @@ constexpr integer::integer(std::initializer_list il) noexcept _impl::wide_integer_from_wide_integer(*this, *il.begin()); else if constexpr (IsTupleLike::value) _impl::wide_integer_from_tuple_like(*this, *il.begin()); + else if constexpr (std::is_same_v, CityHash_v1_0_2::uint128>) + _impl::wide_integer_from_cityhash_uint128(*this, *il.begin()); else _impl::wide_integer_from_builtin(*this, *il.begin()); } @@ -1088,6 +1102,8 @@ constexpr integer & integer::operator=(T rhs) noexce { if constexpr (IsTupleLike::value) _impl::wide_integer_from_tuple_like(*this, rhs); + else if constexpr (std::is_same_v, CityHash_v1_0_2::uint128>) + _impl::wide_integer_from_cityhash_uint128(*this, rhs); else _impl::wide_integer_from_builtin(*this, rhs); return *this; diff --git a/contrib/cityhash102/include/city.h b/contrib/cityhash102/include/city.h index 77d4c988cdd..87363d16444 100644 --- a/contrib/cityhash102/include/city.h +++ b/contrib/cityhash102/include/city.h @@ -61,11 +61,24 @@ namespace CityHash_v1_0_2 typedef uint8_t uint8; typedef uint32_t uint32; typedef uint64_t uint64; -typedef std::pair uint128; +/// Represent an unsigned integer of 128 bits as it's used in CityHash. +/// Originally CityHash used `std::pair` instead of this struct, +/// however the members `first` and `second` could be easily confused so they were renamed to `low64` and `high64`: +/// `first` -> `low64`, `second` -> `high64`. +struct uint128 +{ + uint64 low64 = 0; + uint64 high64 = 0; -inline uint64 Uint128Low64(const uint128& x) { return x.first; } -inline uint64 Uint128High64(const uint128& x) { return x.second; } + uint128() = default; + uint128(uint64 low64_, uint64 high64_) : low64(low64_), high64(high64_) {} + friend bool operator ==(const uint128 & x, const uint128 & y) { return (x.low64 == y.low64) && (x.high64 == y.high64); } + friend bool operator !=(const uint128 & x, const uint128 & y) { return !(x == y); } +}; + +inline uint64 Uint128Low64(const uint128 & x) { return x.low64; } +inline uint64 Uint128High64(const uint128 & x) { return x.high64; } // Hash function for a byte array. uint64 CityHash64(const char *buf, size_t len); diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 278210d770a..662cd6bf337 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.first) + getHexUIntLowercase(expected_checksum.second) - + ". Actual: " + getHexUIntLowercase(calculated_checksum.first) + getHexUIntLowercase(calculated_checksum.second) + " Reference: " + getHexUIntLowercase(expected_checksum.low64) + getHexUIntLowercase(expected_checksum.high64) + + ". Actual: " + getHexUIntLowercase(calculated_checksum.low64) + getHexUIntLowercase(calculated_checksum.high64) + ". Size of compressed block: " + toString(size); const char * message_hardware_failure = "This is most likely due to hardware failure. " @@ -95,8 +95,8 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c } /// Check if the difference caused by single bit flip in stored checksum. - size_t difference = std::popcount(expected_checksum.first ^ calculated_checksum.first) - + std::popcount(expected_checksum.second ^ calculated_checksum.second); + size_t difference = std::popcount(expected_checksum.low64 ^ calculated_checksum.low64) + + std::popcount(expected_checksum.high64 ^ calculated_checksum.high64); if (difference == 1) { @@ -194,8 +194,8 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, { Checksum checksum; ReadBufferFromMemory checksum_in(own_compressed_buffer.data(), sizeof(checksum)); - readBinaryLittleEndian(checksum.first, checksum_in); - readBinaryLittleEndian(checksum.second, checksum_in); + readBinaryLittleEndian(checksum.low64, checksum_in); + readBinaryLittleEndian(checksum.high64, checksum_in); validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum); } @@ -238,8 +238,8 @@ size_t CompressedReadBufferBase::readCompressedDataBlockForAsynchronous(size_t & { Checksum checksum; ReadBufferFromMemory checksum_in(own_compressed_buffer.data(), sizeof(checksum)); - readBinaryLittleEndian(checksum.first, checksum_in); - readBinaryLittleEndian(checksum.second, checksum_in); + readBinaryLittleEndian(checksum.low64, checksum_in); + readBinaryLittleEndian(checksum.high64, checksum_in); validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum); } diff --git a/src/Compression/CompressedWriteBuffer.cpp b/src/Compression/CompressedWriteBuffer.cpp index cb2ee1140d0..f16330332ab 100644 --- a/src/Compression/CompressedWriteBuffer.cpp +++ b/src/Compression/CompressedWriteBuffer.cpp @@ -38,8 +38,8 @@ void CompressedWriteBuffer::nextImpl() CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(out_compressed_ptr, compressed_size); - writeBinaryLittleEndian(checksum.first, out); - writeBinaryLittleEndian(checksum.second, out); + writeBinaryLittleEndian(checksum.low64, out); + writeBinaryLittleEndian(checksum.high64, out); out.position() += compressed_size; } @@ -50,8 +50,8 @@ void CompressedWriteBuffer::nextImpl() CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(compressed_buffer.data(), compressed_size); - writeBinaryLittleEndian(checksum.first, out); - writeBinaryLittleEndian(checksum.second, out); + writeBinaryLittleEndian(checksum.low64, out); + writeBinaryLittleEndian(checksum.high64, out); out.write(compressed_buffer.data(), compressed_size); } diff --git a/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp b/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp index 018c1d863bb..d815f671652 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.first), getHexUIntLowercase(expected_checksum.second), - getHexUIntLowercase(calculated_checksum.first), getHexUIntLowercase(calculated_checksum.second)); + getHexUIntLowercase(expected_checksum.low64), getHexUIntLowercase(expected_checksum.high64), + getHexUIntLowercase(calculated_checksum.low64), getHexUIntLowercase(calculated_checksum.high64)); } /// Read the parts of the header. diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index 30776a8bc50..92e9005751e 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -400,7 +400,7 @@ void DataPartStorageOnDiskBase::backup( if (it != checksums.files.end()) { file_size = it->second.file_size; - file_hash = {it->second.file_hash.first, it->second.file_hash.second}; + file_hash = it->second.file_hash; } BackupEntryPtr backup_entry = std::make_unique(disk, filepath_on_disk, copy_encrypted, file_size, file_hash); diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index 78f68ea72fe..58ba7acb9ba 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -154,9 +154,9 @@ bool MergeTreeDataPartChecksums::readV2(ReadBuffer & in) assertString("\n\tsize: ", in); readText(sum.file_size, in); assertString("\n\thash: ", in); - readText(sum.file_hash.first, in); + readText(sum.file_hash.low64, in); assertString(" ", in); - readText(sum.file_hash.second, in); + readText(sum.file_hash.high64, in); assertString("\n\tcompressed: ", in); readText(sum.is_compressed, in); if (sum.is_compressed) @@ -164,9 +164,9 @@ bool MergeTreeDataPartChecksums::readV2(ReadBuffer & in) assertString("\n\tuncompressed size: ", in); readText(sum.uncompressed_size, in); assertString("\n\tuncompressed hash: ", in); - readText(sum.uncompressed_hash.first, in); + readText(sum.uncompressed_hash.low64, in); assertString(" ", in); - readText(sum.uncompressed_hash.second, in); + readText(sum.uncompressed_hash.high64, in); } assertChar('\n', in); diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp index 7c0aedf699b..7deae69750f 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.first) + getHexUIntUppercase(disk_checksum.second), - getHexUIntUppercase(cache_checksums[i].first) + getHexUIntUppercase(cache_checksums[i].second)); + getHexUIntUppercase(disk_checksum.low64) + getHexUIntUppercase(disk_checksum.high64), + getHexUIntUppercase(cache_checksums[i].low64) + getHexUIntUppercase(cache_checksums[i].high64)); 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.first) + getHexUIntUppercase(disk_checksum.second), - getHexUIntUppercase(cache_checksums[i].first) + getHexUIntUppercase(cache_checksums[i].second)); + getHexUIntUppercase(disk_checksum.low64) + getHexUIntUppercase(disk_checksum.high64), + getHexUIntUppercase(cache_checksums[i].low64) + getHexUIntUppercase(cache_checksums[i].high64)); disk_checksums.push_back(disk_checksum); } return results; diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 86ecb336b51..95bad0a20fe 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.first) + getHexUIntLowercase(checksum.second)); + columns[res_index++]->insert(getHexUIntLowercase(checksum.low64) + getHexUIntLowercase(checksum.high64)); } if (columns_mask[src_index++]) { auto checksum = helper.hash_of_uncompressed_files; - columns[res_index++]->insert(getHexUIntLowercase(checksum.first) + getHexUIntLowercase(checksum.second)); + columns[res_index++]->insert(getHexUIntLowercase(checksum.low64) + getHexUIntLowercase(checksum.high64)); } if (columns_mask[src_index++]) { auto checksum = helper.uncompressed_hash_of_compressed_files; - columns[res_index++]->insert(getHexUIntLowercase(checksum.first) + getHexUIntLowercase(checksum.second)); + columns[res_index++]->insert(getHexUIntLowercase(checksum.low64) + getHexUIntLowercase(checksum.high64)); } } diff --git a/src/Storages/System/StorageSystemProjectionParts.cpp b/src/Storages/System/StorageSystemProjectionParts.cpp index d2c6c3ef287..6508d062d37 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.first) + getHexUIntLowercase(checksum.second)); + columns[res_index++]->insert(getHexUIntLowercase(checksum.low64) + getHexUIntLowercase(checksum.high64)); } if (columns_mask[src_index++]) { auto checksum = helper.hash_of_uncompressed_files; - columns[res_index++]->insert(getHexUIntLowercase(checksum.first) + getHexUIntLowercase(checksum.second)); + columns[res_index++]->insert(getHexUIntLowercase(checksum.low64) + getHexUIntLowercase(checksum.high64)); } if (columns_mask[src_index++]) { auto checksum = helper.uncompressed_hash_of_compressed_files; - columns[res_index++]->insert(getHexUIntLowercase(checksum.first) + getHexUIntLowercase(checksum.second)); + columns[res_index++]->insert(getHexUIntLowercase(checksum.low64) + getHexUIntLowercase(checksum.high64)); } } From f11e1c25ee2864bd06fbd1dd5bf6009ec0a391f9 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 24 Jun 2023 17:24:54 +0000 Subject: [PATCH 055/228] added to fatal errors handler, added comments --- src/Common/Exception.h | 2 ++ src/Common/StackTrace.cpp | 15 +++++++++++++++ src/Common/StackTrace.h | 2 ++ src/Common/ThreadPool.h | 2 ++ src/Daemon/BaseDaemon.cpp | 32 +++++++++++++++++++++++++++++++- src/IO/ReadHelpers.h | 3 +++ src/IO/WriteHelpers.h | 3 +++ 7 files changed, 58 insertions(+), 1 deletion(-) diff --git a/src/Common/Exception.h b/src/Common/Exception.h index c1bbeada506..4514df2159b 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -42,6 +42,7 @@ public: message_format_string = msg.format_string; } + /// Collect call stacks of all previous jobs' schedulings leading to this thread job's execution static thread_local std::vector thread_frame_pointers; protected: @@ -140,6 +141,7 @@ private: protected: std::string_view message_format_string; + /// Local copy of static per-thread thread_frame_pointers, should be mutable to be unpoisoned on printout mutable std::vector capture_thread_frame_pointers; }; diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index e6b7839c4e2..aea0f854fe1 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -412,6 +412,21 @@ void StackTrace::toStringEveryLine(std::function callbac toStringEveryLineImpl(true, {frame_pointers, offset, size}, std::move(callback)); } +void StackTrace::toStringEveryLine(const FramePointers & frame_pointers, std::function callback) +{ + toStringEveryLineImpl(true, {frame_pointers, 0, static_cast(std::ranges::find(frame_pointers, nullptr) - frame_pointers.begin())}, std::move(callback)); +} + +void StackTrace::toStringEveryLine(void ** frame_pointers_raw, size_t offset, size_t size, std::function callback) +{ + __msan_unpoison(frame_pointers_raw, size * sizeof(*frame_pointers_raw)); + + StackTrace::FramePointers frame_pointers{}; + std::copy_n(frame_pointers_raw, size, frame_pointers.begin()); + + toStringEveryLineImpl(true, {frame_pointers, offset, size}, std::move(callback)); +} + using StackTraceCache = std::map>; static StackTraceCache & cacheInstance() diff --git a/src/Common/StackTrace.h b/src/Common/StackTrace.h index 3940c880c5b..656f543d837 100644 --- a/src/Common/StackTrace.h +++ b/src/Common/StackTrace.h @@ -65,6 +65,8 @@ public: static void symbolize(const FramePointers & frame_pointers, size_t offset, size_t size, StackTrace::Frames & frames); void toStringEveryLine(std::function callback) const; + static void toStringEveryLine(const FramePointers & frame_pointers, std::function callback); + static void toStringEveryLine(void ** frame_pointers_raw, size_t offset, size_t size, std::function callback); /// Displaying the addresses can be disabled for security reasons. /// If you turn off addresses, it will be more secure, but we will be unable to help you with debugging. diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 17731de9540..57188572a9d 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -129,6 +129,7 @@ private: Priority priority; DB::OpenTelemetry::TracingContextOnThread thread_trace_context; + /// Call stacks of all jobs' schedulings leading to this one std::vector frame_pointers; JobWithPriority(Job job_, Priority priority_, const DB::OpenTelemetry::TracingContextOnThread & thread_trace_context_, bool capture_frame_pointers = false) @@ -136,6 +137,7 @@ private: { if (!capture_frame_pointers) return; + /// Save all previous jobs call stacks and append with current frame_pointers = DB::Exception::thread_frame_pointers; frame_pointers.push_back(StackTrace().getFramePointers()); } diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index bfd5568b71d..fb5b53201a2 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -19,6 +19,7 @@ #include #include +#include #include #include #include @@ -153,6 +154,7 @@ static void signalHandler(int sig, siginfo_t * info, void * context) writePODBinary(*info, out); writePODBinary(signal_context, out); writePODBinary(stack_trace, out); + writeVectorBinary(Exception::thread_frame_pointers, out); writeBinary(static_cast(getThreadId()), out); writePODBinary(current_thread, out); @@ -250,6 +252,7 @@ public: siginfo_t info{}; ucontext_t * context{}; StackTrace stack_trace(NoCapture{}); + std::vector thread_frame_pointers; UInt32 thread_num{}; ThreadStatus * thread_ptr{}; @@ -260,12 +263,13 @@ public: } readPODBinary(stack_trace, in); + readVectorBinary(thread_frame_pointers, in); readBinary(thread_num, in); readPODBinary(thread_ptr, in); /// This allows to receive more signals if failure happens inside onFault function. /// Example: segfault while symbolizing stack trace. - std::thread([=, this] { onFault(sig, info, context, stack_trace, thread_num, thread_ptr); }).detach(); + std::thread([=, this] { onFault(sig, info, context, stack_trace, thread_frame_pointers, thread_num, thread_ptr); }).detach(); } } } @@ -300,6 +304,7 @@ private: const siginfo_t & info, ucontext_t * context, const StackTrace & stack_trace, + const std::vector & thread_frame_pointers, UInt32 thread_num, ThreadStatus * thread_ptr) const { @@ -375,6 +380,31 @@ private: /// Write symbolized stack trace line by line for better grep-ability. stack_trace.toStringEveryLine([&](std::string_view s) { LOG_FATAL(log, fmt::runtime(s)); }); + /// In case it's a scheduled job write all previous jobs origins call stacks + std::for_each(thread_frame_pointers.rbegin(), thread_frame_pointers.rend(), + [this](const StackTrace::FramePointers & frame_pointers) + { + if (size_t size = std::ranges::find(frame_pointers, nullptr) - frame_pointers.begin()) + { + LOG_FATAL(log, "========================================"); + WriteBufferFromOwnString bare_stacktrace; + writeString("Job's origin stack trace:", bare_stacktrace); + std::for_each_n(frame_pointers.begin(), size, + [&bare_stacktrace](const void * ptr) + { + writeChar(' ', bare_stacktrace); + writePointerHex(ptr, bare_stacktrace); + } + ); + + LOG_FATAL(log, fmt::runtime(bare_stacktrace.str())); + + StackTrace::toStringEveryLine(const_cast(&frame_pointers[0]), 0, size, [this](std::string_view s) { LOG_FATAL(log, fmt::runtime(s)); }); + } + } + ); + + #if defined(OS_LINUX) /// Write information about binary checksum. It can be difficult to calculate, so do it only after printing stack trace. /// Please keep the below log messages in-sync with the ones in programs/server/Server.cpp diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index dabb21c046c..4a05a106de0 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -12,6 +12,7 @@ #include +#include #include #include #include @@ -1093,6 +1094,8 @@ inline void readBinary(Decimal128 & x, ReadBuffer & buf) { readPODBinary(x, buf) inline void readBinary(Decimal256 & x, ReadBuffer & buf) { readPODBinary(x.value, buf); } inline void readBinary(LocalDate & x, ReadBuffer & buf) { readPODBinary(x, buf); } +inline void readBinary(StackTrace::FramePointers & x, ReadBuffer & buf) { readPODBinary(x, buf); } + template inline void readBinaryEndian(T & x, ReadBuffer & buf) { diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 3d1a5aa49ef..ade7956a67d 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -10,6 +10,7 @@ #include +#include #include #include #include @@ -868,6 +869,8 @@ inline void writeBinary(const UUID & x, WriteBuffer & buf) { writePODBinary(x, b inline void writeBinary(const IPv4 & x, WriteBuffer & buf) { writePODBinary(x, buf); } inline void writeBinary(const IPv6 & x, WriteBuffer & buf) { writePODBinary(x, buf); } +inline void writeBinary(const StackTrace::FramePointers & x, WriteBuffer & buf) { writePODBinary(x, buf); } + /// Methods for outputting the value in text form for a tab-separated format. inline void writeText(is_integer auto x, WriteBuffer & buf) From e0485778f54e02d2f4b6ead1b1b3142023b569b1 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 24 Jun 2023 19:37:00 +0000 Subject: [PATCH 056/228] style fix --- src/Daemon/BaseDaemon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index fb5b53201a2..f92dc82476d 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -403,7 +403,7 @@ private: } } ); - + #if defined(OS_LINUX) /// Write information about binary checksum. It can be difficult to calculate, so do it only after printing stack trace. From 8763c601d227a527566fe7f08647e86be205bde5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 25 Jun 2023 04:22:06 +0000 Subject: [PATCH 057/228] 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 d377826513b..b316705245a 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 535fef97d5d36180ac55ef76af3c480daa46c90c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 25 Jun 2023 04:44:21 +0000 Subject: [PATCH 058/228] clang-tidy fix --- src/Daemon/BaseDaemon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index f92dc82476d..634ef5e7f0a 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -399,7 +399,7 @@ private: LOG_FATAL(log, fmt::runtime(bare_stacktrace.str())); - StackTrace::toStringEveryLine(const_cast(&frame_pointers[0]), 0, size, [this](std::string_view s) { LOG_FATAL(log, fmt::runtime(s)); }); + StackTrace::toStringEveryLine(const_cast(frame_pointers.data()), 0, size, [this](std::string_view s) { LOG_FATAL(log, fmt::runtime(s)); }); } } ); From cc3d27c0a30e40532541324b100f2584327b5ba3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 25 Jun 2023 15:14:29 +0000 Subject: [PATCH 059/228] 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 060/228] 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 061/228] 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 062/228] 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 cf2e110c134c6934fd22e0c9ec5a598ac5b107cc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 05:20:35 +0200 Subject: [PATCH 063/228] 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 ae08fb20198a953a31ea5930e5890eded39b0642 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 26 Jun 2023 12:02:09 +0200 Subject: [PATCH 064/228] 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 065/228] 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 7d8d19d8003ebaaac910a6af802ee4874e1821f8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 26 Jun 2023 14:27:13 +0200 Subject: [PATCH 066/228] 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 067/228] 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 068/228] 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 047f656980f01cc339043c1b5ba225bb6f4ce07e Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 26 Jun 2023 15:04:50 +0200 Subject: [PATCH 069/228] 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 d4d85d9fe92430e08c1006f2d6c9d935445473e8 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 26 Jun 2023 16:03:47 +0200 Subject: [PATCH 070/228] 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 071/228] 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 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 072/228] 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 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 073/228] 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 074/228] 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 075/228] 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 076/228] 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 648b647f5f878c4da0265cfae3dc39e50021170b Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Jun 2023 15:49:26 +0200 Subject: [PATCH 077/228] 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 078/228] 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 079/228] 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 080/228] New epoch From 0b6d367bdbe73e1ea9b1f179f315cba2185ffe94 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 27 Jun 2023 17:14:49 +0000 Subject: [PATCH 081/228] Parts mover: lock between getActiveContainingPart and swapActivePart --- src/Storages/MergeTree/MergeTreeData.cpp | 9 +++++++-- src/Storages/MergeTree/MergeTreeData.h | 3 ++- src/Storages/MergeTree/MergeTreePartsMover.cpp | 7 +++++-- 3 files changed, 14 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e9c3a7f66ae..5e17559acc0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4529,9 +4529,8 @@ MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart( } -void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy) +void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy, DataPartsLock &) { - auto lock = lockParts(); for (auto original_active_part : getDataPartsStateRange(DataPartState::Active)) // NOLINT (copy is intended) { if (part_copy->name == original_active_part->name) @@ -4587,6 +4586,12 @@ MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(const String & return getActiveContainingPart(part_info); } +MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(const String & part_name, DataPartsLock & lock) const +{ + auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); + return getActiveContainingPart(part_info, DataPartState::Active, lock); +} + MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVectorInPartition(ContextPtr local_context, const String & partition_id) const { return getVisibleDataPartsVectorInPartition(local_context->getCurrentTransaction().get(), partition_id); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index b27392b355b..c821a436a76 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -504,12 +504,13 @@ public: /// Returns a part in Active state with the given name or a part containing it. If there is no such part, returns nullptr. DataPartPtr getActiveContainingPart(const String & part_name) const; + DataPartPtr getActiveContainingPart(const String & part_name, DataPartsLock & lock) const; DataPartPtr getActiveContainingPart(const MergeTreePartInfo & part_info) const; DataPartPtr getActiveContainingPart(const MergeTreePartInfo & part_info, DataPartState state, DataPartsLock & lock) const; /// Swap part with it's identical copy (possible with another path on another disk). /// If original part is not active or doesn't exist exception will be thrown. - void swapActivePart(MergeTreeData::DataPartPtr part_copy); + void swapActivePart(MergeTreeData::DataPartPtr part_copy, DataPartsLock &); /// Returns all parts in specified partition DataPartsVector getVisibleDataPartsVectorInPartition(MergeTreeTransaction * txn, const String & partition_id, DataPartsLock * acquired_lock = nullptr) const; diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 8fa4ac6c78a..a8f34ba4cec 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -263,7 +263,10 @@ void MergeTreePartsMover::swapClonedPart(TemporaryClonedPart & cloned_part) cons if (moves_blocker.isCancelled()) throw Exception(ErrorCodes::ABORTED, "Cancelled moving parts."); - auto active_part = data->getActiveContainingPart(cloned_part.part->name); + /// `getActiveContainingPart` and `swapActivePart` are called under the same lock + /// to prevent part becoming inactive between calls + auto part_lock = data->lockParts(); + auto active_part = data->getActiveContainingPart(cloned_part.part->name, part_lock); /// It's ok, because we don't block moving parts for merges or mutations if (!active_part || active_part->name != cloned_part.part->name) @@ -284,7 +287,7 @@ void MergeTreePartsMover::swapClonedPart(TemporaryClonedPart & cloned_part) cons cloned_part.part->renameTo(active_part->name, false); /// TODO what happen if server goes down here? - data->swapActivePart(cloned_part.part); + data->swapActivePart(cloned_part.part, part_lock); LOG_TRACE(log, "Part {} was moved to {}", cloned_part.part->name, cloned_part.part->getDataPartStorage().getFullPath()); From 79b6792548c065d6795c5a167b29da44aa91dae7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 27 Jun 2023 17:15:11 +0000 Subject: [PATCH 082/228] Enable allow_remove_stale_moving_parts for stateless tests --- tests/config/config.d/merge_tree_old_dirs_cleanup.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/config.d/merge_tree_old_dirs_cleanup.xml b/tests/config/config.d/merge_tree_old_dirs_cleanup.xml index 2b8ea63b63d..e6b50724c97 100644 --- a/tests/config/config.d/merge_tree_old_dirs_cleanup.xml +++ b/tests/config/config.d/merge_tree_old_dirs_cleanup.xml @@ -5,4 +5,5 @@ 5 + true 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 083/228] 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 f5327e79bb37c3cc061b8704ffdb85aa4f0b31c4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 27 Jun 2023 20:18:38 +0200 Subject: [PATCH 084/228] 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 085/228] 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 086/228] 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 087/228] 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 088/228] 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 089/228] 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 090/228] 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 091/228] 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 092/228] 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 0e72c6dcc934a52184909e3c6b32b084678c185a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 28 Jun 2023 10:33:07 +0200 Subject: [PATCH 093/228] 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 1596b09dc2888690acafa397936d0d7051e95fba Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 28 Jun 2023 10:51:15 +0200 Subject: [PATCH 094/228] 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 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 095/228] 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 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 096/228] 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 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 097/228] 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 098/228] 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 099/228] 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 100/228] 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 101/228] 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 102/228] 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 103/228] 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 104/228] 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 105/228] 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 106/228] 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 107/228] 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 108/228] 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 372f5786c42fd3e96bbcafb80012c04fdaa96bbc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 28 Jun 2023 18:44:54 +0200 Subject: [PATCH 109/228] 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 110/228] 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 111/228] 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 112/228] 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 113/228] 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 114/228] 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 53a30bee8d66c06f69f53b8850a11b8fbde2e1d4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 28 Jun 2023 19:16:10 +0000 Subject: [PATCH 115/228] 00900_orc_load: kill less aggressively The test sporadically produces wrong results because the INSERT takes longer than 3 seconds, likely due to infrastructure latency. Removing the timeout to give it more headroom. Tests are afaik auto-killed after 10 min, so if there is a true issue in ClickHouse (e.g. deadlock), we would notice anyways. --- tests/queries/0_stateless/00900_orc_load.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00900_orc_load.sh b/tests/queries/0_stateless/00900_orc_load.sh index 62149fa554e..5dc5dfb87a7 100755 --- a/tests/queries/0_stateless/00900_orc_load.sh +++ b/tests/queries/0_stateless/00900_orc_load.sh @@ -12,6 +12,6 @@ ${CLICKHOUSE_CLIENT} --query="select * from orc_load FORMAT ORC" > "${CLICKHOUSE ${CLICKHOUSE_CLIENT} --query="truncate table orc_load" cat "${CLICKHOUSE_TMP}"/test.orc | ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" -timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" < "${CLICKHOUSE_TMP}"/test.orc +${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" < "${CLICKHOUSE_TMP}"/test.orc ${CLICKHOUSE_CLIENT} --query="select * from orc_load" ${CLICKHOUSE_CLIENT} --query="drop table orc_load" From 8854f05a7534fc844b5a7b00af72ad0ae7c0ac1b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 28 Jun 2023 19:53:49 +0000 Subject: [PATCH 116/228] 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 117/228] 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 118/228] 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 119/228] 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 120/228] 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 121/228] 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 122/228] 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 123/228] 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 124/228] 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 125/228] 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 126/228] 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 127/228] 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 128/228] 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 129/228] 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 130/228] 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 131/228] 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 132/228] 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 133/228] 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 296f9968c04f6ca49599e281b4092f6cffef5bfd Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 28 Jun 2023 23:20:02 +0200 Subject: [PATCH 134/228] 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 45232770e06079b2b346209ae7b412597ff5996f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 29 Jun 2023 00:41:23 +0300 Subject: [PATCH 135/228] 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 136/228] 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 137/228] 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 340262814a721ca7be1523ff1194c404b0f03de8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 29 Jun 2023 12:24:19 +0200 Subject: [PATCH 138/228] 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 139/228] 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 140/228] 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 141/228] 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 142/228] 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 143/228] 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 144/228] 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 145/228] 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 146/228] 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 147/228] 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 148/228] 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 149/228] 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 150/228] 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 151/228] 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 152/228] 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 153/228] 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 154/228] 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 f12c257dc95de8188e2be8a70344b0bd0b1f6204 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 29 Jun 2023 19:25:46 +0200 Subject: [PATCH 155/228] Fix --- src/Interpreters/Cache/FileCache.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index dc1f012f5c6..463d0d6605c 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -1017,7 +1017,8 @@ void FileCache::deactivateBackgroundOperations() metadata.cancelDownload(); for (auto & thread : download_threads) - thread.join(); + if (thread.joinable()) + thread.join(); } void FileCache::cleanup() From 7d416cbc60c89ce34e03e7e9de5045e0120af119 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 29 Jun 2023 13:39:39 -0400 Subject: [PATCH 156/228] 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 157/228] 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 158/228] 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 159/228] 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 160/228] 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 161/228] 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 162/228] 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 163/228] 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 164/228] 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 b0d4c9c83b210f9d266cc448df4641a07fdfb08c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 03:04:48 +0200 Subject: [PATCH 165/228] Disable hedged requests under TSan --- src/Core/SettingsQuirks.cpp | 9 ++- .../integration/test_hedged_requests/test.py | 75 +++++++++++++++++++ .../test_hedged_requests_parallel/test.py | 25 +++++++ tests/integration/test_secure_socket/test.py | 5 ++ ...851_hedged_connections_external_tables.sql | 1 + 5 files changed, 114 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsQuirks.cpp b/src/Core/SettingsQuirks.cpp index 3326f42adf5..5b981c71403 100644 --- a/src/Core/SettingsQuirks.cpp +++ b/src/Core/SettingsQuirks.cpp @@ -1,10 +1,11 @@ +#include #include #include #include #include #include #include -#include + namespace { @@ -71,6 +72,12 @@ void applySettingsQuirks(Settings & settings, Poco::Logger * log) } } +#if defined(THREAD_SANITIZER) + settings.use_hedged_requests = false; + if (log) + LOG_WARNING(log, "use_hedged_requests has been disabled for the build with Thread Sanitizer, because they are using fibers, leading to a failed assertion inside TSan"); +#endif + if (!queryProfilerWorks()) { if (settings.query_profiler_real_time_period_ns) diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index be6cea80f87..5de92f437c5 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -203,6 +203,11 @@ def update_configs( def test_stuck_replica(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs() cluster.pause_container("node_1") @@ -233,6 +238,11 @@ def test_stuck_replica(started_cluster): def test_long_query(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs() # Restart to reset pool states. @@ -249,12 +259,22 @@ def test_long_query(started_cluster): def test_send_table_status_sleep(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs(node_1_sleep_in_send_tables_status=sleep_time) check_query(expected_replica="node_2") check_changing_replica_events(1) def test_send_table_status_sleep2(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_tables_status=sleep_time, node_2_sleep_in_send_tables_status=sleep_time, @@ -264,12 +284,22 @@ def test_send_table_status_sleep2(started_cluster): def test_send_data(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs(node_1_sleep_in_send_data=sleep_time) check_query(expected_replica="node_2") check_changing_replica_events(1) def test_send_data2(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_data=sleep_time ) @@ -278,6 +308,11 @@ def test_send_data2(started_cluster): def test_combination1(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_tables_status=sleep_time, node_2_sleep_in_send_data=sleep_time, @@ -287,6 +322,11 @@ def test_combination1(started_cluster): def test_combination2(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_tables_status=sleep_time, @@ -296,6 +336,11 @@ def test_combination2(started_cluster): def test_combination3(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_tables_status=1000, @@ -306,6 +351,11 @@ def test_combination3(started_cluster): def test_combination4(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_tables_status=1000, node_1_sleep_in_send_data=sleep_time, @@ -317,6 +367,11 @@ def test_combination4(started_cluster): def test_receive_timeout1(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + # Check the situation when first two replicas get receive timeout # in establishing connection, but the third replica is ok. update_configs( @@ -329,6 +384,11 @@ def test_receive_timeout1(started_cluster): def test_receive_timeout2(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + # Check the situation when first replica get receive timeout # in packet receiving but there are replicas in process of # connection establishing. @@ -342,6 +402,11 @@ def test_receive_timeout2(started_cluster): def test_initial_receive_timeout(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + # Check the situation when replicas don't respond after # receiving query (so, no packets were send to initiator) update_configs( @@ -360,6 +425,11 @@ def test_initial_receive_timeout(started_cluster): def test_async_connect(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs() NODES["node"].restart_clickhouse() @@ -390,6 +460,11 @@ def test_async_connect(started_cluster): def test_async_query_sending(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_after_receiving_query=5000, node_2_sleep_after_receiving_query=5000, diff --git a/tests/integration/test_hedged_requests_parallel/test.py b/tests/integration/test_hedged_requests_parallel/test.py index 492b869614f..34cad7bb754 100644 --- a/tests/integration/test_hedged_requests_parallel/test.py +++ b/tests/integration/test_hedged_requests_parallel/test.py @@ -172,6 +172,11 @@ def update_configs( def test_send_table_status_sleep(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_tables_status=sleep_time, node_2_sleep_in_send_tables_status=sleep_time, @@ -181,6 +186,11 @@ def test_send_table_status_sleep(started_cluster): def test_send_data(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_data=sleep_time ) @@ -189,6 +199,11 @@ def test_send_data(started_cluster): def test_combination1(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_tables_status=1000, node_2_sleep_in_send_tables_status=1000, @@ -199,6 +214,11 @@ def test_combination1(started_cluster): def test_combination2(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_tables_status=1000, @@ -210,6 +230,11 @@ def test_combination2(started_cluster): def test_query_with_no_data_to_sample(started_cluster): + if NODES["node"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_data=sleep_time ) diff --git a/tests/integration/test_secure_socket/test.py b/tests/integration/test_secure_socket/test.py index 2dffbed03d6..827e4b81cfc 100644 --- a/tests/integration/test_secure_socket/test.py +++ b/tests/integration/test_secure_socket/test.py @@ -58,6 +58,11 @@ def test(started_cluster): config.format(sleep_in_send_data_ms=1000000), ) + if NODES["node1"].is_built_with_thread_sanitizer(): + pytest.skip( + "Hedged requests don't work under Thread Sanitizer" + ) + attempts = 0 while attempts < 1000: setting = NODES["node2"].http_query( diff --git a/tests/queries/0_stateless/01851_hedged_connections_external_tables.sql b/tests/queries/0_stateless/01851_hedged_connections_external_tables.sql index c4625720e59..22888d5e68c 100644 --- a/tests/queries/0_stateless/01851_hedged_connections_external_tables.sql +++ b/tests/queries/0_stateless/01851_hedged_connections_external_tables.sql @@ -1 +1,2 @@ +-- Tags: no-tsan select number from remote('127.0.0.{3|2}', numbers(2)) where number global in (select number from numbers(1)) settings async_socket_for_remote=1, use_hedged_requests = 1, sleep_in_send_data_ms=10, receive_data_timeout_ms=1; From 6b7c17fb4c863b5145dca71daaf8c14cf4fa8a42 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 26 Jun 2023 01:24:09 +0000 Subject: [PATCH 166/228] Automatic style fix --- .../integration/test_hedged_requests/test.py | 62 +++++-------------- .../test_hedged_requests_parallel/test.py | 20 ++---- tests/integration/test_secure_socket/test.py | 4 +- 3 files changed, 22 insertions(+), 64 deletions(-) diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index 5de92f437c5..18ea3e50619 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -204,9 +204,7 @@ def update_configs( def test_stuck_replica(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs() @@ -239,9 +237,7 @@ def test_stuck_replica(started_cluster): def test_long_query(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs() @@ -260,9 +256,7 @@ def test_long_query(started_cluster): def test_send_table_status_sleep(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs(node_1_sleep_in_send_tables_status=sleep_time) check_query(expected_replica="node_2") @@ -271,9 +265,7 @@ def test_send_table_status_sleep(started_cluster): def test_send_table_status_sleep2(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_tables_status=sleep_time, @@ -285,9 +277,7 @@ def test_send_table_status_sleep2(started_cluster): def test_send_data(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs(node_1_sleep_in_send_data=sleep_time) check_query(expected_replica="node_2") @@ -296,9 +286,7 @@ def test_send_data(started_cluster): def test_send_data2(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_data=sleep_time @@ -309,9 +297,7 @@ def test_send_data2(started_cluster): def test_combination1(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_tables_status=sleep_time, @@ -323,9 +309,7 @@ def test_combination1(started_cluster): def test_combination2(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_data=sleep_time, @@ -337,9 +321,7 @@ def test_combination2(started_cluster): def test_combination3(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_data=sleep_time, @@ -352,9 +334,7 @@ def test_combination3(started_cluster): def test_combination4(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_tables_status=1000, @@ -368,9 +348,7 @@ def test_combination4(started_cluster): def test_receive_timeout1(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") # Check the situation when first two replicas get receive timeout # in establishing connection, but the third replica is ok. @@ -385,9 +363,7 @@ def test_receive_timeout1(started_cluster): def test_receive_timeout2(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") # Check the situation when first replica get receive timeout # in packet receiving but there are replicas in process of @@ -403,9 +379,7 @@ def test_receive_timeout2(started_cluster): def test_initial_receive_timeout(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") # Check the situation when replicas don't respond after # receiving query (so, no packets were send to initiator) @@ -426,9 +400,7 @@ def test_initial_receive_timeout(started_cluster): def test_async_connect(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs() @@ -461,10 +433,8 @@ def test_async_connect(started_cluster): def test_async_query_sending(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) - + pytest.skip("Hedged requests don't work under Thread Sanitizer") + update_configs( node_1_sleep_after_receiving_query=5000, node_2_sleep_after_receiving_query=5000, diff --git a/tests/integration/test_hedged_requests_parallel/test.py b/tests/integration/test_hedged_requests_parallel/test.py index 34cad7bb754..728697c690d 100644 --- a/tests/integration/test_hedged_requests_parallel/test.py +++ b/tests/integration/test_hedged_requests_parallel/test.py @@ -173,9 +173,7 @@ def update_configs( def test_send_table_status_sleep(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_tables_status=sleep_time, @@ -187,9 +185,7 @@ def test_send_table_status_sleep(started_cluster): def test_send_data(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_data=sleep_time @@ -200,9 +196,7 @@ def test_send_data(started_cluster): def test_combination1(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_tables_status=1000, @@ -215,9 +209,7 @@ def test_combination1(started_cluster): def test_combination2(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_data=sleep_time, @@ -231,9 +223,7 @@ def test_combination2(started_cluster): def test_query_with_no_data_to_sample(started_cluster): if NODES["node"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") update_configs( node_1_sleep_in_send_data=sleep_time, node_2_sleep_in_send_data=sleep_time diff --git a/tests/integration/test_secure_socket/test.py b/tests/integration/test_secure_socket/test.py index 827e4b81cfc..123715e5f05 100644 --- a/tests/integration/test_secure_socket/test.py +++ b/tests/integration/test_secure_socket/test.py @@ -59,9 +59,7 @@ def test(started_cluster): ) if NODES["node1"].is_built_with_thread_sanitizer(): - pytest.skip( - "Hedged requests don't work under Thread Sanitizer" - ) + pytest.skip("Hedged requests don't work under Thread Sanitizer") attempts = 0 while attempts < 1000: From a628bbb1f55bea23662ff65512b2e892310c13af Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 28 Jun 2023 13:58:32 +0200 Subject: [PATCH 167/228] Fix tests --- src/Core/SettingsQuirks.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsQuirks.cpp b/src/Core/SettingsQuirks.cpp index 5b981c71403..37a0f2db3e2 100644 --- a/src/Core/SettingsQuirks.cpp +++ b/src/Core/SettingsQuirks.cpp @@ -73,7 +73,7 @@ void applySettingsQuirks(Settings & settings, Poco::Logger * log) } #if defined(THREAD_SANITIZER) - settings.use_hedged_requests = false; + settings.use_hedged_requests.value = false; if (log) LOG_WARNING(log, "use_hedged_requests has been disabled for the build with Thread Sanitizer, because they are using fibers, leading to a failed assertion inside TSan"); #endif From 24ae56b5dd383ab4577747a873d9bc4aae9f7f16 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 30 Jun 2023 11:02:16 +0000 Subject: [PATCH 168/228] 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 3b73e112165833e8baece650021adbb19e8f635f Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 30 Jun 2023 11:04:40 +0000 Subject: [PATCH 169/228] Fix tests --- tests/queries/0_stateless/01361_fover_remote_num_tries.sh | 2 +- ...6_skip_unavailable_shards_excessive_attempts.reference | 8 ++++---- .../01956_skip_unavailable_shards_excessive_attempts.sh | 2 +- .../02226_parallel_reading_from_replicas_benchmark.sh | 2 ++ 4 files changed, 8 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01361_fover_remote_num_tries.sh b/tests/queries/0_stateless/01361_fover_remote_num_tries.sh index 2ee2ec1bc76..f07ffc02e4f 100755 --- a/tests/queries/0_stateless/01361_fover_remote_num_tries.sh +++ b/tests/queries/0_stateless/01361_fover_remote_num_tries.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --connections_with_failover_max_tries 10 --query "SELECT hostName() FROM remote('128.1.2.3', default.tmp)" 2>&1 | grep -o -P 'Timeout exceeded while connecting to socket|Network is unreachable' | wc -l +$CLICKHOUSE_CLIENT --connections_with_failover_max_tries 10 --query "SELECT hostName() FROM remote('128.1.2.3', default.tmp)" 2>&1 | grep -o -P 'Timeout exceeded while connecting to socket|Network is unreachable|Timeout: connect timed out' | wc -l diff --git a/tests/queries/0_stateless/01956_skip_unavailable_shards_excessive_attempts.reference b/tests/queries/0_stateless/01956_skip_unavailable_shards_excessive_attempts.reference index e39f4b962e6..ad409aac5b8 100644 --- a/tests/queries/0_stateless/01956_skip_unavailable_shards_excessive_attempts.reference +++ b/tests/queries/0_stateless/01956_skip_unavailable_shards_excessive_attempts.reference @@ -1,10 +1,10 @@ 255.255.255.255 -HedgedConnectionsFactory: Connection failed at try №1 +ConnectionPoolWithFailover: Connection failed at try №1 executeQuery: Code: 519.: All attempts to get table structure failed. 127.2,255.255.255.255 0 -HedgedConnectionsFactory: Connection failed at try №1 +ConnectionPoolWithFailover: Connection failed at try №1 255.255.255.255,127.2 0 -HedgedConnectionsFactory: Connection failed at try №1 -HedgedConnectionsFactory: Connection failed at try №1 +ConnectionPoolWithFailover: Connection failed at try №1 +ConnectionPoolWithFailover: Connection failed at try №1 diff --git a/tests/queries/0_stateless/01956_skip_unavailable_shards_excessive_attempts.sh b/tests/queries/0_stateless/01956_skip_unavailable_shards_excessive_attempts.sh index 488e2fe106a..9f9de96ca6e 100755 --- a/tests/queries/0_stateless/01956_skip_unavailable_shards_excessive_attempts.sh +++ b/tests/queries/0_stateless/01956_skip_unavailable_shards_excessive_attempts.sh @@ -25,7 +25,7 @@ function execute_query() # clickhouse-client 2> >(wc -l) # # May dump output of "wc -l" after some other programs. - $CLICKHOUSE_CLIENT "${opts[@]}" --query "select * from remote('$hosts', system.one)" 2>"$stderr" + $CLICKHOUSE_CLIENT "${opts[@]}" --query "select * from remote('$hosts', system.one) settings use_hedged_requests=0" 2>"$stderr" process_log_safe "$stderr" } execute_query 255.255.255.255 diff --git a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh index 4b8f8da5480..941f024825a 100755 --- a/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh +++ b/tests/queries/0_stateless/02226_parallel_reading_from_replicas_benchmark.sh @@ -17,6 +17,8 @@ opts=( --allow_experimental_parallel_reading_from_replicas 1 --parallel_replicas_for_non_replicated_merge_tree 1 --max_parallel_replicas 3 + --use_hedged_requests 0 + --cluster_for_parallel_replicas parallel_replicas --iterations 1 ) 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 170/228] 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 f134153f4e39befdbed703eda439d96e7d539a49 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 30 Jun 2023 13:12:04 +0200 Subject: [PATCH 171/228] Fix race with "user_was_dropped" in ContextAccess. --- src/Access/ContextAccess.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 63604a03b4e..1259d8d72eb 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -221,7 +221,7 @@ private: mutable Poco::Logger * trace_log = nullptr; mutable UserPtr user; mutable String user_name; - mutable bool user_was_dropped = false; + mutable std::atomic user_was_dropped = false; mutable scope_guard subscription_for_user_change; mutable std::shared_ptr enabled_roles; mutable scope_guard subscription_for_roles_changes; From 58f291997176b230395bc906fe642c5dfad4657c Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Jun 2023 12:46:45 +0200 Subject: [PATCH 172/228] 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 173/228] 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 174/228] 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 175/228] 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 176/228] 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 177/228] 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 2311dd9b5ddbeb8587ce7201b889c0ab1442d003 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 30 Jun 2023 15:18:54 +0200 Subject: [PATCH 178/228] Use TSA_GUARDED_BY() macro in the definition of ContextAccess. --- src/Access/ContextAccess.cpp | 16 +++++++------- src/Access/ContextAccess.h | 43 ++++++++++++++++++++++-------------- 2 files changed, 34 insertions(+), 25 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 0d28edc0a10..9c57853679f 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -228,6 +228,12 @@ ContextAccess::ContextAccess(const AccessControl & access_control_, const Params } +ContextAccess::ContextAccess(FullAccess) + : is_full_access(true), access(std::make_shared(AccessRights::getFullAccess())), access_with_implicit(access) +{ +} + + ContextAccess::~ContextAccess() { enabled_settings.reset(); @@ -413,14 +419,8 @@ std::optional ContextAccess::getQuotaUsage() const std::shared_ptr ContextAccess::getFullAccess() { - static const std::shared_ptr res = [] - { - auto full_access = std::make_shared(); - full_access->is_full_access = true; - full_access->access = std::make_shared(AccessRights::getFullAccess()); - full_access->access_with_implicit = full_access->access; - return full_access; - }(); + static const std::shared_ptr res = + [] { return std::shared_ptr(new ContextAccess{kFullAccess}); }(); return res; } diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 1259d8d72eb..60bad0118fc 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -69,7 +69,6 @@ public: using Params = ContextAccessParams; const Params & getParams() const { return params; } - ContextAccess() { } /// NOLINT ContextAccess(const AccessControl & access_control_, const Params & params_); /// Returns the current user. Throws if user is nullptr. @@ -171,10 +170,17 @@ public: private: friend class AccessControl; + struct FullAccess {}; + static const FullAccess kFullAccess; + + /// Makes an instance of ContextAccess which provides full access to everything + /// without any limitations. This is used for the global context. + explicit ContextAccess(FullAccess); + void initialize(); - void setUser(const UserPtr & user_) const; - void setRolesInfo(const std::shared_ptr & roles_info_) const; - void calculateAccessRights() const; + void setUser(const UserPtr & user_) const TSA_REQUIRES(mutex); + void setRolesInfo(const std::shared_ptr & roles_info_) const TSA_REQUIRES(mutex); + void calculateAccessRights() const TSA_REQUIRES(mutex); template bool checkAccessImpl(const AccessFlags & flags) const; @@ -217,20 +223,23 @@ private: const AccessControl * access_control = nullptr; const Params params; - bool is_full_access = false; - mutable Poco::Logger * trace_log = nullptr; - mutable UserPtr user; - mutable String user_name; + const bool is_full_access = false; + mutable std::atomic user_was_dropped = false; - mutable scope_guard subscription_for_user_change; - mutable std::shared_ptr enabled_roles; - mutable scope_guard subscription_for_roles_changes; - mutable std::shared_ptr roles_info; - mutable std::shared_ptr access; - mutable std::shared_ptr access_with_implicit; - mutable std::shared_ptr enabled_row_policies; - mutable std::shared_ptr enabled_quota; - mutable std::shared_ptr enabled_settings; + mutable std::atomic trace_log = nullptr; + + mutable UserPtr user TSA_GUARDED_BY(mutex); + mutable String user_name TSA_GUARDED_BY(mutex); + mutable scope_guard subscription_for_user_change TSA_GUARDED_BY(mutex); + mutable std::shared_ptr enabled_roles TSA_GUARDED_BY(mutex); + mutable scope_guard subscription_for_roles_changes TSA_GUARDED_BY(mutex); + mutable std::shared_ptr roles_info TSA_GUARDED_BY(mutex); + mutable std::shared_ptr access TSA_GUARDED_BY(mutex); + mutable std::shared_ptr access_with_implicit TSA_GUARDED_BY(mutex); + mutable std::shared_ptr enabled_row_policies TSA_GUARDED_BY(mutex); + mutable std::shared_ptr enabled_quota TSA_GUARDED_BY(mutex); + mutable std::shared_ptr enabled_settings TSA_GUARDED_BY(mutex); + mutable std::mutex mutex; }; From 904c533a84e5766b4e5ca5e886d7a826cefdd0d4 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 30 Jun 2023 09:32:54 -0400 Subject: [PATCH 179/228] 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 180/228] 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 181/228] 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 182/228] 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 183/228] 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 184/228] 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 185/228] 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 186/228] 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 187/228] 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 188/228] 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 189/228] 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 190/228] 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 3f29bd1bd42f7ebff737d2196c285aa7be4c6989 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Jun 2023 18:20:59 +0200 Subject: [PATCH 191/228] Try fix test --- tests/queries/0_stateless/02789_filesystem_cache_alignment.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh index b9f9548a7b3..509d1a635b1 100755 --- a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh +++ b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel, no-random-settings +# Tags: no-fasttest, no-parallel, no-random-settings, no-random-merge-tree-settings CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh 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 192/228] 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 193/228] 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 194/228] 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 195/228] 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 196/228] 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 197/228] 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 198/228] 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 199/228] 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 200/228] 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 201/228] 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 202/228] 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 203/228] 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 204/228] 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 292eec247058c6020c48d0d4c3ebd27784fed466 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 3 Jul 2023 09:40:36 +0200 Subject: [PATCH 205/228] Run cargo update to fix build with nightly --- rust/skim/Cargo.lock | 360 ++++++++++++++++++++++++++----------------- 1 file changed, 218 insertions(+), 142 deletions(-) diff --git a/rust/skim/Cargo.lock b/rust/skim/Cargo.lock index badd22dad07..9f948ee1c38 100644 --- a/rust/skim/Cargo.lock +++ b/rust/skim/Cargo.lock @@ -14,13 +14,19 @@ dependencies = [ [[package]] name = "aho-corasick" -version = "0.7.20" +version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc936419f96fa211c1b9166887b38e5e40b19958e5b895be7c1f93adec7071ac" +checksum = "43f6cb1bf222025340178f382c426f13757b2960e89779dfcb319c32542a5a41" dependencies = [ "memchr", ] +[[package]] +name = "android-tzdata" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e999941b234f3131b00bc13c22d06e8c5ff726d1b6318ac7eb276997bbb4fef0" + [[package]] name = "android_system_properties" version = "0.1.5" @@ -32,9 +38,9 @@ dependencies = [ [[package]] name = "arrayvec" -version = "0.7.2" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8da52d66c7071e2e3fa2a1e5c6d088fec47b593032b254f5e980de8ea54454d6" +checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "atty" @@ -42,7 +48,7 @@ version = "0.2.14" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d9b39be18770d11421cdb1b9947a45dd3f37e93092cbf377614828a319d5fee8" dependencies = [ - "hermit-abi", + "hermit-abi 0.1.19", "libc", "winapi", ] @@ -67,15 +73,15 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bumpalo" -version = "3.11.1" +version = "3.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "572f695136211188308f16ad2ca5c851a712c464060ae6974944458eb83880ba" +checksum = "a3e2c3daef883ecc1b5d58c15adae93470a91d425f3532ba1695849656af3fc1" [[package]] name = "cc" -version = "1.0.77" +version = "1.0.79" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9f73505338f7d905b19d18738976aae232eb46b8efc15554ffc56deb5d9ebe4" +checksum = "50d30906286121d95be3d479533b458f87493b30a4b5f79a607db8f5d11aa91f" [[package]] name = "cfg-if" @@ -85,13 +91,13 @@ checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" [[package]] name = "chrono" -version = "0.4.23" +version = "0.4.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16b0a3d9ed01224b22057780a37bb8c5dbfe1be8ba48678e7bf57ec4b385411f" +checksum = "ec837a71355b28f6556dbd569b37b3f363091c0bd4b2e735674521b4c5fd9bc5" dependencies = [ + "android-tzdata", "iana-time-zone", "js-sys", - "num-integer", "num-traits", "time 0.1.45", "wasm-bindgen", @@ -100,9 +106,9 @@ dependencies = [ [[package]] name = "clap" -version = "3.2.23" +version = "3.2.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "71655c45cb9845d3270c9d6df84ebe72b4dad3c2ba3f7023ad47c144e4e473a5" +checksum = "4ea181bf566f71cb9a5d17a59e1871af638180a18fb0035c92ae62b705207123" dependencies = [ "atty", "bitflags", @@ -135,9 +141,9 @@ dependencies = [ [[package]] name = "core-foundation-sys" -version = "0.8.3" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5827cebf4670468b8772dd191856768aedcb1b0278a04f989f7766351917b9dc" +checksum = "e496a50fda8aacccc86d7529e2c1e0892dbd0f898a6b5645b5561b89c3210efa" [[package]] name = "crossbeam" @@ -155,9 +161,9 @@ dependencies = [ [[package]] name = "crossbeam-channel" -version = "0.5.6" +version = "0.5.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2dd04ddaf88237dc3b8d8f9a3c1004b506b54b3313403944054d23c0870c521" +checksum = "a33c2bf77f2df06183c3aa30d1e96c0695a313d4f9c453cc3762a6db39f99200" dependencies = [ "cfg-if", "crossbeam-utils", @@ -165,9 +171,9 @@ dependencies = [ [[package]] name = "crossbeam-deque" -version = "0.8.2" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "715e8152b692bba2d374b53d4875445368fdf21a94751410af607a5ac677d1fc" +checksum = "ce6fd6f855243022dcecf8702fef0c297d4338e226845fe067f6341ad9fa0cef" dependencies = [ "cfg-if", "crossbeam-epoch", @@ -176,14 +182,14 @@ dependencies = [ [[package]] name = "crossbeam-epoch" -version = "0.9.13" +version = "0.9.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01a9af1f4c2ef74bb8aa1f7e19706bc72d03598c8a570bb5de72243c7a9d9d5a" +checksum = "ae211234986c545741a7dc064309f67ee1e5ad243d0e48335adc0484d960bcc7" dependencies = [ "autocfg", "cfg-if", "crossbeam-utils", - "memoffset 0.7.1", + "memoffset 0.9.0", "scopeguard", ] @@ -199,18 +205,18 @@ dependencies = [ [[package]] name = "crossbeam-utils" -version = "0.8.14" +version = "0.8.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fb766fa798726286dbbb842f174001dab8abc7b627a1dd86e0b7222a95d929f" +checksum = "5a22b2d63d4d1dc0b7f1b6b2747dd0088008a9be28b6ddf0b1e7d335e3037294" dependencies = [ "cfg-if", ] [[package]] name = "cxx" -version = "1.0.83" +version = "1.0.97" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bdf07d07d6531bfcdbe9b8b739b104610c6508dcc4d63b410585faf338241daf" +checksum = "e88abab2f5abbe4c56e8f1fb431b784d710b709888f35755a160e62e33fe38e8" dependencies = [ "cc", "cxxbridge-flags", @@ -220,9 +226,9 @@ dependencies = [ [[package]] name = "cxx-build" -version = "1.0.83" +version = "1.0.97" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2eb5b96ecdc99f72657332953d4d9c50135af1bac34277801cc3937906ebd39" +checksum = "5c0c11acd0e63bae27dcd2afced407063312771212b7a823b4fd72d633be30fb" dependencies = [ "cc", "codespan-reporting", @@ -230,31 +236,31 @@ dependencies = [ "proc-macro2", "quote", "scratch", - "syn", + "syn 2.0.23", ] [[package]] name = "cxxbridge-flags" -version = "1.0.83" +version = "1.0.97" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac040a39517fd1674e0f32177648334b0f4074625b5588a64519804ba0553b12" +checksum = "8d3816ed957c008ccd4728485511e3d9aaf7db419aa321e3d2c5a2f3411e36c8" [[package]] name = "cxxbridge-macro" -version = "1.0.83" +version = "1.0.97" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1362b0ddcfc4eb0a1f57b68bd77dd99f0e826958a96abd0ae9bd092e114ffed6" +checksum = "a26acccf6f445af85ea056362561a24ef56cdc15fcc685f03aec50b9c702cb6d" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.23", ] [[package]] name = "darling" -version = "0.14.2" +version = "0.14.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0dd3cd20dc6b5a876612a6e5accfe7f3dd883db6d07acfbf14c128f61550dfa" +checksum = "7b750cb3417fd1b327431a470f388520309479ab0bf5e323505daf0290cd3850" dependencies = [ "darling_core", "darling_macro", @@ -262,27 +268,27 @@ dependencies = [ [[package]] name = "darling_core" -version = "0.14.2" +version = "0.14.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a784d2ccaf7c98501746bf0be29b2022ba41fd62a2e622af997a03e9f972859f" +checksum = "109c1ca6e6b7f82cc233a97004ea8ed7ca123a9af07a8230878fcfda9b158bf0" dependencies = [ "fnv", "ident_case", "proc-macro2", "quote", "strsim", - "syn", + "syn 1.0.109", ] [[package]] name = "darling_macro" -version = "0.14.2" +version = "0.14.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7618812407e9402654622dd402b0a89dff9ba93badd6540781526117b92aab7e" +checksum = "a4aab4dbc9f7611d8b55048a3a16d2d010c2c8334e46304b40ac1cc14bf3b48e" dependencies = [ "darling_core", "quote", - "syn", + "syn 1.0.109", ] [[package]] @@ -313,7 +319,7 @@ dependencies = [ "darling", "proc-macro2", "quote", - "syn", + "syn 1.0.109", ] [[package]] @@ -323,7 +329,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8f0314b72bed045f3a68671b3c86328386762c93f82d98c65c3cb5e5f573dd68" dependencies = [ "derive_builder_core", - "syn", + "syn 1.0.109", ] [[package]] @@ -349,9 +355,9 @@ dependencies = [ [[package]] name = "either" -version = "1.8.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90e5c1c8368803113bf0c9584fc495a58b86dc8a29edbf8fe877d21d9507e797" +checksum = "7fcaabb2fef8c910e7f4c7ce9f67a1283a1715879a7c230ca9d6d1ae31f16d91" [[package]] name = "env_logger" @@ -383,9 +389,9 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.8" +version = "0.2.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c05aeb6a22b8f62540c194aac980f2115af067bfe15a0734d7277a768d396b31" +checksum = "be4136b2a15dd319360be1c07d9933517ccf0be8f16bf62a3bee4f0d618df427" dependencies = [ "cfg-if", "libc", @@ -407,6 +413,12 @@ dependencies = [ "libc", ] +[[package]] +name = "hermit-abi" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fed44880c466736ef9a5c5b5facefb5ed0785676d0c02d612db14e54f0d84286" + [[package]] name = "humantime" version = "2.1.0" @@ -415,26 +427,25 @@ checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" [[package]] name = "iana-time-zone" -version = "0.1.53" +version = "0.1.57" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64c122667b287044802d6ce17ee2ddf13207ed924c712de9a66a5814d5b64765" +checksum = "2fad5b825842d2b38bd206f3e81d6957625fd7f0a361e345c30e01a0ae2dd613" dependencies = [ "android_system_properties", "core-foundation-sys", "iana-time-zone-haiku", "js-sys", "wasm-bindgen", - "winapi", + "windows", ] [[package]] name = "iana-time-zone-haiku" -version = "0.1.1" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0703ae284fc167426161c2e3f1da3ea71d94b21bedbcc9494e92b28e334e3dca" +checksum = "f31827a206f56af32e590ba56d5d2d085f558508192593743f16b2306495269f" dependencies = [ - "cxx", - "cxx-build", + "cc", ] [[package]] @@ -445,9 +456,9 @@ checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" [[package]] name = "indexmap" -version = "1.9.2" +version = "1.9.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1885e79c1fc4b10f0e172c475f458b7f7b93061064d98c3293e98c5ba0c8b399" +checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" dependencies = [ "autocfg", "hashbrown", @@ -455,9 +466,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.60" +version = "0.3.64" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49409df3e3bf0856b916e2ceaca09ee28e6871cf7d9ce97a692cacfdb2a25a47" +checksum = "c5f195fe497f702db0f318b07fdd68edb16955aed830df8363d837542f8f935a" dependencies = [ "wasm-bindgen", ] @@ -470,27 +481,24 @@ checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" [[package]] name = "libc" -version = "0.2.138" +version = "0.2.147" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db6d7e329c562c5dfab7a46a2afabc8b987ab9a4834c9d1ca04dc54c1546cef8" +checksum = "b4668fb0ea861c1df094127ac5f1da3409a82116a4ba74fca2e58ef927159bb3" [[package]] name = "link-cplusplus" -version = "1.0.7" +version = "1.0.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9272ab7b96c9046fbc5bc56c06c117cb639fe2d509df0c421cad82d2915cf369" +checksum = "ecd207c9c713c34f95a097a5b029ac2ce6010530c7b49d7fea24d977dede04f5" dependencies = [ "cc", ] [[package]] name = "log" -version = "0.4.17" +version = "0.4.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "abb12e687cfb44aa40f41fc3978ef76448f9b6038cad6aef4259d3c095a2382e" -dependencies = [ - "cfg-if", -] +checksum = "b06a4cde4c0f271a446782e3eff8de789548ce57dbc8eca9292c27f4a42004b4" [[package]] name = "memchr" @@ -509,9 +517,9 @@ dependencies = [ [[package]] name = "memoffset" -version = "0.7.1" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5de893c32cde5f383baa4c04c5d6dbdd735cfd4a794b0debdb2bb1b421da5ff4" +checksum = "5a634b1c61a95585bd15607c6ab0c4e5b226e695ff2800ba0cdccddf208c406c" dependencies = [ "autocfg", ] @@ -541,16 +549,6 @@ dependencies = [ "pin-utils", ] -[[package]] -name = "num-integer" -version = "0.1.45" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "225d3389fb3509a24c93f5c29eb6bde2586b98d9f016636dff58d7c6f7569cd9" -dependencies = [ - "autocfg", - "num-traits", -] - [[package]] name = "num-traits" version = "0.2.15" @@ -562,25 +560,25 @@ dependencies = [ [[package]] name = "num_cpus" -version = "1.14.0" +version = "1.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f6058e64324c71e02bc2b150e4f3bc8286db6c83092132ffa3f6b1eab0f9def5" +checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" dependencies = [ - "hermit-abi", + "hermit-abi 0.3.1", "libc", ] [[package]] name = "once_cell" -version = "1.16.0" +version = "1.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86f0b0d4bf799edbc74508c1e8bf170ff5f41238e5f8225603ca7caaae2b7860" +checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" [[package]] name = "os_str_bytes" -version = "6.4.1" +version = "6.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b7820b9daea5457c9f21c69448905d723fbd21136ccf521748f23fd49e723ee" +checksum = "4d5d9eb14b174ee9aa2ef96dc2b94637a2d4b6e7cb873c7e171f0c20c6cf3eac" [[package]] name = "pin-utils" @@ -590,27 +588,27 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "proc-macro2" -version = "1.0.47" +version = "1.0.63" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ea3d908b0e36316caf9e9e2c4625cdde190a7e6f440d794667ed17a1855e725" +checksum = "7b368fba921b0dce7e60f5e04ec15e565b3303972b42bcfde1d0713b881959eb" dependencies = [ "unicode-ident", ] [[package]] name = "quote" -version = "1.0.21" +version = "1.0.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bbe448f377a7d6961e30f5955f9b8d106c3f5e449d493ee1b125c1d43c2b5179" +checksum = "573015e8ab27661678357f27dc26460738fd2b6c86e46f386fde94cb5d913105" dependencies = [ "proc-macro2", ] [[package]] name = "rayon" -version = "1.6.1" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6db3a213adf02b3bcfd2d3846bb41cb22857d131789e01df434fb7e7bc0759b7" +checksum = "1d2df5196e37bcc87abebc0053e20787d73847bb33134a69841207dd0a47f03b" dependencies = [ "either", "rayon-core", @@ -618,9 +616,9 @@ dependencies = [ [[package]] name = "rayon-core" -version = "1.10.1" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cac410af5d00ab6884528b4ab69d1e8e146e8d471201800fa1b4524126de6ad3" +checksum = "4b8f95bd6966f5c87776639160a66bd8ab9895d9d4ab01ddba9fc60661aebe8d" dependencies = [ "crossbeam-channel", "crossbeam-deque", @@ -650,9 +648,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.7.0" +version = "1.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e076559ef8e241f2ae3479e36f97bd5741c0330689e217ad51ce2c76808b868a" +checksum = "d0ab3ca65655bb1e41f2a8c8cd662eb4fb035e67c3f78da1d61dffe89d07300f" dependencies = [ "aho-corasick", "memchr", @@ -661,15 +659,15 @@ dependencies = [ [[package]] name = "regex-syntax" -version = "0.6.28" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "456c603be3e8d448b072f410900c09faf164fbce2d480456f50eea6e25f9c848" +checksum = "436b050e76ed2903236f032a59761c1eb99e1b0aead2c257922771dab1fc8c78" [[package]] name = "rustversion" -version = "1.0.9" +version = "1.0.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97477e48b4cf8603ad5f7aaf897467cf42ab4218a38ef76fb14c2d6773a6d6a8" +checksum = "4f3208ce4d8448b3f3e7d168a73f5e0c43a61e32930de3bceeccedb388b6bf06" [[package]] name = "scopeguard" @@ -679,15 +677,15 @@ checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" [[package]] name = "scratch" -version = "1.0.2" +version = "1.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c8132065adcfd6e02db789d9285a0deb2f3fcb04002865ab67d5fb103533898" +checksum = "1792db035ce95be60c3f8853017b3999209281c24e2ba5bc8e59bf97a0c590c1" [[package]] name = "serde" -version = "1.0.149" +version = "1.0.164" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "256b9932320c590e707b94576e3cc1f7c9024d0ee6612dfbcf1cb106cbe8e055" +checksum = "9e8c8cf938e98f769bc164923b06dce91cea1751522f46f8466461af04c9027d" [[package]] name = "shlex" @@ -697,9 +695,9 @@ checksum = "43b2853a4d09f215c24cc5489c992ce46052d359b5109343cbafbf26bc62f8a3" [[package]] name = "skim" -version = "0.10.2" +version = "0.10.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cebed5f897cd6c0d80fbe30adb36c0abf7400e93043a63ae56458495642b3485" +checksum = "e5d28de0a6cb2cdd83a076f1de9d965b973ae08b244df1aa70b432946dda0f32" dependencies = [ "atty", "beef", @@ -717,7 +715,7 @@ dependencies = [ "rayon", "regex", "shlex", - "time 0.3.17", + "time 0.3.22", "timer", "tuikit", "unicode-width", @@ -732,9 +730,20 @@ checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" [[package]] name = "syn" -version = "1.0.105" +version = "1.0.109" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "60b9b43d45702de4c839cb9b51d9f529c5dd26a4aff255b42b1ebc03e88ee908" +checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "syn" +version = "2.0.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "59fb7d6d8281a51045d62b8eb3a7d1ce347b76f312af50cd3dc0af39c87c1737" dependencies = [ "proc-macro2", "quote", @@ -754,9 +763,9 @@ dependencies = [ [[package]] name = "termcolor" -version = "1.1.3" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bab24d30b911b2376f3a13cc2cd443142f0c81dda04c118693e35b3835757755" +checksum = "be55cf8942feac5c765c2c993422806843c9a9a45d4d5c407ad6dd2ea95eb9b6" dependencies = [ "winapi-util", ] @@ -769,30 +778,31 @@ checksum = "222a222a5bfe1bba4a77b45ec488a741b3cb8872e5e499451fd7d0129c9c7c3d" [[package]] name = "thiserror" -version = "1.0.37" +version = "1.0.40" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "10deb33631e3c9018b9baf9dcbbc4f737320d2b576bac10f6aefa048fa407e3e" +checksum = "978c9a314bd8dc99be594bc3c175faaa9794be04a5a5e153caba6915336cebac" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.37" +version = "1.0.40" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "982d17546b47146b28f7c22e3d08465f6b8903d0ea13c1660d9d84a6e7adcdbb" +checksum = "f9456a42c5b0d803c8cd86e73dd7cc9edd429499f37a3550d286d5e86720569f" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.23", ] [[package]] name = "thread_local" -version = "1.1.4" +version = "1.1.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5516c27b78311c50bf42c071425c560ac799b11c30b31f87e3081965fe5e0180" +checksum = "3fdd6f064ccff2d6567adcb3873ca630700f00b5ad3f060c25b5dcfd9a4ce152" dependencies = [ + "cfg-if", "once_cell", ] @@ -809,9 +819,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.17" +version = "0.3.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a561bf4617eebd33bca6434b988f39ed798e527f51a1e797d0ee4f61c0a38376" +checksum = "ea9e1b3cf1243ae005d9e74085d4d542f3125458f3a81af210d901dcd7411efd" dependencies = [ "serde", "time-core", @@ -819,9 +829,9 @@ dependencies = [ [[package]] name = "time-core" -version = "0.1.0" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e153e1f1acaef8acc537e68b44906d2db6436e2b35ac2c6b42640fff91f00fd" +checksum = "7300fbefb4dadc1af235a9cef3737cea692a9d97e1b9cbcd4ebdae6f8868e6fb" [[package]] name = "timer" @@ -848,9 +858,9 @@ dependencies = [ [[package]] name = "unicode-ident" -version = "1.0.5" +version = "1.0.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ceab39d59e4c9499d4e5a8ee0e2735b891bb7308ac83dfb4e80cad195c9f6f3" +checksum = "b15811caf2415fb889178633e7724bad2509101cde276048e013b9def5e51fa0" [[package]] name = "unicode-width" @@ -860,15 +870,15 @@ checksum = "c0edd1e5b14653f783770bce4a4dabb4a5108a5370a5f5d8cfe8710c361f6c8b" [[package]] name = "utf8parse" -version = "0.2.0" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "936e4b492acfd135421d8dca4b1aa80a7bfc26e702ef3af710e0752684df5372" +checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" [[package]] name = "vte" -version = "0.11.0" +version = "0.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1aae21c12ad2ec2d168c236f369c38ff332bc1134f7246350dca641437365045" +checksum = "f5022b5fbf9407086c180e9557be968742d839e68346af7792b8592489732197" dependencies = [ "arrayvec", "utf8parse", @@ -899,9 +909,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.83" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eaf9f5aceeec8be17c128b2e93e031fb8a4d469bb9c4ae2d7dc1888b26887268" +checksum = "7706a72ab36d8cb1f80ffbf0e071533974a60d0a308d01a5d0375bf60499a342" dependencies = [ "cfg-if", "wasm-bindgen-macro", @@ -909,24 +919,24 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.83" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c8ffb332579b0557b52d268b91feab8df3615f265d5270fec2a8c95b17c1142" +checksum = "5ef2b6d3c510e9625e5fe6f509ab07d66a760f0885d858736483c32ed7809abd" dependencies = [ "bumpalo", "log", "once_cell", "proc-macro2", "quote", - "syn", + "syn 2.0.23", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-macro" -version = "0.2.83" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "052be0f94026e6cbc75cdefc9bae13fd6052cdcaf532fa6c45e7ae33a1e6c810" +checksum = "dee495e55982a3bd48105a7b947fd2a9b4a8ae3010041b9e0faab3f9cd028f1d" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -934,22 +944,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.83" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07bc0c051dc5f23e307b13285f9d75df86bfdf816c5721e573dec1f9b8aa193c" +checksum = "54681b18a46765f095758388f2d0cf16eb8d4169b639ab575a8f5693af210c7b" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.23", "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.83" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1c38c045535d93ec4f0b4defec448e4291638ee608530863b1e2ba115d4fff7f" +checksum = "ca6ad05a4870b2bf5fe995117d3728437bd27d7cd5f06f13c17443ef369775a1" [[package]] name = "winapi" @@ -981,3 +991,69 @@ name = "winapi-x86_64-pc-windows-gnu" version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" + +[[package]] +name = "windows" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e686886bc078bc1b0b600cac0147aadb815089b6e4da64016cbd754b6342700f" +dependencies = [ + "windows-targets", +] + +[[package]] +name = "windows-targets" +version = "0.48.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05d4b17490f70499f20b9e791dcf6a299785ce8af4d709018206dc5b4953e95f" +dependencies = [ + "windows_aarch64_gnullvm", + "windows_aarch64_msvc", + "windows_i686_gnu", + "windows_i686_msvc", + "windows_x86_64_gnu", + "windows_x86_64_gnullvm", + "windows_x86_64_msvc", +] + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91ae572e1b79dba883e0d315474df7305d12f569b400fcf90581b06062f7e1bc" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b2ef27e0d7bdfcfc7b868b317c1d32c641a6fe4629c171b8928c7b08d98d7cf3" + +[[package]] +name = "windows_i686_gnu" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "622a1962a7db830d6fd0a69683c80a18fda201879f0f447f065a3b7467daa241" + +[[package]] +name = "windows_i686_msvc" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4542c6e364ce21bf45d69fdd2a8e455fa38d316158cfd43b3ac1c5b1b19f8e00" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ca2b8a661f7628cbd23440e50b05d705db3686f894fc9580820623656af974b1" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7896dbc1f41e08872e9d5e8f8baa8fdd2677f29468c4e156210174edc7f7b953" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a515f5799fe4961cb532f983ce2b23082366b898e52ffbce459c86f67c8378a" From f1709b3cf4360bbd57e7740a305d2a888849197b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 3 Jul 2023 12:58:41 +0300 Subject: [PATCH 206/228] 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 207/228] 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 208/228] 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 209/228] 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 579a446d1901b356fd1f58d98cd1cf6898de2604 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 3 Jul 2023 13:49:56 +0200 Subject: [PATCH 210/228] Update 02789_filesystem_cache_alignment.sh --- tests/queries/0_stateless/02789_filesystem_cache_alignment.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh index 509d1a635b1..912cdd3d1e8 100755 --- a/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh +++ b/tests/queries/0_stateless/02789_filesystem_cache_alignment.sh @@ -61,7 +61,7 @@ WHERE file_segment_size < file_size AND end_offset + 1 != file_size; ") #echo $all -if [ "$all" -gt "10" ]; then +if [ "$all" -gt "1" ]; then echo "OK" else echo "FAIL" @@ -105,7 +105,7 @@ SELECT count() FROM ($query2) WHERE file_segment_size < file_size AND file_segment_range_end + 1 != file_size; ") -if [ "$all" -gt "10" ]; then +if [ "$all" -gt "1" ]; then echo "OK" else echo "FAIL" From d1c4a37473749172406e850a3f0c0bcc6f122a49 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 3 Jul 2023 14:38:38 +0200 Subject: [PATCH 211/228] Remove MemoryPool from Poco because it's useless --- base/poco/Foundation/CMakeLists.txt | 1 - .../poco/Foundation/include/Poco/MemoryPool.h | 116 ------------------ base/poco/Foundation/src/MemoryPool.cpp | 105 ---------------- .../Net/include/Poco/Net/HTTPBasicStreamBuf.h | 5 +- .../include/Poco/Net/HTTPBufferAllocator.h | 53 -------- .../Net/include/Poco/Net/HTTPChunkedStream.h | 13 -- .../include/Poco/Net/HTTPFixedLengthStream.h | 12 -- .../Net/include/Poco/Net/HTTPHeaderStream.h | 13 -- base/poco/Net/include/Poco/Net/HTTPStream.h | 13 -- base/poco/Net/src/HTTPBufferAllocator.cpp | 44 ------- base/poco/Net/src/HTTPChunkedStream.cpp | 50 +------- base/poco/Net/src/HTTPFixedLengthStream.cpp | 47 +------ base/poco/Net/src/HTTPHeaderStream.cpp | 50 +------- base/poco/Net/src/HTTPSession.cpp | 8 +- base/poco/Net/src/HTTPStream.cpp | 48 +------- 15 files changed, 10 insertions(+), 568 deletions(-) delete mode 100644 base/poco/Foundation/include/Poco/MemoryPool.h delete mode 100644 base/poco/Foundation/src/MemoryPool.cpp delete mode 100644 base/poco/Net/include/Poco/Net/HTTPBufferAllocator.h delete mode 100644 base/poco/Net/src/HTTPBufferAllocator.cpp diff --git a/base/poco/Foundation/CMakeLists.txt b/base/poco/Foundation/CMakeLists.txt index 358f49ed055..d0dde8a51a5 100644 --- a/base/poco/Foundation/CMakeLists.txt +++ b/base/poco/Foundation/CMakeLists.txt @@ -87,7 +87,6 @@ set (SRCS src/LoggingRegistry.cpp src/LogStream.cpp src/MD5Engine.cpp - src/MemoryPool.cpp src/MemoryStream.cpp src/Message.cpp src/Mutex.cpp diff --git a/base/poco/Foundation/include/Poco/MemoryPool.h b/base/poco/Foundation/include/Poco/MemoryPool.h deleted file mode 100644 index 9ab12081b5f..00000000000 --- a/base/poco/Foundation/include/Poco/MemoryPool.h +++ /dev/null @@ -1,116 +0,0 @@ -// -// MemoryPool.h -// -// Library: Foundation -// Package: Core -// Module: MemoryPool -// -// Definition of the MemoryPool class. -// -// Copyright (c) 2005-2006, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef Foundation_MemoryPool_INCLUDED -#define Foundation_MemoryPool_INCLUDED - - -#include -#include -#include "Poco/Foundation.h" -#include "Poco/Mutex.h" - - -namespace Poco -{ - - -class Foundation_API MemoryPool -/// A simple pool for fixed-size memory blocks. -/// -/// The main purpose of this class is to speed-up -/// memory allocations, as well as to reduce memory -/// fragmentation in situations where the same blocks -/// are allocated all over again, such as in server -/// applications. -/// -/// All allocated blocks are retained for future use. -/// A limit on the number of blocks can be specified. -/// Blocks can be preallocated. -{ -public: - MemoryPool(std::size_t blockSize, int preAlloc = 0, int maxAlloc = 0); - /// Creates a MemoryPool for blocks with the given blockSize. - /// The number of blocks given in preAlloc are preallocated. - - ~MemoryPool(); - - void * get(); - /// Returns a memory block. If there are no more blocks - /// in the pool, a new block will be allocated. - /// - /// If maxAlloc blocks are already allocated, an - /// OutOfMemoryException is thrown. - - void release(void * ptr); - /// Releases a memory block and returns it to the pool. - - std::size_t blockSize() const; - /// Returns the block size. - - int allocated() const; - /// Returns the number of allocated blocks. - - int available() const; - /// Returns the number of available blocks in the pool. - -private: - MemoryPool(); - MemoryPool(const MemoryPool &); - MemoryPool & operator=(const MemoryPool &); - - void clear(); - - enum - { - BLOCK_RESERVE = 128 - }; - - typedef std::vector BlockVec; - - std::size_t _blockSize; - int _maxAlloc; - int _allocated; - BlockVec _blocks; - FastMutex _mutex; -}; - - -// -// inlines -// -inline std::size_t MemoryPool::blockSize() const -{ - return _blockSize; -} - - -inline int MemoryPool::allocated() const -{ - return _allocated; -} - - -inline int MemoryPool::available() const -{ - return (int)_blocks.size(); -} - - -} // namespace Poco - - -#endif // Foundation_MemoryPool_INCLUDED diff --git a/base/poco/Foundation/src/MemoryPool.cpp b/base/poco/Foundation/src/MemoryPool.cpp deleted file mode 100644 index 01c477be525..00000000000 --- a/base/poco/Foundation/src/MemoryPool.cpp +++ /dev/null @@ -1,105 +0,0 @@ -// -// MemoryPool.cpp -// -// Library: Foundation -// Package: Core -// Module: MemoryPool -// -// Copyright (c) 2005-2006, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/MemoryPool.h" -#include "Poco/Exception.h" - - -namespace Poco { - - -MemoryPool::MemoryPool(std::size_t blockSize, int preAlloc, int maxAlloc): - _blockSize(blockSize), - _maxAlloc(maxAlloc), - _allocated(preAlloc) -{ - poco_assert (maxAlloc == 0 || maxAlloc >= preAlloc); - poco_assert (preAlloc >= 0 && maxAlloc >= 0); - - int r = BLOCK_RESERVE; - if (preAlloc > r) - r = preAlloc; - if (maxAlloc > 0 && maxAlloc < r) - r = maxAlloc; - _blocks.reserve(r); - - try - { - for (int i = 0; i < preAlloc; ++i) - { - _blocks.push_back(new char[_blockSize]); - } - } - catch (...) - { - clear(); - throw; - } -} - - -MemoryPool::~MemoryPool() -{ - clear(); -} - - -void MemoryPool::clear() -{ - for (BlockVec::iterator it = _blocks.begin(); it != _blocks.end(); ++it) - { - delete [] *it; - } - _blocks.clear(); -} - - -void* MemoryPool::get() -{ - FastMutex::ScopedLock lock(_mutex); - - if (_blocks.empty()) - { - if (_maxAlloc == 0 || _allocated < _maxAlloc) - { - ++_allocated; - return new char[_blockSize]; - } - else throw OutOfMemoryException("MemoryPool exhausted"); - } - else - { - char* ptr = _blocks.back(); - _blocks.pop_back(); - return ptr; - } -} - - -void MemoryPool::release(void* ptr) -{ - FastMutex::ScopedLock lock(_mutex); - - try - { - _blocks.push_back(reinterpret_cast(ptr)); - } - catch (...) - { - delete [] reinterpret_cast(ptr); - } -} - - -} // namespace Poco diff --git a/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h b/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h index c4872d95353..c87719b63a4 100644 --- a/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h +++ b/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h @@ -19,7 +19,6 @@ #include "Poco/BufferedStreamBuf.h" -#include "Poco/Net/HTTPBufferAllocator.h" #include "Poco/Net/Net.h" @@ -27,9 +26,9 @@ namespace Poco { namespace Net { + constexpr size_t HTTP_DEFAULT_BUFFER_SIZE = 8 * 1024; - - typedef Poco::BasicBufferedStreamBuf, HTTPBufferAllocator> HTTPBasicStreamBuf; + typedef Poco::BasicBufferedStreamBuf> HTTPBasicStreamBuf; } diff --git a/base/poco/Net/include/Poco/Net/HTTPBufferAllocator.h b/base/poco/Net/include/Poco/Net/HTTPBufferAllocator.h deleted file mode 100644 index 5d088e35297..00000000000 --- a/base/poco/Net/include/Poco/Net/HTTPBufferAllocator.h +++ /dev/null @@ -1,53 +0,0 @@ -// -// HTTPBufferAllocator.h -// -// Library: Net -// Package: HTTP -// Module: HTTPBufferAllocator -// -// Definition of the HTTPBufferAllocator class. -// -// Copyright (c) 2005-2006, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef Net_HTTPBufferAllocator_INCLUDED -#define Net_HTTPBufferAllocator_INCLUDED - - -#include -#include "Poco/MemoryPool.h" -#include "Poco/Net/Net.h" - - -namespace Poco -{ -namespace Net -{ - - - class Net_API HTTPBufferAllocator - /// A BufferAllocator for HTTP streams. - { - public: - static char * allocate(std::streamsize size); - static void deallocate(char * ptr, std::streamsize size); - - enum - { - BUFFER_SIZE = 128 * 1024 - }; - - private: - static Poco::MemoryPool _pool; - }; - - -} -} // namespace Poco::Net - - -#endif // Net_HTTPBufferAllocator_INCLUDED diff --git a/base/poco/Net/include/Poco/Net/HTTPChunkedStream.h b/base/poco/Net/include/Poco/Net/HTTPChunkedStream.h index 47987b18817..5f4729c9278 100644 --- a/base/poco/Net/include/Poco/Net/HTTPChunkedStream.h +++ b/base/poco/Net/include/Poco/Net/HTTPChunkedStream.h @@ -21,7 +21,6 @@ #include #include #include -#include "Poco/MemoryPool.h" #include "Poco/Net/HTTPBasicStreamBuf.h" #include "Poco/Net/Net.h" @@ -80,12 +79,6 @@ namespace Net public: HTTPChunkedInputStream(HTTPSession & session); ~HTTPChunkedInputStream(); - - void * operator new(std::size_t size); - void operator delete(void * ptr); - - private: - static Poco::MemoryPool _pool; }; @@ -95,12 +88,6 @@ namespace Net public: HTTPChunkedOutputStream(HTTPSession & session); ~HTTPChunkedOutputStream(); - - void * operator new(std::size_t size); - void operator delete(void * ptr); - - private: - static Poco::MemoryPool _pool; }; diff --git a/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h b/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h index 4de211fdb92..2f4df102605 100644 --- a/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h +++ b/base/poco/Net/include/Poco/Net/HTTPFixedLengthStream.h @@ -78,12 +78,6 @@ namespace Net public: HTTPFixedLengthInputStream(HTTPSession & session, HTTPFixedLengthStreamBuf::ContentLength length); ~HTTPFixedLengthInputStream(); - - void * operator new(std::size_t size); - void operator delete(void * ptr); - - private: - static Poco::MemoryPool _pool; }; @@ -93,12 +87,6 @@ namespace Net public: HTTPFixedLengthOutputStream(HTTPSession & session, HTTPFixedLengthStreamBuf::ContentLength length); ~HTTPFixedLengthOutputStream(); - - void * operator new(std::size_t size); - void operator delete(void * ptr); - - private: - static Poco::MemoryPool _pool; }; diff --git a/base/poco/Net/include/Poco/Net/HTTPHeaderStream.h b/base/poco/Net/include/Poco/Net/HTTPHeaderStream.h index bcfca984d8b..cf1a6dba2e6 100644 --- a/base/poco/Net/include/Poco/Net/HTTPHeaderStream.h +++ b/base/poco/Net/include/Poco/Net/HTTPHeaderStream.h @@ -21,7 +21,6 @@ #include #include #include -#include "Poco/MemoryPool.h" #include "Poco/Net/HTTPBasicStreamBuf.h" #include "Poco/Net/Net.h" @@ -74,12 +73,6 @@ namespace Net public: HTTPHeaderInputStream(HTTPSession & session); ~HTTPHeaderInputStream(); - - void * operator new(std::size_t size); - void operator delete(void * ptr); - - private: - static Poco::MemoryPool _pool; }; @@ -89,12 +82,6 @@ namespace Net public: HTTPHeaderOutputStream(HTTPSession & session); ~HTTPHeaderOutputStream(); - - void * operator new(std::size_t size); - void operator delete(void * ptr); - - private: - static Poco::MemoryPool _pool; }; diff --git a/base/poco/Net/include/Poco/Net/HTTPStream.h b/base/poco/Net/include/Poco/Net/HTTPStream.h index 0197bc62eb2..48502347b2c 100644 --- a/base/poco/Net/include/Poco/Net/HTTPStream.h +++ b/base/poco/Net/include/Poco/Net/HTTPStream.h @@ -21,7 +21,6 @@ #include #include #include -#include "Poco/MemoryPool.h" #include "Poco/Net/HTTPBasicStreamBuf.h" #include "Poco/Net/Net.h" @@ -75,12 +74,6 @@ namespace Net public: HTTPInputStream(HTTPSession & session); ~HTTPInputStream(); - - void * operator new(std::size_t size); - void operator delete(void * ptr); - - private: - static Poco::MemoryPool _pool; }; @@ -90,12 +83,6 @@ namespace Net public: HTTPOutputStream(HTTPSession & session); ~HTTPOutputStream(); - - void * operator new(std::size_t size); - void operator delete(void * ptr); - - private: - static Poco::MemoryPool _pool; }; diff --git a/base/poco/Net/src/HTTPBufferAllocator.cpp b/base/poco/Net/src/HTTPBufferAllocator.cpp deleted file mode 100644 index 2944e2a6121..00000000000 --- a/base/poco/Net/src/HTTPBufferAllocator.cpp +++ /dev/null @@ -1,44 +0,0 @@ -// -// HTTPBufferAllocator.cpp -// -// Library: Net -// Package: HTTP -// Module: HTTPBufferAllocator -// -// Copyright (c) 2005-2006, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/Net/HTTPBufferAllocator.h" - - -using Poco::MemoryPool; - - -namespace Poco { -namespace Net { - - -MemoryPool HTTPBufferAllocator::_pool(HTTPBufferAllocator::BUFFER_SIZE, 16); - - -char* HTTPBufferAllocator::allocate(std::streamsize size) -{ - poco_assert_dbg (size == BUFFER_SIZE); - - return reinterpret_cast(_pool.get()); -} - - -void HTTPBufferAllocator::deallocate(char* ptr, std::streamsize size) -{ - poco_assert_dbg (size == BUFFER_SIZE); - - _pool.release(ptr); -} - - -} } // namespace Poco::Net diff --git a/base/poco/Net/src/HTTPChunkedStream.cpp b/base/poco/Net/src/HTTPChunkedStream.cpp index f2f79da590b..376e3f55492 100644 --- a/base/poco/Net/src/HTTPChunkedStream.cpp +++ b/base/poco/Net/src/HTTPChunkedStream.cpp @@ -34,7 +34,7 @@ namespace Net { HTTPChunkedStreamBuf::HTTPChunkedStreamBuf(HTTPSession& session, openmode mode): - HTTPBasicStreamBuf(HTTPBufferAllocator::BUFFER_SIZE, mode), + HTTPBasicStreamBuf(HTTP_DEFAULT_BUFFER_SIZE, mode), _session(session), _mode(mode), _chunk(0) @@ -181,10 +181,6 @@ HTTPChunkedStreamBuf* HTTPChunkedIOS::rdbuf() // HTTPChunkedInputStream // - -Poco::MemoryPool HTTPChunkedInputStream::_pool(sizeof(HTTPChunkedInputStream)); - - HTTPChunkedInputStream::HTTPChunkedInputStream(HTTPSession& session): HTTPChunkedIOS(session, std::ios::in), std::istream(&_buf) @@ -196,34 +192,10 @@ HTTPChunkedInputStream::~HTTPChunkedInputStream() { } - -void* HTTPChunkedInputStream::operator new(std::size_t size) -{ - return _pool.get(); -} - - -void HTTPChunkedInputStream::operator delete(void* ptr) -{ - try - { - _pool.release(ptr); - } - catch (...) - { - poco_unexpected(); - } -} - - // // HTTPChunkedOutputStream // - -Poco::MemoryPool HTTPChunkedOutputStream::_pool(sizeof(HTTPChunkedOutputStream)); - - HTTPChunkedOutputStream::HTTPChunkedOutputStream(HTTPSession& session): HTTPChunkedIOS(session, std::ios::out), std::ostream(&_buf) @@ -235,24 +207,4 @@ HTTPChunkedOutputStream::~HTTPChunkedOutputStream() { } - -void* HTTPChunkedOutputStream::operator new(std::size_t size) -{ - return _pool.get(); -} - - -void HTTPChunkedOutputStream::operator delete(void* ptr) -{ - try - { - _pool.release(ptr); - } - catch (...) - { - poco_unexpected(); - } -} - - } } // namespace Poco::Net diff --git a/base/poco/Net/src/HTTPFixedLengthStream.cpp b/base/poco/Net/src/HTTPFixedLengthStream.cpp index d19f6122ee1..fd77ff71cd9 100644 --- a/base/poco/Net/src/HTTPFixedLengthStream.cpp +++ b/base/poco/Net/src/HTTPFixedLengthStream.cpp @@ -30,7 +30,7 @@ namespace Net { HTTPFixedLengthStreamBuf::HTTPFixedLengthStreamBuf(HTTPSession& session, ContentLength length, openmode mode): - HTTPBasicStreamBuf(HTTPBufferAllocator::BUFFER_SIZE, mode), + HTTPBasicStreamBuf(HTTP_DEFAULT_BUFFER_SIZE, mode), _session(session), _length(length), _count(0) @@ -109,9 +109,6 @@ HTTPFixedLengthStreamBuf* HTTPFixedLengthIOS::rdbuf() // -Poco::MemoryPool HTTPFixedLengthInputStream::_pool(sizeof(HTTPFixedLengthInputStream)); - - HTTPFixedLengthInputStream::HTTPFixedLengthInputStream(HTTPSession& session, HTTPFixedLengthStreamBuf::ContentLength length): HTTPFixedLengthIOS(session, length, std::ios::in), std::istream(&_buf) @@ -124,33 +121,10 @@ HTTPFixedLengthInputStream::~HTTPFixedLengthInputStream() } -void* HTTPFixedLengthInputStream::operator new(std::size_t size) -{ - return _pool.get(); -} - - -void HTTPFixedLengthInputStream::operator delete(void* ptr) -{ - try - { - _pool.release(ptr); - } - catch (...) - { - poco_unexpected(); - } -} - - // // HTTPFixedLengthOutputStream // - -Poco::MemoryPool HTTPFixedLengthOutputStream::_pool(sizeof(HTTPFixedLengthOutputStream)); - - HTTPFixedLengthOutputStream::HTTPFixedLengthOutputStream(HTTPSession& session, HTTPFixedLengthStreamBuf::ContentLength length): HTTPFixedLengthIOS(session, length, std::ios::out), std::ostream(&_buf) @@ -163,23 +137,4 @@ HTTPFixedLengthOutputStream::~HTTPFixedLengthOutputStream() } -void* HTTPFixedLengthOutputStream::operator new(std::size_t size) -{ - return _pool.get(); -} - - -void HTTPFixedLengthOutputStream::operator delete(void* ptr) -{ - try - { - _pool.release(ptr); - } - catch (...) - { - poco_unexpected(); - } -} - - } } // namespace Poco::Net diff --git a/base/poco/Net/src/HTTPHeaderStream.cpp b/base/poco/Net/src/HTTPHeaderStream.cpp index 8e0091fcbe3..39b9007062d 100644 --- a/base/poco/Net/src/HTTPHeaderStream.cpp +++ b/base/poco/Net/src/HTTPHeaderStream.cpp @@ -26,7 +26,7 @@ namespace Net { HTTPHeaderStreamBuf::HTTPHeaderStreamBuf(HTTPSession& session, openmode mode): - HTTPBasicStreamBuf(HTTPBufferAllocator::BUFFER_SIZE, mode), + HTTPBasicStreamBuf(HTTP_DEFAULT_BUFFER_SIZE, mode), _session(session), _end(false) { @@ -101,10 +101,6 @@ HTTPHeaderStreamBuf* HTTPHeaderIOS::rdbuf() // HTTPHeaderInputStream // - -Poco::MemoryPool HTTPHeaderInputStream::_pool(sizeof(HTTPHeaderInputStream)); - - HTTPHeaderInputStream::HTTPHeaderInputStream(HTTPSession& session): HTTPHeaderIOS(session, std::ios::in), std::istream(&_buf) @@ -116,34 +112,10 @@ HTTPHeaderInputStream::~HTTPHeaderInputStream() { } - -void* HTTPHeaderInputStream::operator new(std::size_t size) -{ - return _pool.get(); -} - - -void HTTPHeaderInputStream::operator delete(void* ptr) -{ - try - { - _pool.release(ptr); - } - catch (...) - { - poco_unexpected(); - } -} - - // // HTTPHeaderOutputStream // - -Poco::MemoryPool HTTPHeaderOutputStream::_pool(sizeof(HTTPHeaderOutputStream)); - - HTTPHeaderOutputStream::HTTPHeaderOutputStream(HTTPSession& session): HTTPHeaderIOS(session, std::ios::out), std::ostream(&_buf) @@ -155,24 +127,4 @@ HTTPHeaderOutputStream::~HTTPHeaderOutputStream() { } - -void* HTTPHeaderOutputStream::operator new(std::size_t size) -{ - return _pool.get(); -} - - -void HTTPHeaderOutputStream::operator delete(void* ptr) -{ - try - { - _pool.release(ptr); - } - catch (...) - { - poco_unexpected(); - } -} - - } } // namespace Poco::Net diff --git a/base/poco/Net/src/HTTPSession.cpp b/base/poco/Net/src/HTTPSession.cpp index cb6fdc25e9a..1e15b24ddae 100644 --- a/base/poco/Net/src/HTTPSession.cpp +++ b/base/poco/Net/src/HTTPSession.cpp @@ -13,8 +13,8 @@ #include "Poco/Net/HTTPSession.h" -#include "Poco/Net/HTTPBufferAllocator.h" #include "Poco/Net/NetException.h" +#include "Poco/Net/HTTPBasicStreamBuf.h" #include @@ -70,7 +70,7 @@ HTTPSession::~HTTPSession() { try { - if (_pBuffer) HTTPBufferAllocator::deallocate(_pBuffer, HTTPBufferAllocator::BUFFER_SIZE); + if (_pBuffer) delete[] _pBuffer; } catch (...) { @@ -177,10 +177,10 @@ void HTTPSession::refill() { if (!_pBuffer) { - _pBuffer = HTTPBufferAllocator::allocate(HTTPBufferAllocator::BUFFER_SIZE); + _pBuffer = new char[HTTP_DEFAULT_BUFFER_SIZE]; } _pCurrent = _pEnd = _pBuffer; - int n = receive(_pBuffer, HTTPBufferAllocator::BUFFER_SIZE); + int n = receive(_pBuffer, HTTP_DEFAULT_BUFFER_SIZE); _pEnd += n; } diff --git a/base/poco/Net/src/HTTPStream.cpp b/base/poco/Net/src/HTTPStream.cpp index 4acb881c4f3..c2f27600569 100644 --- a/base/poco/Net/src/HTTPStream.cpp +++ b/base/poco/Net/src/HTTPStream.cpp @@ -26,7 +26,7 @@ namespace Net { HTTPStreamBuf::HTTPStreamBuf(HTTPSession& session, openmode mode): - HTTPBasicStreamBuf(HTTPBufferAllocator::BUFFER_SIZE, mode), + HTTPBasicStreamBuf(HTTP_DEFAULT_BUFFER_SIZE, mode), _session(session), _mode(mode) { @@ -96,10 +96,6 @@ HTTPStreamBuf* HTTPIOS::rdbuf() // HTTPInputStream // - -Poco::MemoryPool HTTPInputStream::_pool(sizeof(HTTPInputStream)); - - HTTPInputStream::HTTPInputStream(HTTPSession& session): HTTPIOS(session, std::ios::in), std::istream(&_buf) @@ -112,33 +108,11 @@ HTTPInputStream::~HTTPInputStream() } -void* HTTPInputStream::operator new(std::size_t size) -{ - return _pool.get(); -} - - -void HTTPInputStream::operator delete(void* ptr) -{ - try - { - _pool.release(ptr); - } - catch (...) - { - poco_unexpected(); - } -} - - // // HTTPOutputStream // -Poco::MemoryPool HTTPOutputStream::_pool(sizeof(HTTPOutputStream)); - - HTTPOutputStream::HTTPOutputStream(HTTPSession& session): HTTPIOS(session, std::ios::out), std::ostream(&_buf) @@ -150,24 +124,4 @@ HTTPOutputStream::~HTTPOutputStream() { } - -void* HTTPOutputStream::operator new(std::size_t size) -{ - return _pool.get(); -} - - -void HTTPOutputStream::operator delete(void* ptr) -{ - try - { - _pool.release(ptr); - } - catch (...) - { - poco_unexpected(); - } -} - - } } // namespace Poco::Net From a74bc6190da07904116ffa36f6ee1340afb74c91 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 3 Jul 2023 15:46:30 +0200 Subject: [PATCH 212/228] Implement suggestion --- base/poco/Net/include/Poco/Net/HTTPSession.h | 2 +- base/poco/Net/src/HTTPSession.cpp | 16 ++++------------ 2 files changed, 5 insertions(+), 13 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPSession.h b/base/poco/Net/include/Poco/Net/HTTPSession.h index d0045025f5f..934b34be5d5 100644 --- a/base/poco/Net/include/Poco/Net/HTTPSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPSession.h @@ -192,7 +192,7 @@ namespace Net HTTPSession & operator=(const HTTPSession &); StreamSocket _socket; - char * _pBuffer; + std::unique_ptr _pBuffer; char * _pCurrent; char * _pEnd; bool _keepAlive; diff --git a/base/poco/Net/src/HTTPSession.cpp b/base/poco/Net/src/HTTPSession.cpp index 1e15b24ddae..d2663baaf9f 100644 --- a/base/poco/Net/src/HTTPSession.cpp +++ b/base/poco/Net/src/HTTPSession.cpp @@ -68,14 +68,6 @@ HTTPSession::HTTPSession(const StreamSocket& socket, bool keepAlive): HTTPSession::~HTTPSession() { - try - { - if (_pBuffer) delete[] _pBuffer; - } - catch (...) - { - poco_unexpected(); - } try { close(); @@ -177,10 +169,10 @@ void HTTPSession::refill() { if (!_pBuffer) { - _pBuffer = new char[HTTP_DEFAULT_BUFFER_SIZE]; + _pBuffer = std::make_unique(HTTP_DEFAULT_BUFFER_SIZE); } - _pCurrent = _pEnd = _pBuffer; - int n = receive(_pBuffer, HTTP_DEFAULT_BUFFER_SIZE); + _pCurrent = _pEnd = _pBuffer.get(); + int n = receive(_pBuffer.get(), HTTP_DEFAULT_BUFFER_SIZE); _pEnd += n; } @@ -199,7 +191,7 @@ void HTTPSession::connect(const SocketAddress& address) _socket.setNoDelay(true); // There may be leftover data from a previous (failed) request in the buffer, // so we clear it. - _pCurrent = _pEnd = _pBuffer; + _pCurrent = _pEnd = _pBuffer.get(); } From ebd7ecb230a101bc5bed7d3ad79c9f08cba523f5 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 25 May 2023 14:44:24 +0000 Subject: [PATCH 213/228] Remove unstable queries from performance/join_set_filter --- tests/performance/join_set_filter.xml | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/performance/join_set_filter.xml b/tests/performance/join_set_filter.xml index 7f7804853fc..712b60ae12f 100644 --- a/tests/performance/join_set_filter.xml +++ b/tests/performance/join_set_filter.xml @@ -34,9 +34,6 @@ SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE greater(t1.y, {table_size} - 10000) SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE greater(t1.y, {table_size} - 10000) - SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.y % 100 = 0 - SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE t1.y % 100 = 0 - SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.y % 1000 = 0 SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE t1.y % 1000 = 0 From 9ea5d929a5f80466d0be721266302b67c4e54fa0 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 1 Jun 2023 13:45:58 +0200 Subject: [PATCH 214/228] Update tests/performance/join_set_filter.xml --- tests/performance/join_set_filter.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/performance/join_set_filter.xml b/tests/performance/join_set_filter.xml index 712b60ae12f..38ff9beb827 100644 --- a/tests/performance/join_set_filter.xml +++ b/tests/performance/join_set_filter.xml @@ -33,7 +33,8 @@ SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE greater(t1.y, {table_size} - 10000) SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE greater(t1.y, {table_size} - 10000) - + SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.y % 100 = 0 AND t1.y < 10000 + SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE t1.y % 100 = 0 AND t1.y < 10000 SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.y % 1000 = 0 SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE t1.y % 1000 = 0 From 737cff7e5703941e1e05b4f9c70b18f1f508f2f8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 3 Jul 2023 14:42:58 +0200 Subject: [PATCH 215/228] Remove whole join_set_filter.xml, will resubmit --- tests/performance/join_set_filter.xml | 43 --------------------------- 1 file changed, 43 deletions(-) delete mode 100644 tests/performance/join_set_filter.xml diff --git a/tests/performance/join_set_filter.xml b/tests/performance/join_set_filter.xml deleted file mode 100644 index 38ff9beb827..00000000000 --- a/tests/performance/join_set_filter.xml +++ /dev/null @@ -1,43 +0,0 @@ - - - - table_size - - 100000000 - - - - - - full_sorting_merge - - - - CREATE TABLE t1 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y - AS SELECT - sipHash64(number, 't1_x') % {table_size} AS x, - sipHash64(number, 't1_y') % {table_size} AS y - FROM numbers({table_size}) - - - - CREATE TABLE t2 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y - AS SELECT - sipHash64(number, 't2_x') % {table_size} AS x, - sipHash64(number, 't2_y') % {table_size} AS y - FROM numbers({table_size}) - - - SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE less(t1.y, 10000) - SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE less(t1.y, 10000) - - SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE greater(t1.y, {table_size} - 10000) - SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE greater(t1.y, {table_size} - 10000) - SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.y % 100 = 0 AND t1.y < 10000 - SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE t1.y % 100 = 0 AND t1.y < 10000 - SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.y % 1000 = 0 - SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE t1.y % 1000 = 0 - - DROP TABLE IF EXISTS t1 - DROP TABLE IF EXISTS t2 - From 509a0c6f30543b364f805deaca5ba8a025ab8538 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Mon, 3 Jul 2023 13:50:40 -0400 Subject: [PATCH 216/228] 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 217/228] 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 218/228] 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 219/228] 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 220/228] 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 221/228] 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 222/228] 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: From e99e0ebddb712ab40e22be4ab74ebd2ae3786ba0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 4 Jul 2023 13:26:55 +0300 Subject: [PATCH 223/228] Update 01320_create_sync_race_condition_zookeeper.sh (#51742) --- .../01320_create_sync_race_condition_zookeeper.sh | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh index ef45e8e63bc..aee69e64b1b 100755 --- a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh +++ b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh @@ -12,7 +12,10 @@ $CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 --query "CREATE DATABA function thread1() { - while true; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE test_01320.r (x UInt64) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r') ORDER BY x; DROP TABLE test_01320.r;"; done + while true; do + $CLICKHOUSE_CLIENT -n --query "CREATE TABLE test_01320.r (x UInt64) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r') ORDER BY x; + DROP TABLE test_01320.r;" 2>&1 | grep -F "Code:" | grep -v "UNKNOWN_DATABASE" + done } function thread2() From 35c13219520b8b41c633f41423af91b9fda898e1 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 4 Jul 2023 12:35:16 +0200 Subject: [PATCH 224/228] Version mypy=1.4.1 falsly reports unused ignore comment --- docker/test/style/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index 746cc7bb2d5..2aa0b1a62d6 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -18,7 +18,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ python3-pip \ shellcheck \ yamllint \ - && pip3 install black==23.1.0 boto3 codespell==2.2.1 dohq-artifactory mypy PyGithub unidiff pylint==2.6.2 \ + && pip3 install black==23.1.0 boto3 codespell==2.2.1 mypy==1.3.0 PyGithub unidiff pylint==2.6.2 \ && apt-get clean \ && rm -rf /root/.cache/pip From 463c2ba5ce13a6839dfaf12223826b9e90f143c8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 4 Jul 2023 14:07:10 +0300 Subject: [PATCH 225/228] Revert "Add documentation for building in docker" --- docker/packager/README.md | 2 +- docker/packager/packager | 16 ++++++++-------- docs/en/development/build.md | 14 -------------- tests/ci/build_check.py | 6 +++--- tests/ci/build_report_check.py | 4 ++-- tests/ci/ci_config.py | 34 +++++++++++++++++----------------- tests/ci/report.py | 6 +++--- 7 files changed, 34 insertions(+), 48 deletions(-) diff --git a/docker/packager/README.md b/docker/packager/README.md index 3a91f9a63f0..a78feb8d7fc 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 --debug-build +$ ./packager --output-dir deb/test_output/ --package-type deb --compiler=clang-14 --build-type=debug $ 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 3c3304165b3..1b3df858cd2 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(debug_build: bool, package_type: str, sanitizer: str) -> bool: - return not debug_build and package_type == "deb" and sanitizer == "" +def is_release_build(build_type: str, package_type: str, sanitizer: str) -> bool: + return build_type == "" and package_type == "deb" and sanitizer == "" def parse_env_variables( - debug_build: bool, + build_type: str, 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(debug_build, package_type, sanitizer): + if is_release_build(build_type, 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 debug_build: - result.append("BUILD_TYPE=DEBUG") + if build_type: + result.append(f"BUILD_TYPE={build_type.capitalize()}") 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("--debug-build", action="store_true") + parser.add_argument("--build-type", choices=("debug", ""), default="") parser.add_argument( "--compiler", @@ -464,7 +464,7 @@ def main(): build_image(image_with_version, dockerfile) env_prepared = parse_env_variables( - args.debug_build, + args.build_type, args.compiler, args.sanitizer, args.package_type, diff --git a/docs/en/development/build.md b/docs/en/development/build.md index ab47ee8aac5..83a4550df88 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -13,20 +13,6 @@ 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. diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 2a636faf967..35b98a7c3bb 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["debug_build"]: + if build_config["build_type"] != "": return True return False @@ -66,8 +66,8 @@ def get_packager_cmd( f"--package-type={package_type} --compiler={comp}" ) - if build_config["debug_build"]: - cmd += " --debug-build" + if build_config["build_type"]: + cmd += f" --build-type={build_config['build_type']}" 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 295b6cf9740..1362f3c8934 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", - debug_build=False, + build_type="unknown", 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"], - debug_build=build_config["debug_build"], + build_type=build_config["build_type"], 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 875c5a3c8bd..c680b5810fc 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", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "deb", "static_binary_name": "amd64", @@ -21,7 +21,7 @@ CI_CONFIG = { }, "coverity": { "compiler": "clang-16", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "coverity", "tidy": "disable", @@ -31,7 +31,7 @@ CI_CONFIG = { }, "package_aarch64": { "compiler": "clang-16-aarch64", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "deb", "static_binary_name": "aarch64", @@ -42,7 +42,7 @@ CI_CONFIG = { }, "package_asan": { "compiler": "clang-16", - "debug_build": False, + "build_type": "", "sanitizer": "address", "package_type": "deb", "tidy": "disable", @@ -51,7 +51,7 @@ CI_CONFIG = { }, "package_ubsan": { "compiler": "clang-16", - "debug_build": False, + "build_type": "", "sanitizer": "undefined", "package_type": "deb", "tidy": "disable", @@ -60,7 +60,7 @@ CI_CONFIG = { }, "package_tsan": { "compiler": "clang-16", - "debug_build": False, + "build_type": "", "sanitizer": "thread", "package_type": "deb", "tidy": "disable", @@ -69,7 +69,7 @@ CI_CONFIG = { }, "package_msan": { "compiler": "clang-16", - "debug_build": False, + "build_type": "", "sanitizer": "memory", "package_type": "deb", "tidy": "disable", @@ -78,7 +78,7 @@ CI_CONFIG = { }, "package_debug": { "compiler": "clang-16", - "debug_build": True, + "build_type": "debug", "sanitizer": "", "package_type": "deb", "tidy": "disable", @@ -87,7 +87,7 @@ CI_CONFIG = { }, "binary_release": { "compiler": "clang-16", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "binary", "tidy": "disable", @@ -96,7 +96,7 @@ CI_CONFIG = { }, "binary_tidy": { "compiler": "clang-16", - "debug_build": True, + "build_type": "debug", "sanitizer": "", "package_type": "binary", "static_binary_name": "debug-amd64", @@ -106,7 +106,7 @@ CI_CONFIG = { }, "binary_darwin": { "compiler": "clang-16-darwin", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "binary", "static_binary_name": "macos", @@ -116,7 +116,7 @@ CI_CONFIG = { }, "binary_aarch64": { "compiler": "clang-16-aarch64", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "binary", "tidy": "disable", @@ -125,7 +125,7 @@ CI_CONFIG = { }, "binary_aarch64_v80compat": { "compiler": "clang-16-aarch64-v80compat", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "binary", "static_binary_name": "aarch64v80compat", @@ -135,7 +135,7 @@ CI_CONFIG = { }, "binary_freebsd": { "compiler": "clang-16-freebsd", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "binary", "static_binary_name": "freebsd", @@ -145,7 +145,7 @@ CI_CONFIG = { }, "binary_darwin_aarch64": { "compiler": "clang-16-darwin-aarch64", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "binary", "static_binary_name": "macos-aarch64", @@ -155,7 +155,7 @@ CI_CONFIG = { }, "binary_ppc64le": { "compiler": "clang-16-ppc64le", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "binary", "static_binary_name": "powerpc64le", @@ -165,7 +165,7 @@ CI_CONFIG = { }, "binary_amd64_compat": { "compiler": "clang-16-amd64-compat", - "debug_build": False, + "build_type": "", "sanitizer": "", "package_type": "binary", "static_binary_name": "amd64compat", diff --git a/tests/ci/report.py b/tests/ci/report.py index 0f84fbcaeb2..a9014acec12 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 - debug_build: bool + build_type: str 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.debug_build: - row += "debug" + if build_result.build_type: + row += f"{build_result.build_type}" else: row += "relwithdebuginfo" if build_result.sanitizer: From 5230503651e091f4bf1944a1176aeac47c834907 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 4 Jul 2023 12:59:25 +0000 Subject: [PATCH 226/228] remove unused type: ignore --- tests/ci/commit_status_helper.py | 2 +- tests/ci/github_helper.py | 2 +- tests/ci/merge_pr.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 4437ddc52c6..efe149b0aa4 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -7,7 +7,7 @@ from typing import Dict, List, Literal, Optional, Union import logging from github import Github -from github.GithubObject import _NotSetType, NotSet as NotSet # type: ignore +from github.GithubObject import _NotSetType, NotSet as NotSet from github.Commit import Commit from github.CommitStatus import CommitStatus from github.IssueComment import IssueComment diff --git a/tests/ci/github_helper.py b/tests/ci/github_helper.py index 834c8247cb8..f7256e523b1 100644 --- a/tests/ci/github_helper.py +++ b/tests/ci/github_helper.py @@ -111,7 +111,7 @@ class GitHub(github.Github): # See https://github.com/PyGithub/PyGithub/issues/2202, # obj._rawData doesn't spend additional API requests # pylint: disable=protected-access - repo_url = issue._rawData["repository_url"] # type: ignore + repo_url = issue._rawData["repository_url"] if repo_url not in repos: repos[repo_url] = issue.repository prs.append( diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 2d6d81a152a..14844ed9b25 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -154,7 +154,7 @@ def get_workflows_for_head(repo: Repository, head_sha: str) -> List[WorkflowRun] return list( PaginatedList( WorkflowRun, - repo._requester, # type:ignore # pylint:disable=protected-access + repo._requester, # pylint:disable=protected-access f"{repo.url}/actions/runs", {"head_sha": head_sha}, list_item="workflow_runs", From 3cb459bd04d141ca0fffe7c1f6c389e4be434167 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 4 Jul 2023 15:50:56 +0000 Subject: [PATCH 227/228] Docs: Fix description of output field NON_UNIQUE for statement SHOW INDEXES --- docs/en/sql-reference/statements/show.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index f96eb55aa45..336b93db9d5 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -283,7 +283,7 @@ The optional keyword `EXTENDED` currently has no effect, it only exists for MySQ `SHOW INDEX` produces a result table with the following structure: - table - The name of the table (String) -- non_unique - 0 if the index can contain duplicates, 1 otherwise (UInt8) +- non_unique - 0 if the index cannot contain duplicates, 1 otherwise (UInt8) - key_name - The name of the index, `PRIMARY` if the index is a primary key index (String) - seq_in_index - Currently unused - column_name - Currently unused From 32ee0e7d08d0f525c69cad6df1639c3d27888a7c Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 4 Jul 2023 17:35:26 +0000 Subject: [PATCH 228/228] Add assert in ThreadStatus destructor for correct current_thread --- src/Common/ThreadStatus.cpp | 9 ++++++--- src/Common/ThreadStatus.h | 4 +++- src/Processors/Transforms/buildPushingToViewsChain.cpp | 2 +- 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 9b0743d89c3..7a602afe7e7 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -67,8 +67,8 @@ ThreadGroup::ThreadGroup() : master_thread_id(CurrentThread::get().thread_id) {} -ThreadStatus::ThreadStatus() - : thread_id{getThreadId()} +ThreadStatus::ThreadStatus(bool check_current_thread_on_destruction_) + : thread_id{getThreadId()}, check_current_thread_on_destruction(check_current_thread_on_destruction_) { last_rusage = std::make_unique(); @@ -201,8 +201,11 @@ ThreadStatus::~ThreadStatus() /// Only change current_thread if it's currently being used by this ThreadStatus /// For example, PushingToViews chain creates and deletes ThreadStatus instances while running in the main query thread - if (current_thread == this) + if (check_current_thread_on_destruction) + { + assert(current_thread == this); current_thread = nullptr; + } } void ThreadStatus::updatePerformanceCounters() diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 061959d9f1f..6e4f074a162 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -224,8 +224,10 @@ private: Poco::Logger * log = nullptr; + bool check_current_thread_on_destruction; + public: - ThreadStatus(); + explicit ThreadStatus(bool check_current_thread_on_destruction_ = true); ~ThreadStatus(); ThreadGroupPtr getThreadGroup() const; diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 43085690519..7f7f9058f1b 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -282,7 +282,7 @@ Chain buildPushingToViewsChain( auto * original_thread = current_thread; SCOPE_EXIT({ current_thread = original_thread; }); - std::unique_ptr view_thread_status_ptr = std::make_unique(); + std::unique_ptr view_thread_status_ptr = std::make_unique(/*check_current_thread_on_destruction=*/ false); /// Copy of a ThreadStatus should be internal. view_thread_status_ptr->setInternalThread(); view_thread_status_ptr->attachToGroup(running_group);