From f29700bd2fe1fc79cb6fd5bf2b31129119e6978c Mon Sep 17 00:00:00 2001 From: LiuYangkuan Date: Wed, 1 Feb 2023 18:50:43 +0800 Subject: [PATCH 01/43] 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 bf6afd27a655bb159583af2020113e101ca82e00 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 17 May 2023 13:32:51 +0000 Subject: [PATCH 02/43] 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 03/43] 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 04/43] 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 05/43] 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 06/43] 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 07/43] 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 08/43] 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 09/43] 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 10/43] 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 11/43] 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 12/43] 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 13/43] 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 14/43] 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 15/43] 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 16/43] 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 17/43] 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 18/43] 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 19/43] 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 20/43] 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 21/43] 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 22/43] 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 25f08f8d194f77d0ee56e7c5132b9d5c4244a30f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 12 Jun 2023 15:35:44 +0200 Subject: [PATCH 23/43] 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 24/43] 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 f9f1e870c8468b829b4aa449c8a9b3736b733056 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 14 Jun 2023 10:09:01 +0000 Subject: [PATCH 25/43] 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 f5327e79bb37c3cc061b8704ffdb85aa4f0b31c4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 27 Jun 2023 20:18:38 +0200 Subject: [PATCH 26/43] 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 594ec09edf0d63b73e02c2dbbee6cc500eb52d87 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 28 Jun 2023 18:19:16 +0200 Subject: [PATCH 27/43] 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 28/43] 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 296f9968c04f6ca49599e281b4092f6cffef5bfd Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 28 Jun 2023 23:20:02 +0200 Subject: [PATCH 29/43] 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 b27cf4e7ba0d9d579722202f693f820777c6bfc7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 29 Jun 2023 00:15:45 +0200 Subject: [PATCH 30/43] 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 bac126fe3edbaec15834ba21ae57b3aec0f3d299 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 29 Jun 2023 15:11:02 +0200 Subject: [PATCH 31/43] 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 32/43] 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 542b03bfd1bf8b7f6af69c76c2ef1d7f86d535b0 Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Fri, 30 Jun 2023 10:41:31 -0400 Subject: [PATCH 33/43] 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 185e106c1faff2201db3f33f7497ab4a43d1c7e6 Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Fri, 30 Jun 2023 10:53:04 -0400 Subject: [PATCH 34/43] 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 4511213a1f61ae7d67c8c44b6bab0ff3fe6ebaaa Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Fri, 30 Jun 2023 13:52:22 -0400 Subject: [PATCH 35/43] 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 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 36/43] 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 37/43] 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 38/43] 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 39/43] 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 aeea3dc8247fecc8f4be75c3330cce696bd054ba Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 3 Jul 2023 07:40:05 +0000 Subject: [PATCH 40/43] Add odbcinst dep to sqllogic docker --- docker/test/sqllogic/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/sqllogic/Dockerfile b/docker/test/sqllogic/Dockerfile index 83dcf7e1f56..5cf71e4d3f8 100644 --- a/docker/test/sqllogic/Dockerfile +++ b/docker/test/sqllogic/Dockerfile @@ -13,6 +13,7 @@ RUN apt-get update --yes \ sqlite3 \ unixodbc \ unixodbc-dev \ + odbcinst \ sudo \ && apt-get clean From f1709b3cf4360bbd57e7740a305d2a888849197b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 3 Jul 2023 12:58:41 +0300 Subject: [PATCH 41/43] 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 42/43] 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 138e28cf6b98697297c35430f877782c437ddbe0 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Mon, 3 Jul 2023 13:45:39 +0200 Subject: [PATCH 43/43] 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.