From eeb71672a0217f6b7660872ec8f84f3851627f0a Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Apr 2021 03:05:43 +0300 Subject: [PATCH 01/55] Change in Storages/* --- src/Storages/Distributed/DirectoryMonitor.cpp | 50 ++++++++----------- .../DistributedBlockOutputStream.cpp | 18 +++---- src/Storages/MergeTree/DataPartsExchange.cpp | 12 ++--- src/Storages/MergeTree/MergeTreeData.cpp | 9 ++-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 - .../MergeTreeIndexGranularityInfo.cpp | 7 +-- .../MergeTree/MergeTreeReaderInMemory.cpp | 1 - .../RocksDB/StorageEmbeddedRocksDB.cpp | 12 +++-- src/Storages/StorageDistributed.cpp | 4 +- src/Storages/StorageFile.cpp | 34 ++++++------- src/Storages/StorageFile.h | 4 -- 11 files changed, 71 insertions(+), 82 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 2afa9747c60..ac376f92e73 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -25,11 +25,9 @@ #include #include #include - #include #include - -#include +#include namespace CurrentMetrics @@ -38,6 +36,8 @@ namespace CurrentMetrics extern const Metric DistributedFilesToInsert; } +namespace fs = std::filesystem; + namespace DB { @@ -332,7 +332,7 @@ void StorageDistributedDirectoryMonitor::shutdownAndDropAllData() } auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path); - Poco::File(path).remove(true); + fs::remove_all(path); } @@ -475,16 +475,15 @@ std::map StorageDistributedDirectoryMonitor::getFiles() std::map files; size_t new_bytes_count = 0; - Poco::DirectoryIterator end; - for (Poco::DirectoryIterator it{path}; it != end; ++it) + fs::directory_iterator end; + for (fs::directory_iterator it{path}; it != end; ++it) { const auto & file_path_str = it->path(); - Poco::Path file_path{file_path_str}; - - if (!it->isDirectory() && startsWith(file_path.getExtension(), "bin")) + fs::path fs_file_path(file_path_str); + if (!it->is_directory() && startsWith(fs_file_path.extension(), ".bin")) { - files[parse(file_path.getBaseName())] = file_path_str; - new_bytes_count += Poco::File(file_path).getSize(); + files[parse(fs_file_path.stem())] = file_path_str; + new_bytes_count += fs::file_size(fs_file_path); } } @@ -646,8 +645,7 @@ struct StorageDistributedDirectoryMonitor::Batch String tmp_file{parent.current_batch_file_path + ".tmp"}; auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, parent.disk, parent.relative_path); - - if (Poco::File{tmp_file}.exists()) + if (fs::exists(tmp_file)) LOG_ERROR(parent.log, "Temporary file {} exists. Unclean shutdown?", backQuote(tmp_file)); { @@ -659,7 +657,7 @@ struct StorageDistributedDirectoryMonitor::Batch out.sync(); } - Poco::File{tmp_file}.renameTo(parent.current_batch_file_path); + fs::rename(tmp_file, parent.current_batch_file_path); } auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(parent.storage.getContext()->getSettingsRef()); auto connection = parent.pool->get(timeouts); @@ -732,7 +730,7 @@ struct StorageDistributedDirectoryMonitor::Batch total_bytes = 0; recovered = false; - Poco::File{parent.current_batch_file_path}.setSize(0); + fs::resize_file(parent.current_batch_file_path, 0); } void writeText(WriteBuffer & out) @@ -832,7 +830,7 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map { std::unordered_set file_indices_to_skip; - if (Poco::File{current_batch_file_path}.exists()) + if (fs::exists(current_batch_file_path)) { /// Possibly, we failed to send a batch on the previous iteration. Try to send exactly the same batch. Batch batch(*this, files); @@ -933,8 +931,8 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map /// current_batch.txt will not exist if there was no send /// (this is the case when all batches that was pending has been marked as pending) - if (Poco::File{current_batch_file_path}.exists()) - Poco::File{current_batch_file_path}.remove(); + if (fs::exists(current_batch_file_path)) + fs::remove(current_batch_file_path); } } @@ -946,30 +944,26 @@ void StorageDistributedDirectoryMonitor::markAsBroken(const std::string & file_p const auto & broken_path = base_path + "broken/"; const auto & broken_file_path = broken_path + file_name; - Poco::File{broken_path}.createDirectory(); + fs::create_directory(broken_path); auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path); auto broken_dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path + "/broken/"); - Poco::File file(file_path); - { std::lock_guard metrics_lock(metrics_mutex); - size_t file_size = file.getSize(); + size_t file_size = fs::file_size(file_path); --files_count; bytes_count -= file_size; } - file.renameTo(broken_file_path); - + fs::rename(file_path, broken_file_path); LOG_ERROR(log, "Renamed `{}` to `{}`", file_path, broken_file_path); } + void StorageDistributedDirectoryMonitor::markAsSend(const std::string & file_path) { - Poco::File file(file_path); - - size_t file_size = file.getSize(); + size_t file_size = fs::file_size(file_path); { std::lock_guard metrics_lock(metrics_mutex); @@ -978,7 +972,7 @@ void StorageDistributedDirectoryMonitor::markAsSend(const std::string & file_pat bytes_count -= file_size; } - file.remove(); + fs::remove(file_path); } bool StorageDistributedDirectoryMonitor::maybeMarkAsBroken(const std::string & file_path, const Exception & e) diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index d05fbae60d9..c5d4dc7486c 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -33,11 +33,10 @@ #include #include -#include - #include #include #include +#include namespace CurrentMetrics @@ -50,10 +49,11 @@ namespace ProfileEvents extern const Event DistributedSyncInsertionTimeoutExceeded; } +namespace fs = std::filesystem; + namespace DB { - namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -656,10 +656,10 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: /// hardlinking to ensure the inode is not freed until we're done { const std::string path(disk_path + data_path + *it); - Poco::File(path).createDirectory(); - const std::string tmp_path(path + "/tmp/"); - Poco::File(tmp_path).createDirectory(); + + fs::create_directory(path); + fs::create_directory(tmp_path); const std::string file_name(toString(storage.file_names_increment.get()) + ".bin"); @@ -723,17 +723,17 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: for (; it != dir_names.end(); ++it) { const std::string path(disk_path + data_path + *it); - Poco::File(path).createDirectory(); + fs::create_directory(path); const std::string block_file_path(path + '/' + toString(storage.file_names_increment.get()) + ".bin"); createHardLink(first_file_tmp_path, block_file_path); auto dir_sync_guard = make_directory_sync_guard(*it); } - auto file_size = Poco::File(first_file_tmp_path).getSize(); + auto file_size = fs::file_size(first_file_tmp_path); /// remove the temporary file, enabling the OS to reclaim inode after all threads /// have removed their corresponding files - Poco::File(first_file_tmp_path).remove(); + fs::remove(first_file_tmp_path); /// Notify auto sleep_ms = context->getSettingsRef().distributed_directory_monitor_sleep_time_ms; diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 205d57f533e..dfeb31af7b7 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -13,11 +13,11 @@ #include #include #include - -#include #include +namespace fs = std::filesystem; + namespace CurrentMetrics { extern const Metric ReplicatedSend; @@ -281,13 +281,13 @@ void Service::sendPartS3Metadata(const MergeTreeData::DataPartPtr & part, WriteB String metadata_file = disk->getPath() + part->getFullRelativePath() + file_name; - Poco::File metadata(metadata_file); + fs::path metadata(metadata_file); - if (!metadata.exists()) + if (!fs::exists(metadata)) throw Exception("S3 metadata '" + file_name + "' is not exists", ErrorCodes::CORRUPTED_DATA); - if (!metadata.isFile()) + if (!fs::is_regular_file(metadata)) throw Exception("S3 metadata '" + file_name + "' is not a file", ErrorCodes::CORRUPTED_DATA); - UInt64 file_size = metadata.getSize(); + UInt64 file_size = fs::file_size(metadata); writeStringBinary(it.first, out); writeBinary(file_size, out); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ee8e15008cb..426b95968b6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -51,8 +51,6 @@ #include #include -#include - #include #include @@ -66,8 +64,11 @@ #include #include #include +#include +namespace fs = std::filesystem; + namespace ProfileEvents { extern const Event RejectedInserts; @@ -3834,9 +3835,9 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( const String & with_name, ContextPtr local_context) { - String clickhouse_path = Poco::Path(local_context->getPath()).makeAbsolute().toString(); + String clickhouse_path = fs::absolute(local_context->getPath()); String default_shadow_path = clickhouse_path + "shadow/"; - Poco::File(default_shadow_path).createDirectories(); + fs::create_directories(default_shadow_path); auto increment = Increment(default_shadow_path + "increment.txt").get(true); const String shadow_path = "shadow/"; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index ddb140989f6..0f30929eeb8 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -3,8 +3,6 @@ #include #include -#include - #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp index ed0e7e55fc8..6da0a822f7f 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp @@ -1,6 +1,8 @@ #include #include -#include + + +namespace fs = std::filesystem; namespace DB { @@ -17,8 +19,7 @@ std::optional MergeTreeIndexGranularityInfo::getMarksExtensionFromF { for (DiskDirectoryIteratorPtr it = disk->iterateDirectory(path_to_part); it->isValid(); it->next()) { - Poco::Path path(it->path()); - const auto & ext = "." + path.getExtension(); + const auto & ext = fs::path(it->path()).extension(); if (ext == getNonAdaptiveMrkExtension() || ext == getAdaptiveMrkExtension(MergeTreeDataPartType::WIDE) || ext == getAdaptiveMrkExtension(MergeTreeDataPartType::COMPACT)) diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index 5ee4aa555e6..cf5c850daa1 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 9173c23ec5a..d878fc63655 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -26,14 +26,16 @@ #include #include -#include -#include #include #include #include #include +#include + + +namespace fs = std::filesystem; namespace DB { @@ -253,7 +255,7 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, rocksdb_dir = context_->getPath() + relative_data_path_; if (!attach) { - Poco::File(rocksdb_dir).createDirectories(); + fs::create_directories(rocksdb_dir); } initDb(); } @@ -261,8 +263,8 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, void StorageEmbeddedRocksDB::truncate(const ASTPtr &, const StorageMetadataPtr & , ContextPtr, TableExclusiveLockHolder &) { rocksdb_ptr->Close(); - Poco::File(rocksdb_dir).remove(true); - Poco::File(rocksdb_dir).createDirectories(); + fs::remove_all(rocksdb_dir); + fs::create_directories(rocksdb_dir); initDb(); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index e42e53d3f1b..6c3278d9598 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -76,6 +76,8 @@ #include +namespace fs = std::filesystem; + namespace { const UInt64 FORCE_OPTIMIZE_SKIP_UNUSED_SHARDS_HAS_SHARDING_KEY = 1; @@ -845,7 +847,7 @@ StoragePolicyPtr StorageDistributed::getStoragePolicy() const void StorageDistributed::createDirectoryMonitors(const DiskPtr & disk) { const std::string path(disk->getPath() + relative_data_path); - Poco::File{path}.createDirectories(); + fs::create_directories(path); std::filesystem::directory_iterator begin(path); std::filesystem::directory_iterator end; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 14b91d29805..be052f91b39 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -28,9 +28,6 @@ #include #include -#include -#include - #include #include #include @@ -39,6 +36,7 @@ #include #include + namespace fs = std::filesystem; namespace DB @@ -78,10 +76,9 @@ std::vector listFilesWithRegexpMatching(const std::string & path_fo std::vector result; const std::string prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); - if (!fs::exists(fs::path(prefix_without_globs))) - { + if (!fs::exists(prefix_without_globs)) return result; - } + const fs::directory_iterator end; for (fs::directory_iterator it(prefix_without_globs); it != end; ++it) { @@ -125,21 +122,20 @@ void checkCreationIsAllowed(ContextPtr context_global, const std::string & db_di if (!startsWith(table_path, db_dir_path) && table_path != "/dev/null") throw Exception("File is not inside " + db_dir_path, ErrorCodes::DATABASE_ACCESS_DENIED); - Poco::File table_path_poco_file = Poco::File(table_path); - if (table_path_poco_file.exists() && table_path_poco_file.isDirectory()) + if (fs::exists(table_path) && fs::is_directory(table_path)) throw Exception("File must not be a directory", ErrorCodes::INCORRECT_FILE_NAME); } } Strings StorageFile::getPathsList(const String & table_path, const String & user_files_path, ContextPtr context) { - String user_files_absolute_path = Poco::Path(user_files_path).makeAbsolute().makeDirectory().toString(); - Poco::Path poco_path = Poco::Path(table_path); - if (poco_path.isRelative()) - poco_path = Poco::Path(user_files_absolute_path, poco_path); + fs::path user_files_absolute_path = fs::absolute(user_files_path); + fs::path fs_table_path(table_path); + if (fs_table_path.is_relative()) + fs_table_path = user_files_absolute_path / fs_table_path; Strings paths; - const String path = poco_path.absolute().toString(); + const String path = fs::absolute(fs_table_path); if (path.find_first_of("*?{") == std::string::npos) paths.push_back(path); else @@ -205,7 +201,7 @@ StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArgu throw Exception("Distributed format is allowed only with explicit file path", ErrorCodes::INCORRECT_FILE_NAME); String table_dir_path = base_path + relative_table_dir_path + "/"; - Poco::File(table_dir_path).createDirectories(); + fs::create_directories(table_dir_path); paths = {getTablePath(table_dir_path, format_name)}; } @@ -455,7 +451,7 @@ Pipe StorageFile::read( if (use_table_fd) /// need to call ctr BlockInputStream paths = {""}; /// when use fd, paths are empty else - if (paths.size() == 1 && !Poco::File(paths[0]).exists()) + if (paths.size() == 1 && !fs::exists(paths[0])) { if (context->getSettingsRef().engine_file_empty_if_not_exists) return Pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); @@ -598,7 +594,7 @@ BlockOutputStreamPtr StorageFile::write( if (!paths.empty()) { path = paths[0]; - Poco::File(Poco::Path(path).makeParent()).createDirectories(); + fs::create_directories(fs::path(path).parent_path()); } return std::make_shared( @@ -635,8 +631,8 @@ void StorageFile::rename(const String & new_path_to_table_data, const StorageID if (path_new == paths[0]) return; - Poco::File(Poco::Path(path_new).parent()).createDirectories(); - Poco::File(paths[0]).renameTo(path_new); + fs::create_directories(fs::path(path_new).parent_path()); + fs::rename(paths[0], path_new); paths[0] = std::move(path_new); renameInMemory(new_table_id); @@ -658,7 +654,7 @@ void StorageFile::truncate( } else { - if (!Poco::File(paths[0]).exists()) + if (!fs::exists(paths[0])) return; if (0 != ::truncate(paths[0].c_str(), 0)) diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index a277dda7cc0..b30e5eeb5d8 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -1,10 +1,6 @@ #pragma once #include - -#include -#include - #include #include From 1e4a61ce63347212bc41c5a91554e97a4cc97bd1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Apr 2021 20:22:39 +0300 Subject: [PATCH 02/55] Fix build --- src/Storages/Distributed/DirectoryMonitor.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index ac376f92e73..7381468a399 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -479,11 +479,10 @@ std::map StorageDistributedDirectoryMonitor::getFiles() for (fs::directory_iterator it{path}; it != end; ++it) { const auto & file_path_str = it->path(); - fs::path fs_file_path(file_path_str); - if (!it->is_directory() && startsWith(fs_file_path.extension(), ".bin")) + if (!it->is_directory() && startsWith(fs::path(file_path_str).extension(), ".bin")) { - files[parse(fs_file_path.stem())] = file_path_str; - new_bytes_count += fs::file_size(fs_file_path); + files[parse(fs::path(file_path_str).stem())] = file_path_str; + new_bytes_count += fs::file_size(fs::path(file_path_str)); } } From 5e429b1266feb652accb1ded3ca90e6dc04d6433 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 28 Apr 2021 13:42:07 +0300 Subject: [PATCH 03/55] Change in Interpreters/* --- src/Interpreters/DatabaseCatalog.cpp | 22 +++++------ .../ExternalLoaderXMLConfigRepository.cpp | 4 +- src/Interpreters/loadMetadata.cpp | 39 ++++++++++--------- src/Server/StaticRequestHandler.cpp | 16 ++++---- 4 files changed, 43 insertions(+), 38 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index d88b87a73d4..f9014063b6b 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include "config_core.h" @@ -26,8 +27,7 @@ # include #endif -#include - +namespace fs = std::filesystem; namespace CurrentMetrics { @@ -353,10 +353,9 @@ DatabasePtr DatabaseCatalog::detachDatabase(const String & database_name, bool d db->drop(getContext()); /// Old ClickHouse versions did not store database.sql files - Poco::File database_metadata_file( - getContext()->getPath() + "metadata/" + escapeForFileName(database_name) + ".sql"); - if (database_metadata_file.exists()) - database_metadata_file.remove(false); + fs::path database_metadata_file = fs::path(getContext()->getPath()) / "metadata" / (escapeForFileName(database_name) + ".sql"); + if (fs::exists(database_metadata_file)) + fs::remove_all(database_metadata_file); } return db; @@ -889,16 +888,15 @@ void DatabaseCatalog::dropTableFinally(const TableMarkedAsDropped & table) /// Even if table is not loaded, try remove its data from disk. /// TODO remove data from all volumes - String data_path = getContext()->getPath() + "store/" + getPathForUUID(table.table_id.uuid); - Poco::File table_data_dir{data_path}; - if (table_data_dir.exists()) + fs::path data_path = fs::path(getContext()->getPath()) / "store" / getPathForUUID(table.table_id.uuid); + if (fs::exists(data_path)) { - LOG_INFO(log, "Removing data directory {} of dropped table {}", data_path, table.table_id.getNameForLogs()); - table_data_dir.remove(true); + LOG_INFO(log, "Removing data directory {} of dropped table {}", data_path.string(), table.table_id.getNameForLogs()); + fs::remove_all(data_path); } LOG_INFO(log, "Removing metadata {} of dropped table {}", table.metadata_path, table.table_id.getNameForLogs()); - Poco::File(table.metadata_path).remove(); + fs::remove(fs::path(table.metadata_path)); removeUUIDMappingFinally(table.table_id.uuid); CurrentMetrics::sub(CurrentMetrics::TablesToDropQueueSize, 1); diff --git a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp index 63755ee1839..e1052817407 100644 --- a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp +++ b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp @@ -7,7 +7,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -59,7 +61,7 @@ std::set ExternalLoaderXMLConfigRepository::getAllLoadablesDefiniti bool ExternalLoaderXMLConfigRepository::exists(const std::string & definition_entity_name) { - return Poco::File(definition_entity_name).exists(); + return fs::exists(fs::path(definition_entity_name)); } Poco::AutoPtr ExternalLoaderXMLConfigRepository::load( diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 79076e57328..c109c53af20 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -18,7 +18,9 @@ #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -54,13 +56,13 @@ static void loadDatabase( String database_attach_query; String database_metadata_file = database_path + ".sql"; - if (Poco::File(database_metadata_file).exists()) + if (fs::exists(fs::path(database_metadata_file))) { /// There is .sql file with database creation statement. ReadBufferFromFile in(database_metadata_file, 1024); readStringUntilEOF(database_attach_query, in); } - else if (Poco::File(database_path).exists()) + else if (fs::exists(fs::path(database_path))) { /// Database exists, but .sql file is absent. It's old-style Ordinary database (e.g. system or default) database_attach_query = "ATTACH DATABASE " + backQuoteIfNeed(database) + " ENGINE = Ordinary"; @@ -95,34 +97,35 @@ void loadMetadata(ContextPtr context, const String & default_database_name) * This file is deleted after successful loading of tables. * (flag is "one-shot") */ - Poco::File force_restore_data_flag_file(context->getFlagsPath() + "force_restore_data"); - bool has_force_restore_data_flag = force_restore_data_flag_file.exists(); + auto force_restore_data_flag_file = fs::path(context->getFlagsPath()) / "force_restore_data"; + bool has_force_restore_data_flag = fs::exists(force_restore_data_flag_file); /// Loop over databases. std::map databases; - Poco::DirectoryIterator dir_end; - for (Poco::DirectoryIterator it(path); it != dir_end; ++it) + fs::directory_iterator dir_end; + for (fs::directory_iterator it(path); it != dir_end; ++it) { - if (it->isLink()) + if (it->is_symlink()) continue; - if (!it->isDirectory()) + const auto current_file = it->path().filename().string(); + if (!it->is_directory()) { /// TODO: DETACH DATABASE PERMANENTLY ? - if (endsWith(it.name(), ".sql")) + if (endsWith(current_file, ".sql")) { - String db_name = it.name().substr(0, it.name().size() - 4); + String db_name = current_file.substr(0, current_file.size() - 4); if (db_name != DatabaseCatalog::SYSTEM_DATABASE) databases.emplace(unescapeForFileName(db_name), path + "/" + db_name); } /// Temporary fails may be left from previous server runs. - if (endsWith(it.name(), ".tmp")) + if (endsWith(current_file, ".tmp")) { - LOG_WARNING(log, "Removing temporary file {}", it->path()); + LOG_WARNING(log, "Removing temporary file {}", it->path().string()); try { - it->remove(); + fs::remove(it->path()); } catch (...) { @@ -135,13 +138,13 @@ void loadMetadata(ContextPtr context, const String & default_database_name) } /// For '.svn', '.gitignore' directory and similar. - if (it.name().at(0) == '.') + if (current_file.at(0) == '.') continue; - if (it.name() == DatabaseCatalog::SYSTEM_DATABASE) + if (current_file == DatabaseCatalog::SYSTEM_DATABASE) continue; - databases.emplace(unescapeForFileName(it.name()), it.path().toString()); + databases.emplace(unescapeForFileName(current_file), it->path().string()); } /// clickhouse-local creates DatabaseMemory as default database by itself @@ -158,7 +161,7 @@ void loadMetadata(ContextPtr context, const String & default_database_name) { try { - force_restore_data_flag_file.remove(); + fs::remove(force_restore_data_flag_file); } catch (...) { @@ -172,7 +175,7 @@ void loadMetadataSystem(ContextPtr context) { String path = context->getPath() + "metadata/" + DatabaseCatalog::SYSTEM_DATABASE; String metadata_file = path + ".sql"; - if (Poco::File(path).exists() || Poco::File(metadata_file).exists()) + if (fs::exists(fs::path(path)) || fs::exists(fs::path(metadata_file))) { /// 'has_force_restore_data_flag' is true, to not fail on loading query_log table, if it is corrupted. loadDatabase(context, DatabaseCatalog::SYSTEM_DATABASE, path, true); diff --git a/src/Server/StaticRequestHandler.cpp b/src/Server/StaticRequestHandler.cpp index 169d6859b43..978271c1f04 100644 --- a/src/Server/StaticRequestHandler.cpp +++ b/src/Server/StaticRequestHandler.cpp @@ -14,14 +14,15 @@ #include -#include -#include #include #include #include #include +#include +namespace fs = std::filesystem; + namespace DB { @@ -137,14 +138,15 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out) if (startsWith(response_expression, file_prefix)) { - const auto & user_files_absolute_path = Poco::Path(server.context()->getUserFilesPath()).makeAbsolute().makeDirectory().toString(); const auto & file_name = response_expression.substr(file_prefix.size(), response_expression.size() - file_prefix.size()); - const auto & file_path = Poco::Path(user_files_absolute_path, file_name).makeAbsolute().toString(); - if (!Poco::File(file_path).exists()) - throw Exception("Invalid file name " + file_path + " for static HTTPHandler. ", ErrorCodes::INCORRECT_FILE_NAME); + fs::path user_files_absolute_path = fs::absolute(fs::path(server.context()->getUserFilesPath())); + fs::path file_path = user_files_absolute_path / file_name; - ReadBufferFromFile in(file_path); + if (!fs::exists(file_path)) + throw Exception("Invalid file name " + file_path.string() + " for static HTTPHandler. ", ErrorCodes::INCORRECT_FILE_NAME); + + ReadBufferFromFile in(file_path.string()); copyData(in, out); } else if (startsWith(response_expression, config_prefix)) From 3386dc32d3a4d42adfe177dc79ef38a9ee868e5f Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 28 Apr 2021 23:48:34 +0300 Subject: [PATCH 04/55] Change in Functions/* --- src/Functions/FunctionFile.cpp | 32 ++++++++++++++++---------------- src/Interpreters/Context.cpp | 15 +++++++++------ 2 files changed, 25 insertions(+), 22 deletions(-) diff --git a/src/Functions/FunctionFile.cpp b/src/Functions/FunctionFile.cpp index 9247152367b..7c87f27c731 100644 --- a/src/Functions/FunctionFile.cpp +++ b/src/Functions/FunctionFile.cpp @@ -3,10 +3,11 @@ #include #include #include -#include -#include #include #include +#include + +namespace fs = std::filesystem; namespace DB { @@ -68,21 +69,20 @@ public: { const char * filename = reinterpret_cast(&chars[source_offset]); - const String user_files_path = getContext()->getUserFilesPath(); - String user_files_absolute_path = Poco::Path(user_files_path).makeAbsolute().makeDirectory().toString(); - Poco::Path poco_filepath = Poco::Path(filename); - if (poco_filepath.isRelative()) - poco_filepath = Poco::Path(user_files_absolute_path, poco_filepath); - const String file_absolute_path = poco_filepath.absolute().toString(); - checkReadIsAllowedOrThrow(user_files_absolute_path, file_absolute_path); + fs::path user_files_absolute_path = fs::canonical(fs::path(getContext()->getUserFilesPath())); + fs::path file_path(filename); + if (file_path.is_relative()) + file_path = user_files_absolute_path / file_path; - checked_filenames[row] = file_absolute_path; - auto file = Poco::File(file_absolute_path); + fs::path file_absolute_path = fs::absolute(file_path); + checkReadIsAllowedOrThrow(user_files_absolute_path.string(), file_absolute_path); - if (!file.exists()) - throw Exception(fmt::format("File {} doesn't exist.", file_absolute_path), ErrorCodes::FILE_DOESNT_EXIST); + checked_filenames[row] = file_absolute_path.string(); - const auto current_file_size = Poco::File(file_absolute_path).getSize(); + if (!fs::exists(file_absolute_path)) + throw Exception(fmt::format("File {} doesn't exist.", file_absolute_path.string()), ErrorCodes::FILE_DOESNT_EXIST); + + const auto current_file_size = fs::file_size(file_absolute_path); result_offset += current_file_size + 1; res_offsets[row] = result_offset; @@ -117,8 +117,8 @@ private: if (file_absolute_path.find(user_files_absolute_path) != 0) throw Exception("File is not inside " + user_files_absolute_path, ErrorCodes::DATABASE_ACCESS_DENIED); - Poco::File path_poco_file = Poco::File(file_absolute_path); - if (path_poco_file.exists() && path_poco_file.isDirectory()) + fs::path fs_path(file_absolute_path); + if (fs::exists(fs_path) && fs::is_directory(fs_path)) throw Exception("File can't be a directory", ErrorCodes::INCORRECT_FILE_NAME); } }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 8c0d8f9d48b..0a0c6f9638d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -74,8 +74,11 @@ #include #include #include +#include +namespace fs = std::filesystem; + namespace ProfileEvents { extern const Event ContextLock; @@ -2186,14 +2189,14 @@ void Context::checkCanBeDropped(const String & database, const String & table, c if (!max_size_to_drop || size <= max_size_to_drop) return; - Poco::File force_file(getFlagsPath() + "force_drop_table"); - bool force_file_exists = force_file.exists(); + fs::path force_file(getFlagsPath() + "force_drop_table"); + bool force_file_exists = fs::exists(force_file); if (force_file_exists) { try { - force_file.remove(); + fs::remove(force_file); return; } catch (...) @@ -2215,9 +2218,9 @@ void Context::checkCanBeDropped(const String & database, const String & table, c "Example:\nsudo touch '{}' && sudo chmod 666 '{}'", backQuoteIfNeed(database), backQuoteIfNeed(table), size_str, max_size_to_drop_str, - force_file.path(), force_file_exists ? "exists but not writeable (could not be removed)" : "doesn't exist", - force_file.path(), - force_file.path(), force_file.path()); + force_file.string(), force_file_exists ? "exists but not writeable (could not be removed)" : "doesn't exist", + force_file.string(), + force_file.string(), force_file.string()); } From 984d82c3f23ca8ba672b6e83167fae4d5d1ea8b1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 29 Apr 2021 02:00:04 +0300 Subject: [PATCH 05/55] Update DiskLocal --- src/Disks/DiskLocal.cpp | 45 +++++++++++++++++----------------- src/Functions/FunctionFile.cpp | 3 +-- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index d0cf6a00344..f567ec4c0ad 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -6,11 +6,12 @@ #include #include #include - #include +#include #include +namespace fs = std::filesystem; namespace DB { @@ -152,45 +153,43 @@ UInt64 DiskLocal::getUnreservedSpace() const bool DiskLocal::exists(const String & path) const { - return Poco::File(disk_path + path).exists(); + return fs::exists(fs::path(disk_path) / path); } bool DiskLocal::isFile(const String & path) const { - return Poco::File(disk_path + path).isFile(); + return fs::is_regular_file(fs::path(disk_path) / path); } bool DiskLocal::isDirectory(const String & path) const { - return Poco::File(disk_path + path).isDirectory(); + return fs::is_directory(fs::path(disk_path) / path); } size_t DiskLocal::getFileSize(const String & path) const { - return Poco::File(disk_path + path).getSize(); + return fs::file_size(fs::path(disk_path) / path); } void DiskLocal::createDirectory(const String & path) { - Poco::File(disk_path + path).createDirectory(); + fs::create_directory(fs::path(disk_path) / path); } void DiskLocal::createDirectories(const String & path) { - Poco::File(disk_path + path).createDirectories(); + fs::create_directories(fs::path(disk_path) / path); } void DiskLocal::clearDirectory(const String & path) { - std::vector files; - Poco::File(disk_path + path).list(files); - for (auto & file : files) - file.remove(); + for (auto & entry : fs::directory_iterator(fs::path(disk_path) / path)) + fs::remove(entry.path()); } void DiskLocal::moveDirectory(const String & from_path, const String & to_path) { - Poco::File(disk_path + from_path).renameTo(disk_path + to_path); + fs::rename(fs::path(disk_path) / from_path, fs::path(disk_path) / to_path); } DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) @@ -200,22 +199,24 @@ DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) void DiskLocal::moveFile(const String & from_path, const String & to_path) { - Poco::File(disk_path + from_path).renameTo(disk_path + to_path); + fs::rename(fs::path(disk_path) / from_path, fs::path(disk_path) / to_path); } void DiskLocal::replaceFile(const String & from_path, const String & to_path) { - Poco::File from_file(disk_path + from_path); - Poco::File to_file(disk_path + to_path); - if (to_file.exists()) + fs::path from_file = fs::path(disk_path) / from_path; + fs::path to_file = fs::path(disk_path) / to_path; + if (fs::exists(to_file)) { - Poco::File tmp_file(disk_path + to_path + ".old"); - to_file.renameTo(tmp_file.path()); - from_file.renameTo(disk_path + to_path); - tmp_file.remove(); + fs::path tmp_file(to_file.string() + ".old"); + fs::rename(to_file, tmp_file); + fs::rename(from_file, fs::path(disk_path) / to_path); + fs::remove(tmp_file); } else - from_file.renameTo(to_file.path()); + { + fs::rename(from_file, to_file); + } } std::unique_ptr @@ -255,7 +256,7 @@ void DiskLocal::removeDirectory(const String & path) void DiskLocal::removeRecursive(const String & path) { - Poco::File(disk_path + path).remove(true); + fs::remove_all(fs::path(disk_path) / path); } void DiskLocal::listFiles(const String & path, std::vector & file_names) diff --git a/src/Functions/FunctionFile.cpp b/src/Functions/FunctionFile.cpp index 7c87f27c731..3239705281e 100644 --- a/src/Functions/FunctionFile.cpp +++ b/src/Functions/FunctionFile.cpp @@ -73,8 +73,7 @@ public: fs::path file_path(filename); if (file_path.is_relative()) file_path = user_files_absolute_path / file_path; - - fs::path file_absolute_path = fs::absolute(file_path); + fs::path file_absolute_path = fs::canonical(file_path); checkReadIsAllowedOrThrow(user_files_absolute_path.string(), file_absolute_path); checked_filenames[row] = file_absolute_path.string(); From 654e58b77c2b376918847986e43cc07ff4d4f187 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 29 Apr 2021 22:58:42 +0300 Subject: [PATCH 06/55] Adjust test --- .../0_stateless/01658_read_file_to_stringcolumn.reference | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01658_read_file_to_stringcolumn.reference b/tests/queries/0_stateless/01658_read_file_to_stringcolumn.reference index 87659c32e39..1d0901cf9f6 100644 --- a/tests/queries/0_stateless/01658_read_file_to_stringcolumn.reference +++ b/tests/queries/0_stateless/01658_read_file_to_stringcolumn.reference @@ -8,11 +8,11 @@ ccccccccc aaaaaaaaa bbbbbbbbb aaaaaaaaa bbbbbbbbb ccccccccc -:107 +:233 :79 :35 :35 -:35 +:233 699415 aaaaaaaaa bbbbbbbbb ccccccccc aaaaaaaaa bbbbbbbbb @@ -21,5 +21,5 @@ ccccccccc aaaaaaaaa bbbbbbbbb ccccccccc aaaaaaaaa bbbbbbbbb 699415 0 :0 -:107 +:70 :79 From 7f2f138d5692d4b9e6a3ab3d72936fa3287a2751 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Apr 2021 09:52:40 +0300 Subject: [PATCH 07/55] Update DiskLocal and DiskMemory --- src/Disks/DiskLocal.cpp | 88 ++++++++++++++++++++-------------------- src/Disks/DiskLocal.h | 14 ++++--- src/Disks/DiskMemory.cpp | 15 +++---- 3 files changed, 62 insertions(+), 55 deletions(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index f567ec4c0ad..3e109ad8da4 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -60,28 +60,28 @@ private: class DiskLocalDirectoryIterator : public IDiskDirectoryIterator { public: - explicit DiskLocalDirectoryIterator(const String & disk_path_, const String & dir_path_) - : dir_path(dir_path_), iter(disk_path_ + dir_path_) + explicit DiskLocalDirectoryIterator(const fs::path & disk_path_, const String & dir_path_) + : dir_path(dir_path_), entry(disk_path_ / dir_path_) { } - void next() override { ++iter; } + void next() override { ++entry; } - bool isValid() const override { return iter != Poco::DirectoryIterator(); } + bool isValid() const override { return entry != fs::directory_iterator(); } String path() const override { - if (iter->isDirectory()) - return dir_path + iter.name() + '/'; + if (entry->is_directory()) + return dir_path / entry->path().filename() / ""; else - return dir_path + iter.name(); + return dir_path / entry->path().filename(); } - String name() const override { return iter.name(); } + String name() const override { return entry->path().filename(); } private: - String dir_path; - Poco::DirectoryIterator iter; + fs::path dir_path; + fs::directory_iterator entry; }; @@ -119,9 +119,9 @@ UInt64 DiskLocal::getTotalSpace() const { struct statvfs fs; if (name == "default") /// for default disk we get space from path/data/ - fs = getStatVFS(disk_path + "data/"); + fs = getStatVFS(fs::path(disk_path / "data/").string()); else - fs = getStatVFS(disk_path); + fs = getStatVFS(disk_path.string()); UInt64 total_size = fs.f_blocks * fs.f_bsize; if (total_size < keep_free_space_bytes) return 0; @@ -134,9 +134,9 @@ UInt64 DiskLocal::getAvailableSpace() const /// available for superuser only and for system purposes struct statvfs fs; if (name == "default") /// for default disk we get space from path/data/ - fs = getStatVFS(disk_path + "data/"); + fs = getStatVFS(fs::path(disk_path / "data/").string()); else - fs = getStatVFS(disk_path); + fs = getStatVFS(disk_path.string()); UInt64 total_size = fs.f_bavail * fs.f_bsize; if (total_size < keep_free_space_bytes) return 0; @@ -153,43 +153,43 @@ UInt64 DiskLocal::getUnreservedSpace() const bool DiskLocal::exists(const String & path) const { - return fs::exists(fs::path(disk_path) / path); + return fs::exists(disk_path / path); } bool DiskLocal::isFile(const String & path) const { - return fs::is_regular_file(fs::path(disk_path) / path); + return fs::is_regular_file(disk_path / path); } bool DiskLocal::isDirectory(const String & path) const { - return fs::is_directory(fs::path(disk_path) / path); + return fs::is_directory(disk_path / path); } size_t DiskLocal::getFileSize(const String & path) const { - return fs::file_size(fs::path(disk_path) / path); + return fs::file_size(disk_path / path); } void DiskLocal::createDirectory(const String & path) { - fs::create_directory(fs::path(disk_path) / path); + fs::create_directory(disk_path / path); } void DiskLocal::createDirectories(const String & path) { - fs::create_directories(fs::path(disk_path) / path); + fs::create_directories(disk_path / path); } void DiskLocal::clearDirectory(const String & path) { - for (auto & entry : fs::directory_iterator(fs::path(disk_path) / path)) + for (auto & entry : fs::directory_iterator(disk_path / path)) fs::remove(entry.path()); } void DiskLocal::moveDirectory(const String & from_path, const String & to_path) { - fs::rename(fs::path(disk_path) / from_path, fs::path(disk_path) / to_path); + fs::rename(disk_path / from_path, disk_path / to_path); } DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) @@ -199,18 +199,18 @@ DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) void DiskLocal::moveFile(const String & from_path, const String & to_path) { - fs::rename(fs::path(disk_path) / from_path, fs::path(disk_path) / to_path); + fs::rename(disk_path / from_path, disk_path / to_path); } void DiskLocal::replaceFile(const String & from_path, const String & to_path) { - fs::path from_file = fs::path(disk_path) / from_path; - fs::path to_file = fs::path(disk_path) / to_path; + fs::path from_file = disk_path / from_path; + fs::path to_file = disk_path / to_path; if (fs::exists(to_file)) { fs::path tmp_file(to_file.string() + ".old"); fs::rename(to_file, tmp_file); - fs::rename(from_file, fs::path(disk_path) / to_path); + fs::rename(from_file, disk_path / to_path); fs::remove(tmp_file); } else @@ -223,35 +223,35 @@ std::unique_ptr DiskLocal::readFile( const String & path, size_t buf_size, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold, MMappedFileCache * mmap_cache) const { - return createReadBufferFromFileBase(disk_path + path, estimated_size, aio_threshold, mmap_threshold, mmap_cache, buf_size); + return createReadBufferFromFileBase(disk_path / path, estimated_size, aio_threshold, mmap_threshold, mmap_cache, buf_size); } std::unique_ptr DiskLocal::writeFile(const String & path, size_t buf_size, WriteMode mode) { int flags = (mode == WriteMode::Append) ? (O_APPEND | O_CREAT | O_WRONLY) : -1; - return std::make_unique(disk_path + path, buf_size, flags); + return std::make_unique(disk_path / path, buf_size, flags); } void DiskLocal::removeFile(const String & path) { - auto fs_path = disk_path + path; + auto fs_path = disk_path / path; if (0 != unlink(fs_path.c_str())) - throwFromErrnoWithPath("Cannot unlink file " + fs_path, fs_path, ErrorCodes::CANNOT_UNLINK); + throwFromErrnoWithPath("Cannot unlink file " + fs_path.string(), fs_path, ErrorCodes::CANNOT_UNLINK); } void DiskLocal::removeFileIfExists(const String & path) { - auto fs_path = disk_path + path; + auto fs_path = disk_path / path; if (0 != unlink(fs_path.c_str()) && errno != ENOENT) - throwFromErrnoWithPath("Cannot unlink file " + fs_path, fs_path, ErrorCodes::CANNOT_UNLINK); + throwFromErrnoWithPath("Cannot unlink file " + fs_path.string(), fs_path, ErrorCodes::CANNOT_UNLINK); } void DiskLocal::removeDirectory(const String & path) { - auto fs_path = disk_path + path; + auto fs_path = disk_path / path; if (0 != rmdir(fs_path.c_str())) - throwFromErrnoWithPath("Cannot rmdir " + fs_path, fs_path, ErrorCodes::CANNOT_RMDIR); + throwFromErrnoWithPath("Cannot rmdir " + fs_path.string(), fs_path, ErrorCodes::CANNOT_RMDIR); } void DiskLocal::removeRecursive(const String & path) @@ -261,39 +261,41 @@ void DiskLocal::removeRecursive(const String & path) void DiskLocal::listFiles(const String & path, std::vector & file_names) { - Poco::File(disk_path + path).list(file_names); + file_names.clear(); + for (auto & entry : fs::directory_iterator(disk_path / path)) + file_names.emplace_back(entry.path().filename()); } void DiskLocal::setLastModified(const String & path, const Poco::Timestamp & timestamp) { - Poco::File(disk_path + path).setLastModified(timestamp); + Poco::File(disk_path / path).setLastModified(timestamp); } Poco::Timestamp DiskLocal::getLastModified(const String & path) { - return Poco::File(disk_path + path).getLastModified(); + return Poco::File(disk_path / path).getLastModified(); } void DiskLocal::createHardLink(const String & src_path, const String & dst_path) { - DB::createHardLink(disk_path + src_path, disk_path + dst_path); + DB::createHardLink(disk_path / src_path, disk_path / dst_path); } void DiskLocal::truncateFile(const String & path, size_t size) { - int res = truncate((disk_path + path).c_str(), size); + int res = truncate((disk_path / path).string().data(), size); if (-1 == res) throwFromErrnoWithPath("Cannot truncate file " + path, path, ErrorCodes::CANNOT_TRUNCATE_FILE); } void DiskLocal::createFile(const String & path) { - Poco::File(disk_path + path).createFile(); + Poco::File(disk_path / path).createFile(); } void DiskLocal::setReadOnly(const String & path) { - Poco::File(disk_path + path).setReadOnly(true); + Poco::File(disk_path / path).setReadOnly(true); } bool inline isSameDiskType(const IDisk & one, const IDisk & another) @@ -304,14 +306,14 @@ bool inline isSameDiskType(const IDisk & one, const IDisk & another) void DiskLocal::copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) { if (isSameDiskType(*this, *to_disk)) - Poco::File(disk_path + from_path).copyTo(to_disk->getPath() + to_path); /// Use more optimal way. + Poco::File(disk_path / from_path).copyTo(to_disk->getPath() + to_path); /// Use more optimal way. else IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. } SyncGuardPtr DiskLocal::getDirectorySyncGuard(const String & path) const { - return std::make_unique(disk_path + path); + return std::make_unique(disk_path / path); } DiskPtr DiskLocalReservation::getDisk(size_t i) const diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 567ca24eb50..8cdadd095c6 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -8,6 +8,9 @@ #include #include +#include + +namespace fs = std::filesystem; namespace DB { @@ -24,15 +27,15 @@ public: friend class DiskLocalReservation; DiskLocal(const String & name_, const String & path_, UInt64 keep_free_space_bytes_) - : name(name_), disk_path(path_), keep_free_space_bytes(keep_free_space_bytes_) + : name(name_), disk_path(path_), disk_path_str(path_), keep_free_space_bytes(keep_free_space_bytes_) { - if (disk_path.back() != '/') - throw Exception("Disk path must ends with '/', but '" + disk_path + "' doesn't.", ErrorCodes::LOGICAL_ERROR); + if (disk_path_str.back() != '/') + throw Exception("Disk path must ends with '/', but '" + disk_path_str + "' doesn't.", ErrorCodes::LOGICAL_ERROR); } const String & getName() const override { return name; } - const String & getPath() const override { return disk_path; } + const String & getPath() const override { return disk_path_str; } ReservationPtr reserve(UInt64 bytes) override; @@ -109,7 +112,8 @@ private: private: const String name; - const String disk_path; + const fs::path disk_path; + const String disk_path_str; const UInt64 keep_free_space_bytes; UInt64 reserved_bytes = 0; diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index 68257ec4948..0e93d469df7 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -6,8 +6,9 @@ #include #include #include -#include +#include +namespace fs = std::filesystem; namespace DB { @@ -24,7 +25,7 @@ namespace ErrorCodes class DiskMemoryDirectoryIterator final : public IDiskDirectoryIterator { public: - explicit DiskMemoryDirectoryIterator(std::vector && dir_file_paths_) + explicit DiskMemoryDirectoryIterator(std::vector && dir_file_paths_) : dir_file_paths(std::move(dir_file_paths_)), iter(dir_file_paths.begin()) { } @@ -33,13 +34,13 @@ public: bool isValid() const override { return iter != dir_file_paths.end(); } - String path() const override { return (*iter).toString(); } + String path() const override { return iter->string(); } - String name() const override { return (*iter).getFileName(); } + String name() const override { return iter->filename(); } private: - std::vector dir_file_paths; - std::vector::iterator iter; + std::vector dir_file_paths; + std::vector::iterator iter; }; @@ -268,7 +269,7 @@ DiskDirectoryIteratorPtr DiskMemory::iterateDirectory(const String & path) if (!path.empty() && files.find(path) == files.end()) throw Exception("Directory '" + path + "' does not exist", ErrorCodes::DIRECTORY_DOESNT_EXIST); - std::vector dir_file_paths; + std::vector dir_file_paths; for (const auto & file : files) if (parentPath(file.first) == path) dir_file_paths.emplace_back(file.first); From c609119daf710f1ccfebb0ebd26c109294d579f1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Apr 2021 18:13:19 +0300 Subject: [PATCH 08/55] Update Disks/* --- src/Disks/DiskLocal.cpp | 32 ++++++++++++++++++++++---------- src/Disks/DiskLocal.h | 5 ----- src/Disks/DiskMemory.cpp | 2 -- src/Disks/IDisk.cpp | 7 +++---- src/Disks/IDisk.h | 10 ++++++---- 5 files changed, 31 insertions(+), 25 deletions(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 3e109ad8da4..aa03cd02a56 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -8,10 +8,9 @@ #include #include -#include +#include #include -namespace fs = std::filesystem; namespace DB { @@ -25,6 +24,7 @@ namespace ErrorCodes extern const int CANNOT_TRUNCATE_FILE; extern const int CANNOT_UNLINK; extern const int CANNOT_RMDIR; + extern const int CANNOT_OPEN_FILE; } std::mutex DiskLocal::reservation_mutex; @@ -268,12 +268,14 @@ void DiskLocal::listFiles(const String & path, std::vector & file_names) void DiskLocal::setLastModified(const String & path, const Poco::Timestamp & timestamp) { - Poco::File(disk_path / path).setLastModified(timestamp); + fs::last_write_time(disk_path / path, static_cast(std::chrono::microseconds(timestamp.epochMicroseconds()))); } Poco::Timestamp DiskLocal::getLastModified(const String & path) { - return Poco::File(disk_path / path).getLastModified(); + fs::file_time_type fs_time = fs::last_write_time(disk_path / path); + auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); + return Poco::Timestamp(micro_sec.count()); } void DiskLocal::createHardLink(const String & src_path, const String & dst_path) @@ -290,12 +292,16 @@ void DiskLocal::truncateFile(const String & path, size_t size) void DiskLocal::createFile(const String & path) { - Poco::File(disk_path / path).createFile(); + FILE * file = fopen((disk_path / path).string().data(), "a+"); + if (file == nullptr) + throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Cannot create file {}", path); } void DiskLocal::setReadOnly(const String & path) { - Poco::File(disk_path / path).setReadOnly(true); + fs::permissions(disk_path / path, + fs::perms::owner_read | fs::perms::group_read | fs::perms::others_read, + fs::perm_options::remove); /// bitwise AND } bool inline isSameDiskType(const IDisk & one, const IDisk & another) @@ -306,7 +312,7 @@ bool inline isSameDiskType(const IDisk & one, const IDisk & another) void DiskLocal::copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) { if (isSameDiskType(*this, *to_disk)) - Poco::File(disk_path / from_path).copyTo(to_disk->getPath() + to_path); /// Use more optimal way. + fs::copy_file(disk_path / from_path, to_disk->getPath() + to_path); else IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. } @@ -384,10 +390,16 @@ void registerDiskLocal(DiskFactory & factory) throw Exception("Disk path must end with /. Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); } - if (Poco::File disk{path}; !disk.canRead() || !disk.canWrite()) - { + fs::path disk(path); + fs::perms p = fs::status(disk).permissions(); + bool is_readable = (p & fs::perms::owner_read) != fs::perms::none + | (p & fs::perms::group_read) != fs::perms::none + | (p & fs::perms::others_read) != fs::perms::none; + bool is_writable = (p & fs::perms::owner_write) != fs::perms::none + | (p & fs::perms::group_write) != fs::perms::none + | (p & fs::perms::others_write) != fs::perms::none; + if (!is_readable || !is_writable) throw Exception("There is no RW access to the disk " + name + " (" + path + ")", ErrorCodes::PATH_ACCESS_DENIED); - } bool has_space_ratio = config.has(config_prefix + ".keep_free_space_ratio"); diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 8cdadd095c6..91835707725 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -6,11 +6,6 @@ #include #include -#include -#include -#include - -namespace fs = std::filesystem; namespace DB { diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index 0e93d469df7..423dba6bed6 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -6,9 +6,7 @@ #include #include #include -#include -namespace fs = std::filesystem; namespace DB { diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index ee7f57af771..8c69398d3ad 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -48,10 +48,9 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p } else { - Poco::Path path(from_path); - const String & dir_name = path.directory(path.depth() - 1); - const String dest = to_path + dir_name + "/"; - to_disk.createDirectories(dest); + const String & dir_name = directoryPath(from_path); + fs::path dest(fs::path(to_path) / dir_name); + fs::create_directories(dest); for (auto it = from_disk.iterateDirectory(from_path); it->isValid(); it->next()) asyncCopy(from_disk, it->path(), to_disk, dest, exec, results); diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 726145cb5d2..778b3dc9bf3 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -11,9 +11,10 @@ #include #include #include -#include #include +#include +namespace fs = std::filesystem; namespace CurrentMetrics { @@ -288,19 +289,20 @@ inline String fullPath(const DiskPtr & disk, const String & path) /// Return parent path for the specified path. inline String parentPath(const String & path) { - return Poco::Path(path).parent().toString(); + auto fs_path = fs::path(path).parent_path() / ""; + return fs_path.string(); } /// Return file name for the specified path. inline String fileName(const String & path) { - return Poco::Path(path).getFileName(); + return fs::path(path).filename(); } /// Return directory path for the specified path. inline String directoryPath(const String & path) { - return Poco::Path(path).setFileName("").toString(); + return fs::is_directory(path) ? path : fs::path(path).parent_path().string(); } } From 6867ea2c73b0aabcc425be1e7e5ec533b989eefd Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Apr 2021 19:14:04 +0300 Subject: [PATCH 09/55] Temporary fix --- src/Disks/IDisk.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 778b3dc9bf3..34936373d67 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -13,6 +13,7 @@ #include #include #include +#include namespace fs = std::filesystem; @@ -289,8 +290,7 @@ inline String fullPath(const DiskPtr & disk, const String & path) /// Return parent path for the specified path. inline String parentPath(const String & path) { - auto fs_path = fs::path(path).parent_path() / ""; - return fs_path.string(); + return Poco::Path(path).parent().toString(); } /// Return file name for the specified path. From 1b75f12fcb93e5e2dc4b0ee2df73ff3cbc0e3dd4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Apr 2021 19:48:42 +0300 Subject: [PATCH 10/55] Update Databases/* --- src/Databases/DatabaseAtomic.cpp | 5 ++- src/Databases/DatabaseOrdinary.cpp | 8 ++-- .../PostgreSQL/DatabasePostgreSQL.cpp | 40 +++++++++---------- 3 files changed, 25 insertions(+), 28 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 9b9ea572c3d..ad6af3ff8ab 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -12,6 +12,9 @@ #include #include #include +#include + +namespace fs = std::filesystem; namespace DB { @@ -42,7 +45,7 @@ DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, c , db_uuid(uuid) { assert(db_uuid != UUIDHelpers::Nil); - Poco::File(path_to_table_symlinks).createDirectories(); + fs::create_directories(path_to_table_symlinks); tryCreateMetadataSymlink(); } diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index cea31d88d57..6b91649c3b2 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -17,7 +17,6 @@ #include #include #include -#include #include #include #include @@ -112,8 +111,7 @@ void DatabaseOrdinary::loadStoredObjects(ContextPtr local_context, bool has_forc auto * create_query = ast->as(); create_query->database = database_name; - auto detached_permanently_flag = Poco::File(full_path.string() + detached_suffix); - if (detached_permanently_flag.exists()) + if (fs::exists(full_path.string() + detached_suffix)) { /// FIXME: even if we don't load the table we can still mark the uuid of it as taken. /// if (create_query->uuid != UUIDHelpers::Nil) @@ -282,11 +280,11 @@ void DatabaseOrdinary::commitAlterTable(const StorageID &, const String & table_ try { /// rename atomically replaces the old file with the new one. - Poco::File(table_metadata_tmp_path).renameTo(table_metadata_path); + fs::rename(table_metadata_tmp_path, table_metadata_path); } catch (...) { - Poco::File(table_metadata_tmp_path).remove(); + fs::remove(table_metadata_tmp_path); throw; } } diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index e12d7f975ce..d0864d619b1 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -12,12 +12,12 @@ #include #include #include -#include -#include #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -29,6 +29,7 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; extern const int TABLE_IS_DROPPED; extern const int TABLE_ALREADY_EXISTS; + extern const int CANNOT_OPEN_FILE; } static const auto suffix = ".removed"; @@ -205,9 +206,9 @@ void DatabasePostgreSQL::attachTable(const String & table_name, const StoragePtr detached_or_dropped.erase(table_name); - Poco::File table_marked_as_removed(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix); - if (table_marked_as_removed.exists()) - table_marked_as_removed.remove(); + fs::path table_marked_as_removed = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix); + if (fs::exists(table_marked_as_removed)) + fs::remove(table_marked_as_removed); } @@ -252,16 +253,11 @@ void DatabasePostgreSQL::dropTable(ContextPtr, const String & table_name, bool / if (detached_or_dropped.count(table_name)) throw Exception(fmt::format("Table {}.{} is already dropped/detached", database_name, table_name), ErrorCodes::TABLE_IS_DROPPED); - Poco::File mark_table_removed(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix); + fs::path mark_table_removed = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix); - try - { - mark_table_removed.createFile(); - } - catch (...) - { - throw; - } + FILE * file = fopen(mark_table_removed.string().data(), "a+"); + if (file == nullptr) + throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Cannot create file {}", mark_table_removed.string()); if (cache_tables) cached_tables.erase(table_name); @@ -272,7 +268,7 @@ void DatabasePostgreSQL::dropTable(ContextPtr, const String & table_name, bool / void DatabasePostgreSQL::drop(ContextPtr /*context*/) { - Poco::File(getMetadataPath()).remove(true); + fs::remove_all(getMetadataPath()); } @@ -280,14 +276,14 @@ void DatabasePostgreSQL::loadStoredObjects(ContextPtr /* context */, bool, bool { { std::lock_guard lock{mutex}; - Poco::DirectoryIterator iterator(getMetadataPath()); + fs::directory_iterator iter(getMetadataPath()); /// Check for previously dropped tables - for (Poco::DirectoryIterator end; iterator != end; ++iterator) + for (fs::directory_iterator end; iter != end; ++iter) { - if (iterator->isFile() && endsWith(iterator.name(), suffix)) + if (fs::is_regular_file(iter->path()) && endsWith(iter->path().filename(), suffix)) { - const auto & file_name = iterator.name(); + const auto & file_name = iter->path().filename().string(); const auto & table_name = unescapeForFileName(file_name.substr(0, file_name.size() - strlen(suffix))); detached_or_dropped.emplace(table_name); } @@ -321,9 +317,9 @@ void DatabasePostgreSQL::removeOutdatedTables() { auto table_name = *iter; iter = detached_or_dropped.erase(iter); - Poco::File table_marked_as_removed(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix); - if (table_marked_as_removed.exists()) - table_marked_as_removed.remove(); + fs::path table_marked_as_removed = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix); + if (fs::exists(table_marked_as_removed)) + fs::remove(table_marked_as_removed); } else ++iter; From 6e0cfdaeb569241b7a719979abae9085b7e02507 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Apr 2021 20:20:53 +0300 Subject: [PATCH 11/55] Update Databases/* part 2 --- src/Databases/DatabaseAtomic.cpp | 38 +++++++++++----------- src/Databases/DatabaseFactory.cpp | 12 +++---- src/Databases/DatabaseOnDisk.cpp | 53 ++++++++++++++++--------------- 3 files changed, 52 insertions(+), 51 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index ad6af3ff8ab..12d1fa48670 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -1,7 +1,5 @@ #include #include -#include -#include #include #include #include @@ -76,14 +74,14 @@ void DatabaseAtomic::drop(ContextPtr) assert(tables.empty()); try { - Poco::File(path_to_metadata_symlink).remove(); - Poco::File(path_to_table_symlinks).remove(true); + fs::remove(path_to_metadata_symlink); + fs::remove_all(path_to_table_symlinks); } catch (...) { LOG_WARNING(log, getCurrentExceptionMessage(true)); } - Poco::File(getMetadataPath()).remove(true); + fs::remove_all(getMetadataPath()); } void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) @@ -135,8 +133,8 @@ void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_na /// (it's more likely to lost connection, than to fail before applying local changes). /// TODO better detection and recovery - Poco::File(table_metadata_path).renameTo(table_metadata_path_drop); /// Mark table as dropped - DatabaseOrdinary::detachTableUnlocked(table_name, lock); /// Should never throw + fs::rename(table_metadata_path, table_metadata_path_drop); /// Mark table as dropped + DatabaseOrdinary::detachTableUnlocked(table_name, lock); /// Should never throw table_name_to_path.erase(table_name); } @@ -319,7 +317,7 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora } catch (...) { - Poco::File(table_metadata_tmp_path).remove(); + fs::remove(table_metadata_tmp_path); if (locked_uuid) DatabaseCatalog::instance().removeUUIDMappingFinally(query.uuid); throw; @@ -423,7 +421,7 @@ void DatabaseAtomic::loadStoredObjects(ContextPtr local_context, bool has_force_ { /// Recreate symlinks to table data dirs in case of force restore, because some of them may be broken if (has_force_restore_data_flag) - Poco::File(path_to_table_symlinks).remove(true); + fs::remove_all(path_to_table_symlinks); DatabaseOrdinary::loadStoredObjects(local_context, has_force_restore_data_flag, force_attach); @@ -435,7 +433,7 @@ void DatabaseAtomic::loadStoredObjects(ContextPtr local_context, bool has_force_ table_names = table_name_to_path; } - Poco::File(path_to_table_symlinks).createDirectories(); + fs::create_directories(path_to_table_symlinks); for (const auto & table : table_names) tryCreateSymlink(table.first, table.second, true); } @@ -446,9 +444,9 @@ void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String & try { String link = path_to_table_symlinks + escapeForFileName(table_name); - Poco::File data = Poco::Path(getContext()->getPath()).makeAbsolute().toString() + actual_data_path; - if (!if_data_path_exist || data.exists()) - data.linkTo(link, Poco::File::LINK_SYMBOLIC); + fs::path data = fs::absolute(getContext()->getPath()).string() + actual_data_path; + if (!if_data_path_exist || fs::exists(data)) + fs::create_symlink(data, link); } catch (...) { @@ -461,7 +459,7 @@ void DatabaseAtomic::tryRemoveSymlink(const String & table_name) try { String path = path_to_table_symlinks + escapeForFileName(table_name); - Poco::File{path}.remove(); + fs::remove(path); } catch (...) { @@ -474,17 +472,17 @@ void DatabaseAtomic::tryCreateMetadataSymlink() /// Symlinks in data/db_name/ directory and metadata/db_name/ are not used by ClickHouse, /// it's needed only for convenient introspection. assert(path_to_metadata_symlink != metadata_path); - Poco::File metadata_symlink(path_to_metadata_symlink); - if (metadata_symlink.exists()) + fs::path metadata_symlink(path_to_metadata_symlink); + if (fs::exists(metadata_symlink)) { - if (!metadata_symlink.isLink()) + if (!fs::is_symlink(metadata_symlink)) throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Directory {} exists", path_to_metadata_symlink); } else { try { - Poco::File{metadata_path}.linkTo(path_to_metadata_symlink, Poco::File::LINK_SYMBOLIC); + fs::create_symlink(metadata_path, path_to_metadata_symlink); } catch (...) { @@ -498,7 +496,7 @@ void DatabaseAtomic::renameDatabase(const String & new_name) /// CREATE, ATTACH, DROP, DETACH and RENAME DATABASE must hold DDLGuard try { - Poco::File(path_to_metadata_symlink).remove(); + fs::remove(path_to_metadata_symlink); } catch (...) { @@ -529,7 +527,7 @@ void DatabaseAtomic::renameDatabase(const String & new_name) path_to_table_symlinks = getContext()->getPath() + "data/" + new_name_escaped + "/"; } - Poco::File(old_path_to_table_symlinks).renameTo(path_to_table_symlinks); + fs::rename(old_path_to_table_symlinks, path_to_table_symlinks); tryCreateMetadataSymlink(); } diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 9d09ac731d2..93511a99879 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include "config_core.h" @@ -40,6 +41,8 @@ #include #endif +namespace fs = std::filesystem; + namespace DB { @@ -62,7 +65,7 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m /// Before 20.7 it's possible that .sql metadata file does not exist for some old database. /// In this case Ordinary database is created on server startup if the corresponding metadata directory exists. /// So we should remove metadata directory if database creation failed. - created = Poco::File(metadata_path).createDirectory(); + created = fs::create_directory(metadata_path); DatabasePtr impl = getImpl(create, metadata_path, context); @@ -74,11 +77,8 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m } catch (...) { - Poco::File metadata_dir(metadata_path); - - if (created && metadata_dir.exists()) - metadata_dir.remove(true); - + if (created && fs::exists(metadata_path)) + fs::remove_all(metadata_path); throw; } } diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 2ad961ccce4..44ec5d27250 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -21,7 +21,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -199,8 +201,8 @@ DatabaseOnDisk::DatabaseOnDisk( , metadata_path(metadata_path_) , data_path(data_path_) { - Poco::File(local_context->getPath() + data_path).createDirectories(); - Poco::File(metadata_path).createDirectories(); + fs::create_directories(local_context->getPath() + data_path); + fs::create_directories(metadata_path); } @@ -243,7 +245,7 @@ void DatabaseOnDisk::createTable( if (!create.attach) checkMetadataFilenameAvailability(table_name); - if (create.attach && Poco::File(table_metadata_path).exists()) + if (create.attach && fs::exists(table_metadata_path)) { ASTPtr ast_detached = parseQueryFromMetadata(log, local_context, table_metadata_path); auto & create_detached = ast_detached->as(); @@ -283,10 +285,10 @@ void DatabaseOnDisk::removeDetachedPermanentlyFlag(ContextPtr, const String & ta { try { - auto detached_permanently_flag = Poco::File(table_metadata_path + detached_suffix); + fs::path detached_permanently_flag(table_metadata_path + detached_suffix); - if (detached_permanently_flag.exists()) - detached_permanently_flag.remove(); + if (fs::exists(detached_permanently_flag)) + fs::remove(detached_permanently_flag); } catch (Exception & e) { @@ -306,11 +308,11 @@ void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const Stora /// If it was ATTACH query and file with table metadata already exist /// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one. - Poco::File(table_metadata_tmp_path).renameTo(table_metadata_path); + fs::rename(table_metadata_tmp_path, table_metadata_path); } catch (...) { - Poco::File(table_metadata_tmp_path).remove(); + fs::remove(table_metadata_tmp_path); throw; } } @@ -348,25 +350,25 @@ void DatabaseOnDisk::dropTable(ContextPtr local_context, const String & table_na bool renamed = false; try { - Poco::File(table_metadata_path).renameTo(table_metadata_path_drop); + fs::rename(table_metadata_path, table_metadata_path_drop); renamed = true; table->drop(); table->is_dropped = true; - Poco::File table_data_dir{local_context->getPath() + table_data_path_relative}; - if (table_data_dir.exists()) - table_data_dir.remove(true); + fs::path table_data_dir(local_context->getPath() + table_data_path_relative); + if (fs::exists(table_data_dir)) + fs::remove_all(table_data_dir); } catch (...) { LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__)); attachTable(table_name, table, table_data_path_relative); if (renamed) - Poco::File(table_metadata_path_drop).renameTo(table_metadata_path); + fs::rename(table_metadata_path_drop, table_metadata_path); throw; } - Poco::File(table_metadata_path_drop).remove(); + fs::remove(table_metadata_path_drop); } void DatabaseOnDisk::checkMetadataFilenameAvailability(const String & to_table_name) const @@ -379,11 +381,11 @@ void DatabaseOnDisk::checkMetadataFilenameAvailabilityUnlocked(const String & to { String table_metadata_path = getObjectMetadataPath(to_table_name); - if (Poco::File(table_metadata_path).exists()) + if (fs::exists(table_metadata_path)) { - auto detached_permanently_flag = Poco::File(table_metadata_path + detached_suffix); + fs::path detached_permanently_flag(table_metadata_path + detached_suffix); - if (detached_permanently_flag.exists()) + if (fs::exists(detached_permanently_flag)) throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists (detached permanently)", backQuote(database_name), backQuote(to_table_name)); else throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists (detached)", backQuote(database_name), backQuote(to_table_name)); @@ -461,7 +463,7 @@ void DatabaseOnDisk::renameTable( /// Now table data are moved to new database, so we must add metadata and attach table to new database to_database.createTable(local_context, to_table_name, table, attach_query); - Poco::File(table_metadata_path).remove(); + fs::remove(table_metadata_path); if (from_atomic_to_ordinary) { @@ -526,8 +528,8 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const void DatabaseOnDisk::drop(ContextPtr local_context) { assert(tables.empty()); - Poco::File(local_context->getPath() + getDataPath()).remove(false); - Poco::File(getMetadataPath()).remove(false); + fs::remove(local_context->getPath() + getDataPath()); + fs::remove(getMetadataPath()); } String DatabaseOnDisk::getObjectMetadataPath(const String & object_name) const @@ -553,16 +555,17 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat assert(getUUID() == UUIDHelpers::Nil); static const char * tmp_drop_ext = ".sql.tmp_drop"; const std::string object_name = file_name.substr(0, file_name.size() - strlen(tmp_drop_ext)); - if (Poco::File(local_context->getPath() + getDataPath() + '/' + object_name).exists()) + + if (fs::exists(local_context->getPath() + getDataPath() + '/' + object_name)) { - Poco::File(getMetadataPath() + file_name).renameTo(getMetadataPath() + object_name + ".sql"); + fs::rename(getMetadataPath() + file_name, getMetadataPath() + object_name + ".sql"); LOG_WARNING(log, "Object {} was not dropped previously and will be restored", backQuote(object_name)); process_metadata_file(object_name + ".sql"); } else { LOG_INFO(log, "Removing file {}", getMetadataPath() + file_name); - Poco::File(getMetadataPath() + file_name).remove(); + fs::remove(getMetadataPath() + file_name); } }; @@ -593,7 +596,7 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat { /// There are files .sql.tmp - delete LOG_INFO(log, "Removing file {}", dir_it->path()); - Poco::File(dir_it->path()).remove(); + fs::remove(dir_it->path()); } else if (endsWith(dir_it.name(), ".sql")) { @@ -649,7 +652,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata( { if (logger) LOG_ERROR(logger, "File {} is empty. Removing.", metadata_file_path); - Poco::File(metadata_file_path).remove(); + fs::remove(metadata_file_path); return nullptr; } From e717b4c5c76e982d7ea3d3129d8f1ea0dc4dad3c Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Apr 2021 22:38:07 +0300 Subject: [PATCH 12/55] Update databases/* --- src/Databases/DatabaseLazy.cpp | 7 ++++--- src/Databases/DatabaseMemory.cpp | 8 ++++---- .../MySQL/DatabaseConnectionMySQL.cpp | 20 +++++++++---------- .../MySQL/DatabaseMaterializeMySQL.cpp | 10 ++++++---- src/Databases/MySQL/MaterializeMetadata.cpp | 11 +++++----- 5 files changed, 30 insertions(+), 26 deletions(-) diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index d498cb96062..416a94db929 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -12,8 +12,9 @@ #include #include #include -#include +#include +namespace fs = std::filesystem; namespace DB { @@ -43,8 +44,8 @@ void DatabaseLazy::loadStoredObjects( { const std::string table_name = file_name.substr(0, file_name.size() - 4); - auto detached_permanently_flag = Poco::File(getMetadataPath() + "/" + file_name + detached_suffix); - if (detached_permanently_flag.exists()) + fs::path detached_permanently_flag = fs::path(getMetadataPath()) / (file_name + detached_suffix); + if (fs::exists(detached_permanently_flag)) { LOG_DEBUG(log, "Skipping permanently detached table {}.", backQuote(table_name)); return; diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index f21a145df55..c0af027e027 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -4,9 +4,9 @@ #include #include #include -#include #include +namespace fs = std::filesystem; namespace DB { @@ -42,9 +42,9 @@ void DatabaseMemory::dropTable( try { table->drop(); - Poco::File table_data_dir{getTableDataPath(table_name)}; - if (table_data_dir.exists()) - table_data_dir.remove(true); + fs::path table_data_dir{getTableDataPath(table_name)}; + if (fs::exists(table_data_dir)) + fs::remove_all(table_data_dir); } catch (...) { diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index 4e63bed9c6d..2b6c6d243a7 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -23,10 +23,10 @@ # include # include # include - -# include +# include # include +namespace fs = std::filesystem; namespace DB { @@ -314,7 +314,7 @@ void DatabaseConnectionMySQL::shutdown() void DatabaseConnectionMySQL::drop(ContextPtr /*context*/) { - Poco::File(getMetadataPath()).remove(true); + fs::remove_all(getMetadataPath()); } void DatabaseConnectionMySQL::cleanOutdatedTables() @@ -360,10 +360,10 @@ void DatabaseConnectionMySQL::attachTable(const String & table_name, const Stora local_tables_cache[table_name].second = storage; remove_or_detach_tables.erase(table_name); - Poco::File remove_flag(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix); + fs::path remove_flag = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix); - if (remove_flag.exists()) - remove_flag.remove(); + if (fs::exists(remove_flag)) + fs::remove(remove_flag); } StoragePtr DatabaseConnectionMySQL::detachTable(const String & table_name) @@ -391,13 +391,13 @@ void DatabaseConnectionMySQL::loadStoredObjects(ContextPtr, bool, bool /*force_a { std::lock_guard lock{mutex}; - Poco::DirectoryIterator iterator(getMetadataPath()); + fs::directory_iterator iter(getMetadataPath()); - for (Poco::DirectoryIterator end; iterator != end; ++iterator) + for (fs::directory_iterator end; iter != end; ++iter) { - if (iterator->isFile() && endsWith(iterator.name(), suffix)) + if (fs::is_regular_file(iter->path()) && endsWith(iter->path().filename(), suffix)) { - const auto & filename = iterator.name(); + const auto & filename = iter->path().filename().string(); const auto & table_name = unescapeForFileName(filename.substr(0, filename.size() - strlen(suffix))); remove_or_detach_tables.emplace(table_name); } diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp index 62a66b22c93..e4470f9371f 100644 --- a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -13,9 +13,11 @@ # include # include # include -# include # include # include +# include + +namespace fs = std::filesystem; namespace DB { @@ -158,10 +160,10 @@ template void DatabaseMaterializeMySQL::drop(ContextPtr context_) { /// Remove metadata info - Poco::File metadata(Base::getMetadataPath() + "/.metadata"); + fs::path metadata(Base::getMetadataPath() + "/.metadata"); - if (metadata.exists()) - metadata.remove(false); + if (fs::exists(metadata)) + fs::remove(metadata); Base::drop(context_); } diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index c389ab5a1b0..4fd99ca3438 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -8,11 +8,13 @@ #include #include #include -#include #include #include #include #include +#include + +namespace fs = std::filesystem; namespace DB { @@ -193,12 +195,11 @@ void commitMetadata(const std::function & function, const String & persi try { function(); - - Poco::File(persistent_tmp_path).renameTo(persistent_path); + fs::rename(persistent_tmp_path, persistent_path); } catch (...) { - Poco::File(persistent_tmp_path).remove(); + fs::remove(persistent_tmp_path); throw; } } @@ -231,7 +232,7 @@ void MaterializeMetadata::transaction(const MySQLReplication::Position & positio MaterializeMetadata::MaterializeMetadata(const String & path_, const Settings & settings_) : persistent_path(path_), settings(settings_) { - if (Poco::File(persistent_path).exists()) + if (fs::exists(persistent_path)) { ReadBufferFromFile in(persistent_path, DBMS_DEFAULT_BUFFER_SIZE); assertString("Version:\t" + toString(meta_version), in); From 549b5f8a0cd037bc9032d976921d64500bfe52e6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Apr 2021 23:16:35 +0300 Subject: [PATCH 13/55] Update Common/* --- src/Common/Config/ConfigProcessor.cpp | 2 -- src/Common/Config/configReadClient.cpp | 10 ++++++---- src/Common/FileUpdatesTracker.h | 8 +++++--- src/Common/StatusFile.cpp | 5 +++-- src/Common/filesystemHelpers.cpp | 4 +++- src/Common/renameat2.cpp | 8 +++++--- 6 files changed, 22 insertions(+), 15 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 5399826fe20..149de3cfcbf 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -20,10 +20,8 @@ #include #include - #define PREPROCESSED_SUFFIX "-preprocessed" - namespace fs = std::filesystem; using namespace Poco::XML; diff --git a/src/Common/Config/configReadClient.cpp b/src/Common/Config/configReadClient.cpp index 61d8e507c05..8ce2a8b03e9 100644 --- a/src/Common/Config/configReadClient.cpp +++ b/src/Common/Config/configReadClient.cpp @@ -1,8 +1,10 @@ #include "configReadClient.h" #include -#include #include "ConfigProcessor.h" +#include + +namespace fs = std::filesystem; namespace DB { @@ -11,11 +13,11 @@ bool configReadClient(Poco::Util::LayeredConfiguration & config, const std::stri std::string config_path; if (config.has("config-file")) config_path = config.getString("config-file"); - else if (Poco::File("./clickhouse-client.xml").exists()) + else if (fs::exists("./clickhouse-client.xml")) config_path = "./clickhouse-client.xml"; - else if (!home_path.empty() && Poco::File(home_path + "/.clickhouse-client/config.xml").exists()) + else if (!home_path.empty() && fs::exists(home_path + "/.clickhouse-client/config.xml")) config_path = home_path + "/.clickhouse-client/config.xml"; - else if (Poco::File("/etc/clickhouse-client/config.xml").exists()) + else if (fs::exists("/etc/clickhouse-client/config.xml")) config_path = "/etc/clickhouse-client/config.xml"; if (!config_path.empty()) diff --git a/src/Common/FileUpdatesTracker.h b/src/Common/FileUpdatesTracker.h index 0914c75693c..471a62cf07a 100644 --- a/src/Common/FileUpdatesTracker.h +++ b/src/Common/FileUpdatesTracker.h @@ -1,10 +1,10 @@ #pragma once -#include #include - #include +#include +namespace fs = std::filesystem; class FileUpdatesTracker { @@ -31,6 +31,8 @@ public: private: Poco::Timestamp getLastModificationTime() const { - return Poco::File(path).getLastModified(); + fs::file_time_type fs_time = fs::last_write_time(path); + auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); + return Poco::Timestamp(micro_sec.count()); } }; diff --git a/src/Common/StatusFile.cpp b/src/Common/StatusFile.cpp index b21454c9ed8..ceedf518608 100644 --- a/src/Common/StatusFile.cpp +++ b/src/Common/StatusFile.cpp @@ -4,7 +4,6 @@ #include #include -#include #include #include #include @@ -14,7 +13,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -45,7 +46,7 @@ StatusFile::StatusFile(std::string path_, FillFunction fill_) : path(std::move(path_)), fill(std::move(fill_)) { /// If file already exists. NOTE Minor race condition. - if (Poco::File(path).exists()) + if (fs::exists(path)) { std::string contents; { diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 51a66fba3aa..bfcb75235c0 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -9,7 +9,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -48,7 +50,7 @@ bool enoughSpaceInDirectory(const std::string & path [[maybe_unused]], size_t da std::unique_ptr createTemporaryFile(const std::string & path) { - Poco::File(path).createDirectories(); + fs::create_directories(path); /// NOTE: std::make_shared cannot use protected constructors return std::make_unique(path); diff --git a/src/Common/renameat2.cpp b/src/Common/renameat2.cpp index a735a9d72d4..26d90427889 100644 --- a/src/Common/renameat2.cpp +++ b/src/Common/renameat2.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #if defined(linux) || defined(__linux) || defined(__linux__) #include @@ -10,6 +10,8 @@ #include #endif +namespace fs = std::filesystem; + namespace DB { @@ -93,9 +95,9 @@ static bool renameat2(const std::string &, const std::string &, int) static void renameNoReplaceFallback(const std::string & old_path, const std::string & new_path) { /// NOTE it's unsafe - if (Poco::File{new_path}.exists()) + if (fs::exists(new_path)) throw Exception("File " + new_path + " exists", ErrorCodes::FILE_ALREADY_EXISTS); - Poco::File{old_path}.renameTo(new_path); + fs::rename(old_path, new_path); } /// Do not use [[noreturn]] to avoid warnings like "code will never be executed" in other places From 378195a6d8990b8e62f5fa6f36d45111c5ae8653 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Apr 2021 23:35:44 +0300 Subject: [PATCH 14/55] Update Dictionaries/* --- src/Common/Config/ConfigProcessor.cpp | 19 +++++++++---------- .../GeodataProviders/HierarchiesProvider.cpp | 4 +++- .../GeodataProviders/NamesProvider.cpp | 6 ++++-- src/Dictionaries/FileDictionarySource.cpp | 8 +++++++- src/Dictionaries/LibraryDictionarySource.cpp | 7 ++++--- 5 files changed, 27 insertions(+), 17 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 149de3cfcbf..957934770e7 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -424,20 +424,19 @@ ConfigProcessor::Files ConfigProcessor::getConfigMergeFiles(const std::string & for (const std::string & merge_dir_name : merge_dirs) { - Poco::File merge_dir(merge_dir_name); - if (!merge_dir.exists() || !merge_dir.isDirectory()) + fs::path merge_dir(merge_dir_name); + if (!fs::exists(merge_dir) || !is_directory(merge_dir)) continue; - for (Poco::DirectoryIterator it(merge_dir_name); it != Poco::DirectoryIterator(); ++it) + for (fs::directory_iterator it(merge_dir_name); it != fs::directory_iterator(); ++it) { - Poco::File & file = *it; - Poco::Path path(file.path()); - std::string extension = path.getExtension(); - std::string base_name = path.getBaseName(); + fs::path path(it->path()); + std::string extension = path.extension(); + std::string base_name = path.stem(); // Skip non-config and temporary files - if (file.isFile() && (extension == "xml" || extension == "conf") && !startsWith(base_name, ".")) - files.push_back(file.path()); + if (fs::is_regular_file(path) && (extension == ".xml" || extension == ".conf") && !startsWith(base_name, ".")) + files.push_back(it->path()); } } @@ -512,7 +511,7 @@ XMLDocumentPtr ConfigProcessor::processConfig( else { std::string default_path = "/etc/metrika.xml"; - if (Poco::File(default_path).exists()) + if (fs::exists(default_path)) include_from_path = default_path; } if (!include_from_path.empty()) diff --git a/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp b/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp index 3b8848ab19b..7257caa066f 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp +++ b/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp @@ -5,7 +5,9 @@ #include #include #include "HierarchyFormatReader.h" +#include +namespace fs = std::filesystem; bool RegionsHierarchyDataSource::isModified() const { @@ -27,7 +29,7 @@ RegionsHierarchiesDataProvider::RegionsHierarchiesDataProvider(const std::string void RegionsHierarchiesDataProvider::discoverFilesWithCustomHierarchies() { - std::string basename = Poco::Path(path).getBaseName(); + std::string basename = fs::path(path).stem(); Poco::Path dir_path = Poco::Path(path).absolute().parent(); diff --git a/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.cpp b/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.cpp index f3e49545481..5f79fda070f 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.cpp +++ b/src/Dictionaries/Embedded/GeodataProviders/NamesProvider.cpp @@ -2,7 +2,9 @@ #include #include "NamesFormatReader.h" +#include +namespace fs = std::filesystem; bool LanguageRegionsNamesDataSource::isModified() const { @@ -11,7 +13,7 @@ bool LanguageRegionsNamesDataSource::isModified() const size_t LanguageRegionsNamesDataSource::estimateTotalSize() const { - return Poco::File(path).getSize(); + return fs::file_size(path); } ILanguageRegionsNamesReaderPtr LanguageRegionsNamesDataSource::createReader() @@ -39,7 +41,7 @@ RegionsNamesDataProvider::RegionsNamesDataProvider(const std::string & directory ILanguageRegionsNamesDataSourcePtr RegionsNamesDataProvider::getLanguageRegionsNamesSource(const std::string & language) const { const auto data_file = getDataFilePath(language); - if (Poco::File(data_file).exists()) + if (fs::exists(data_file)) return std::make_unique(data_file, language); else return {}; diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index 378c6f11857..d4bcc97d563 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -12,6 +12,9 @@ #include "DictionaryStructure.h" #include "registerDictionaries.h" #include "DictionarySourceHelpers.h" +#include + +namespace fs = std::filesystem; namespace DB { @@ -80,9 +83,12 @@ std::string FileDictionarySource::toString() const Poco::Timestamp FileDictionarySource::getLastModification() const { - return Poco::File{filepath}.getLastModified(); + fs::file_time_type fs_time = fs::last_write_time(filepath); + auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); + return Poco::Timestamp(micro_sec.count()); } + void registerDictionarySourceFile(DictionarySourceFactory & factory) { auto create_table_source = [=](const DictionaryStructure & dict_struct, diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index a971ba4b1be..ce580d6b803 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -13,7 +12,9 @@ #include "registerDictionaries.h" #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -49,8 +50,8 @@ LibraryDictionarySource::LibraryDictionarySource( throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "LibraryDictionarySource: Library path {} is not inside {}", path, dictionaries_lib_path); } - if (!Poco::File(path).exists()) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "LibraryDictionarySource: Can't load library {}: file doesn't exist", Poco::File(path).path()); + if (!fs::exists(path)) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "LibraryDictionarySource: Can't load library {}: file doesn't exist", path); description.init(sample_block); bridge_helper = std::make_shared(context, description.sample_block, dictionary_id); From a2753a46c609ba77a1974c721b86434885b95309 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 1 May 2021 00:37:10 +0300 Subject: [PATCH 15/55] Adjust test --- .../queries/0_stateless/01601_detach_permanently.reference | 2 +- tests/queries/0_stateless/01601_detach_permanently.sql | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01601_detach_permanently.reference b/tests/queries/0_stateless/01601_detach_permanently.reference index 6683603c972..0232c1694e4 100644 --- a/tests/queries/0_stateless/01601_detach_permanently.reference +++ b/tests/queries/0_stateless/01601_detach_permanently.reference @@ -50,7 +50,7 @@ After database reattachement the table is still absent (it was detached permamen And we can not detach it permanently But we can attach it back And detach permanently again to check how database drop will behave -DROP database - Directory not empty error, but database deteched +DROP database - Directory not empty error, but database detached DROP database - now success ----------------------- database lazy tests diff --git a/tests/queries/0_stateless/01601_detach_permanently.sql b/tests/queries/0_stateless/01601_detach_permanently.sql index 0e21fd8f463..3af8ed573ef 100644 --- a/tests/queries/0_stateless/01601_detach_permanently.sql +++ b/tests/queries/0_stateless/01601_detach_permanently.sql @@ -128,8 +128,8 @@ ATTACH TABLE test1601_detach_permanently_ordinary.test_name_reuse; SELECT 'And detach permanently again to check how database drop will behave'; DETACH table test1601_detach_permanently_ordinary.test_name_reuse PERMANENTLY; -SELECT 'DROP database - Directory not empty error, but database deteched'; -DROP DATABASE test1601_detach_permanently_ordinary; -- { serverError 1000 } +SELECT 'DROP database - Directory not empty error, but database detached'; +DROP DATABASE test1601_detach_permanently_ordinary; -- { serverError 1001 } ATTACH DATABASE test1601_detach_permanently_ordinary; @@ -203,7 +203,7 @@ SELECT 'And detach permanently again to check how database drop will behave'; DETACH table test1601_detach_permanently_lazy.test_name_reuse PERMANENTLY; SELECT 'DROP database - Directory not empty error, but database deteched'; -DROP DATABASE test1601_detach_permanently_lazy; -- { serverError 1000 } +DROP DATABASE test1601_detach_permanently_lazy; -- { serverError 1001 } ATTACH DATABASE test1601_detach_permanently_lazy; From 2887ce1f69c430ea053a6b472ee9406b337b0f68 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 1 May 2021 01:55:14 +0300 Subject: [PATCH 16/55] Temporary rollback --- src/Dictionaries/FileDictionarySource.cpp | 1 - src/Disks/DiskLocal.cpp | 22 +++++++++------------- src/Disks/tests/gtest_disk.cpp | 7 +++++-- 3 files changed, 14 insertions(+), 16 deletions(-) diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index d4bcc97d563..eb1ecb8c723 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -12,7 +12,6 @@ #include "DictionaryStructure.h" #include "registerDictionaries.h" #include "DictionarySourceHelpers.h" -#include namespace fs = std::filesystem; diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index aa03cd02a56..726b8181fc5 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -24,7 +25,6 @@ namespace ErrorCodes extern const int CANNOT_TRUNCATE_FILE; extern const int CANNOT_UNLINK; extern const int CANNOT_RMDIR; - extern const int CANNOT_OPEN_FILE; } std::mutex DiskLocal::reservation_mutex; @@ -292,16 +292,12 @@ void DiskLocal::truncateFile(const String & path, size_t size) void DiskLocal::createFile(const String & path) { - FILE * file = fopen((disk_path / path).string().data(), "a+"); - if (file == nullptr) - throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Cannot create file {}", path); + Poco::File(disk_path / path).createFile(); } void DiskLocal::setReadOnly(const String & path) { - fs::permissions(disk_path / path, - fs::perms::owner_read | fs::perms::group_read | fs::perms::others_read, - fs::perm_options::remove); /// bitwise AND + Poco::File(disk_path / path).setReadOnly(true); } bool inline isSameDiskType(const IDisk & one, const IDisk & another) @@ -392,12 +388,12 @@ void registerDiskLocal(DiskFactory & factory) fs::path disk(path); fs::perms p = fs::status(disk).permissions(); - bool is_readable = (p & fs::perms::owner_read) != fs::perms::none - | (p & fs::perms::group_read) != fs::perms::none - | (p & fs::perms::others_read) != fs::perms::none; - bool is_writable = (p & fs::perms::owner_write) != fs::perms::none - | (p & fs::perms::group_write) != fs::perms::none - | (p & fs::perms::others_write) != fs::perms::none; + bool is_readable = ((p & fs::perms::owner_read) != fs::perms::none) + | ((p & fs::perms::group_read) != fs::perms::none) + | ((p & fs::perms::others_read) != fs::perms::none); + bool is_writable = ((p & fs::perms::owner_write) != fs::perms::none) + | ((p & fs::perms::group_write) != fs::perms::none) + | ((p & fs::perms::others_write) != fs::perms::none); if (!is_readable || !is_writable) throw Exception("There is no RW access to the disk " + name + " (" + path + ")", ErrorCodes::PATH_ACCESS_DENIED); diff --git a/src/Disks/tests/gtest_disk.cpp b/src/Disks/tests/gtest_disk.cpp index 3b9dca63002..cffbcf57e76 100644 --- a/src/Disks/tests/gtest_disk.cpp +++ b/src/Disks/tests/gtest_disk.cpp @@ -3,6 +3,9 @@ #include #include #include "gtest_disk.h" +#include + +namespace fs = std::filesystem; #if !defined(__clang__) # pragma GCC diagnostic push @@ -21,7 +24,7 @@ DB::DiskPtr createDisk() template <> DB::DiskPtr createDisk() { - Poco::File("tmp/").createDirectory(); + fs::create_directory("tmp/"); return std::make_shared("local_disk", "tmp/", 0); } @@ -42,7 +45,7 @@ template <> void destroyDisk(DB::DiskPtr & disk) { disk.reset(); - Poco::File("tmp/").remove(true); + fs::remove_all("tmp/"); } From 32b291714109092f716067a815888d9b277ee0d8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 1 May 2021 17:25:38 +0300 Subject: [PATCH 17/55] Update s3/* --- src/Common/CounterInFile.h | 5 +- src/Databases/DatabaseAtomic.cpp | 1 - src/Disks/DiskLocal.cpp | 4 +- src/Disks/S3/DiskS3.cpp | 54 ++++++++++--------- .../gtest_cascade_and_memory_write_buffer.cpp | 5 +- 5 files changed, 35 insertions(+), 34 deletions(-) diff --git a/src/Common/CounterInFile.h b/src/Common/CounterInFile.h index 8cd4534d413..6326261234d 100644 --- a/src/Common/CounterInFile.h +++ b/src/Common/CounterInFile.h @@ -7,7 +7,6 @@ #include #include -#include #include #include @@ -59,7 +58,7 @@ public: Int64 res = -1; - bool file_doesnt_exists = !Poco::File(path).exists(); + bool file_doesnt_exists = !fs::exists(path); if (file_doesnt_exists && !create_if_need) { throw Poco::Exception("File " + path + " does not exist. " @@ -138,7 +137,7 @@ public: // Not thread-safe and not synchronized between processes. void fixIfBroken(UInt64 value) { - bool file_exists = Poco::File(path).exists(); + bool file_exists = fs::exists(path); int fd = ::open(path.c_str(), O_RDWR | O_CREAT | O_CLOEXEC, 0666); if (-1 == fd) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 12d1fa48670..fa4e2eed382 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -10,7 +10,6 @@ #include #include #include -#include namespace fs = std::filesystem; diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 726b8181fc5..49ca4a32293 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -183,7 +183,7 @@ void DiskLocal::createDirectories(const String & path) void DiskLocal::clearDirectory(const String & path) { - for (auto & entry : fs::directory_iterator(disk_path / path)) + for (const auto & entry : fs::directory_iterator(disk_path / path)) fs::remove(entry.path()); } @@ -262,7 +262,7 @@ void DiskLocal::removeRecursive(const String & path) void DiskLocal::listFiles(const String & path, std::vector & file_names) { file_names.clear(); - for (auto & entry : fs::directory_iterator(disk_path / path)) + for (const auto & entry : fs::directory_iterator(disk_path / path)) file_names.emplace_back(entry.path().filename()); } diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 325e887073c..07e2d254632 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -443,20 +443,20 @@ public: void next() override { ++iter; } - bool isValid() const override { return iter != Poco::DirectoryIterator(); } + bool isValid() const override { return iter != fs::directory_iterator(); } String path() const override { - if (iter->isDirectory()) - return folder_path + iter.name() + '/'; + if (fs::is_directory(iter->path())) + return folder_path + iter->path().filename().string() + '/'; else - return folder_path + iter.name(); + return folder_path + iter->path().filename().string(); } - String name() const override { return iter.name(); } + String name() const override { return iter->path().filename(); } private: - Poco::DirectoryIterator iter; + fs::directory_iterator iter; String folder_path; }; @@ -590,17 +590,17 @@ ReservationPtr DiskS3::reserve(UInt64 bytes) bool DiskS3::exists(const String & path) const { - return Poco::File(metadata_path + path).exists(); + return fs::exists(fs::path(metadata_path) / path); } bool DiskS3::isFile(const String & path) const { - return Poco::File(metadata_path + path).isFile(); + return fs::is_regular_file(fs::path(metadata_path) / path); } bool DiskS3::isDirectory(const String & path) const { - return Poco::File(metadata_path + path).isDirectory(); + return fs::is_directory(fs::path(metadata_path) / path); } size_t DiskS3::getFileSize(const String & path) const @@ -611,12 +611,12 @@ size_t DiskS3::getFileSize(const String & path) const void DiskS3::createDirectory(const String & path) { - Poco::File(metadata_path + path).createDirectory(); + fs::create_directory(fs::path(metadata_path) / path); } void DiskS3::createDirectories(const String & path) { - Poco::File(metadata_path + path).createDirectories(); + fs::create_directories(fs::path(metadata_path) / path); } String DiskS3::getUniqueId(const String & path) const @@ -662,7 +662,7 @@ void DiskS3::moveFile(const String & from_path, const String & to_path, bool sen createFileOperationObject("rename", revision, object_metadata); } - Poco::File(metadata_path + from_path).renameTo(metadata_path + to_path); + fs::rename(fs::path(metadata_path) / from_path, fs::path(metadata_path) / to_path); } void DiskS3::replaceFile(const String & from_path, const String & to_path) @@ -727,9 +727,9 @@ void DiskS3::removeMeta(const String & path, AwsS3KeyKeeper & keys) { LOG_DEBUG(log, "Remove file by path: {}", backQuote(metadata_path + path)); - Poco::File file(metadata_path + path); + fs::path file = fs::path(metadata_path) / path; - if (!file.isFile()) + if (!is_regular_file(file)) throw Exception(ErrorCodes::CANNOT_DELETE_DIRECTORY, "Path '{}' is a directory", path); try @@ -739,7 +739,7 @@ void DiskS3::removeMeta(const String & path, AwsS3KeyKeeper & keys) /// If there is no references - delete content from S3. if (metadata.ref_count == 0) { - file.remove(); + fs::remove(file); for (const auto & [s3_object_path, _] : metadata.s3_objects) keys.addKey(s3_root_path + s3_object_path); @@ -748,7 +748,7 @@ void DiskS3::removeMeta(const String & path, AwsS3KeyKeeper & keys) { --metadata.ref_count; metadata.save(); - file.remove(); + fs::remove(file); } } catch (const Exception & e) @@ -762,7 +762,7 @@ void DiskS3::removeMeta(const String & path, AwsS3KeyKeeper & keys) backQuote(path), e.nested() ? e.nested()->message() : e.message()); - file.remove(); + fs::remove(file); } else throw; @@ -773,8 +773,8 @@ void DiskS3::removeMetaRecursive(const String & path, AwsS3KeyKeeper & keys) { checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks. - Poco::File file(metadata_path + path); - if (file.isFile()) + fs::path file = fs::path(metadata_path) / path; + if (fs::is_regular_file(file)) { removeMeta(path, keys); } @@ -782,7 +782,7 @@ void DiskS3::removeMetaRecursive(const String & path, AwsS3KeyKeeper & keys) { for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) removeMetaRecursive(it->path(), keys); - file.remove(); + fs::remove(file); } } @@ -810,7 +810,7 @@ void DiskS3::removeAws(const AwsS3KeyKeeper & keys) void DiskS3::removeFileIfExists(const String & path) { AwsS3KeyKeeper keys; - if (Poco::File(metadata_path + path).exists()) + if (fs::exists(fs::path(metadata_path) / path)) { removeMeta(path, keys); removeAws(keys); @@ -819,7 +819,7 @@ void DiskS3::removeFileIfExists(const String & path) void DiskS3::removeDirectory(const String & path) { - Poco::File(metadata_path + path).remove(); + fs::remove(fs::path(metadata_path) / path); } void DiskS3::removeSharedFile(const String & path, bool keep_s3) @@ -869,12 +869,14 @@ void DiskS3::listFiles(const String & path, std::vector & file_names) void DiskS3::setLastModified(const String & path, const Poco::Timestamp & timestamp) { - Poco::File(metadata_path + path).setLastModified(timestamp); + fs::last_write_time(fs::path(metadata_path) / path, static_cast(std::chrono::microseconds(timestamp.epochMicroseconds()))); } Poco::Timestamp DiskS3::getLastModified(const String & path) { - return Poco::File(metadata_path + path).getLastModified(); + fs::file_time_type fs_time = fs::last_write_time(fs::path(metadata_path) / path); + auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); + return Poco::Timestamp(micro_sec.count()); } void DiskS3::createHardLink(const String & src_path, const String & dst_path) @@ -1306,8 +1308,8 @@ void DiskS3::restore() restoreFiles(information); restoreFileOperations(information); - Poco::File restore_file(metadata_path + RESTORE_FILE_NAME); - restore_file.remove(); + fs::path restore_file = fs::path(metadata_path) / RESTORE_FILE_NAME; + fs::remove(restore_file); saveSchemaVersion(RESTORABLE_SCHEMA_VERSION); diff --git a/src/IO/tests/gtest_cascade_and_memory_write_buffer.cpp b/src/IO/tests/gtest_cascade_and_memory_write_buffer.cpp index 4936307a5e3..d013077b7c2 100644 --- a/src/IO/tests/gtest_cascade_and_memory_write_buffer.cpp +++ b/src/IO/tests/gtest_cascade_and_memory_write_buffer.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include #include @@ -9,7 +8,9 @@ #include #include #include +#include +namespace fs = std::filesystem; using namespace DB; @@ -236,7 +237,7 @@ try buf.reset(); reread_buf.reset(); - ASSERT_TRUE(!Poco::File(tmp_filename).exists()); + ASSERT_TRUE(!fs::exists(tmp_filename)); } } catch (...) From ad69fceea52d17db774cf2620ad9fdb2adad0e84 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 2 May 2021 19:59:18 +0300 Subject: [PATCH 18/55] Restore copyFile --- src/Disks/DiskLocal.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 49ca4a32293..2dbee7fd782 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -308,7 +308,7 @@ bool inline isSameDiskType(const IDisk & one, const IDisk & another) void DiskLocal::copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) { if (isSameDiskType(*this, *to_disk)) - fs::copy_file(disk_path / from_path, to_disk->getPath() + to_path); + Poco::File(disk_path / from_path).copyTo(to_disk->getPath() + to_path); /// Use more optimal way. else IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. } From 2b67341f281b53021a3be0faad54f5665a3668c1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 4 May 2021 15:55:21 +0300 Subject: [PATCH 19/55] Finish s3 --- src/Disks/S3/DiskS3.cpp | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 07e2d254632..a28e7e0f387 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -16,7 +16,6 @@ #include #include #include -#include #include #include #include @@ -1496,8 +1495,8 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio continue; /// Skip not finished parts. They shouldn't be in 'detached' directory, because CH wouldn't be able to finish processing them. - Poco::Path directory_path (path); - auto directory_name = directory_path.directory(directory_path.depth() - 1); + fs::path directory_path(path); + auto directory_name = directory_path.parent_path().filename().string(); auto predicate = [&directory_name](String & prefix) { return directory_name.starts_with(prefix); }; if (std::any_of(not_finished_prefixes.begin(), not_finished_prefixes.end(), predicate)) continue; @@ -1506,7 +1505,7 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio LOG_DEBUG(log, "Move directory to 'detached' {} -> {}", path, detached_path); - Poco::File(metadata_path + path).moveTo(metadata_path + detached_path); + fs::rename(fs::path(metadata_path) / path, fs::path(metadata_path) / detached_path); } } @@ -1538,7 +1537,7 @@ String DiskS3::revisionToString(UInt64 revision) String DiskS3::pathToDetached(const String & source_path) { - return Poco::Path(source_path).parent().append(Poco::Path("detached")).toString() + '/'; + return fs::path(source_path).parent_path() / "detached" / ""; } void DiskS3::onFreeze(const String & path) From 9ec92ec5145d3620cfa0030034fc625ffe7efbae Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 5 May 2021 18:10:14 +0300 Subject: [PATCH 20/55] Fix tests, less manual concatination of paths --- src/Disks/DiskLocal.cpp | 70 ++++++++-------- src/Disks/DiskLocal.h | 11 ++- src/Disks/IDisk.cpp | 9 ++- src/Disks/IDisk.h | 10 +-- src/Disks/S3/DiskS3.cpp | 80 +++++++++---------- src/Storages/MergeTree/DataPartsExchange.cpp | 16 ++-- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 78 +++++++++--------- .../MergeTree/MergeTreeDataMergerMutator.cpp | 6 +- .../MergeTree/MergedBlockOutputStream.cpp | 8 +- src/Storages/MergeTree/checkDataPart.cpp | 6 +- src/Storages/StorageMergeTree.cpp | 4 +- 11 files changed, 146 insertions(+), 152 deletions(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 2dbee7fd782..64c599c5b81 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -69,13 +69,7 @@ public: bool isValid() const override { return entry != fs::directory_iterator(); } - String path() const override - { - if (entry->is_directory()) - return dir_path / entry->path().filename() / ""; - else - return dir_path / entry->path().filename(); - } + String path() const override { return dir_path / entry->path().filename(); } String name() const override { return entry->path().filename(); } @@ -119,9 +113,9 @@ UInt64 DiskLocal::getTotalSpace() const { struct statvfs fs; if (name == "default") /// for default disk we get space from path/data/ - fs = getStatVFS(fs::path(disk_path / "data/").string()); + fs = getStatVFS((fs::path(disk_path) / "data/").string()); else - fs = getStatVFS(disk_path.string()); + fs = getStatVFS(disk_path); UInt64 total_size = fs.f_blocks * fs.f_bsize; if (total_size < keep_free_space_bytes) return 0; @@ -134,9 +128,9 @@ UInt64 DiskLocal::getAvailableSpace() const /// available for superuser only and for system purposes struct statvfs fs; if (name == "default") /// for default disk we get space from path/data/ - fs = getStatVFS(fs::path(disk_path / "data/").string()); + fs = getStatVFS((fs::path(disk_path) / "data/").string()); else - fs = getStatVFS(disk_path.string()); + fs = getStatVFS(disk_path); UInt64 total_size = fs.f_bavail * fs.f_bsize; if (total_size < keep_free_space_bytes) return 0; @@ -153,64 +147,64 @@ UInt64 DiskLocal::getUnreservedSpace() const bool DiskLocal::exists(const String & path) const { - return fs::exists(disk_path / path); + return fs::exists(fs::path(disk_path) / path); } bool DiskLocal::isFile(const String & path) const { - return fs::is_regular_file(disk_path / path); + return fs::is_regular_file(fs::path(disk_path) / path); } bool DiskLocal::isDirectory(const String & path) const { - return fs::is_directory(disk_path / path); + return fs::is_directory(fs::path(disk_path) / path); } size_t DiskLocal::getFileSize(const String & path) const { - return fs::file_size(disk_path / path); + return fs::file_size(fs::path(disk_path) / path); } void DiskLocal::createDirectory(const String & path) { - fs::create_directory(disk_path / path); + fs::create_directory(fs::path(disk_path) / path); } void DiskLocal::createDirectories(const String & path) { - fs::create_directories(disk_path / path); + fs::create_directories(fs::path(disk_path) / path); } void DiskLocal::clearDirectory(const String & path) { - for (const auto & entry : fs::directory_iterator(disk_path / path)) + for (const auto & entry : fs::directory_iterator(fs::path(disk_path) / path)) fs::remove(entry.path()); } void DiskLocal::moveDirectory(const String & from_path, const String & to_path) { - fs::rename(disk_path / from_path, disk_path / to_path); + fs::rename(fs::path(disk_path) / from_path, fs::path(disk_path) / to_path); } DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) { - return std::make_unique(disk_path, path); + return std::make_unique(fs::path(disk_path), path); } void DiskLocal::moveFile(const String & from_path, const String & to_path) { - fs::rename(disk_path / from_path, disk_path / to_path); + fs::rename(fs::path(disk_path) / from_path, fs::path(disk_path) / to_path); } void DiskLocal::replaceFile(const String & from_path, const String & to_path) { - fs::path from_file = disk_path / from_path; - fs::path to_file = disk_path / to_path; + fs::path from_file = fs::path(disk_path) / from_path; + fs::path to_file = fs::path(disk_path) / to_path; if (fs::exists(to_file)) { fs::path tmp_file(to_file.string() + ".old"); fs::rename(to_file, tmp_file); - fs::rename(from_file, disk_path / to_path); + fs::rename(from_file, fs::path(disk_path) / to_path); fs::remove(tmp_file); } else @@ -223,33 +217,33 @@ std::unique_ptr DiskLocal::readFile( const String & path, size_t buf_size, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold, MMappedFileCache * mmap_cache) const { - return createReadBufferFromFileBase(disk_path / path, estimated_size, aio_threshold, mmap_threshold, mmap_cache, buf_size); + return createReadBufferFromFileBase(fs::path(disk_path) / path, estimated_size, aio_threshold, mmap_threshold, mmap_cache, buf_size); } std::unique_ptr DiskLocal::writeFile(const String & path, size_t buf_size, WriteMode mode) { int flags = (mode == WriteMode::Append) ? (O_APPEND | O_CREAT | O_WRONLY) : -1; - return std::make_unique(disk_path / path, buf_size, flags); + return std::make_unique(fs::path(disk_path) / path, buf_size, flags); } void DiskLocal::removeFile(const String & path) { - auto fs_path = disk_path / path; + auto fs_path = fs::path(disk_path) / path; if (0 != unlink(fs_path.c_str())) throwFromErrnoWithPath("Cannot unlink file " + fs_path.string(), fs_path, ErrorCodes::CANNOT_UNLINK); } void DiskLocal::removeFileIfExists(const String & path) { - auto fs_path = disk_path / path; + auto fs_path = fs::path(disk_path) / path; if (0 != unlink(fs_path.c_str()) && errno != ENOENT) throwFromErrnoWithPath("Cannot unlink file " + fs_path.string(), fs_path, ErrorCodes::CANNOT_UNLINK); } void DiskLocal::removeDirectory(const String & path) { - auto fs_path = disk_path / path; + auto fs_path = fs::path(disk_path) / path; if (0 != rmdir(fs_path.c_str())) throwFromErrnoWithPath("Cannot rmdir " + fs_path.string(), fs_path, ErrorCodes::CANNOT_RMDIR); } @@ -262,42 +256,42 @@ void DiskLocal::removeRecursive(const String & path) void DiskLocal::listFiles(const String & path, std::vector & file_names) { file_names.clear(); - for (const auto & entry : fs::directory_iterator(disk_path / path)) + for (const auto & entry : fs::directory_iterator(fs::path(disk_path) / path)) file_names.emplace_back(entry.path().filename()); } void DiskLocal::setLastModified(const String & path, const Poco::Timestamp & timestamp) { - fs::last_write_time(disk_path / path, static_cast(std::chrono::microseconds(timestamp.epochMicroseconds()))); + fs::last_write_time(fs::path(disk_path) / path, static_cast(std::chrono::microseconds(timestamp.epochMicroseconds()))); } Poco::Timestamp DiskLocal::getLastModified(const String & path) { - fs::file_time_type fs_time = fs::last_write_time(disk_path / path); + fs::file_time_type fs_time = fs::last_write_time(fs::path(disk_path) / path); auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); return Poco::Timestamp(micro_sec.count()); } void DiskLocal::createHardLink(const String & src_path, const String & dst_path) { - DB::createHardLink(disk_path / src_path, disk_path / dst_path); + DB::createHardLink(fs::path(disk_path) / src_path, fs::path(disk_path) / dst_path); } void DiskLocal::truncateFile(const String & path, size_t size) { - int res = truncate((disk_path / path).string().data(), size); + int res = truncate((fs::path(disk_path) / path).string().data(), size); if (-1 == res) throwFromErrnoWithPath("Cannot truncate file " + path, path, ErrorCodes::CANNOT_TRUNCATE_FILE); } void DiskLocal::createFile(const String & path) { - Poco::File(disk_path / path).createFile(); + Poco::File(fs::path(disk_path) / path).createFile(); } void DiskLocal::setReadOnly(const String & path) { - Poco::File(disk_path / path).setReadOnly(true); + Poco::File(fs::path(disk_path) / path).setReadOnly(true); } bool inline isSameDiskType(const IDisk & one, const IDisk & another) @@ -308,14 +302,14 @@ bool inline isSameDiskType(const IDisk & one, const IDisk & another) void DiskLocal::copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) { if (isSameDiskType(*this, *to_disk)) - Poco::File(disk_path / from_path).copyTo(to_disk->getPath() + to_path); /// Use more optimal way. + Poco::File(fs::path(disk_path) / from_path).copyTo(fs::path(to_disk->getPath()) / to_path); /// Use more optimal way. else IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. } SyncGuardPtr DiskLocal::getDirectorySyncGuard(const String & path) const { - return std::make_unique(disk_path / path); + return std::make_unique(fs::path(disk_path) / path); } DiskPtr DiskLocalReservation::getDisk(size_t i) const diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 91835707725..47482ad8d67 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -22,15 +22,15 @@ public: friend class DiskLocalReservation; DiskLocal(const String & name_, const String & path_, UInt64 keep_free_space_bytes_) - : name(name_), disk_path(path_), disk_path_str(path_), keep_free_space_bytes(keep_free_space_bytes_) + : name(name_), disk_path(path_), keep_free_space_bytes(keep_free_space_bytes_) { - if (disk_path_str.back() != '/') - throw Exception("Disk path must ends with '/', but '" + disk_path_str + "' doesn't.", ErrorCodes::LOGICAL_ERROR); + if (disk_path.back() != '/') + throw Exception("Disk path must end with '/', but '" + disk_path + "' doesn't.", ErrorCodes::LOGICAL_ERROR); } const String & getName() const override { return name; } - const String & getPath() const override { return disk_path_str; } + const String & getPath() const override { return disk_path; } ReservationPtr reserve(UInt64 bytes) override; @@ -107,8 +107,7 @@ private: private: const String name; - const fs::path disk_path; - const String disk_path_str; + const String disk_path; const UInt64 keep_free_space_bytes; UInt64 reserved_bytes = 0; diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 8c69398d3ad..82705b5dcc8 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -22,7 +22,8 @@ bool IDisk::isDirectoryEmpty(const String & path) void copyFile(IDisk & from_disk, const String & from_path, IDisk & to_disk, const String & to_path) { - LOG_DEBUG(&Poco::Logger::get("IDisk"), "Copying from {} {} to {} {}.", from_disk.getName(), from_path, to_disk.getName(), to_path); + LOG_DEBUG(&Poco::Logger::get("IDisk"), "Copying from {} (path: {}) {} to {} (path: {}) {}.", + from_disk.getName(), from_disk.getPath(), from_path, to_disk.getName(), to_disk.getPath(), to_path); auto in = from_disk.readFile(from_path); auto out = to_disk.writeFile(to_path); @@ -41,16 +42,16 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p [&from_disk, from_path, &to_disk, to_path]() { setThreadName("DiskCopier"); - DB::copyFile(from_disk, from_path, to_disk, to_path + fileName(from_path)); + DB::copyFile(from_disk, from_path, to_disk, fs::path(to_path) / fileName(from_path)); }); results.push_back(std::move(result)); } else { - const String & dir_name = directoryPath(from_path); + fs::path dir_name = fs::path(from_path).parent_path().filename(); fs::path dest(fs::path(to_path) / dir_name); - fs::create_directories(dest); + to_disk.createDirectories(dest); for (auto it = from_disk.iterateDirectory(from_path); it->isValid(); it->next()) asyncCopy(from_disk, it->path(), to_disk, dest, exec, results); diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index a70404cbfb4..62ab54e90c2 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -213,10 +213,10 @@ public: virtual DiskType::Type getType() const = 0; /// Invoked when Global Context is shutdown. - virtual void shutdown() { } + virtual void shutdown() {} /// Performs action on disk startup. - virtual void startup() { } + virtual void startup() {} /// Return some uniq string for file, overrode for S3 /// Required for distinguish different copies of the same part on S3 @@ -234,7 +234,7 @@ public: virtual SyncGuardPtr getDirectorySyncGuard(const String & path) const; /// Applies new settings for disk in runtime. - virtual void applyNewSettings(const Poco::Util::AbstractConfiguration &, ContextConstPtr) { } + virtual void applyNewSettings(const Poco::Util::AbstractConfiguration &, ContextConstPtr) {} protected: friend class DiskDecorator; @@ -295,7 +295,7 @@ public: /// Return full path to a file on disk. inline String fullPath(const DiskPtr & disk, const String & path) { - return disk->getPath() + path; + return fs::path(disk->getPath()) / path; } /// Return parent path for the specified path. @@ -313,7 +313,7 @@ inline String fileName(const String & path) /// Return directory path for the specified path. inline String directoryPath(const String & path) { - return fs::is_directory(path) ? path : fs::path(path).parent_path().string(); + return Poco::Path(path).setFileName("").toString(); } } diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index a28e7e0f387..442f015b3ec 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -117,12 +117,12 @@ struct DiskS3::Metadata using PathAndSize = std::pair; /// S3 root path. - const String & s3_root_path; + fs::path s3_root_path; /// Disk path. - const String & disk_path; + fs::path disk_path; /// Relative path to metadata file on local FS. - String metadata_file_path; + fs::path metadata_file_path; /// Total size of all S3 objects. size_t total_size; /// S3 objects paths and their sizes. @@ -141,14 +141,14 @@ struct DiskS3::Metadata try { - ReadBufferFromFile buf(disk_path + metadata_file_path, 1024); /* reasonable buffer size for small file */ + ReadBufferFromFile buf(disk_path / metadata_file_path, 1024); /* reasonable buffer size for small file */ UInt32 version; readIntText(version, buf); if (version < VERSION_ABSOLUTE_PATHS || version > VERSION_READ_ONLY_FLAG) throw Exception( - "Unknown metadata file version. Path: " + disk_path + metadata_file_path + "Unknown metadata file version. Path: " + (disk_path / metadata_file_path).string() + " Version: " + std::to_string(version) + ", Maximum expected version: " + std::to_string(VERSION_READ_ONLY_FLAG), ErrorCodes::UNKNOWN_FORMAT); @@ -169,12 +169,12 @@ struct DiskS3::Metadata readEscapedString(s3_object_path, buf); if (version == VERSION_ABSOLUTE_PATHS) { - if (!boost::algorithm::starts_with(s3_object_path, s3_root_path)) + if (!boost::algorithm::starts_with(s3_object_path, s3_root_path.string())) throw Exception( "Path in metadata does not correspond S3 root path. Path: " + s3_object_path - + ", root path: " + s3_root_path + ", disk path: " + disk_path_, + + ", root path: " + s3_root_path.string() + ", disk path: " + disk_path_, ErrorCodes::UNKNOWN_FORMAT); - s3_object_path = s3_object_path.substr(s3_root_path.size()); + s3_object_path = s3_object_path.substr(s3_root_path.string().size()); } assertChar('\n', buf); s3_objects[i] = {s3_object_path, s3_object_size}; @@ -207,7 +207,7 @@ struct DiskS3::Metadata /// Fsync metadata file if 'sync' flag is set. void save(bool sync = false) { - WriteBufferFromFile buf(disk_path + metadata_file_path, 1024); + WriteBufferFromFile buf(disk_path / metadata_file_path, 1024); writeIntText(VERSION_RELATIVE_PATHS, buf); writeChar('\n', buf); @@ -338,7 +338,7 @@ private: const auto & [path, size] = metadata.s3_objects[i]; if (size > offset) { - auto buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, s3_max_single_read_retries, buf_size); + auto buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path / path, s3_max_single_read_retries, buf_size); buf->seek(offset, SEEK_SET); return buf; } @@ -367,7 +367,7 @@ private: ++current_buf_idx; const auto & path = metadata.s3_objects[current_buf_idx].first; - current_buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path + path, s3_max_single_read_retries, buf_size); + current_buf = std::make_unique(client_ptr, bucket, metadata.s3_root_path / path, s3_max_single_read_retries, buf_size); current_buf->next(); working_buffer = current_buf->buffer(); absolute_position += working_buffer.size(); @@ -447,16 +447,16 @@ public: String path() const override { if (fs::is_directory(iter->path())) - return folder_path + iter->path().filename().string() + '/'; + return folder_path / iter->path().filename().string() / ""; else - return folder_path + iter->path().filename().string(); + return folder_path / iter->path().filename().string(); } String name() const override { return iter->path().filename(); } private: fs::directory_iterator iter; - String folder_path; + fs::path folder_path; }; @@ -623,13 +623,13 @@ String DiskS3::getUniqueId(const String & path) const Metadata metadata(s3_root_path, metadata_path, path); String id; if (!metadata.s3_objects.empty()) - id = metadata.s3_root_path + metadata.s3_objects[0].first; + id = metadata.s3_root_path / metadata.s3_objects[0].first; return id; } DiskDirectoryIteratorPtr DiskS3::iterateDirectory(const String & path) { - return std::make_unique(metadata_path + path, path); + return std::make_unique(fs::path(metadata_path) / path, path); } void DiskS3::clearDirectory(const String & path) @@ -683,7 +683,7 @@ std::unique_ptr DiskS3::readFile(const String & path, si auto metadata = readMeta(path); LOG_DEBUG(log, "Read from file by path: {}. Existing S3 objects: {}", - backQuote(metadata_path + path), metadata.s3_objects.size()); + backQuote((fs::path(metadata_path) / path).string()), metadata.s3_objects.size()); auto reader = std::make_unique(settings->client, bucket, metadata, settings->s3_max_single_read_retries, buf_size); return std::make_unique(std::move(reader), settings->min_bytes_for_seek); @@ -708,12 +708,12 @@ std::unique_ptr DiskS3::writeFile(const String & path, } LOG_DEBUG(log, "{} to file by path: {}. S3 path: {}", - mode == WriteMode::Rewrite ? "Write" : "Append", backQuote(metadata_path + path), s3_root_path + s3_path); + mode == WriteMode::Rewrite ? "Write" : "Append", backQuote((fs::path(metadata_path) / path).string()), (fs::path(s3_root_path) / s3_path).string()); auto s3_buffer = std::make_unique( settings->client, bucket, - metadata.s3_root_path + s3_path, + fs::path(metadata.s3_root_path) / s3_path, settings->s3_min_upload_part_size, settings->s3_max_single_part_upload_size, std::move(object_metadata), @@ -724,7 +724,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, void DiskS3::removeMeta(const String & path, AwsS3KeyKeeper & keys) { - LOG_DEBUG(log, "Remove file by path: {}", backQuote(metadata_path + path)); + LOG_DEBUG(log, "Remove file by path: {}", backQuote((fs::path(metadata_path) / path).string())); fs::path file = fs::path(metadata_path) / path; @@ -741,7 +741,7 @@ void DiskS3::removeMeta(const String & path, AwsS3KeyKeeper & keys) fs::remove(file); for (const auto & [s3_object_path, _] : metadata.s3_objects) - keys.addKey(s3_root_path + s3_object_path); + keys.addKey(fs::path(s3_root_path) / s3_object_path); } else /// In other case decrement number of references, save metadata and delete file. { @@ -904,7 +904,7 @@ void DiskS3::createHardLink(const String & src_path, const String & dst_path, bo src.save(); /// Create FS hardlink to metadata file. - DB::createHardLink(metadata_path + src_path, metadata_path + dst_path); + DB::createHardLink(fs::path(metadata_path) / src_path, fs::path(metadata_path) / dst_path); } void DiskS3::createFile(const String & path) @@ -940,7 +940,7 @@ void DiskS3::createFileOperationObject(const String & operation_name, UInt64 rev WriteBufferFromS3 buffer( settings->client, bucket, - s3_root_path + key, + fs::path(s3_root_path) / key, settings->s3_min_upload_part_size, settings->s3_max_single_part_upload_size, metadata); @@ -993,14 +993,14 @@ void DiskS3::findLastRevision() int DiskS3::readSchemaVersion(const String & source_bucket, const String & source_path) { int version = 0; - if (!checkObjectExists(source_bucket, source_path + SCHEMA_VERSION_OBJECT)) + if (!checkObjectExists(source_bucket, fs::path(source_path) / SCHEMA_VERSION_OBJECT)) return version; auto settings = current_settings.get(); ReadBufferFromS3 buffer( settings->client, source_bucket, - source_path + SCHEMA_VERSION_OBJECT, + fs::path(source_path) / SCHEMA_VERSION_OBJECT, settings->s3_max_single_read_retries); readIntText(version, buffer); @@ -1015,7 +1015,7 @@ void DiskS3::saveSchemaVersion(const int & version) WriteBufferFromS3 buffer( settings->client, bucket, - s3_root_path + SCHEMA_VERSION_OBJECT, + fs::path(s3_root_path) / SCHEMA_VERSION_OBJECT, settings->s3_min_upload_part_size, settings->s3_max_single_part_upload_size); @@ -1027,7 +1027,7 @@ void DiskS3::updateObjectMetadata(const String & key, const ObjectMetadata & met { auto settings = current_settings.get(); Aws::S3::Model::CopyObjectRequest request; - request.SetCopySource(bucket + "/" + key); + request.SetCopySource(fs::path(bucket) / key); request.SetBucket(bucket); request.SetKey(key); request.SetMetadata(metadata); @@ -1039,7 +1039,7 @@ void DiskS3::updateObjectMetadata(const String & key, const ObjectMetadata & met void DiskS3::migrateFileToRestorableSchema(const String & path) { - LOG_DEBUG(log, "Migrate file {} to restorable schema", metadata_path + path); + LOG_DEBUG(log, "Migrate file {} to restorable schema", (fs::path(metadata_path) / path).string()); auto meta = readMeta(path); @@ -1048,7 +1048,7 @@ void DiskS3::migrateFileToRestorableSchema(const String & path) ObjectMetadata metadata { {"path", path} }; - updateObjectMetadata(s3_root_path + key, metadata); + updateObjectMetadata(fs::path(s3_root_path) / key, metadata); } } @@ -1056,7 +1056,7 @@ void DiskS3::migrateToRestorableSchemaRecursive(const String & path, Futures & r { checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks. - LOG_DEBUG(log, "Migrate directory {} to restorable schema", metadata_path + path); + LOG_DEBUG(log, "Migrate directory {} to restorable schema", (fs::path(metadata_path) / path).string()); bool dir_contains_only_files = true; for (auto it = iterateDirectory(path); it->isValid(); it->next()) @@ -1105,7 +1105,7 @@ void DiskS3::migrateToRestorableSchema() for (const auto & root : data_roots) if (exists(root)) - migrateToRestorableSchemaRecursive(root + '/', results); + migrateToRestorableSchemaRecursive(root, results); for (auto & result : results) result.wait(); @@ -1194,7 +1194,7 @@ void DiskS3::copyObject(const String & src_bucket, const String & src_key, const { auto settings = current_settings.get(); Aws::S3::Model::CopyObjectRequest request; - request.SetCopySource(src_bucket + "/" + src_key); + request.SetCopySource(fs::path(src_bucket) / src_key); request.SetBucket(dst_bucket); request.SetKey(dst_key); @@ -1212,7 +1212,7 @@ struct DiskS3::RestoreInformation void DiskS3::readRestoreInformation(DiskS3::RestoreInformation & restore_information) { - ReadBufferFromFile buffer(metadata_path + RESTORE_FILE_NAME, 512); + ReadBufferFromFile buffer(fs::path(metadata_path) / RESTORE_FILE_NAME, 512); buffer.next(); try @@ -1302,7 +1302,7 @@ void DiskS3::restore() bool cleanup_s3 = information.source_bucket != bucket || information.source_path != s3_root_path; for (const auto & root : data_roots) if (exists(root)) - removeSharedRecursive(root + '/', !cleanup_s3); + removeSharedRecursive(root, !cleanup_s3); restoreFiles(information); restoreFileOperations(information); @@ -1393,8 +1393,8 @@ void DiskS3::processRestoreFiles(const String & source_bucket, const String & so auto relative_key = shrinkKey(source_path, key); /// Copy object if we restore to different bucket / path. - if (bucket != source_bucket || s3_root_path != source_path) - copyObject(source_bucket, key, bucket, s3_root_path + relative_key); + if (bucket != source_bucket || fs::path(s3_root_path) != fs::path(source_path)) + copyObject(source_bucket, key, bucket, fs::path(s3_root_path) / relative_key); metadata.addObject(relative_key, head_result.GetContentLength()); metadata.save(); @@ -1482,7 +1482,7 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio }; /// Execute. - listObjects(restore_information.source_bucket, restore_information.source_path + "operations/", restore_file_operations); + listObjects(restore_information.source_bucket, fs::path(restore_information.source_path) / "operations/", restore_file_operations); if (restore_information.detached) { @@ -1505,7 +1505,7 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio LOG_DEBUG(log, "Move directory to 'detached' {} -> {}", path, detached_path); - fs::rename(fs::path(metadata_path) / path, fs::path(metadata_path) / detached_path); + Poco::File(fs::path(metadata_path) / path).moveTo(fs::path(metadata_path) / detached_path); } } @@ -1537,13 +1537,13 @@ String DiskS3::revisionToString(UInt64 revision) String DiskS3::pathToDetached(const String & source_path) { - return fs::path(source_path).parent_path() / "detached" / ""; + return Poco::Path(source_path).parent().append(Poco::Path("detached")).toString() + '/'; } void DiskS3::onFreeze(const String & path) { createDirectories(path); - WriteBufferFromFile revision_file_buf(metadata_path + path + "revision.txt", 32); + WriteBufferFromFile revision_file_buf(fs::path(metadata_path) / path / "revision.txt", 32); writeIntText(revision_counter.load(), revision_file_buf); revision_file_buf.finalize(); } diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index dfeb31af7b7..82617e6e0af 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -230,7 +230,7 @@ void Service::sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuf { String file_name = it.first; - String path = part->getFullRelativePath() + file_name; + String path = fs::path(part->getFullRelativePath()) / file_name; UInt64 size = disk->getFileSize(path); @@ -279,7 +279,7 @@ void Service::sendPartS3Metadata(const MergeTreeData::DataPartPtr & part, WriteB { String file_name = it.first; - String metadata_file = disk->getPath() + part->getFullRelativePath() + file_name; + String metadata_file = fs::path(disk->getPath()) / part->getFullRelativePath() / file_name; fs::path metadata(metadata_file); @@ -480,7 +480,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( readUUIDText(part_uuid, in); auto storage_id = data.getStorageID(); - String new_part_path = part_type == "InMemory" ? "memory" : data.getFullPathOnDisk(reservation->getDisk()) + part_name + "/"; + String new_part_path = part_type == "InMemory" ? "memory" : fs::path(data.getFullPathOnDisk(reservation->getDisk())) / part_name / ""; auto entry = data.getContext()->getReplicatedFetchList().insert( storage_id.getDatabaseName(), storage_id.getTableName(), part_info.partition_id, part_name, new_part_path, @@ -551,7 +551,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( throw Exception("Logical error: tmp_prefix and part_name cannot be empty or contain '.' or '/' characters.", ErrorCodes::LOGICAL_ERROR); String part_relative_path = String(to_detached ? "detached/" : "") + tmp_prefix + part_name; - String part_download_path = data.getRelativeDataPath() + part_relative_path + "/"; + String part_download_path = fs::path(data.getRelativeDataPath()) / part_relative_path / ""; if (disk->exists(part_download_path)) { @@ -583,7 +583,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( " This may happen if we are trying to download part from malicious replica or logical error.", ErrorCodes::INSECURE_PATH); - auto file_out = disk->writeFile(part_download_path + file_name); + auto file_out = disk->writeFile(fs::path(part_download_path) / file_name); HashingWriteBuffer hashing_out(*file_out); copyData(in, hashing_out, file_size, blocker.getCounter()); @@ -600,7 +600,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( readPODBinary(expected_hash, in); if (expected_hash != hashing_out.getHash()) - throw Exception("Checksum mismatch for file " + fullPath(disk, part_download_path + file_name) + " transferred from " + replica_path, + throw Exception("Checksum mismatch for file " + fullPath(disk, (fs::path(part_download_path) / file_name).string()) + " transferred from " + replica_path, ErrorCodes::CHECKSUM_DOESNT_MATCH); if (file_name != "checksums.txt" && @@ -654,7 +654,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToS3( String tmp_prefix = tmp_prefix_.empty() ? TMP_PREFIX : tmp_prefix_; String part_relative_path = String(to_detached ? "detached/" : "") + tmp_prefix + part_name; - String part_download_path = data.getRelativeDataPath() + part_relative_path + "/"; + String part_download_path = fs::path(data.getRelativeDataPath()) / part_relative_path / ""; if (disk->exists(part_download_path)) throw Exception("Directory " + fullPath(disk, part_download_path) + " already exists.", ErrorCodes::DIRECTORY_ALREADY_EXISTS); @@ -677,7 +677,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToS3( readStringBinary(file_name, in); readBinary(file_size, in); - String data_path = new_data_part->getFullRelativePath() + file_name; + String data_path = fs::path(new_data_part->getFullRelativePath()) / file_name; String metadata_file = fullPath(disk, data_path); { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 36032f9208f..437718954d4 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -69,7 +69,7 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Dis hyperrectangle.reserve(minmax_idx_size); for (size_t i = 0; i < minmax_idx_size; ++i) { - String file_name = part_path + "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; + String file_name = fs::path(part_path) / ("minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"); auto file = openForReading(disk_, file_name); auto serialization = minmax_column_types[i]->getDefaultSerialization(); @@ -111,7 +111,7 @@ void IMergeTreeDataPart::MinMaxIndex::store( String file_name = "minmax_" + escapeForFileName(column_names[i]) + ".idx"; auto serialization = data_types.at(i)->getDefaultSerialization(); - auto out = disk_->writeFile(part_path + file_name); + auto out = disk_->writeFile(fs::path(part_path) / file_name); HashingWriteBuffer out_hashing(*out); serialization->serializeBinary(hyperrectangle[i].left, out_hashing); serialization->serializeBinary(hyperrectangle[i].right, out_hashing); @@ -543,7 +543,7 @@ String IMergeTreeDataPart::getFullPath() const if (relative_path.empty()) throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR); - return storage.getFullPathOnDisk(volume->getDisk()) + relative_path + "/"; + return fs::path(storage.getFullPathOnDisk(volume->getDisk())) / relative_path / ""; } String IMergeTreeDataPart::getFullRelativePath() const @@ -551,7 +551,7 @@ String IMergeTreeDataPart::getFullRelativePath() const if (relative_path.empty()) throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR); - return storage.relative_data_path + relative_path + "/"; + return fs::path(storage.relative_data_path) / relative_path / ""; } void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency) @@ -604,7 +604,7 @@ void IMergeTreeDataPart::loadIndex() loaded_index[i]->reserve(index_granularity.getMarksCount()); } - String index_path = getFullRelativePath() + "primary.idx"; + String index_path = fs::path(getFullRelativePath()) / "primary.idx"; auto index_file = openForReading(volume->getDisk(), index_path); size_t marks_count = index_granularity.getMarksCount(); @@ -639,7 +639,7 @@ NameSet IMergeTreeDataPart::getFileNamesWithoutChecksums() const return {}; NameSet result = {"checksums.txt", "columns.txt"}; - String default_codec_path = getFullRelativePath() + DEFAULT_COMPRESSION_CODEC_FILE_NAME; + String default_codec_path = fs::path(getFullRelativePath()) / DEFAULT_COMPRESSION_CODEC_FILE_NAME; if (volume->getDisk()->exists(default_codec_path)) result.emplace(DEFAULT_COMPRESSION_CODEC_FILE_NAME); @@ -656,7 +656,7 @@ void IMergeTreeDataPart::loadDefaultCompressionCodec() return; } - String path = getFullRelativePath() + DEFAULT_COMPRESSION_CODEC_FILE_NAME; + String path = fs::path(getFullRelativePath()) / DEFAULT_COMPRESSION_CODEC_FILE_NAME; if (!volume->getDisk()->exists(path)) { default_codec = detectDefaultCompressionCodec(); @@ -717,7 +717,7 @@ CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const { if (path_to_data_file.empty()) { - String candidate_path = getFullRelativePath() + ISerialization::getFileNameForStream(part_column, substream_path) + ".bin"; + String candidate_path = fs::path(getFullRelativePath()) / (ISerialization::getFileNameForStream(part_column, substream_path) + ".bin"); /// We can have existing, but empty .bin files. Example: LowCardinality(Nullable(...)) columns and column_name.dict.null.bin file. if (volume->getDisk()->exists(candidate_path) && volume->getDisk()->getFileSize(candidate_path) != 0) @@ -773,7 +773,7 @@ void IMergeTreeDataPart::loadPartitionAndMinMaxIndex() void IMergeTreeDataPart::loadChecksums(bool require) { - const String path = getFullRelativePath() + "checksums.txt"; + const String path = fs::path(getFullRelativePath()) / "checksums.txt"; if (volume->getDisk()->exists(path)) { @@ -798,11 +798,11 @@ void IMergeTreeDataPart::loadChecksums(bool require) checksums = checkDataPart(shared_from_this(), false); { - auto out = volume->getDisk()->writeFile(getFullRelativePath() + "checksums.txt.tmp", 4096); + auto out = volume->getDisk()->writeFile(fs::path(getFullRelativePath()) / "checksums.txt.tmp", 4096); checksums.write(*out); } - volume->getDisk()->moveFile(getFullRelativePath() + "checksums.txt.tmp", getFullRelativePath() + "checksums.txt"); + volume->getDisk()->moveFile(fs::path(getFullRelativePath()) / "checksums.txt.tmp", fs::path(getFullRelativePath()) / "checksums.txt"); bytes_on_disk = checksums.getTotalSizeOnDisk(); } @@ -810,7 +810,7 @@ void IMergeTreeDataPart::loadChecksums(bool require) void IMergeTreeDataPart::loadRowsCount() { - String path = getFullRelativePath() + "count.txt"; + String path = fs::path(getFullRelativePath()) / "count.txt"; if (index_granularity.empty()) { rows_count = 0; @@ -911,7 +911,7 @@ void IMergeTreeDataPart::loadRowsCount() void IMergeTreeDataPart::loadTTLInfos() { - String path = getFullRelativePath() + "ttl.txt"; + String path = fs::path(getFullRelativePath()) / "ttl.txt"; if (volume->getDisk()->exists(path)) { auto in = openForReading(volume->getDisk(), path); @@ -938,7 +938,7 @@ void IMergeTreeDataPart::loadTTLInfos() void IMergeTreeDataPart::loadUUID() { - String path = getFullRelativePath() + UUID_FILE_NAME; + String path = fs::path(getFullRelativePath()) / UUID_FILE_NAME; if (volume->getDisk()->exists(path)) { @@ -951,7 +951,7 @@ void IMergeTreeDataPart::loadUUID() void IMergeTreeDataPart::loadColumns(bool require) { - String path = getFullRelativePath() + "columns.txt"; + String path = fs::path(getFullRelativePath()) / "columns.txt"; auto metadata_snapshot = storage.getInMemoryMetadataPtr(); NamesAndTypesList loaded_columns; @@ -964,7 +964,7 @@ void IMergeTreeDataPart::loadColumns(bool require) /// If there is no file with a list of columns, write it down. for (const NameAndTypePair & column : metadata_snapshot->getColumns().getAllPhysical()) - if (volume->getDisk()->exists(getFullRelativePath() + getFileNameForColumn(column) + ".bin")) + if (volume->getDisk()->exists(fs::path(getFullRelativePath()) / (getFileNameForColumn(column) + ".bin"))) loaded_columns.push_back(column); if (columns.empty()) @@ -1002,7 +1002,7 @@ UInt64 IMergeTreeDataPart::calculateTotalSizeOnDisk(const DiskPtr & disk_, const disk_->listFiles(from, files); UInt64 res = 0; for (const auto & file : files) - res += calculateTotalSizeOnDisk(disk_, from + file); + res += calculateTotalSizeOnDisk(disk_, fs::path(from) / file); return res; } @@ -1012,7 +1012,7 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_ assertOnDisk(); String from = getFullRelativePath(); - String to = storage.relative_data_path + new_relative_path + "/"; + String to = fs::path(storage.relative_data_path) / new_relative_path / ""; if (!volume->getDisk()->exists(from)) throw Exception("Part directory " + fullPath(volume->getDisk(), from) + " doesn't exist. Most likely it is a logical error.", ErrorCodes::FILE_DOESNT_EXIST); @@ -1066,8 +1066,8 @@ void IMergeTreeDataPart::remove(bool keep_s3) const * And a race condition can happen that will lead to "File not found" error here. */ - String from = storage.relative_data_path + relative_path; - String to = storage.relative_data_path + "delete_tmp_" + name; + fs::path from = fs::path(storage.relative_data_path) / relative_path; + fs::path to = fs::path(storage.relative_data_path) / ("delete_tmp_" + name); // TODO directory delete_tmp_ is never removed if server crashes before returning from this function if (volume->getDisk()->exists(to)) @@ -1076,7 +1076,7 @@ void IMergeTreeDataPart::remove(bool keep_s3) const try { - volume->getDisk()->removeSharedRecursive(to + "/", keep_s3); + volume->getDisk()->removeSharedRecursive(to / "", keep_s3); } catch (...) { @@ -1099,7 +1099,7 @@ void IMergeTreeDataPart::remove(bool keep_s3) const if (checksums.empty()) { /// If the part is not completely written, we cannot use fast path by listing files. - volume->getDisk()->removeSharedRecursive(to + "/", keep_s3); + volume->getDisk()->removeSharedRecursive(to / "", keep_s3); } else { @@ -1112,16 +1112,16 @@ void IMergeTreeDataPart::remove(bool keep_s3) const # pragma GCC diagnostic ignored "-Wunused-variable" #endif for (const auto & [file, _] : checksums.files) - volume->getDisk()->removeSharedFile(to + "/" + file, keep_s3); + volume->getDisk()->removeSharedFile(to / file, keep_s3); #if !defined(__clang__) # pragma GCC diagnostic pop #endif for (const auto & file : {"checksums.txt", "columns.txt"}) - volume->getDisk()->removeSharedFile(to + "/" + file, keep_s3); + volume->getDisk()->removeSharedFile(to / file, keep_s3); - volume->getDisk()->removeSharedFileIfExists(to + "/" + DEFAULT_COMPRESSION_CODEC_FILE_NAME, keep_s3); - volume->getDisk()->removeSharedFileIfExists(to + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME, keep_s3); + volume->getDisk()->removeSharedFileIfExists(to / DEFAULT_COMPRESSION_CODEC_FILE_NAME, keep_s3); + volume->getDisk()->removeSharedFileIfExists(to / DELETE_ON_DESTROY_MARKER_FILE_NAME, keep_s3); volume->getDisk()->removeDirectory(to); } @@ -1131,7 +1131,7 @@ void IMergeTreeDataPart::remove(bool keep_s3) const LOG_ERROR(storage.log, "Cannot quickly remove directory {} by removing files; fallback to recursive removal. Reason: {}", fullPath(volume->getDisk(), to), getCurrentExceptionMessage(false)); - volume->getDisk()->removeSharedRecursive(to + "/", keep_s3); + volume->getDisk()->removeSharedRecursive(to / "", keep_s3); } } } @@ -1149,7 +1149,7 @@ String IMergeTreeDataPart::getRelativePathForPrefix(const String & prefix) const { res = (prefix.empty() ? "" : prefix + "_") + name + (try_no ? "_try" + DB::toString(try_no) : ""); - if (!volume->getDisk()->exists(getFullRelativePath() + res)) + if (!volume->getDisk()->exists(fs::path(getFullRelativePath()) / res)) return res; LOG_WARNING(storage.log, "Directory {} (to detach to) already exists. Will detach to directory with '_tryN' suffix.", res); @@ -1172,11 +1172,11 @@ void IMergeTreeDataPart::renameToDetached(const String & prefix) const void IMergeTreeDataPart::makeCloneInDetached(const String & prefix, const StorageMetadataPtr & /*metadata_snapshot*/) const { - String destination_path = storage.relative_data_path + getRelativePathForDetachedPart(prefix); + String destination_path = fs::path(storage.relative_data_path) / getRelativePathForDetachedPart(prefix); /// Backup is not recursive (max_level is 0), so do not copy inner directories localBackup(volume->getDisk(), getFullRelativePath(), destination_path, 0); - volume->getDisk()->removeFileIfExists(destination_path + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME); + volume->getDisk()->removeFileIfExists(fs::path(destination_path) / DELETE_ON_DESTROY_MARKER_FILE_NAME); } void IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & disk, const String & directory_name) const @@ -1188,16 +1188,16 @@ void IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & disk, const String & di if (directory_name.empty()) throw Exception("Can not clone data part " + name + " to empty directory.", ErrorCodes::LOGICAL_ERROR); - String path_to_clone = storage.relative_data_path + directory_name + '/'; + String path_to_clone = fs::path(storage.relative_data_path) / directory_name / ""; - if (disk->exists(path_to_clone + relative_path)) + if (disk->exists(fs::path(path_to_clone) / relative_path)) { LOG_WARNING(storage.log, "Path " + fullPath(disk, path_to_clone + relative_path) + " already exists. Will remove it and clone again."); - disk->removeRecursive(path_to_clone + relative_path + '/'); + disk->removeRecursive(fs::path(path_to_clone) / relative_path / ""); } disk->createDirectories(path_to_clone); volume->getDisk()->copy(getFullRelativePath(), disk, path_to_clone); - volume->getDisk()->removeFileIfExists(path_to_clone + '/' + DELETE_ON_DESTROY_MARKER_FILE_NAME); + volume->getDisk()->removeFileIfExists(fs::path(path_to_clone) / DELETE_ON_DESTROY_MARKER_FILE_NAME); } void IMergeTreeDataPart::checkConsistencyBase() const @@ -1244,17 +1244,17 @@ void IMergeTreeDataPart::checkConsistencyBase() const /// Check that the primary key index is not empty. if (!pk.column_names.empty()) - check_file_not_empty(volume->getDisk(), path + "primary.idx"); + check_file_not_empty(volume->getDisk(), fs::path(path) / "primary.idx"); if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { - check_file_not_empty(volume->getDisk(), path + "count.txt"); + check_file_not_empty(volume->getDisk(), fs::path(path) / "count.txt"); if (metadata_snapshot->hasPartitionKey()) - check_file_not_empty(volume->getDisk(), path + "partition.dat"); + check_file_not_empty(volume->getDisk(), fs::path(path) / "partition.dat"); for (const String & col_name : storage.getMinMaxColumnsNames(partition_key)) - check_file_not_empty(volume->getDisk(), path + "minmax_" + escapeForFileName(col_name) + ".idx"); + check_file_not_empty(volume->getDisk(), fs::path(path) / ("minmax_" + escapeForFileName(col_name) + ".idx")); } } } @@ -1348,7 +1348,7 @@ String IMergeTreeDataPart::getUniqueId() const auto disk = volume->getDisk(); if (disk->getType() == DB::DiskType::Type::S3) - id = disk->getUniqueId(getFullRelativePath() + "checksums.txt"); + id = disk->getUniqueId(fs::path(getFullRelativePath()) / "checksums.txt"); if (id.empty()) throw Exception("Can't get unique S3 object", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index dfebd88abe9..17673bbdd97 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1868,7 +1868,7 @@ void MergeTreeDataMergerMutator::finalizeMutatedPart( if (need_remove_expired_values) { /// Write a file with ttl infos in json format. - auto out_ttl = disk->writeFile(new_data_part->getFullRelativePath() + "ttl.txt", 4096); + auto out_ttl = disk->writeFile(fs::path(new_data_part->getFullRelativePath()) / "ttl.txt", 4096); HashingWriteBuffer out_hashing(*out_ttl); new_data_part->ttl_infos.write(out_hashing); new_data_part->checksums.files["ttl.txt"].file_size = out_hashing.count(); @@ -1877,7 +1877,7 @@ void MergeTreeDataMergerMutator::finalizeMutatedPart( { /// Write file with checksums. - auto out_checksums = disk->writeFile(new_data_part->getFullRelativePath() + "checksums.txt", 4096); + auto out_checksums = disk->writeFile(fs::path(new_data_part->getFullRelativePath()) / "checksums.txt", 4096); new_data_part->checksums.write(*out_checksums); } /// close fd @@ -1888,7 +1888,7 @@ void MergeTreeDataMergerMutator::finalizeMutatedPart( { /// Write a file with a description of columns. - auto out_columns = disk->writeFile(new_data_part->getFullRelativePath() + "columns.txt", 4096); + auto out_columns = disk->writeFile(fs::path(new_data_part->getFullRelativePath()) / "columns.txt", 4096); new_data_part->getColumns().writeText(*out_columns); } /// close fd diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index ab364e0e5aa..bc330ce3dae 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -121,7 +121,7 @@ void MergedBlockOutputStream::finalizePartOnDisk( throw Exception("MinMax index was not initialized for new non-empty part " + new_part->name + ". It is a bug.", ErrorCodes::LOGICAL_ERROR); - auto count_out = volume->getDisk()->writeFile(part_path + "count.txt", 4096); + auto count_out = volume->getDisk()->writeFile(fs::path(part_path) / "count.txt", 4096); HashingWriteBuffer count_out_hashing(*count_out); writeIntText(rows_count, count_out_hashing); count_out_hashing.next(); @@ -135,7 +135,7 @@ void MergedBlockOutputStream::finalizePartOnDisk( if (!new_part->ttl_infos.empty()) { /// Write a file with ttl infos in json format. - auto out = volume->getDisk()->writeFile(part_path + "ttl.txt", 4096); + auto out = volume->getDisk()->writeFile(fs::path(part_path) / "ttl.txt", 4096); HashingWriteBuffer out_hashing(*out); new_part->ttl_infos.write(out_hashing); checksums.files["ttl.txt"].file_size = out_hashing.count(); @@ -149,7 +149,7 @@ void MergedBlockOutputStream::finalizePartOnDisk( { /// Write a file with a description of columns. - auto out = volume->getDisk()->writeFile(part_path + "columns.txt", 4096); + auto out = volume->getDisk()->writeFile(fs::path(part_path) / "columns.txt", 4096); part_columns.writeText(*out); out->finalize(); if (sync) @@ -170,7 +170,7 @@ void MergedBlockOutputStream::finalizePartOnDisk( { /// Write file with checksums. - auto out = volume->getDisk()->writeFile(part_path + "checksums.txt", 4096); + auto out = volume->getDisk()->writeFile(fs::path(part_path) / "checksums.txt", 4096); checksums.write(*out); out->finalize(); if (sync) diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index ac28f84db43..35c73145e66 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -68,7 +68,7 @@ IMergeTreeDataPart::Checksums checkDataPart( NamesAndTypesList columns_txt; { - auto buf = disk->readFile(path + "columns.txt"); + auto buf = disk->readFile(fs::path(path) / "columns.txt"); columns_txt.readText(*buf); assertEOF(*buf); } @@ -141,9 +141,9 @@ IMergeTreeDataPart::Checksums checkDataPart( /// Checksums from the rest files listed in checksums.txt. May be absent. If present, they are subsequently compared with the actual data checksums. IMergeTreeDataPart::Checksums checksums_txt; - if (require_checksums || disk->exists(path + "checksums.txt")) + if (require_checksums || disk->exists(fs::path(path) / "checksums.txt")) { - auto buf = disk->readFile(path + "checksums.txt"); + auto buf = disk->readFile(fs::path(path) / "checksums.txt"); checksums_txt.read(*buf); assertEOF(*buf); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b3165febd7c..b9daa4a1f41 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1482,8 +1482,8 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ auto disk = part->volume->getDisk(); String part_path = part->getFullRelativePath(); /// If the checksums file is not present, calculate the checksums and write them to disk. - String checksums_path = part_path + "checksums.txt"; - String tmp_checksums_path = part_path + "checksums.txt.tmp"; + String checksums_path = fs::path(part_path) / "checksums.txt"; + String tmp_checksums_path = fs::path(part_path) / "checksums.txt.tmp"; if (part->isStoredOnDisk() && !disk->exists(checksums_path)) { try From 5068b163b81950ef775ac567aea1a271eabe3985 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 5 May 2021 19:55:02 +0300 Subject: [PATCH 21/55] Fix --- src/Disks/DiskLocal.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 64c599c5b81..99f0162cf12 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -69,7 +69,14 @@ public: bool isValid() const override { return entry != fs::directory_iterator(); } - String path() const override { return dir_path / entry->path().filename(); } + String path() const override + { + if (entry->is_directory()) + return dir_path / entry->path().filename() / ""; + else + return dir_path / entry->path().filename(); + } + String name() const override { return entry->path().filename(); } From 140bf7e2eaa8901f4ba6abbc13e44c5b88713361 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 7 May 2021 15:29:26 +0300 Subject: [PATCH 22/55] Less Poco --- src/Common/Config/ConfigReloader.cpp | 4 +++- .../MySQL/DatabaseConnectionMySQL.cpp | 2 +- src/Disks/DiskLocal.cpp | 2 +- src/Disks/S3/DiskS3.cpp | 2 +- src/Disks/S3/registerDiskS3.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 24 +++++++++---------- 6 files changed, 19 insertions(+), 17 deletions(-) diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index afff08e82bb..ad0ce7cc30e 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -5,7 +5,9 @@ #include #include #include "ConfigProcessor.h" +#include +namespace fs = std::filesystem; namespace DB { @@ -167,7 +169,7 @@ struct ConfigReloader::FileWithTimestamp void ConfigReloader::FilesChangesTracker::addIfExists(const std::string & path_to_add) { - if (!path_to_add.empty() && Poco::File(path_to_add).exists()) + if (!path_to_add.empty() && fs::exists(path_to_add)) files.emplace(path_to_add, Poco::File(path_to_add).getLastModified().epochTime()); } diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index 2b6c6d243a7..7b1bb7a0494 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -408,7 +408,7 @@ void DatabaseConnectionMySQL::detachTablePermanently(ContextPtr, const String & { std::lock_guard lock{mutex}; - Poco::File remove_flag(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix); + Poco::File remove_flag(fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix)); if (remove_or_detach_tables.count(table_name)) throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " is dropped", diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 99f0162cf12..85023dba464 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -309,7 +309,7 @@ bool inline isSameDiskType(const IDisk & one, const IDisk & another) void DiskLocal::copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) { if (isSameDiskType(*this, *to_disk)) - Poco::File(fs::path(disk_path) / from_path).copyTo(fs::path(to_disk->getPath()) / to_path); /// Use more optimal way. + fs::copy(fs::path(disk_path) / from_path, fs::path(to_disk->getPath()) / to_path); /// Use more optimal way. else IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. } diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 442f015b3ec..faff486eeae 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -1505,7 +1505,7 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio LOG_DEBUG(log, "Move directory to 'detached' {} -> {}", path, detached_path); - Poco::File(fs::path(metadata_path) / path).moveTo(fs::path(metadata_path) / detached_path); + fs::rename(fs::path(metadata_path) / path, fs::path(metadata_path) / detached_path); } } diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index 639bb46c033..56726cfcca1 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -172,7 +172,7 @@ void registerDiskS3(DiskFactory & factory) throw Exception("S3 path must ends with '/', but '" + uri.key + "' doesn't.", ErrorCodes::BAD_ARGUMENTS); String metadata_path = config.getString(config_prefix + ".metadata_path", context->getPath() + "disks/" + name + "/"); - Poco::File (metadata_path).createDirectories(); + fs::create_directories(metadata_path); std::shared_ptr s3disk = std::make_shared( name, diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 383e504aad8..1deed68f239 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -209,8 +209,8 @@ MergeTreeData::MergeTreeData( for (const auto & [path, disk] : getRelativeDataPathsWithDisks()) { disk->createDirectories(path); - disk->createDirectories(path + MergeTreeData::DETACHED_DIR_NAME); - auto current_version_file_path = path + MergeTreeData::FORMAT_VERSION_FILE_NAME; + disk->createDirectories(fs::path(path) / MergeTreeData::DETACHED_DIR_NAME); + String current_version_file_path = fs::path(path) / MergeTreeData::FORMAT_VERSION_FILE_NAME; if (disk->exists(current_version_file_path)) { if (!version_file.first.empty()) @@ -224,7 +224,7 @@ MergeTreeData::MergeTreeData( /// If not choose any if (version_file.first.empty()) - version_file = {relative_data_path + MergeTreeData::FORMAT_VERSION_FILE_NAME, getStoragePolicy()->getAnyDisk()}; + version_file = {fs::path(relative_data_path) / MergeTreeData::FORMAT_VERSION_FILE_NAME, getStoragePolicy()->getAnyDisk()}; bool version_file_exists = version_file.second->exists(version_file.first); @@ -3854,10 +3854,10 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( const String & with_name, ContextPtr local_context) { - String clickhouse_path = fs::absolute(local_context->getPath()); - String default_shadow_path = clickhouse_path + "shadow/"; + String clickhouse_path = fs::canonical(local_context->getPath()); + String default_shadow_path = fs::path(clickhouse_path) / "shadow/"; fs::create_directories(default_shadow_path); - auto increment = Increment(default_shadow_path + "increment.txt").get(true); + auto increment = Increment(fs::path(default_shadow_path) / "increment.txt").get(true); const String shadow_path = "shadow/"; @@ -3865,7 +3865,7 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( const auto data_parts = getDataParts(); String backup_name = (!with_name.empty() ? escapeForFileName(with_name) : toString(increment)); - String backup_path = shadow_path + backup_name + "/"; + String backup_path = fs::path(shadow_path) / backup_name / ""; for (const auto & disk : getStoragePolicy()->getDisks()) disk->onFreeze(backup_path); @@ -3882,20 +3882,20 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( part->volume->getDisk()->createDirectories(backup_path); - String backup_part_path = backup_path + relative_data_path + part->relative_path; + String backup_part_path = fs::path(backup_path) / relative_data_path / part->relative_path; if (auto part_in_memory = asInMemoryPart(part)) - part_in_memory->flushToDisk(backup_path + relative_data_path, part->relative_path, metadata_snapshot); + part_in_memory->flushToDisk(fs::path(backup_path) / relative_data_path, part->relative_path, metadata_snapshot); else localBackup(part->volume->getDisk(), part->getFullRelativePath(), backup_part_path); - part->volume->getDisk()->removeFileIfExists(backup_part_path + "/" + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); + part->volume->getDisk()->removeFileIfExists(fs::path(backup_part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); part->is_frozen.store(true, std::memory_order_relaxed); result.push_back(PartitionCommandResultInfo{ .partition_id = part->info.partition_id, .part_name = part->name, - .backup_path = part->volume->getDisk()->getPath() + backup_path, - .part_backup_path = part->volume->getDisk()->getPath() + backup_part_path, + .backup_path = fs::path(part->volume->getDisk()->getPath()) / backup_path, + .part_backup_path = fs::path(part->volume->getDisk()->getPath()) / backup_part_path, .backup_name = backup_name, }); ++parts_processed; From a2cfbd74ec22cf283c2d392699b877d8cb3d74ac Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 7 May 2021 22:00:36 +0300 Subject: [PATCH 23/55] Fix fs::copy to work the same as Poco::copy --- src/Disks/DiskLocal.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 85023dba464..7c192f70cd2 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -309,7 +309,15 @@ bool inline isSameDiskType(const IDisk & one, const IDisk & another) void DiskLocal::copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) { if (isSameDiskType(*this, *to_disk)) - fs::copy(fs::path(disk_path) / from_path, fs::path(to_disk->getPath()) / to_path); /// Use more optimal way. + { + fs::path from = fs::path(disk_path) / from_path; + if (from_path.ends_with('/')) + from = (fs::path(disk_path) / from_path.substr(0, from_path.size() - 1)).parent_path(); + else if (fs::is_directory(from)) + from = from.parent_path(); + + fs::copy(from, fs::path(to_disk->getPath()) / to_path, fs::copy_options::recursive | fs::copy_options::overwrite_existing); /// Use more optimal way. + } else IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. } From 35f999bf0406a46c423ec5d87ed5359c373bf585 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 8 May 2021 00:53:44 +0300 Subject: [PATCH 24/55] Poco::createFile to fs::createFile --- src/Common/ErrorCodes.cpp | 2 + src/Common/Exception.cpp | 1 + src/Common/createFile.cpp | 52 +++++++++++++++++++ src/Common/createFile.h | 7 +++ src/Databases/DatabaseOnDisk.cpp | 36 ++++++------- .../MySQL/DatabaseConnectionMySQL.cpp | 18 +++---- .../PostgreSQL/DatabasePostgreSQL.cpp | 6 +-- src/Disks/DiskLocal.cpp | 3 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 11 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 20 +++++-- src/Storages/MergeTree/localBackup.cpp | 15 +++--- .../examples/remove_symlink_directory.cpp | 18 +++---- 12 files changed, 131 insertions(+), 58 deletions(-) create mode 100644 src/Common/createFile.cpp create mode 100644 src/Common/createFile.h diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index ad0463db889..40be6a64336 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -550,6 +550,8 @@ M(580, CANNOT_SET_ROUNDING_MODE) \ M(581, TOO_LARGE_DISTRIBUTED_DEPTH) \ \ + M(996, OPERATION_NOT_PERMITTED) \ + M(997, CANNOT_CREATE_FILE) \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index dca19eea7f2..f17eaea8d7f 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -251,6 +251,7 @@ static std::string getExtraExceptionInfo(const std::exception & e) String msg; try { + /// TODO: this has to be adjusted for std::filesystem if (const auto * file_exception = dynamic_cast(&e)) { if (file_exception->code() == ENOSPC) diff --git a/src/Common/createFile.cpp b/src/Common/createFile.cpp new file mode 100644 index 00000000000..fc637f4d4f2 --- /dev/null +++ b/src/Common/createFile.cpp @@ -0,0 +1,52 @@ +#include "createFile.h" +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ +extern const int FILE_ALREADY_EXISTS; +extern const int OPERATION_NOT_PERMITTED; +extern const int NOT_ENOUGH_SPACE; +extern const int CANNOT_CREATE_FILE; +} +} + +namespace std::filesystem +{ +[[noreturn]] void handleLastError(const std::string & path) +{ + switch (errno) + { + case EEXIST: + throw DB::Exception(DB::ErrorCodes::FILE_ALREADY_EXISTS, "File {} already exist", path); + case EPERM: + throw DB::Exception(DB::ErrorCodes::OPERATION_NOT_PERMITTED, "Not enough permissions to create file {}", path); + case ENOSPC: + throw DB::Exception(DB::ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space to create file {}", path); + case ENAMETOOLONG: + throw DB::Exception(DB::ErrorCodes::CANNOT_CREATE_FILE, "File name {} is too long"); + default: + throw DB::Exception(DB::ErrorCodes::CANNOT_CREATE_FILE, "Cannot create file {}. Error: {}", path, strerror(errno)); + } +} + +/// Copy from Poco::createFile +bool createFile(const path & path) +{ + int n = open(path.c_str(), O_WRONLY | O_CREAT | O_EXCL, S_IRUSR | S_IWUSR | S_IRGRP | S_IWGRP | S_IROTH | S_IWOTH); + if (n != -1) + { + close(n); + return true; + } + if (n == -1 && errno == EEXIST) + return false; + handleLastError(path); +} +} diff --git a/src/Common/createFile.h b/src/Common/createFile.h new file mode 100644 index 00000000000..40ec74e6288 --- /dev/null +++ b/src/Common/createFile.h @@ -0,0 +1,7 @@ +#pragma once +#include + +namespace std::filesystem +{ +bool createFile(const path & path); +} diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 44ec5d27250..95cd47e3796 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -14,14 +14,14 @@ #include #include #include - #include -#include - #include #include #include #include +#include +#include +#include namespace fs = std::filesystem; @@ -321,10 +321,10 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr, const String & table_nam { auto table = detachTable(table_name); - Poco::File detached_permanently_flag(getObjectMetadataPath(table_name) + detached_suffix); + fs::path detached_permanently_flag(getObjectMetadataPath(table_name) + detached_suffix); try { - detached_permanently_flag.createFile(); + fs::createFile(detached_permanently_flag); } catch (Exception & e) { @@ -572,40 +572,40 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat /// Metadata files to load: name and flag for .tmp_drop files std::set> metadata_files; - Poco::DirectoryIterator dir_end; - for (Poco::DirectoryIterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it) + fs::directory_iterator dir_end; + for (fs::directory_iterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it) { + String file_name = dir_it->path().filename(); /// For '.svn', '.gitignore' directory and similar. - if (dir_it.name().at(0) == '.') + if (file_name.at(0) == '.') continue; /// There are .sql.bak files - skip them. - if (endsWith(dir_it.name(), ".sql.bak")) + if (endsWith(file_name, ".sql.bak")) continue; /// Permanently detached table flag - if (endsWith(dir_it.name(), ".sql.detached")) + if (endsWith(file_name, ".sql.detached")) continue; - if (endsWith(dir_it.name(), ".sql.tmp_drop")) + if (endsWith(file_name, ".sql.tmp_drop")) { /// There are files that we tried to delete previously - metadata_files.emplace(dir_it.name(), false); + metadata_files.emplace(file_name, false); } - else if (endsWith(dir_it.name(), ".sql.tmp")) + else if (endsWith(file_name, ".sql.tmp")) { /// There are files .sql.tmp - delete - LOG_INFO(log, "Removing file {}", dir_it->path()); + LOG_INFO(log, "Removing file {}", dir_it->path().string()); fs::remove(dir_it->path()); } - else if (endsWith(dir_it.name(), ".sql")) + else if (endsWith(file_name, ".sql")) { /// The required files have names like `table_name.sql` - metadata_files.emplace(dir_it.name(), true); + metadata_files.emplace(file_name, true); } else - throw Exception("Incorrect file extension: " + dir_it.name() + " in metadata directory " + getMetadataPath(), - ErrorCodes::INCORRECT_FILE_NAME); + throw Exception(ErrorCodes::INCORRECT_FILE_NAME, "Incorrect file extension: {} in metadata directory {}", file_name, getMetadataPath()); } /// Read and parse metadata in parallel diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index 7b1bb7a0494..07da307c507 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -24,7 +24,7 @@ # include # include # include -# include +# include namespace fs = std::filesystem; @@ -408,27 +408,25 @@ void DatabaseConnectionMySQL::detachTablePermanently(ContextPtr, const String & { std::lock_guard lock{mutex}; - Poco::File remove_flag(fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix)); + fs::path remove_flag = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix); if (remove_or_detach_tables.count(table_name)) - throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " is dropped", - ErrorCodes::TABLE_IS_DROPPED); + throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {}.{} is dropped", backQuoteIfNeed(database_name), backQuoteIfNeed(table_name)); - if (remove_flag.exists()) - throw Exception("The remove flag file already exists but the " + backQuoteIfNeed(database_name) + - "." + backQuoteIfNeed(table_name) + " does not exists remove tables, it is bug.", ErrorCodes::LOGICAL_ERROR); + if (fs::exists(remove_flag)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "The remove flag file already exists but the {}.{} does not exists remove tables, it is bug.", + backQuoteIfNeed(database_name), backQuoteIfNeed(table_name)); auto table_iter = local_tables_cache.find(table_name); if (table_iter == local_tables_cache.end()) - throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " doesn't exist.", - ErrorCodes::UNKNOWN_TABLE); + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuoteIfNeed(database_name), backQuoteIfNeed(table_name)); remove_or_detach_tables.emplace(table_name); try { table_iter->second.second->drop(); - remove_flag.createFile(); + fs::createFile(remove_flag); } catch (...) { diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 9a60e579d9e..37e678a1c20 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include namespace fs = std::filesystem; @@ -253,10 +254,7 @@ void DatabasePostgreSQL::dropTable(ContextPtr, const String & table_name, bool / throw Exception(fmt::format("Table {}.{} is already dropped/detached", database_name, table_name), ErrorCodes::TABLE_IS_DROPPED); fs::path mark_table_removed = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix); - - FILE * file = fopen(mark_table_removed.string().data(), "a+"); - if (file == nullptr) - throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Cannot create file {}", mark_table_removed.string()); + fs::createFile(mark_table_removed); if (cache_tables) cached_tables.erase(table_name); diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 7c192f70cd2..31a8d217909 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -293,7 +294,7 @@ void DiskLocal::truncateFile(const String & path, size_t size) void DiskLocal::createFile(const String & path) { - Poco::File(fs::path(disk_path) / path).createFile(); + fs::createFile(fs::path(disk_path) / path); } void DiskLocal::setReadOnly(const String & path) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 437718954d4..02560ca3e48 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1089,11 +1089,14 @@ void IMergeTreeDataPart::remove(bool keep_s3) const { volume->getDisk()->moveDirectory(from, to); } - catch (const Poco::FileNotFoundException &) + catch (const fs::filesystem_error & e) { - LOG_ERROR(storage.log, "Directory {} (part to remove) doesn't exist or one of nested files has gone. Most likely this is due to manual removing. This should be discouraged. Ignoring.", fullPath(volume->getDisk(), to)); - - return; + if (e.code() == std::errc::no_such_file_or_directory) + { + LOG_ERROR(storage.log, "Directory {} (part to remove) doesn't exist or one of nested files has gone. Most likely this is due to manual removing. This should be discouraged. Ignoring.", fullPath(volume->getDisk(), to)); + return; + } + throw; } if (checksums.empty()) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1deed68f239..59297d0bc7c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1085,9 +1085,14 @@ void MergeTreeData::clearOldTemporaryDirectories(ssize_t custom_directories_life disk->removeRecursive(it->path()); } } - catch (const Poco::FileNotFoundException &) + catch (const fs::filesystem_error & e) { - /// If the file is already deleted, do nothing. + if (e.code() == std::errc::no_such_file_or_directory) + { + /// If the file is already deleted, do nothing. + } + else + throw; } } } @@ -1373,10 +1378,15 @@ void MergeTreeData::dropAllData() { disk->removeRecursive(path); } - catch (const Poco::FileNotFoundException &) + catch (const fs::filesystem_error & e) { - /// If the file is already deleted, log the error message and do nothing. - tryLogCurrentException(__PRETTY_FUNCTION__); + if (e.code() == std::errc::no_such_file_or_directory) + { + /// If the file is already deleted, log the error message and do nothing. + tryLogCurrentException(__PRETTY_FUNCTION__); + } + else + throw; } } diff --git a/src/Storages/MergeTree/localBackup.cpp b/src/Storages/MergeTree/localBackup.cpp index 7d7dacaeaf1..9f9f894c535 100644 --- a/src/Storages/MergeTree/localBackup.cpp +++ b/src/Storages/MergeTree/localBackup.cpp @@ -74,13 +74,16 @@ void localBackup(const DiskPtr & disk, const String & source_path, const String continue; } - catch (const Poco::FileNotFoundException &) + catch (const fs::filesystem_error & e) { - ++try_no; - if (try_no == max_tries) - throw; - - continue; + if (e.code() == std::errc::no_such_file_or_directory) + { + ++try_no; + if (try_no == max_tries) + throw; + continue; + } + throw; } break; diff --git a/src/Storages/examples/remove_symlink_directory.cpp b/src/Storages/examples/remove_symlink_directory.cpp index ae5fa72fa66..05fdc18be2f 100644 --- a/src/Storages/examples/remove_symlink_directory.cpp +++ b/src/Storages/examples/remove_symlink_directory.cpp @@ -1,9 +1,10 @@ #include #include -#include -#include #include +#include +#include +namespace fs = std::filesystem; namespace DB { @@ -16,18 +17,15 @@ namespace DB int main(int, char **) try { - Poco::File dir("./test_dir/"); - dir.createDirectories(); - - Poco::File("./test_dir/file").createFile(); + fs::path dir("./test_dir/"); + fs::create_directories(dir); + fs::createFile("./test_dir/file"); if (0 != symlink("./test_dir", "./test_link")) DB::throwFromErrnoWithPath("Cannot create symlink", "./test_link", DB::ErrorCodes::SYSTEM_ERROR); - Poco::File link("./test_link"); - link.renameTo("./test_link2"); - - Poco::File("./test_link2").remove(true); + fs::rename("./test_link", "./test_link2"); + fs::remove_all("./test_link2"); return 0; } catch (...) From 02288359c5f3cbb117a230dc3abc64afffaaf872 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 8 May 2021 13:59:55 +0300 Subject: [PATCH 25/55] Less manual concatenation of paths --- src/Access/AllowedClientHosts.h | 6 +- src/Common/ZooKeeper/ZooKeeper.cpp | 10 +- src/Databases/DatabaseAtomic.cpp | 4 +- src/Databases/DatabaseReplicatedWorker.cpp | 19 +- src/Storages/Distributed/DirectoryMonitor.cpp | 4 +- .../DistributedBlockOutputStream.cpp | 6 +- src/Storages/HDFS/StorageHDFS.cpp | 5 +- src/Storages/MergeTree/MergeTreeData.cpp | 32 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- .../MergeTree/MergeTreePartsMover.cpp | 12 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 52 +-- .../ReplicatedMergeTreeRestartingThread.cpp | 12 +- src/Storages/MergeTree/localBackup.cpp | 2 +- src/Storages/StorageFile.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 361 +++++++++--------- src/Storages/StorageS3.cpp | 4 +- .../get_current_inserts_in_replicated.cpp | 10 +- 17 files changed, 280 insertions(+), 265 deletions(-) diff --git a/src/Access/AllowedClientHosts.h b/src/Access/AllowedClientHosts.h index a6895b120e0..7b21fd7e236 100644 --- a/src/Access/AllowedClientHosts.h +++ b/src/Access/AllowedClientHosts.h @@ -7,7 +7,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -198,9 +200,9 @@ inline String AllowedClientHosts::IPSubnet::toString() const if (isMaskAllBitsOne()) return prefix.toString(); else if (IPAddress{prefix_length, mask.family()} == mask) - return prefix.toString() + "/" + std::to_string(prefix_length); + return fs::path(prefix.toString()) / std::to_string(prefix_length); else - return prefix.toString() + "/" + mask.toString(); + return fs::path(prefix.toString()) / mask.toString(); } inline bool AllowedClientHosts::IPSubnet::isMaskAllBitsOne() const diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 9f59da233fc..74c35f4f0e9 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -4,6 +4,7 @@ #include "TestKeeper.h" #include +#include #include #include @@ -17,6 +18,7 @@ #define ZOOKEEPER_CONNECTION_TIMEOUT_MS 1000 +namespace fs = std::filesystem; namespace DB { @@ -593,7 +595,7 @@ void ZooKeeper::removeChildren(const std::string & path) Coordination::Requests ops; for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i) { - ops.emplace_back(makeRemoveRequest(path + "/" + children.back(), -1)); + ops.emplace_back(makeRemoveRequest(fs::path(path) / children.back(), -1)); children.pop_back(); } multi(ops); @@ -609,9 +611,9 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path, const String & Coordination::Requests ops; for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i) { - removeChildrenRecursive(path + "/" + children.back()); + removeChildrenRecursive(fs::path(path) / children.back()); if (likely(keep_child_node.empty() || keep_child_node != children.back())) - ops.emplace_back(makeRemoveRequest(path + "/" + children.back(), -1)); + ops.emplace_back(makeRemoveRequest(fs::path(path) / children.back(), -1)); children.pop_back(); } multi(ops); @@ -629,7 +631,7 @@ void ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, const Strin Strings batch; for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i) { - String child_path = path + "/" + children.back(); + String child_path = fs::path(path) / children.back(); tryRemoveChildrenRecursive(child_path); if (likely(keep_child_node.empty() || keep_child_node != children.back())) { diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index fa4e2eed382..0dae31ac906 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -37,8 +37,8 @@ public: DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, const String & logger_name, ContextPtr context_) : DatabaseOrdinary(name_, std::move(metadata_path_), "store/", logger_name, context_) - , path_to_table_symlinks(getContext()->getPath() + "data/" + escapeForFileName(name_) + "/") - , path_to_metadata_symlink(getContext()->getPath() + "metadata/" + escapeForFileName(name_)) + , path_to_table_symlinks(fs::path(getContext()->getPath()) / "data" / escapeForFileName(name_) / "") + , path_to_metadata_symlink(fs::path(getContext()->getPath()) / "metadata" / escapeForFileName(name_)) , db_uuid(uuid) { assert(db_uuid != UUIDHelpers::Nil); diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 9ae4d026bf0..760300d6750 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -1,6 +1,9 @@ #include #include #include +#include + +namespace fs = std::filesystem; namespace DB { @@ -156,7 +159,7 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na } } - UInt32 our_log_ptr = parse(current_zookeeper->get(database->replica_path + "/log_ptr")); + UInt32 our_log_ptr = parse(current_zookeeper->get(fs::path(database->replica_path) / "log_ptr")); UInt32 entry_num = DatabaseReplicatedTask::getLogEntryNumber(entry_name); if (entry_num <= our_log_ptr) @@ -165,13 +168,13 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na return {}; } - String entry_path = queue_dir + "/" + entry_name; + String entry_path = fs::path(queue_dir) / entry_name; auto task = std::make_unique(entry_name, entry_path, database); String initiator_name; zkutil::EventPtr wait_committed_or_failed = std::make_shared(); - String try_node_path = entry_path + "/try"; + String try_node_path = fs::path(entry_path) / "try"; if (zookeeper->tryGet(try_node_path, initiator_name, nullptr, wait_committed_or_failed)) { task->is_initial_query = initiator_name == task->host_id_str; @@ -203,7 +206,7 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE) throw Coordination::Exception(code, try_node_path); - if (!zookeeper->exists(entry_path + "/committed")) + if (!zookeeper->exists(fs::path(entry_path) / "committed")) { out_reason = fmt::format("Entry {} was forcefully cancelled due to timeout", entry_name); return {}; @@ -212,7 +215,7 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na } } - if (!zookeeper->exists(entry_path + "/committed")) + if (!zookeeper->exists(fs::path(entry_path) / "committed")) { out_reason = fmt::format("Entry {} hasn't been committed", entry_name); return {}; @@ -220,8 +223,8 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na if (task->is_initial_query) { - assert(!zookeeper->exists(entry_path + "/try")); - assert(zookeeper->exists(entry_path + "/committed") == (zookeeper->get(task->getFinishedNodePath()) == ExecutionStatus(0).serializeText())); + assert(!zookeeper->exists(fs::path(entry_path) / "try")); + assert(zookeeper->exists(fs::path(entry_path) / "committed") == (zookeeper->get(task->getFinishedNodePath()) == ExecutionStatus(0).serializeText())); out_reason = fmt::format("Entry {} has been executed as initial query", entry_name); return {}; } @@ -257,7 +260,7 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na bool DatabaseReplicatedDDLWorker::canRemoveQueueEntry(const String & entry_name, const Coordination::Stat &) { UInt32 entry_number = DDLTaskBase::getLogEntryNumber(entry_name); - UInt32 max_log_ptr = parse(getAndSetZooKeeper()->get(database->zookeeper_path + "/max_log_ptr")); + UInt32 max_log_ptr = parse(getAndSetZooKeeper()->get(fs::path(database->zookeeper_path) / "max_log_ptr")); return entry_number + logs_to_keep < max_log_ptr; } diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index ce19d3ec89f..bae81a56b30 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -293,7 +293,7 @@ StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( , pool(std::move(pool_)) , disk(disk_) , relative_path(relative_path_) - , path(disk->getPath() + relative_path + '/') + , path(fs::path(disk->getPath()) / relative_path / "") , should_batch_inserts(storage.getContext()->getSettingsRef().distributed_directory_monitor_batch_inserts) , dir_fsync(storage.getDistributedSettingsRef().fsync_directories) , min_batched_block_size_rows(storage.getContext()->getSettingsRef().min_insert_block_size_rows) @@ -1023,7 +1023,7 @@ void StorageDistributedDirectoryMonitor::updatePath(const std::string & new_rela { std::lock_guard status_lock(status_mutex); relative_path = new_relative_path; - path = disk->getPath() + relative_path + '/'; + path = fs::path(disk->getPath()) / relative_path / ""; } current_batch_file_path = path + "current_batch.txt"; diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 2f0187fc115..ea694fb0cfe 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -717,7 +717,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: } // Create hardlink here to reuse increment number - const std::string block_file_path(path + '/' + file_name); + const std::string block_file_path(fs::path(path) / file_name); createHardLink(first_file_tmp_path, block_file_path); auto dir_sync_guard = make_directory_sync_guard(*it); } @@ -726,10 +726,10 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: /// Make hardlinks for (; it != dir_names.end(); ++it) { - const std::string path(disk_path + data_path + *it); + const std::string path(fs::path(disk_path) / (data_path + *it)); fs::create_directory(path); - const std::string block_file_path(path + '/' + toString(storage.file_names_increment.get()) + ".bin"); + const std::string block_file_path(fs::path(path) / (toString(storage.file_names_increment.get()) + ".bin")); createHardLink(first_file_tmp_path, block_file_path); auto dir_sync_guard = make_directory_sync_guard(*it); } diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index c08e487f179..4f1aec29e6b 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -18,7 +18,6 @@ #include #include #include - #include #include #include @@ -26,7 +25,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -250,7 +251,7 @@ Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, c { if (re2::RE2::FullMatch(file_name, matcher)) { - Strings result_part = LSWithRegexpMatching(full_path + "/", fs, suffix_with_globs.substr(next_slash)); + Strings result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, suffix_with_globs.substr(next_slash)); /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 59297d0bc7c..1849b2efebd 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -852,8 +852,8 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) auto part = createPart(part_name, part_info, single_disk_volume, part_name); bool broken = false; - String part_path = relative_data_path + "/" + part_name; - String marker_path = part_path + "/" + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME; + String part_path = fs::path(relative_data_path) / part_name; + String marker_path = fs::path(part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME; if (part_disk_ptr->exists(marker_path)) { LOG_WARNING(log, "Detaching stale part {}{}, which should have been deleted after a move. That can only happen after unclean restart of ClickHouse after move of a part having an operation blocking that stale copy of part.", getFullPathOnDisk(part_disk_ptr), part_name); @@ -941,7 +941,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) else has_adaptive_parts.store(true, std::memory_order_relaxed); - part->modification_time = part_disk_ptr->getLastModified(relative_data_path + part_name).epochTime(); + part->modification_time = part_disk_ptr->getLastModified(fs::path(relative_data_path) / part_name).epochTime(); /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later part->setState(DataPartState::Committed); @@ -1409,8 +1409,8 @@ void MergeTreeData::dropIfEmpty() for (const auto & [path, disk] : getRelativeDataPathsWithDisks()) { /// Non recursive, exception is thrown if there are more files. - disk->removeFileIfExists(path + MergeTreeData::FORMAT_VERSION_FILE_NAME); - disk->removeDirectory(path + MergeTreeData::DETACHED_DIR_NAME); + disk->removeFileIfExists(fs::path(path) / MergeTreeData::FORMAT_VERSION_FILE_NAME); + disk->removeDirectory(fs::path(path) / MergeTreeData::DETACHED_DIR_NAME); disk->removeDirectory(path); } } @@ -1854,7 +1854,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createPart( const VolumePtr & volume, const String & relative_path) const { MergeTreeDataPartType type; - auto full_path = relative_data_path + relative_path + "/"; + auto full_path = fs::path(relative_data_path) / relative_path / ""; auto mrk_ext = MergeTreeIndexGranularityInfo::getMarksExtensionFromFilesystem(volume->getDisk(), full_path); if (mrk_ext) @@ -1907,7 +1907,7 @@ void MergeTreeData::changeSettings( { auto disk = new_storage_policy->getDiskByName(disk_name); disk->createDirectories(relative_data_path); - disk->createDirectories(relative_data_path + MergeTreeData::DETACHED_DIR_NAME); + disk->createDirectories(fs::path(relative_data_path) / MergeTreeData::DETACHED_DIR_NAME); } /// FIXME how would that be done while reloading configuration??? @@ -1936,7 +1936,7 @@ void MergeTreeData::PartsTemporaryRename::addPart(const String & old_name, const old_and_new_names.push_back({old_name, new_name}); for (const auto & [path, disk] : storage.getRelativeDataPathsWithDisks()) { - for (auto it = disk->iterateDirectory(path + source_dir); it->isValid(); it->next()) + for (auto it = disk->iterateDirectory(fs::path(path) / source_dir); it->isValid(); it->next()) { if (it->name() == old_name) { @@ -1958,8 +1958,8 @@ void MergeTreeData::PartsTemporaryRename::tryRenameAll() if (old_name.empty() || new_name.empty()) throw DB::Exception("Empty part name. Most likely it's a bug.", ErrorCodes::INCORRECT_FILE_NAME); const auto & [path, disk] = old_part_name_to_path_and_disk[old_name]; - const auto full_path = path + source_dir; /// for old_name - disk->moveFile(full_path + old_name, full_path + new_name); + const auto full_path = fs::path(path) / source_dir; /// for old_name + disk->moveFile(fs::path(full_path) / old_name, fs::path(full_path) / new_name); } catch (...) { @@ -1983,8 +1983,8 @@ MergeTreeData::PartsTemporaryRename::~PartsTemporaryRename() try { const auto & [path, disk] = old_part_name_to_path_and_disk[old_name]; - const auto full_path = path + source_dir; /// for old_name - disk->moveFile(full_path + new_name, full_path + old_name); + const String full_path = fs::path(path) / source_dir; /// for old_name + disk->moveFile(fs::path(full_path) / new_name, fs::path(full_path) / old_name); } catch (...) { @@ -2672,7 +2672,7 @@ void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy) addPartContributionToDataVolume(part_copy); auto disk = original_active_part->volume->getDisk(); - String marker_path = original_active_part->getFullRelativePath() + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME; + String marker_path = fs::path(original_active_part->getFullRelativePath()) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME; try { disk->createFile(marker_path); @@ -3747,12 +3747,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk( const auto & src_relative_data_path = src_part_in_memory->storage.relative_data_path; auto flushed_part_path = src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix); src_part_in_memory->flushToDisk(src_relative_data_path, flushed_part_path, metadata_snapshot); - src_part_path = src_relative_data_path + flushed_part_path + "/"; + src_part_path = fs::path(src_relative_data_path) / flushed_part_path / ""; } LOG_DEBUG(log, "Cloning part {} to {}", fullPath(disk, src_part_path), fullPath(disk, dst_part_path)); localBackup(disk, src_part_path, dst_part_path); - disk->removeFileIfExists(dst_part_path + "/" + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); + disk->removeFileIfExists(fs::path(dst_part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); auto single_disk_volume = std::make_shared(disk->getName(), disk, 0); auto dst_data_part = createPart(dst_part_name, dst_part_info, single_disk_volume, tmp_dst_part_name); @@ -3934,7 +3934,7 @@ PartitionCommandsResultInfo MergeTreeData::unfreezeAll( PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn matcher, const String & backup_name, ContextPtr) { - auto backup_path = std::filesystem::path("shadow") / escapeForFileName(backup_name) / relative_data_path; + auto backup_path = fs::path("shadow") / escapeForFileName(backup_name) / relative_data_path; LOG_DEBUG(log, "Unfreezing parts by path {}", backup_path.generic_string()); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index ac078eee6bd..8debf961b89 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -133,7 +133,7 @@ using RelativeSize = boost::rational; static std::string toString(const RelativeSize & x) { - return ASTSampleRatio::toString(x.numerator()) + "/" + ASTSampleRatio::toString(x.denominator()); + return fs::path(ASTSampleRatio::toString(x.numerator())) / ASTSampleRatio::toString(x.denominator()); } /// Converts sample size to an approximate number of rows (ex. `SAMPLE 1000000`) to relative value (ex. `SAMPLE 0.1`). diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index f9e3883d5e2..5b77ac9ec4a 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -206,18 +206,18 @@ MergeTreeData::DataPartPtr MergeTreePartsMover::clonePart(const MergeTreeMoveEnt /// Try to fetch part from S3 without copy and fallback to default copy /// if it's not possible moving_part.part->assertOnDisk(); - String path_to_clone = data->getRelativeDataPath() + directory_to_move + "/"; + String path_to_clone = fs::path(data->getRelativeDataPath()) / directory_to_move / ""; String relative_path = part->relative_path; if (disk->exists(path_to_clone + relative_path)) { LOG_WARNING(log, "Path " + fullPath(disk, path_to_clone + relative_path) + " already exists. Will remove it and clone again."); - disk->removeRecursive(path_to_clone + relative_path + "/"); + disk->removeRecursive(fs::path(path_to_clone) / relative_path / ""); } disk->createDirectories(path_to_clone); - bool is_fetched = data->tryToFetchIfShared(*part, disk, path_to_clone + "/" + part->name); + bool is_fetched = data->tryToFetchIfShared(*part, disk, fs::path(path_to_clone) / part->name); if (!is_fetched) - part->volume->getDisk()->copy(data->getRelativeDataPath() + relative_path + "/", disk, path_to_clone); - part->volume->getDisk()->removeFileIfExists(path_to_clone + "/" + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); + part->volume->getDisk()->copy(fs::path(data->getRelativeDataPath()) / relative_path / "", disk, path_to_clone); + part->volume->getDisk()->removeFileIfExists(fs::path(path_to_clone) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); } else { @@ -226,7 +226,7 @@ MergeTreeData::DataPartPtr MergeTreePartsMover::clonePart(const MergeTreeMoveEnt auto single_disk_volume = std::make_shared("volume_" + part->name, moving_part.reserved_space->getDisk(), 0); MergeTreeData::MutableDataPartPtr cloned_part = - data->createPart(part->name, single_disk_volume, directory_to_move + '/' + part->name); + data->createPart(part->name, single_disk_volume, fs::path(directory_to_move) / part->name); LOG_TRACE(log, "Part {} was cloned to {}", part->name, cloned_part->getFullPath()); cloned_part->loadColumnsChecksumsIndexes(true, true); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index ad41bbe1a08..f24e0b7b87d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -63,7 +63,7 @@ bool ReplicatedMergeTreeQueue::isVirtualPart(const MergeTreeData::DataPartPtr & bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper) { - auto queue_path = replica_path + "/queue"; + String queue_path = fs::path(replica_path) / "queue"; LOG_DEBUG(log, "Loading queue from {}", queue_path); bool updated = false; @@ -75,7 +75,7 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper) /// Reset batch size on initialization to recover from possible errors of too large batch size. current_multi_batch_size = 1; - String log_pointer_str = zookeeper->get(replica_path + "/log_pointer"); + String log_pointer_str = zookeeper->get(fs::path(replica_path) / "log_pointer"); log_pointer = log_pointer_str.empty() ? 0 : parse(log_pointer_str); std::unordered_set already_loaded_paths; @@ -102,7 +102,7 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper) futures.reserve(children.size()); for (const String & child : children) - futures.emplace_back(child, zookeeper->asyncGet(queue_path + "/" + child)); + futures.emplace_back(child, zookeeper->asyncGet(fs::path(queue_path) / child)); for (auto & future : futures) { @@ -117,7 +117,7 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper) updated = true; } - zookeeper->tryGet(replica_path + "/mutation_pointer", mutation_pointer); + zookeeper->tryGet(fs::path(replica_path) / "mutation_pointer", mutation_pointer); } updateTimesInZooKeeper(zookeeper, min_unprocessed_insert_time_changed, {}); @@ -401,7 +401,7 @@ void ReplicatedMergeTreeQueue::removeProcessedEntry(zkutil::ZooKeeperPtr zookeep if (!need_remove_from_zk) return; - auto code = zookeeper->tryRemove(replica_path + "/queue/" + entry->znode_name); + auto code = zookeeper->tryRemove(fs::path(replica_path) / "queue" / entry->znode_name); if (code != Coordination::Error::ZOK) LOG_ERROR(log, "Couldn't remove {}/queue/{}: {}. This shouldn't happen often.", replica_path, entry->znode_name, Coordination::errorMessage(code)); @@ -457,7 +457,7 @@ bool ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, const Stri notifySubscribers(queue_size); - zookeeper->tryRemove(replica_path + "/queue/" + found->znode_name); + zookeeper->tryRemove(fs::path(replica_path) / "queue" / found->znode_name); updateTimesInZooKeeper(zookeeper, min_unprocessed_insert_time_changed, max_processed_insert_time_changed); return true; @@ -476,14 +476,14 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper if (pull_log_blocker.isCancelled()) throw Exception("Log pulling is cancelled", ErrorCodes::ABORTED); - String index_str = zookeeper->get(replica_path + "/log_pointer"); + String index_str = zookeeper->get(fs::path(replica_path) / "log_pointer"); UInt64 index; /// The version of "/log" is modified when new entries to merge/mutate/drop appear. Coordination::Stat stat; - zookeeper->get(zookeeper_path + "/log", &stat); + zookeeper->get(fs::path(zookeeper_path) / "log", &stat); - Strings log_entries = zookeeper->getChildrenWatch(zookeeper_path + "/log", nullptr, watch_callback); + Strings log_entries = zookeeper->getChildrenWatch(fs::path(zookeeper_path) / "log", nullptr, watch_callback); /// We update mutations after we have loaded the list of log entries, but before we insert them /// in the queue. @@ -496,7 +496,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper /// If we do not already have a pointer to the log, put a pointer to the first entry in it. index = log_entries.empty() ? 0 : parse(std::min_element(log_entries.begin(), log_entries.end())->substr(strlen("log-"))); - zookeeper->set(replica_path + "/log_pointer", toString(index)); + zookeeper->set(fs::path(replica_path) / "log_pointer", toString(index)); } else { @@ -543,7 +543,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper futures.reserve(end - begin); for (auto it = begin; it != end; ++it) - futures.emplace_back(*it, zookeeper->asyncGet(zookeeper_path + "/log/" + *it)); + futures.emplace_back(*it, zookeeper->asyncGet(fs::path(zookeeper_path) / "log" / *it)); /// Simultaneously add all new entries to the queue and move the pointer to the log. @@ -560,7 +560,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper copied_entries.emplace_back(LogEntry::parse(res.data, res.stat)); ops.emplace_back(zkutil::makeCreateRequest( - replica_path + "/queue/queue-", res.data, zkutil::CreateMode::PersistentSequential)); + fs::path(replica_path) / "queue/queue-", res.data, zkutil::CreateMode::PersistentSequential)); const auto & entry = *copied_entries.back(); if (entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART) @@ -575,11 +575,11 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper } ops.emplace_back(zkutil::makeSetRequest( - replica_path + "/log_pointer", toString(last_entry_index + 1), -1)); + fs::path(replica_path) / "log_pointer", toString(last_entry_index + 1), -1)); if (min_unprocessed_insert_time_changed) ops.emplace_back(zkutil::makeSetRequest( - replica_path + "/min_unprocessed_insert_time", toString(*min_unprocessed_insert_time_changed), -1)); + fs::path(replica_path) / "min_unprocessed_insert_time", toString(*min_unprocessed_insert_time_changed), -1)); auto responses = zookeeper->multi(ops); @@ -657,7 +657,7 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C { std::lock_guard lock(update_mutations_mutex); - Strings entries_in_zk = zookeeper->getChildrenWatch(zookeeper_path + "/mutations", nullptr, watch_callback); + Strings entries_in_zk = zookeeper->getChildrenWatch(fs::path(zookeeper_path) / "mutations", nullptr, watch_callback); StringSet entries_in_zk_set(entries_in_zk.begin(), entries_in_zk.end()); /// Compare with the local state, delete obsolete entries and determine which new entries to load. @@ -714,7 +714,7 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C std::vector> futures; for (const String & entry : entries_to_load) - futures.emplace_back(zookeeper->asyncGet(zookeeper_path + "/mutations/" + entry)); + futures.emplace_back(zookeeper->asyncGet(fs::path(zookeeper_path) / "mutations" / entry)); std::vector new_mutations; for (size_t i = 0; i < entries_to_load.size(); ++i) @@ -798,7 +798,7 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation( { std::lock_guard lock(update_mutations_mutex); - auto rc = zookeeper->tryRemove(zookeeper_path + "/mutations/" + mutation_id); + auto rc = zookeeper->tryRemove(fs::path(zookeeper_path) / "mutations" / mutation_id); if (rc == Coordination::Error::ZOK) LOG_DEBUG(log, "Removed mutation {} from ZooKeeper.", mutation_id); @@ -935,12 +935,12 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( { if ((*it)->currently_executing) to_wait.push_back(*it); - auto code = zookeeper->tryRemove(replica_path + "/queue/" + (*it)->znode_name); + auto code = zookeeper->tryRemove(fs::path(replica_path) / "queue" / (*it)->znode_name); /// FIXME it's probably unsafe to remove entries non-atomically /// when this method called directly from alter query (not from replication queue task), /// because entries will be lost if ALTER fails. if (code != Coordination::Error::ZOK) - LOG_INFO(log, "Couldn't remove {}: {}", replica_path + "/queue/" + (*it)->znode_name, Coordination::errorMessage(code)); + LOG_INFO(log, "Couldn't remove {}: {}", (fs::path(replica_path) / "queue" / (*it)->znode_name).string(), Coordination::errorMessage(code)); updateStateOnQueueEntryRemoval( *it, /* is_successful = */ false, @@ -1593,7 +1593,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep if (!finished.empty()) { - zookeeper->set(replica_path + "/mutation_pointer", finished.back()->znode_name); + zookeeper->set(fs::path(replica_path) / "mutation_pointer", finished.back()->znode_name); std::lock_guard lock(state_mutex); @@ -1796,22 +1796,22 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( } /// Load current quorum status. - auto quorum_status_future = zookeeper->asyncTryGet(queue.zookeeper_path + "/quorum/status"); + auto quorum_status_future = zookeeper->asyncTryGet(fs::path(queue.zookeeper_path) / "quorum" / "status"); /// Load current inserts std::unordered_set lock_holder_paths; - for (const String & entry : zookeeper->getChildren(queue.zookeeper_path + "/temp")) + for (const String & entry : zookeeper->getChildren(fs::path(queue.zookeeper_path) / "temp")) { if (startsWith(entry, "abandonable_lock-")) - lock_holder_paths.insert(queue.zookeeper_path + "/temp/" + entry); + lock_holder_paths.insert(fs::path(queue.zookeeper_path) / "temp" / entry); } if (!lock_holder_paths.empty()) { - Strings partitions = zookeeper->getChildren(queue.zookeeper_path + "/block_numbers"); + Strings partitions = zookeeper->getChildren(fs::path(queue.zookeeper_path) / "block_numbers"); std::vector> lock_futures; for (const String & partition : partitions) - lock_futures.push_back(zookeeper->asyncGetChildren(queue.zookeeper_path + "/block_numbers/" + partition)); + lock_futures.push_back(zookeeper->asyncGetChildren(fs::path(queue.zookeeper_path) / "block_numbers" / partition)); struct BlockInfoInZooKeeper { @@ -1832,7 +1832,7 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( if (startsWith(entry, "block-")) { Int64 block_number = parse(entry.substr(strlen("block-"))); - String zk_path = queue.zookeeper_path + "/block_numbers/" + partitions[i] + "/" + entry; + String zk_path = fs::path(queue.zookeeper_path) / "block_numbers" / partitions[i] / entry; block_infos.emplace_back( BlockInfoInZooKeeper{partitions[i], block_number, zk_path, zookeeper->asyncTryGet(zk_path)}); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index ca6ea3103d1..df57499ae90 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -230,7 +230,7 @@ void ReplicatedMergeTreeRestartingThread::updateQuorumIfWeHavePart() auto zookeeper = storage.getZooKeeper(); String quorum_str; - if (zookeeper->tryGet(storage.zookeeper_path + "/quorum/status", quorum_str)) + if (zookeeper->tryGet(fs::path(storage.zookeeper_path) / "quorum" / "status", quorum_str)) { ReplicatedMergeTreeQuorumEntry quorum_entry(quorum_str); @@ -243,12 +243,12 @@ void ReplicatedMergeTreeRestartingThread::updateQuorumIfWeHavePart() } Strings part_names; - String parallel_quorum_parts_path = storage.zookeeper_path + "/quorum/parallel"; + String parallel_quorum_parts_path = fs::path(storage.zookeeper_path) / "quorum" / "parallel"; if (zookeeper->tryGetChildren(parallel_quorum_parts_path, part_names) == Coordination::Error::ZOK) { for (auto & part_name : part_names) { - if (zookeeper->tryGet(parallel_quorum_parts_path + "/" + part_name, quorum_str)) + if (zookeeper->tryGet(fs::path(parallel_quorum_parts_path) / part_name, quorum_str)) { ReplicatedMergeTreeQuorumEntry quorum_entry(quorum_str); if (!quorum_entry.replicas.count(storage.replica_name) @@ -270,7 +270,7 @@ void ReplicatedMergeTreeRestartingThread::activateReplica() /// How other replicas can access this one. ReplicatedMergeTreeAddress address = storage.getReplicatedMergeTreeAddress(); - String is_active_path = storage.replica_path + "/is_active"; + String is_active_path = fs::path(storage.replica_path) / "is_active"; /** If the node is marked as active, but the mark is made in the same instance, delete it. * This is possible only when session in ZooKeeper expires. @@ -294,7 +294,7 @@ void ReplicatedMergeTreeRestartingThread::activateReplica() /// Simultaneously declare that this replica is active, and update the host. Coordination::Requests ops; ops.emplace_back(zkutil::makeCreateRequest(is_active_path, active_node_identifier, zkutil::CreateMode::Ephemeral)); - ops.emplace_back(zkutil::makeSetRequest(storage.replica_path + "/host", address.toString(), -1)); + ops.emplace_back(zkutil::makeSetRequest(fs::path(storage.replica_path) / "host", address.toString(), -1)); try { @@ -303,7 +303,7 @@ void ReplicatedMergeTreeRestartingThread::activateReplica() catch (const Coordination::Exception & e) { String existing_replica_host; - zookeeper->tryGet(storage.replica_path + "/host", existing_replica_host); + zookeeper->tryGet(fs::path(storage.replica_path) / "host", existing_replica_host); if (existing_replica_host.empty()) existing_replica_host = "without host node"; diff --git a/src/Storages/MergeTree/localBackup.cpp b/src/Storages/MergeTree/localBackup.cpp index 9f9f894c535..2e02481e065 100644 --- a/src/Storages/MergeTree/localBackup.cpp +++ b/src/Storages/MergeTree/localBackup.cpp @@ -29,7 +29,7 @@ static void localBackupImpl(const DiskPtr & disk, const String & source_path, co for (auto it = disk->iterateDirectory(source_path); it->isValid(); it->next()) { auto source = it->path(); - auto destination = destination_path + "/" + it->name(); + auto destination = fs::path(destination_path) / it->name(); if (!disk->isDirectory(source)) { diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index be052f91b39..46b95908e2c 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -99,7 +99,7 @@ std::vector listFilesWithRegexpMatching(const std::string & path_fo if (re2::RE2::FullMatch(file_name, matcher)) { /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. - Strings result_part = listFilesWithRegexpMatching(full_path + "/", suffix_with_globs.substr(next_slash)); + Strings result_part = listFilesWithRegexpMatching(fs::path(full_path) / "", suffix_with_globs.substr(next_slash)); std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } } @@ -200,7 +200,7 @@ StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArgu if (args.format_name == "Distributed") throw Exception("Distributed format is allowed only with explicit file path", ErrorCodes::INCORRECT_FILE_NAME); - String table_dir_path = base_path + relative_table_dir_path + "/"; + String table_dir_path = fs::path(base_path) / relative_table_dir_path / ""; fs::create_directories(table_dir_path); paths = {getTablePath(table_dir_path, format_name)}; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 864c31ec05d..38d0f75bfe5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -66,9 +66,12 @@ #include #include #include +#include #include +namespace fs = std::filesystem; + namespace ProfileEvents { extern const Event ReplicatedPartMerges; @@ -848,12 +851,12 @@ bool StorageReplicatedMergeTree::removeTableNodesFromZooKeeper(zkutil::ZooKeeper for (const auto & child : children) if (child != "dropped") - zookeeper->tryRemoveRecursive(zookeeper_path + "/" + child); + zookeeper->tryRemoveRecursive(fs::path(zookeeper_path) / child); Coordination::Requests ops; Coordination::Responses responses; ops.emplace_back(zkutil::makeRemoveRequest(metadata_drop_lock->getPath(), -1)); - ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/dropped", -1)); + ops.emplace_back(zkutil::makeRemoveRequest(fs::path(zookeeper_path) / "dropped", -1)); ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path, -1)); code = zookeeper->tryMulti(ops, responses); @@ -892,12 +895,12 @@ void StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_pr ReplicatedMergeTreeTableMetadata old_metadata(*this, metadata_snapshot); Coordination::Stat metadata_stat; - String metadata_str = zookeeper->get(zookeeper_prefix + "/metadata", &metadata_stat); + String metadata_str = zookeeper->get(fs::path(zookeeper_prefix) / "metadata", &metadata_stat); auto metadata_from_zk = ReplicatedMergeTreeTableMetadata::parse(metadata_str); old_metadata.checkEquals(metadata_from_zk, metadata_snapshot->getColumns(), getContext()); Coordination::Stat columns_stat; - auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(zookeeper_prefix + "/columns", &columns_stat)); + auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_prefix) / "columns", &columns_stat)); const ColumnsDescription & old_columns = metadata_snapshot->getColumns(); if (columns_from_zk != old_columns) @@ -1039,7 +1042,7 @@ static time_t tryGetPartCreateTime(zkutil::ZooKeeperPtr & zookeeper, const Strin /// We get creation time of part, if it still exists (was not merged, for example). Coordination::Stat stat; String unused; - if (zookeeper->tryGet(replica_path + "/parts/" + part_name, unused, &stat)) + if (zookeeper->tryGet(fs::path(replica_path) / "parts" / part_name, unused, &stat)) res = stat.ctime / 1000; return res; @@ -1050,7 +1053,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) { auto zookeeper = getZooKeeper(); - Strings expected_parts_vec = zookeeper->getChildren(replica_path + "/parts"); + Strings expected_parts_vec = zookeeper->getChildren(fs::path(replica_path) / "parts"); /// Parts in ZK. NameSet expected_parts(expected_parts_vec.begin(), expected_parts_vec.end()); @@ -1153,7 +1156,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) exists_futures.reserve(parts_to_fetch.size()); for (const String & part_name : parts_to_fetch) { - String part_path = replica_path + "/parts/" + part_name; + String part_path = fs::path(replica_path) / "parts" / part_name; exists_futures.emplace_back(zookeeper->asyncExists(part_path)); } @@ -1182,7 +1185,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) /// We assume that this occurs before the queue is loaded (queue.initialize). ops.emplace_back(zkutil::makeCreateRequest( - replica_path + "/queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + fs::path(replica_path) / "queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); enqueue_futures.emplace_back(zookeeper->asyncMulti(ops)); } @@ -1208,13 +1211,13 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil: auto local_part_header = ReplicatedMergeTreePartHeader::fromColumnsAndChecksums( part->getColumns(), part->checksums); - Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + Strings replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); bool has_been_already_added = false; for (const String & replica : replicas) { - String current_part_path = zookeeper_path + "/replicas/" + replica + "/parts/" + part_name; + String current_part_path = fs::path(zookeeper_path) / "replicas" / replica / "parts" / part_name; String part_zk_str; if (!zookeeper->tryGet(current_part_path, part_zk_str)) @@ -1235,9 +1238,9 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil: String checksums_str; /// Let's check that the node's version with the columns did not change while we were reading the checksums. /// This ensures that the columns and the checksum refer to the same - if (!zookeeper->tryGet(current_part_path + "/columns", columns_str, &columns_stat_before) || - !zookeeper->tryGet(current_part_path + "/checksums", checksums_str) || - !zookeeper->exists(current_part_path + "/columns", &columns_stat_after) || + if (!zookeeper->tryGet(fs::path(current_part_path) / "columns", columns_str, &columns_stat_before) || + !zookeeper->tryGet(fs::path(current_part_path) / "checksums", checksums_str) || + !zookeeper->exists(fs::path(current_part_path) / "columns", &columns_stat_after) || columns_stat_before.version != columns_stat_after.version) { LOG_INFO(log, "Not checking checksums of part {} with replica {} because part changed while we were reading its checksums", part_name, replica); @@ -1271,7 +1274,7 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil: if (!has_been_already_added) { const auto storage_settings_ptr = getSettings(); - String part_path = replica_path + "/parts/" + part_name; + String part_path = fs::path(replica_path) / "parts" / part_name; //ops.emplace_back(zkutil::makeCheckRequest( // zookeeper_path + "/columns", expected_columns_version)); @@ -1286,14 +1289,15 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil: ops.emplace_back(zkutil::makeCreateRequest( part_path, "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( - part_path + "/columns", part->getColumns().toString(), zkutil::CreateMode::Persistent)); + fs::path(part_path) / "columns", part->getColumns().toString(), zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( - part_path + "/checksums", getChecksumsForZooKeeper(part->checksums), zkutil::CreateMode::Persistent)); + fs::path(part_path) / "checksums", getChecksumsForZooKeeper(part->checksums), zkutil::CreateMode::Persistent)); } } else { - LOG_WARNING(log, "checkPartAndAddToZooKeeper: node {} already exists. Will not commit any nodes.", replica_path + "/parts/" + part_name); + LOG_WARNING(log, "checkPartAndAddToZooKeeper: node {} already exists. Will not commit any nodes.", + (fs::path(replica_path) / "parts" / part_name).string()); } } @@ -1361,7 +1365,7 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo const String part_new_name = actual_part_info.getPartName(); for (const DiskPtr & disk : getStoragePolicy()->getDisks()) - for (const auto it = disk->iterateDirectory(relative_data_path + "detached/"); it->isValid(); it->next()) + for (const auto it = disk->iterateDirectory(fs::path(relative_data_path) / "detached/"); it->isValid(); it->next()) { MergeTreePartInfo part_info; @@ -1370,7 +1374,7 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::attachPartHelperFo continue; const String part_old_name = part_info.getPartName(); - const String part_path = "detached/" + part_old_name; + const String part_path = fs::path("detached") / part_old_name; const VolumePtr volume = std::make_shared("volume_" + part_old_name, disk); @@ -1425,7 +1429,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) existing_part = getActiveContainingPart(entry.new_part_name); /// Even if the part is local, it (in exceptional cases) may not be in ZooKeeper. Let's check that it is there. - if (existing_part && getZooKeeper()->exists(replica_path + "/parts/" + existing_part->name)) + if (existing_part && getZooKeeper()->exists(fs::path(replica_path) / "parts" / existing_part->name)) { if (!is_get_or_attach || entry.source_replica != replica_name) LOG_DEBUG(log, "Skipping action for part {} because part {} already exists.", @@ -1460,7 +1464,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) /// Perhaps we don't need this part, because during write with quorum, the quorum has failed /// (see below about `/quorum/failed_parts`). - if (entry.quorum && getZooKeeper()->exists(zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name)) + if (entry.quorum && getZooKeeper()->exists(fs::path(zookeeper_path) / "quorum" / "failed_parts" / entry.new_part_name)) { LOG_DEBUG(log, "Skipping action for part {} because quorum for that part was failed.", entry.new_part_name); return true; /// NOTE Deletion from `virtual_parts` is not done, but it is only necessary for merge. @@ -1916,14 +1920,14 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) auto zookeeper = getZooKeeper(); - Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + Strings replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); Coordination::Requests ops; for (const auto & path_part : replicas) { Coordination::Stat stat; - String path = zookeeper_path + "/replicas/" + path_part + "/host"; + String path = fs::path(zookeeper_path) / "replicas" / path_part / "host"; zookeeper->get(path, &stat); ops.emplace_back(zkutil::makeCheckRequest(path, stat.version)); } @@ -1937,8 +1941,8 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) if (replica.empty()) { Coordination::Stat quorum_stat; - const String quorum_unparallel_path = zookeeper_path + "/quorum/status"; - const String quorum_parallel_path = zookeeper_path + "/quorum/parallel/" + entry.new_part_name; + const String quorum_unparallel_path = fs::path(zookeeper_path) / "quorum" / "status"; + const String quorum_parallel_path = fs::path(zookeeper_path) / "quorum" / "parallel" / entry.new_part_name; String quorum_str, quorum_path; ReplicatedMergeTreeQuorumEntry quorum_entry; @@ -1962,13 +1966,13 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) ErrorCodes::LOGICAL_ERROR); ops.emplace_back(zkutil::makeCreateRequest( - zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name, + fs::path(zookeeper_path) / "quorum" / "failed_parts" / entry.new_part_name, "", zkutil::CreateMode::Persistent)); /// Deleting from `blocks`. - if (!entry.block_id.empty() && zookeeper->exists(zookeeper_path + "/blocks/" + entry.block_id)) - ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/blocks/" + entry.block_id, -1)); + if (!entry.block_id.empty() && zookeeper->exists(fs::path(zookeeper_path) / "blocks" / entry.block_id)) + ops.emplace_back(zkutil::makeRemoveRequest(fs::path(zookeeper_path) / "blocks" / entry.block_id, -1)); Coordination::Responses responses; auto code = zookeeper->tryMulti(ops, responses); @@ -1981,7 +1985,8 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) } else if (code == Coordination::Error::ZBADVERSION || code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) { - LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part {} as failed. Code: {}", entry.new_part_name, Coordination::errorMessage(code)); + LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part {} as failed. Code: {}", + entry.new_part_name, Coordination::errorMessage(code)); } else throw Coordination::Exception(code); @@ -2003,7 +2008,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) try { String part_name = entry.actual_new_part_name.empty() ? entry.new_part_name : entry.actual_new_part_name; - if (!fetchPart(part_name, metadata_snapshot, zookeeper_path + "/replicas/" + replica, false, entry.quorum)) + if (!fetchPart(part_name, metadata_snapshot, fs::path(zookeeper_path) / "replicas" / replica, false, entry.quorum)) return false; } catch (Exception & e) @@ -2087,7 +2092,7 @@ bool StorageReplicatedMergeTree::executeFetchShared( try { - if (!fetchExistsPart(new_part_name, metadata_snapshot, zookeeper_path + "/replicas/" + source_replica, disk, path)) + if (!fetchExistsPart(new_part_name, metadata_snapshot, fs::path(zookeeper_path) / "replicas" / source_replica, disk, path)) return false; } catch (Exception & e) @@ -2406,8 +2411,8 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) } else if (!part_desc->replica.empty()) { - String source_replica_path = zookeeper_path + "/replicas/" + part_desc->replica; - ReplicatedMergeTreeAddress address(getZooKeeper()->get(source_replica_path + "/host")); + String source_replica_path = fs::path(zookeeper_path) / "replicas" / part_desc->replica; + ReplicatedMergeTreeAddress address(getZooKeeper()->get(fs::path(source_replica_path) / "host")); auto timeouts = getFetchPartHTTPTimeouts(getContext()); auto credentials = getContext()->getInterserverCredentials(); @@ -2486,18 +2491,18 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coordination::Stat source_is_lost_stat, zkutil::ZooKeeperPtr & zookeeper) { - String source_path = zookeeper_path + "/replicas/" + source_replica; + String source_path = fs::path(zookeeper_path) / "replicas" / source_replica; /** TODO: it will be deleted! (It is only to support old version of CH server). * In current code, the replica is created in single transaction. * If the reference/master replica is not yet fully created, let's wait. */ - while (!zookeeper->exists(source_path + "/columns")) + while (!zookeeper->exists(fs::path(source_path) / "columns")) { LOG_INFO(log, "Waiting for replica {} to be fully created", source_path); zkutil::EventPtr event = std::make_shared(); - if (zookeeper->exists(source_path + "/columns", nullptr, event)) + if (zookeeper->exists(fs::path(source_path) / "columns", nullptr, event)) { LOG_WARNING(log, "Oops, a watch has leaked"); break; @@ -2514,29 +2519,29 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo while (true) { Coordination::Stat log_pointer_stat; - String raw_log_pointer = zookeeper->get(source_path + "/log_pointer", &log_pointer_stat); + String raw_log_pointer = zookeeper->get(fs::path(source_path) / "log_pointer", &log_pointer_stat); Coordination::Requests ops; - ops.push_back(zkutil::makeSetRequest(replica_path + "/log_pointer", raw_log_pointer, -1)); + ops.push_back(zkutil::makeSetRequest(fs::path(replica_path) / "log_pointer", raw_log_pointer, -1)); /// For support old versions CH. if (source_is_lost_stat.version == -1) { /// We check that it was not suddenly upgraded to new version. /// Otherwise it can be upgraded and instantly become lost, but we cannot notice that. - ops.push_back(zkutil::makeCreateRequest(source_path + "/is_lost", "0", zkutil::CreateMode::Persistent)); - ops.push_back(zkutil::makeRemoveRequest(source_path + "/is_lost", -1)); + ops.push_back(zkutil::makeCreateRequest(fs::path(source_path) / "is_lost", "0", zkutil::CreateMode::Persistent)); + ops.push_back(zkutil::makeRemoveRequest(fs::path(source_path) / "is_lost", -1)); } else /// The replica we clone should not suddenly become lost. - ops.push_back(zkutil::makeCheckRequest(source_path + "/is_lost", source_is_lost_stat.version)); + ops.push_back(zkutil::makeCheckRequest(fs::path(source_path) / "is_lost", source_is_lost_stat.version)); Coordination::Responses responses; /// Let's remember the queue of the reference/master replica. - source_queue_names = zookeeper->getChildren(source_path + "/queue"); + source_queue_names = zookeeper->getChildren(fs::path(source_path) / "queue"); /// Check that our log pointer didn't changed while we read queue entries - ops.push_back(zkutil::makeCheckRequest(source_path + "/log_pointer", log_pointer_stat.version)); + ops.push_back(zkutil::makeCheckRequest(fs::path(source_path) / "log_pointer", log_pointer_stat.version)); auto rc = zookeeper->tryMulti(ops, responses); @@ -2576,19 +2581,19 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo for (const String & entry_name : source_queue_names) { String entry; - if (!zookeeper->tryGet(source_path + "/queue/" + entry_name, entry)) + if (!zookeeper->tryGet(fs::path(source_path) / "queue" / entry_name, entry)) continue; source_queue.push_back(entry); } /// Add to the queue jobs to receive all the active parts that the reference/master replica has. - Strings source_replica_parts = zookeeper->getChildren(source_path + "/parts"); + Strings source_replica_parts = zookeeper->getChildren(fs::path(source_path) / "parts"); ActiveDataPartSet active_parts_set(format_version, source_replica_parts); Strings active_parts = active_parts_set.getParts(); /// Remove local parts if source replica does not have them, because such parts will never be fetched by other replicas. - Strings local_parts_in_zk = zookeeper->getChildren(replica_path + "/parts"); + Strings local_parts_in_zk = zookeeper->getChildren(fs::path(replica_path) / "parts"); Strings parts_to_remove_from_zk; for (const auto & part : local_parts_in_zk) { @@ -2632,7 +2637,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo log_entry.new_part_name = name; log_entry.create_time = tryGetPartCreateTime(zookeeper, source_path, name); - zookeeper->create(replica_path + "/queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential); + zookeeper->create(fs::path(replica_path) / "queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential); } LOG_DEBUG(log, "Queued {} parts to be fetched", active_parts.size()); @@ -2640,7 +2645,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo /// Add content of the reference/master replica queue to the queue. for (const String & entry : source_queue) { - zookeeper->create(replica_path + "/queue/queue-", entry, zkutil::CreateMode::PersistentSequential); + zookeeper->create(fs::path(replica_path) / "queue/queue-", entry, zkutil::CreateMode::PersistentSequential); } LOG_DEBUG(log, "Copied {} queue entries", source_queue.size()); @@ -2652,7 +2657,7 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke Coordination::Stat is_lost_stat; bool is_new_replica = true; String res; - if (zookeeper->tryGet(replica_path + "/is_lost", res, &is_lost_stat)) + if (zookeeper->tryGet(fs::path(replica_path) / "is_lost", res, &is_lost_stat)) { if (res == "0") return; @@ -2664,14 +2669,14 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke /// Replica was created by old version of CH, so me must create "/is_lost". /// Note that in old version of CH there was no "lost" replicas possible. /// TODO is_lost node should always exist since v18.12, maybe we can replace `tryGet` with `get` and remove old code? - zookeeper->create(replica_path + "/is_lost", "0", zkutil::CreateMode::Persistent); + zookeeper->create(fs::path(replica_path) / "is_lost", "0", zkutil::CreateMode::Persistent); return; } /// is_lost is "1": it means that we are in repair mode. /// Try choose source replica to clone. /// Source replica must not be lost and should have minimal queue size and maximal log pointer. - Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + Strings replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); std::vector futures; for (const String & source_replica_name : replicas) { @@ -2679,20 +2684,20 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke if (source_replica_name == replica_name) continue; - String source_replica_path = zookeeper_path + "/replicas/" + source_replica_name; + String source_replica_path = fs::path(zookeeper_path) / "replicas" / source_replica_name; /// Obviously the following get operations are not atomic, but it's ok to choose good enough replica, not the best one. /// NOTE: We may count some entries twice if log_pointer is moved. - futures.emplace_back(zookeeper->asyncTryGet(source_replica_path + "/is_lost")); - futures.emplace_back(zookeeper->asyncTryGet(source_replica_path + "/log_pointer")); - futures.emplace_back(zookeeper->asyncTryGet(source_replica_path + "/queue")); + futures.emplace_back(zookeeper->asyncTryGet(fs::path(source_replica_path) / "is_lost")); + futures.emplace_back(zookeeper->asyncTryGet(fs::path(source_replica_path) / "log_pointer")); + futures.emplace_back(zookeeper->asyncTryGet(fs::path(source_replica_path) / "queue")); } /// Wait for results before getting log entries for (auto & future : futures) future.wait(); - Strings log_entries = zookeeper->getChildren(zookeeper_path + "/log"); + Strings log_entries = zookeeper->getChildren(fs::path(zookeeper_path) / "log"); size_t max_log_entry = 0; if (!log_entries.empty()) { @@ -2763,14 +2768,14 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke LOG_WARNING(log, "Will mimic {}", source_replica); /// Clear obsolete queue that we no longer need. - zookeeper->removeChildren(replica_path + "/queue"); + zookeeper->removeChildren(fs::path(replica_path) / "queue"); /// Will do repair from the selected replica. cloneReplica(source_replica, source_is_lost_stat, zookeeper); /// If repair fails to whatever reason, the exception is thrown, is_lost will remain "1" and the replica will be repaired later. /// If replica is repaired successfully, we remove is_lost flag. - zookeeper->set(replica_path + "/is_lost", "0"); + zookeeper->set(fs::path(replica_path) / "is_lost", "0"); } @@ -3097,7 +3102,7 @@ StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::c std::vector> exists_futures; exists_futures.reserve(parts.size()); for (const auto & part : parts) - exists_futures.emplace_back(zookeeper->asyncExists(replica_path + "/parts/" + part->name)); + exists_futures.emplace_back(zookeeper->asyncExists(fs::path(replica_path) / "parts" / part->name)); bool all_in_zk = true; for (size_t i = 0; i < parts.size(); ++i) @@ -3137,11 +3142,11 @@ StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::c Coordination::Responses responses; ops.emplace_back(zkutil::makeCreateRequest( - zookeeper_path + "/log/log-", entry.toString(), + fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); ops.emplace_back(zkutil::makeSetRequest( - zookeeper_path + "/log", "", log_version)); /// Check and update version. + fs::path(zookeeper_path) / "log", "", log_version)); /// Check and update version. Coordination::Error code = zookeeper->tryMulti(ops, responses); @@ -3177,7 +3182,7 @@ StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::c auto zookeeper = getZooKeeper(); /// If there is no information about part in ZK, we will not mutate it. - if (!zookeeper->exists(replica_path + "/parts/" + part.name)) + if (!zookeeper->exists(fs::path(replica_path) / "parts" / part.name)) { if (part.modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time(nullptr)) { @@ -3207,11 +3212,11 @@ StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::c Coordination::Responses responses; ops.emplace_back(zkutil::makeCreateRequest( - zookeeper_path + "/log/log-", entry.toString(), + fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); ops.emplace_back(zkutil::makeSetRequest( - zookeeper_path + "/log", "", log_version)); /// Check and update version. + fs::path(zookeeper_path) / "log", "", log_version)); /// Check and update version. Coordination::Error code = zookeeper->tryMulti(ops, responses); @@ -3232,12 +3237,12 @@ StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::c void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_name, Coordination::Requests & ops, bool has_children) { - String part_path = replica_path + "/parts/" + part_name; + String part_path = fs::path(replica_path) / "parts" / part_name; if (has_children) { - ops.emplace_back(zkutil::makeRemoveRequest(part_path + "/checksums", -1)); - ops.emplace_back(zkutil::makeRemoveRequest(part_path + "/columns", -1)); + ops.emplace_back(zkutil::makeRemoveRequest(fs::path(part_path) / "checksums", -1)); + ops.emplace_back(zkutil::makeRemoveRequest(fs::path(part_path) / "columns", -1)); } ops.emplace_back(zkutil::makeRemoveRequest(part_path, -1)); } @@ -3245,7 +3250,7 @@ void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_nam void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_name) { auto zookeeper = getZooKeeper(); - String part_path = replica_path + "/parts/" + part_name; + String part_path = fs::path(replica_path) / "parts" / part_name; Coordination::Stat stat; /// Part doesn't exist, nothing to remove @@ -3262,7 +3267,7 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n { auto zookeeper = getZooKeeper(); - String part_path = replica_path + "/parts/" + part_name; + String part_path = fs::path(replica_path) / "parts" / part_name; Coordination::Requests ops; @@ -3281,7 +3286,7 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n log_entry->new_part_name = part_name; ops.emplace_back(zkutil::makeCreateRequest( - replica_path + "/queue/queue-", log_entry->toString(), + fs::path(replica_path) / "queue/queue-", log_entry->toString(), zkutil::CreateMode::PersistentSequential)); auto results = zookeeper->multi(ops); @@ -3306,7 +3311,7 @@ void StorageReplicatedMergeTree::enterLeaderElection() { leader_election = std::make_shared( getContext()->getSchedulePool(), - zookeeper_path + "/leader_election", + fs::path(zookeeper_path) / "leader_election", *current_zookeeper, /// current_zookeeper lives for the lifetime of leader_election, /// since before changing `current_zookeeper`, `leader_election` object is destroyed in `partialShutdown` method. callback, @@ -3361,13 +3366,13 @@ ConnectionTimeouts StorageReplicatedMergeTree::getFetchPartHTTPTimeouts(ContextP bool StorageReplicatedMergeTree::checkReplicaHavePart(const String & replica, const String & part_name) { auto zookeeper = getZooKeeper(); - return zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name); + return zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "parts" / part_name); } String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_name, bool active) { auto zookeeper = getZooKeeper(); - Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + Strings replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); /// Select replicas in uniformly random order. std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); @@ -3383,7 +3388,7 @@ String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_nam LOG_TRACE(log, "Candidate replica: {}", replica); if (checkReplicaHavePart(replica, part_name) && - (!active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))) + (!active || zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "is_active"))) return replica; /// Obviously, replica could become inactive or even vanish after return from this method. @@ -3395,7 +3400,7 @@ String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_nam String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(LogEntry & entry, bool active) { auto zookeeper = getZooKeeper(); - Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + Strings replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); /// Select replicas in uniformly random order. std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); @@ -3405,11 +3410,11 @@ String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(LogEntry & entr if (replica == replica_name) continue; - if (active && !zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) + if (active && !zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "is_active")) continue; String largest_part_found; - Strings parts = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/parts"); + Strings parts = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas" / replica / "parts"); for (const String & part_on_replica : parts) { if (part_on_replica == entry.new_part_name @@ -3450,7 +3455,7 @@ String StorageReplicatedMergeTree::findReplicaHavingCoveringPart( const String & part_name, bool active, String & found_part_name) { auto zookeeper = getZooKeeper(); - Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + Strings replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); /// Select replicas in uniformly random order. std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); @@ -3463,10 +3468,10 @@ String StorageReplicatedMergeTree::findReplicaHavingCoveringPart( if (replica == replica_name) continue; - if (active && !zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) + if (active && !zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "is_active")) continue; - Strings parts = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/parts"); + Strings parts = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas" / replica / "parts"); for (const String & part_on_replica : parts) { if (part_on_replica == part_name @@ -3494,11 +3499,11 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name, bool is_ auto zookeeper = getZooKeeper(); /// Information on which replicas a part has been added, if the quorum has not yet been reached. - String quorum_status_path = zookeeper_path + "/quorum/status"; + String quorum_status_path = fs::path(zookeeper_path) / "quorum" / "status"; if (is_parallel) - quorum_status_path = zookeeper_path + "/quorum/parallel/" + part_name; + quorum_status_path = fs::path(zookeeper_path) / "quorum" / "parallel" / part_name; /// The name of the previous part for which the quorum was reached. - const String quorum_last_part_path = zookeeper_path + "/quorum/last_part"; + const String quorum_last_part_path = fs::path(zookeeper_path) / "quorum" / "last_part"; String value; Coordination::Stat stat; @@ -3600,7 +3605,7 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id) auto zookeeper = getZooKeeper(); /// The name of the previous part for which the quorum was reached. - const String quorum_last_part_path = zookeeper_path + "/quorum/last_part"; + const String quorum_last_part_path = fs::path(zookeeper_path) / "quorum" / "last_part"; /// Delete information from "last_part" node. @@ -3650,7 +3655,7 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id) bool StorageReplicatedMergeTree::partIsInsertingWithParallelQuorum(const MergeTreePartInfo & part_info) const { auto zookeeper = getZooKeeper(); - return zookeeper->exists(zookeeper_path + "/quorum/parallel/" + part_info.getPartName()); + return zookeeper->exists(fs::path(zookeeper_path) / "quorum" / "parallel" / part_info.getPartName()); } @@ -3658,7 +3663,7 @@ bool StorageReplicatedMergeTree::partIsLastQuorumPart(const MergeTreePartInfo & { auto zookeeper = getZooKeeper(); - const String parts_with_quorum_path = zookeeper_path + "/quorum/last_part"; + const String parts_with_quorum_path = fs::path(zookeeper_path) / "quorum" / "last_part"; String parts_with_quorum_str = zookeeper->get(parts_with_quorum_path); @@ -3742,13 +3747,13 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora source_part_checksums.computeTotalChecksums(source_part->checksums); MinimalisticDataPartChecksums desired_checksums; - String part_path = source_replica_path + "/parts/" + part_name; + String part_path = fs::path(source_replica_path) / "parts" / part_name; String part_znode = zookeeper->get(part_path); if (!part_znode.empty()) desired_checksums = ReplicatedMergeTreePartHeader::fromString(part_znode).getChecksums(); else { - String desired_checksums_str = zookeeper->get(part_path + "/checksums"); + String desired_checksums_str = zookeeper->get(fs::path(part_path) / "checksums"); desired_checksums = MinimalisticDataPartChecksums::deserializeFrom(desired_checksums_str); } @@ -3777,7 +3782,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora } else { - address.fromString(zookeeper->get(source_replica_path + "/host")); + address.fromString(zookeeper->get(fs::path(source_replica_path) / "host")); timeouts = getFetchPartHTTPTimeouts(getContext()); credentials = getContext()->getInterserverCredentials(); @@ -3824,16 +3829,16 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora if (quorum) { /// Check if this quorum insert is parallel or not - if (zookeeper->exists(zookeeper_path + "/quorum/parallel/" + part_name)) + if (zookeeper->exists(fs::path(zookeeper_path) / "quorum" / "parallel" / part_name)) updateQuorum(part_name, true); - else if (zookeeper->exists(zookeeper_path + "/quorum/status")) + else if (zookeeper->exists(fs::path(zookeeper_path) / "quorum" / "status")) updateQuorum(part_name, false); } /// merged parts that are still inserted with quorum. if it only contains one block, it hasn't been merged before if (part_info.level != 0 || part_info.mutation != 0) { - Strings quorum_parts = zookeeper->getChildren(zookeeper_path + "/quorum/parallel"); + Strings quorum_parts = zookeeper->getChildren(fs::path(zookeeper_path) / "quorum" / "parallel"); for (const String & quorum_part : quorum_parts) { auto quorum_part_info = MergeTreePartInfo::fromPartName(quorum_part, format_version); @@ -3856,7 +3861,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora { // The fetched part is valuable and should not be cleaned like a temp part. part->is_temp = false; - part->renameTo("detached/" + part_name, true); + part->renameTo(fs::path("detached") / part_name, true); } } catch (const Exception & e) @@ -3934,7 +3939,7 @@ bool StorageReplicatedMergeTree::fetchExistsPart(const String & part_name, const std::function get_part; - ReplicatedMergeTreeAddress address(zookeeper->get(source_replica_path + "/host")); + ReplicatedMergeTreeAddress address(zookeeper->get(fs::path(source_replica_path) / "host")); auto timeouts = ConnectionTimeouts::getHTTPTimeouts(getContext()); auto credentials = getContext()->getInterserverCredentials(); String interserver_scheme = getContext()->getInterserverScheme(); @@ -4092,7 +4097,7 @@ ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock StorageReplicatedMerg auto zookeeper = getZooKeeper(); - const String quorum_status_path = zookeeper_path + "/quorum/status"; + const String quorum_status_path = fs::path(zookeeper_path) / "quorum" / "status"; String value; Coordination::Stat stat; @@ -4108,7 +4113,7 @@ ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock StorageReplicatedMerg } String added_parts_str; - if (zookeeper->tryGet(zookeeper_path + "/quorum/last_part", added_parts_str)) + if (zookeeper->tryGet(fs::path(zookeeper_path) / "quorum" / "last_part", added_parts_str)) { if (!added_parts_str.empty()) { @@ -4423,8 +4428,8 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer /// If metadata nodes have changed, we will update table structure locally. Coordination::Requests requests; - requests.emplace_back(zkutil::makeSetRequest(replica_path + "/columns", entry.columns_str, -1)); - requests.emplace_back(zkutil::makeSetRequest(replica_path + "/metadata", entry.metadata_str, -1)); + requests.emplace_back(zkutil::makeSetRequest(fs::path(replica_path) / "columns", entry.columns_str, -1)); + requests.emplace_back(zkutil::makeSetRequest(fs::path(replica_path) / "metadata", entry.metadata_str, -1)); zookeeper->multi(requests); @@ -4440,7 +4445,7 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer } /// This transaction may not happen, but it's OK, because on the next retry we will eventually create/update this node - zookeeper->createOrUpdate(replica_path + "/metadata_version", std::to_string(metadata_version), zkutil::CreateMode::Persistent); + zookeeper->createOrUpdate(fs::path(replica_path) / "metadata_version", std::to_string(metadata_version), zkutil::CreateMode::Persistent); return true; } @@ -4486,7 +4491,7 @@ PartitionBlockNumbersHolder StorageReplicatedMergeTree::allocateBlockNumbersInAf { /// TODO: Implement optimal block number aqcuisition algorithm in multiple (but not all) partitions EphemeralLocksInAllPartitions lock_holder( - zookeeper_path + "/block_numbers", "block-", zookeeper_path + "/temp", *zookeeper); + fs::path(zookeeper_path) / "block_numbers", "block-", fs::path(zookeeper_path) / "temp", *zookeeper); PartitionBlockNumbersHolder::BlockNumbersType block_numbers; for (const auto & lock : lock_holder.getLocks()) @@ -4582,10 +4587,10 @@ void StorageReplicatedMergeTree::alter( size_t mutation_path_idx = std::numeric_limits::max(); String new_metadata_str = future_metadata_in_zk.toString(); - ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/metadata", new_metadata_str, metadata_version)); + ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "metadata", new_metadata_str, metadata_version)); String new_columns_str = future_metadata.columns.toString(); - ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/columns", new_columns_str, -1)); + ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "columns", new_columns_str, -1)); if (ast_to_str(current_metadata->settings_changes) != ast_to_str(future_metadata.settings_changes)) { @@ -4613,12 +4618,12 @@ void StorageReplicatedMergeTree::alter( alter_path_idx = ops.size(); ops.emplace_back(zkutil::makeCreateRequest( - zookeeper_path + "/log/log-", alter_entry->toString(), zkutil::CreateMode::PersistentSequential)); + fs::path(zookeeper_path) / "log/log-", alter_entry->toString(), zkutil::CreateMode::PersistentSequential)); PartitionBlockNumbersHolder partition_block_numbers_holder; if (alter_entry->have_mutation) { - const String mutations_path(zookeeper_path + "/mutations"); + const String mutations_path(fs::path(zookeeper_path) / "mutations"); ReplicatedMergeTreeMutationEntry mutation_entry; mutation_entry.alter_version = new_metadata_version; @@ -4637,7 +4642,7 @@ void StorageReplicatedMergeTree::alter( ops.emplace_back(zkutil::makeSetRequest(mutations_path, String(), mutations_stat.version)); mutation_path_idx = ops.size(); ops.emplace_back( - zkutil::makeCreateRequest(mutations_path + "/", mutation_entry.toString(), zkutil::CreateMode::PersistentSequential)); + zkutil::makeCreateRequest(fs::path(mutations_path) / "", mutation_entry.toString(), zkutil::CreateMode::PersistentSequential)); } if (auto txn = query_context->getZooKeeperMetadataTransaction()) @@ -4645,7 +4650,7 @@ void StorageReplicatedMergeTree::alter( txn->moveOpsTo(ops); /// NOTE: IDatabase::alterTable(...) is called when executing ALTER_METADATA queue entry without query context, /// so we have to update metadata of DatabaseReplicated here. - String metadata_zk_path = txn->getDatabaseZooKeeperPath() + "/metadata/" + escapeForFileName(table_id.table_name); + String metadata_zk_path = fs::path(txn->getDatabaseZooKeeperPath()) / "metadata" / escapeForFileName(table_id.table_name); auto ast = DatabaseCatalog::instance().getDatabase(table_id.database_name)->getCreateTableQuery(table_id.table_name, query_context); applyMetadataChangesToCreateQuery(ast, future_metadata); ops.emplace_back(zkutil::makeSetRequest(metadata_zk_path, getObjectDefinitionFromCreateQuery(ast), -1)); @@ -4830,7 +4835,7 @@ void StorageReplicatedMergeTree::truncate( zkutil::ZooKeeperPtr zookeeper = getZooKeeper(); - Strings partitions = zookeeper->getChildren(zookeeper_path + "/block_numbers"); + Strings partitions = zookeeper->getChildren(fs::path(zookeeper_path) / "block_numbers"); for (String & partition_id : partitions) { @@ -4905,7 +4910,7 @@ void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, c try { auto zookeeper = getZooKeeper(); - zookeeper->set(replica_path + "/host", getReplicatedMergeTreeAddress().toString()); + zookeeper->set(fs::path(replica_path) / "host", getReplicatedMergeTreeAddress().toString()); } catch (Coordination::Exception & e) { @@ -4949,8 +4954,8 @@ StorageReplicatedMergeTree::allocateBlockNumber( deduplication_check_ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_block_id_path, -1)); } - String block_numbers_path = zookeeper_path + "/block_numbers"; - String partition_path = block_numbers_path + "/" + partition_id; + String block_numbers_path = fs::path(zookeeper_path) / "block_numbers"; + String partition_path = fs::path(block_numbers_path) / partition_id; if (!existsNodeCached(partition_path)) { @@ -4972,7 +4977,7 @@ StorageReplicatedMergeTree::allocateBlockNumber( try { lock = EphemeralLockInZooKeeper( - partition_path + "/block-", zookeeper_path + "/temp", *zookeeper, &deduplication_check_ops); + partition_path + "/block-", fs::path(zookeeper_path) / "temp", *zookeeper, &deduplication_check_ops); } catch (const zkutil::KeeperMultiException & e) { @@ -4996,11 +5001,11 @@ Strings StorageReplicatedMergeTree::waitForAllTableReplicasToProcessLogEntry( LOG_DEBUG(log, "Waiting for all replicas to process {}", entry.znode_name); auto zookeeper = getZooKeeper(); - Strings replicas = zookeeper->getChildren(table_zookeeper_path + "/replicas"); + Strings replicas = zookeeper->getChildren(fs::path(table_zookeeper_path) / "replicas"); Strings unwaited; for (const String & replica : replicas) { - if (wait_for_non_active || zookeeper->exists(table_zookeeper_path + "/replicas/" + replica + "/is_active")) + if (wait_for_non_active || zookeeper->exists(fs::path(table_zookeeper_path) / "replicas" / replica / "is_active")) { if (!waitForTableReplicaToProcessLogEntry(table_zookeeper_path, replica, entry, wait_for_non_active)) unwaited.push_back(replica); @@ -5050,7 +5055,7 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry( const auto & stop_waiting = [&]() { bool stop_waiting_itself = waiting_itself && (partial_shutdown_called || is_dropped); - bool stop_waiting_non_active = !wait_for_non_active && !getZooKeeper()->exists(table_zookeeper_path + "/replicas/" + replica + "/is_active"); + bool stop_waiting_non_active = !wait_for_non_active && !getZooKeeper()->exists(fs::path(table_zookeeper_path) / "replicas" / replica / "is_active"); return stop_waiting_itself || stop_waiting_non_active; }; @@ -5072,7 +5077,7 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry( { zkutil::EventPtr event = std::make_shared(); - String log_pointer = getZooKeeper()->get(table_zookeeper_path + "/replicas/" + replica + "/log_pointer", nullptr, event); + String log_pointer = getZooKeeper()->get(fs::path(table_zookeeper_path) / "replicas" / replica / "log_pointer", nullptr, event); if (!log_pointer.empty() && parse(log_pointer) > log_index) break; @@ -5089,9 +5094,9 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry( * looking for a node with the same content. And if we do not find it - then the replica has already taken this entry in its queue. */ - String log_pointer = getZooKeeper()->get(table_zookeeper_path + "/replicas/" + replica + "/log_pointer"); + String log_pointer = getZooKeeper()->get(fs::path(table_zookeeper_path) / "replicas" / replica / "log_pointer"); - Strings log_entries = getZooKeeper()->getChildren(table_zookeeper_path + "/log"); + Strings log_entries = getZooKeeper()->getChildren(fs::path(table_zookeeper_path) / "log"); UInt64 log_index = 0; bool found = false; @@ -5103,7 +5108,7 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry( continue; String log_entry_str; - bool exists = getZooKeeper()->tryGet(table_zookeeper_path + "/log/" + log_entry_name, log_entry_str); + bool exists = getZooKeeper()->tryGet(fs::path(table_zookeeper_path) / "log" / log_entry_name, log_entry_str); if (exists && entry_str == log_entry_str) { found = true; @@ -5121,7 +5126,7 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry( { zkutil::EventPtr event = std::make_shared(); - String log_pointer_new = getZooKeeper()->get(table_zookeeper_path + "/replicas/" + replica + "/log_pointer", nullptr, event); + String log_pointer_new = getZooKeeper()->get(fs::path(table_zookeeper_path) / "replicas" / replica / "log_pointer", nullptr, event); if (!log_pointer_new.empty() && parse(log_pointer_new) > log_index) break; @@ -5146,13 +5151,13 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry( * Therefore, we search by comparing the content. */ - Strings queue_entries = getZooKeeper()->getChildren(table_zookeeper_path + "/replicas/" + replica + "/queue"); + Strings queue_entries = getZooKeeper()->getChildren(fs::path(table_zookeeper_path) / "replicas" / replica / "queue"); String queue_entry_to_wait_for; for (const String & entry_name : queue_entries) { String queue_entry_str; - bool exists = getZooKeeper()->tryGet(table_zookeeper_path + "/replicas/" + replica + "/queue/" + entry_name, queue_entry_str); + bool exists = getZooKeeper()->tryGet(fs::path(table_zookeeper_path) / "replicas" / replica / "queue" / entry_name, queue_entry_str); if (exists && queue_entry_str == entry_str) { queue_entry_to_wait_for = entry_name; @@ -5170,7 +5175,7 @@ bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry( LOG_DEBUG(log, "Waiting for {} to disappear from {} queue", queue_entry_to_wait_for, replica); /// Third - wait until the entry disappears from the replica queue or replica become inactive. - String path_to_wait_on = table_zookeeper_path + "/replicas/" + replica + "/queue/" + queue_entry_to_wait_for; + String path_to_wait_on = fs::path(table_zookeeper_path) / "replicas" / replica / "queue" / queue_entry_to_wait_for; return getZooKeeper()->waitForDisappear(path_to_wait_on, stop_waiting); } @@ -5212,7 +5217,7 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields) { try { - auto log_entries = zookeeper->getChildren(zookeeper_path + "/log"); + auto log_entries = zookeeper->getChildren(fs::path(zookeeper_path) / "log"); if (!log_entries.empty()) { @@ -5220,14 +5225,14 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields) res.log_max_index = parse(last_log_entry.substr(strlen("log-"))); } - String log_pointer_str = zookeeper->get(replica_path + "/log_pointer"); + String log_pointer_str = zookeeper->get(fs::path(replica_path) / "log_pointer"); res.log_pointer = log_pointer_str.empty() ? 0 : parse(log_pointer_str); - auto all_replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + auto all_replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); res.total_replicas = all_replicas.size(); for (const String & replica : all_replicas) - if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) + if (zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "is_active")) ++res.active_replicas; } catch (const Coordination::Exception &) @@ -5305,7 +5310,7 @@ void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, t time_t max_replicas_unprocessed_insert_time = 0; bool have_replica_with_nothing_unprocessed = false; - Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + Strings replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); for (const auto & replica : replicas) { @@ -5313,11 +5318,11 @@ void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, t continue; /// Skip dead replicas. - if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) + if (!zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "is_active")) continue; String value; - if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/min_unprocessed_insert_time", value)) + if (!zookeeper->tryGet(fs::path(zookeeper_path) / "replicas" / replica / "min_unprocessed_insert_time", value)) continue; time_t replica_time = value.empty() ? 0 : parse(value); @@ -5424,13 +5429,13 @@ void StorageReplicatedMergeTree::fetchPartition( { /// List of replicas of source shard. - replicas = zookeeper->getChildren(from + "/replicas"); + replicas = zookeeper->getChildren(fs::path(from) / "replicas"); /// Leave only active replicas. active_replicas.reserve(replicas.size()); for (const String & replica : replicas) - if (zookeeper->exists(from + "/replicas/" + replica + "/is_active")) + if (zookeeper->exists(fs::path(from) / "replicas" / replica / "is_active")) active_replicas.push_back(replica); if (active_replicas.empty()) @@ -5447,13 +5452,13 @@ void StorageReplicatedMergeTree::fetchPartition( for (const String & replica : active_replicas) { - String current_replica_path = from + "/replicas/" + replica; + String current_replica_path = fs::path(from) / "replicas" / replica; - String log_pointer_str = zookeeper->get(current_replica_path + "/log_pointer"); + String log_pointer_str = zookeeper->get(fs::path(current_replica_path) / "log_pointer"); Int64 log_pointer = log_pointer_str.empty() ? 0 : parse(log_pointer_str); Coordination::Stat stat; - zookeeper->get(current_replica_path + "/queue", &stat); + zookeeper->get(fs::path(current_replica_path) / "queue", &stat); size_t queue_size = stat.numChildren; if (log_pointer > max_log_pointer @@ -5471,7 +5476,7 @@ void StorageReplicatedMergeTree::fetchPartition( LOG_INFO(log, "Found {} replicas, {} of them are active. Selected {} to fetch from.", replicas.size(), active_replicas.size(), best_replica); - String best_replica_path = from + "/replicas/" + best_replica; + String best_replica_path = fs::path(from) / "replicas" / best_replica; /// Let's find out which parts are on the best replica. @@ -5490,7 +5495,7 @@ void StorageReplicatedMergeTree::fetchPartition( if (try_no >= query_context->getSettings().max_fetch_partition_retries_count) throw Exception("Too many retries to fetch parts from " + best_replica_path, ErrorCodes::TOO_MANY_RETRIES_TO_FETCH_PARTS); - Strings parts = zookeeper->getChildren(best_replica_path + "/parts"); + Strings parts = zookeeper->getChildren(fs::path(best_replica_path) / "parts"); ActiveDataPartSet active_parts_set(format_version, parts); Strings parts_to_fetch; @@ -5611,7 +5616,7 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, Conte mutation_entry.source_replica = replica_name; mutation_entry.commands = commands; - const String mutations_path = zookeeper_path + "/mutations"; + const String mutations_path = fs::path(zookeeper_path) / "mutations"; const auto zookeeper = getZooKeeper(); /// Update the mutations_path node when creating the mutation and check its version to ensure that @@ -5634,7 +5639,7 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, Conte Coordination::Requests requests; requests.emplace_back(zkutil::makeSetRequest(mutations_path, String(), mutations_stat.version)); requests.emplace_back(zkutil::makeCreateRequest( - mutations_path + "/", mutation_entry.toString(), zkutil::CreateMode::PersistentSequential)); + fs::path(mutations_path) / "", mutation_entry.toString(), zkutil::CreateMode::PersistentSequential)); if (auto txn = query_context->getZooKeeperMetadataTransaction()) txn->moveOpsTo(requests); @@ -5673,7 +5678,7 @@ void StorageReplicatedMergeTree::waitMutation(const String & znode_name, size_t auto zookeeper = getZooKeeper(); Strings replicas; if (mutations_sync == 2) /// wait for all replicas - replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas"); else if (mutations_sync == 1) /// just wait for ourself replicas.push_back(replica_name); @@ -5868,7 +5873,7 @@ bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(const St exists_futures.reserve(part_names.size()); for (const String & part_name : part_names) { - String part_path = replica_path + "/parts/" + part_name; + String part_path = fs::path(replica_path) / "parts" / part_name; exists_futures.emplace_back(zookeeper->asyncExists(part_path)); } @@ -5931,7 +5936,7 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper( /// if zk session will be dropped for (const String & part_name : part_names) { - String part_path = replica_path + "/parts/" + part_name; + String part_path = fs::path(replica_path) / "parts" / part_name; exists_futures.emplace_back(zookeeper->asyncExists(part_path)); } @@ -5990,7 +5995,7 @@ void StorageReplicatedMergeTree::getClearBlocksInPartitionOps( Coordination::Requests & ops, zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num) { Strings blocks; - if (Coordination::Error::ZOK != zookeeper.tryGetChildren(zookeeper_path + "/blocks", blocks)) + if (Coordination::Error::ZOK != zookeeper.tryGetChildren(fs::path(zookeeper_path) / "blocks", blocks)) throw Exception(zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE); String partition_prefix = partition_id + "_"; @@ -5999,7 +6004,7 @@ void StorageReplicatedMergeTree::getClearBlocksInPartitionOps( { if (startsWith(block_id, partition_prefix)) { - String path = zookeeper_path + "/blocks/" + block_id; + String path = fs::path(zookeeper_path) / "blocks" / block_id; get_futures.emplace_back(path, zookeeper.asyncTryGet(path)); } } @@ -6107,7 +6112,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( else LOG_INFO(log, "Trying to attach {} with hash_hex {}", src_part->name, hash_hex); - String block_id_path = replace ? "" : (zookeeper_path + "/blocks/" + partition_id + "_replace_from_" + hash_hex); + String block_id_path = replace ? "" : (fs::path(zookeeper_path) / "blocks" / (partition_id + "_replace_from_" + hash_hex)); auto lock = allocateBlockNumber(partition_id, zookeeper, block_id_path); if (!lock) @@ -6178,8 +6183,8 @@ void StorageReplicatedMergeTree::replacePartitionFrom( if (auto txn = query_context->getZooKeeperMetadataTransaction()) txn->moveOpsTo(ops); - ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); /// Just update version - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); + ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); /// Just update version + ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); Transaction transaction(*this); { @@ -6364,7 +6369,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta } } - ops.emplace_back(zkutil::makeCreateRequest(dest_table_storage->zookeeper_path + "/log/log-", + ops.emplace_back(zkutil::makeCreateRequest(fs::path(dest_table_storage->zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); { @@ -6413,8 +6418,8 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta Coordination::Requests ops_dest; ops_dest.emplace_back(zkutil::makeCreateRequest( - zookeeper_path + "/log/log-", entry_delete.toString(), zkutil::CreateMode::PersistentSequential)); - ops_dest.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); /// Just update version + fs::path(zookeeper_path) / "log/log-", entry_delete.toString(), zkutil::CreateMode::PersistentSequential)); + ops_dest.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); /// Just update version op_results = zookeeper->multi(ops_dest); @@ -6453,22 +6458,22 @@ void StorageReplicatedMergeTree::getCommitPartOps( if (storage_settings_ptr->use_minimalistic_part_header_in_zookeeper) { ops.emplace_back(zkutil::makeCreateRequest( - replica_path + "/parts/" + part->name, + fs::path(replica_path) / "parts" / part->name, ReplicatedMergeTreePartHeader::fromColumnsAndChecksums(part->getColumns(), part->checksums).toString(), zkutil::CreateMode::Persistent)); } else { ops.emplace_back(zkutil::makeCreateRequest( - replica_path + "/parts/" + part->name, + fs::path(replica_path) / "parts" / part->name, "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( - replica_path + "/parts/" + part->name + "/columns", + fs::path(replica_path) / "parts" / part->name / "columns", part->getColumns().toString(), zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( - replica_path + "/parts/" + part->name + "/checksums", + fs::path(replica_path) / "parts" / part->name / "checksums", getChecksumsForZooKeeper(part->checksums), zkutil::CreateMode::Persistent)); } @@ -6619,9 +6624,9 @@ bool StorageReplicatedMergeTree::dropPart( entry.detach = detach; entry.create_time = time(nullptr); - ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/log", merge_pred.getVersion())); /// Make sure no new events were added to the log. - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); - ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); /// Just update version. + ops.emplace_back(zkutil::makeCheckRequest(fs::path(zookeeper_path) / "log", merge_pred.getVersion())); /// Make sure no new events were added to the log. + ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); + ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); /// Just update version. Coordination::Responses responses; Coordination::Error rc = zookeeper->tryMulti(ops, responses); @@ -6676,8 +6681,8 @@ bool StorageReplicatedMergeTree::dropAllPartsInPartition( entry.create_time = time(nullptr); Coordination::Requests ops; - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); - ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); /// Just update version. + ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); + ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); /// Just update version. if (auto txn = query_context->getZooKeeperMetadataTransaction()) txn->moveOpsTo(ops); Coordination::Responses responses = zookeeper.multi(ops); @@ -6755,7 +6760,7 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part) String id = part.getUniqueId(); boost::replace_all(id, "/", "_"); - String zookeeper_node = zookeeper_path + "/zero_copy_s3/shared/" + part.name + "/" + id + "/" + replica_name; + String zookeeper_node = fs::path(zookeeper_path) / "zero_copy_s3" / "shared" / part.name / id / replica_name; LOG_TRACE(log, "Set zookeeper lock {}", zookeeper_node); @@ -6796,9 +6801,9 @@ bool StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & par String id = part.getUniqueId(); boost::replace_all(id, "/", "_"); - String zookeeper_part_node = zookeeper_path + "/zero_copy_s3/shared/" + part.name; - String zookeeper_part_uniq_node = zookeeper_part_node + "/" + id; - String zookeeper_node = zookeeper_part_uniq_node + "/" + replica_name; + String zookeeper_part_node = fs::path(zookeeper_path) / "zero_copy_s3" / "shared" / part.name; + String zookeeper_part_uniq_node = fs::path(zookeeper_part_node) / id; + String zookeeper_node = fs::path(zookeeper_part_uniq_node) / replica_name; LOG_TRACE(log, "Remove zookeeper lock {}", zookeeper_node); @@ -6857,7 +6862,7 @@ String StorageReplicatedMergeTree::getSharedDataReplica( if (!zookeeper) return best_replica; - String zookeeper_part_node = zookeeper_path + "/zero_copy_s3/shared/" + part.name; + String zookeeper_part_node = fs::path(zookeeper_path) / "zero_copy_s3" / "shared" / part.name; Strings ids; zookeeper->tryGetChildren(zookeeper_part_node, ids); @@ -6865,7 +6870,7 @@ String StorageReplicatedMergeTree::getSharedDataReplica( Strings replicas; for (const auto & id : ids) { - String zookeeper_part_uniq_node = zookeeper_part_node + "/" + id; + String zookeeper_part_uniq_node = fs::path(zookeeper_part_node) / id; Strings id_replicas; zookeeper->tryGetChildren(zookeeper_part_uniq_node, id_replicas); LOG_TRACE(log, "Found zookeper replicas for {}: {}", zookeeper_part_uniq_node, id_replicas.size()); @@ -6882,7 +6887,7 @@ String StorageReplicatedMergeTree::getSharedDataReplica( active_replicas.reserve(replicas.size()); for (const String & replica : replicas) - if ((replica != replica_name) && (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))) + if ((replica != replica_name) && (zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "is_active"))) active_replicas.push_back(replica); LOG_TRACE(log, "Found zookeper active replicas for part {}: {}", part.name, active_replicas.size()); @@ -6901,13 +6906,13 @@ String StorageReplicatedMergeTree::getSharedDataReplica( for (const String & replica : active_replicas) { - String current_replica_path = zookeeper_path + "/replicas/" + replica; + String current_replica_path = fs::path(zookeeper_path) / "replicas" / replica; - String log_pointer_str = zookeeper->get(current_replica_path + "/log_pointer"); + String log_pointer_str = zookeeper->get(fs::path(current_replica_path) / "log_pointer"); Int64 log_pointer = log_pointer_str.empty() ? 0 : parse(log_pointer_str); Coordination::Stat stat; - zookeeper->get(current_replica_path + "/queue", &stat); + zookeeper->get(fs::path(current_replica_path) / "queue", &stat); size_t queue_size = stat.numChildren; if (log_pointer > max_log_pointer @@ -6925,16 +6930,16 @@ String StorageReplicatedMergeTree::getSharedDataReplica( String StorageReplicatedMergeTree::findReplicaHavingPart( const String & part_name, const String & zookeeper_path_, zkutil::ZooKeeper::Ptr zookeeper_) { - Strings replicas = zookeeper_->getChildren(zookeeper_path_ + "/replicas"); + Strings replicas = zookeeper_->getChildren(fs::path(zookeeper_path_) / "replicas"); /// Select replicas in uniformly random order. std::shuffle(replicas.begin(), replicas.end(), thread_local_rng); for (const String & replica : replicas) { - if (zookeeper_->exists(zookeeper_path_ + "/replicas/" + replica + "/parts/" + part_name) - && zookeeper_->exists(zookeeper_path_ + "/replicas/" + replica + "/is_active")) - return zookeeper_path_ + "/replicas/" + replica; + if (zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica / "parts" / part_name) + && zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica / "is_active")) + return fs::path(zookeeper_path_) / "replicas" / replica; } return {}; @@ -6942,23 +6947,23 @@ String StorageReplicatedMergeTree::findReplicaHavingPart( bool StorageReplicatedMergeTree::checkIfDetachedPartExists(const String & part_name) { - Poco::DirectoryIterator dir_end; + fs::directory_iterator dir_end; for (const std::string & path : getDataPaths()) - for (Poco::DirectoryIterator dir_it{path + "detached/"}; dir_it != dir_end; ++dir_it) - if (dir_it.name() == part_name) + for (fs::directory_iterator dir_it{fs::path(path) / "detached/"}; dir_it != dir_end; ++dir_it) + if (dir_it->path().filename().string() == part_name) return true; return false; } bool StorageReplicatedMergeTree::checkIfDetachedPartitionExists(const String & partition_name) { - Poco::DirectoryIterator dir_end; + fs::directory_iterator dir_end; for (const std::string & path : getDataPaths()) { - for (Poco::DirectoryIterator dir_it{path + "detached/"}; dir_it != dir_end; ++dir_it) + for (fs::directory_iterator dir_it{fs::path(path) / "detached/"}; dir_it != dir_end; ++dir_it) { MergeTreePartInfo part_info; - if (MergeTreePartInfo::tryParsePartName(dir_it.name(), &part_info, format_version) && part_info.partition_id == partition_name) + if (MergeTreePartInfo::tryParsePartName(dir_it->path().filename(), &part_info, format_version) && part_info.partition_id == partition_name) return true; } } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 2f25fb43e74..db2866ee23b 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -35,7 +35,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -196,7 +198,7 @@ bool StorageS3Source::initialize() if (current_key.empty()) return false; - file_path = bucket + "/" + current_key; + file_path = fs::path(bucket) / current_key; read_buf = wrapReadBufferWithCompressionMethod( std::make_unique(client, bucket, current_key, s3_max_single_read_retries), chooseCompressionMethod(current_key, compression_hint)); diff --git a/src/Storages/examples/get_current_inserts_in_replicated.cpp b/src/Storages/examples/get_current_inserts_in_replicated.cpp index fa998d20f66..62dcce37d4e 100644 --- a/src/Storages/examples/get_current_inserts_in_replicated.cpp +++ b/src/Storages/examples/get_current_inserts_in_replicated.cpp @@ -36,10 +36,10 @@ try Stopwatch stage; /// Load current inserts std::unordered_set lock_holder_paths; - for (const String & entry : zookeeper->getChildren(zookeeper_path + "/temp")) + for (const String & entry : zookeeper->getChildren(fs::path(zookeeper_path) / "temp")) { if (startsWith(entry, "abandonable_lock-")) - lock_holder_paths.insert(zookeeper_path + "/temp/" + entry); + lock_holder_paths.insert(fs::path(zookeeper_path) / "temp" / entry); } std::cerr << "Stage 1 (get lock holders): " << lock_holder_paths.size() << " lock holders, elapsed: " << stage.elapsedSeconds() << "s." << std::endl; @@ -47,14 +47,14 @@ try if (!lock_holder_paths.empty()) { - Strings partitions = zookeeper->getChildren(zookeeper_path + "/block_numbers"); + Strings partitions = zookeeper->getChildren(fs::path(zookeeper_path) / "block_numbers"); std::cerr << "Stage 2 (get partitions): " << partitions.size() << " partitions, elapsed: " << stage.elapsedSeconds() << "s." << std::endl; stage.restart(); std::vector> lock_futures; for (const String & partition : partitions) - lock_futures.push_back(zookeeper->asyncGetChildren(zookeeper_path + "/block_numbers/" + partition)); + lock_futures.push_back(zookeeper->asyncGetChildren(fs::path(zookeeper_path) / "block_numbers" / partition)); struct BlockInfo { @@ -71,7 +71,7 @@ try for (const String & entry : partition_block_numbers) { Int64 block_number = parse(entry.substr(strlen("block-"))); - String zk_path = zookeeper_path + "/block_numbers/" + partitions[i] + "/" + entry; + String zk_path = fs::path(zookeeper_path) / "block_numbers" / partitions[i] / entry; block_infos.push_back( BlockInfo{partitions[i], block_number, zk_path, zookeeper->asyncTryGet(zk_path)}); } From dc38da5e321bd4b191ebd39ee58ed790eb6f7f41 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 8 May 2021 14:00:31 +0300 Subject: [PATCH 26/55] Fix fs::copy to be like Poco::copyTo --- src/Disks/DiskLocal.cpp | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 31a8d217909..151d0d94597 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -311,13 +311,17 @@ void DiskLocal::copy(const String & from_path, const std::shared_ptr & to { if (isSameDiskType(*this, *to_disk)) { - fs::path from = fs::path(disk_path) / from_path; + fs::path to = fs::path(to_disk->getPath()) / to_path; + fs::path from; if (from_path.ends_with('/')) - from = (fs::path(disk_path) / from_path.substr(0, from_path.size() - 1)).parent_path(); - else if (fs::is_directory(from)) - from = from.parent_path(); + from = fs::path(disk_path) / from_path.substr(0, from_path.size() - 1); + else + from = fs::path(disk_path) / from_path; - fs::copy(from, fs::path(to_disk->getPath()) / to_path, fs::copy_options::recursive | fs::copy_options::overwrite_existing); /// Use more optimal way. + if (fs::is_directory(from)) + to /= from.filename(); + + fs::copy(from, to, fs::copy_options::recursive | fs::copy_options::overwrite_existing); /// Use more optimal way. } else IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers. From 2d4c8b394c85f7ccf0365e93e05f5d92f3d0c263 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 8 May 2021 14:34:50 +0300 Subject: [PATCH 27/55] Return Poco::moveTo --- src/Disks/S3/DiskS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index faff486eeae..442f015b3ec 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -1505,7 +1505,7 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio LOG_DEBUG(log, "Move directory to 'detached' {} -> {}", path, detached_path); - fs::rename(fs::path(metadata_path) / path, fs::path(metadata_path) / detached_path); + Poco::File(fs::path(metadata_path) / path).moveTo(fs::path(metadata_path) / detached_path); } } From e26cb4bee02a94a1017ad516a276a4be2d88e2a0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 9 May 2021 13:45:50 +0300 Subject: [PATCH 28/55] Fix style check, fix yandex check --- src/Common/ya.make | 1 + src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 1 - src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 3 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/ya.make b/src/Common/ya.make index c6605e436c0..e918db8a8a6 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -93,6 +93,7 @@ SRCS( ZooKeeper/ZooKeeperNodeCache.cpp checkStackSize.cpp clearPasswordFromCommandLine.cpp + createFile.cpp createHardLink.cpp escapeForFileName.cpp filesystemHelpers.cpp diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 37e678a1c20..ba7c8c0cfc1 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -29,7 +29,6 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; extern const int TABLE_IS_DROPPED; extern const int TABLE_ALREADY_EXISTS; - extern const int CANNOT_OPEN_FILE; } static const auto suffix = ".removed"; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 56114665fd1..23fe86947ef 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2541,7 +2541,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo source_queue_names = zookeeper->getChildren(fs::path(source_path) / "queue"); /// Check that log pointer of source replica didn't changed while we read queue entries - ops.push_back(zkutil::makeCheckRequest(fs:path(source_path) / "log_pointer", log_pointer_stat.version)); + ops.push_back(zkutil::makeCheckRequest(fs::path(source_path) / "log_pointer", log_pointer_stat.version)); auto rc = zookeeper->tryMulti(ops, responses); From ab1a05a1f4b0f2176469c510dac2c125c8fac815 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 9 May 2021 14:59:49 +0300 Subject: [PATCH 29/55] Poco::Path to fs::path, less concatination --- src/Bridge/IBridgeHelper.cpp | 8 +++++--- src/Common/Config/ConfigProcessor.cpp | 12 ++++++------ src/Databases/DatabaseFactory.cpp | 3 ++- src/Databases/DatabaseLazy.cpp | 2 +- src/Databases/DatabaseOnDisk.cpp | 3 +-- src/Databases/DatabaseReplicated.cpp | 2 +- src/Disks/S3/DiskS3.cpp | 4 +++- src/Interpreters/DDLTask.h | 9 ++++++--- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- src/Storages/Distributed/DirectoryMonitor.cpp | 6 +++--- src/Storages/MergeTree/LeaderElection.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++----- 13 files changed, 36 insertions(+), 29 deletions(-) diff --git a/src/Bridge/IBridgeHelper.cpp b/src/Bridge/IBridgeHelper.cpp index 16886bae4fe..93ca359c578 100644 --- a/src/Bridge/IBridgeHelper.cpp +++ b/src/Bridge/IBridgeHelper.cpp @@ -5,7 +5,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -87,10 +89,10 @@ std::unique_ptr IBridgeHelper::startBridgeCommand() const const auto & config = getConfig(); /// Path to executable folder - Poco::Path path{config.getString("application.dir", "/usr/bin")}; + fs::path path(config.getString("application.dir", "/usr/bin")); std::vector cmd_args; - path.setFileName(serviceFileName()); + path /= serviceFileName(); cmd_args.push_back("--http-port"); cmd_args.push_back(std::to_string(config.getUInt(configPrefix() + ".port", getDefaultPort()))); @@ -126,7 +128,7 @@ std::unique_ptr IBridgeHelper::startBridgeCommand() const LOG_TRACE(getLog(), "Starting {}", serviceAlias()); - return ShellCommand::executeDirect(path.toString(), cmd_args, ShellCommandDestructorStrategy(true)); + return ShellCommand::executeDirect(path.string(), cmd_args, ShellCommandDestructorStrategy(true)); } } diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 36f6a18a85f..d2b1d751422 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -58,7 +58,7 @@ static std::string numberFromHost(const std::string & s) bool ConfigProcessor::isPreprocessedFile(const std::string & path) { - return endsWith(Poco::Path(path).getBaseName(), PREPROCESSED_SUFFIX); + return endsWith(fs::path(path).stem(), PREPROCESSED_SUFFIX); } @@ -412,15 +412,15 @@ ConfigProcessor::Files ConfigProcessor::getConfigMergeFiles(const std::string & { Files files; - Poco::Path merge_dir_path(config_path); + fs::path merge_dir_path(config_path); std::set merge_dirs; /// Add path_to_config/config_name.d dir - merge_dir_path.setExtension("d"); - merge_dirs.insert(merge_dir_path.toString()); + merge_dir_path = merge_dir_path.parent_path() / (merge_dir_path.stem().string() + ".d"); + merge_dirs.insert(merge_dir_path); /// Add path_to_config/conf.d dir - merge_dir_path.setBaseName("conf"); - merge_dirs.insert(merge_dir_path.toString()); + merge_dir_path = merge_dir_path.parent_path() / "conf.d"; + merge_dirs.insert(merge_dir_path); for (const std::string & merge_dir_name : merge_dirs) { diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 3efd92694a6..d179adb18cf 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -61,7 +61,8 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m try { /// Creates store/xxx/ for Atomic - Poco::File(Poco::Path(metadata_path).makeParent()).createDirectories(); + fs::create_directories(fs::path(metadata_path).parent_path()); + /// Before 20.7 it's possible that .sql metadata file does not exist for some old database. /// In this case Ordinary database is created on server startup if the corresponding metadata directory exists. /// So we should remove metadata directory if database creation failed. diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 416a94db929..a0d5f5adbc3 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -229,7 +229,7 @@ StoragePtr DatabaseLazy::loadTable(const String & table_name) const LOG_DEBUG(log, "Load table {} to cache.", backQuote(table_name)); - const String table_metadata_path = getMetadataPath() + "/" + escapeForFileName(table_name) + ".sql"; + const String table_metadata_path = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + ".sql"); try { diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 95cd47e3796..75788cb1a02 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -671,8 +671,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata( auto & create = ast->as(); if (!create.table.empty() && create.uuid != UUIDHelpers::Nil) { - String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName(); - table_name = unescapeForFileName(table_name); + String table_name = unescapeForFileName(fs::path(metadata_file_path).stem()); if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER && logger) LOG_WARNING( diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index b7214917ce8..cce02694635 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -212,7 +212,7 @@ void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(bool force_attach) createDatabaseNodesInZooKeeper(current_zookeeper); } - replica_path = zookeeper_path + "/replicas/" + getFullReplicaName(); + replica_path = fs::path(zookeeper_path) / "replicas" / getFullReplicaName(); String replica_host_id; if (current_zookeeper->tryGet(replica_path, replica_host_id)) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 442f015b3ec..b15b7703557 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -1537,7 +1537,9 @@ String DiskS3::revisionToString(UInt64 revision) String DiskS3::pathToDetached(const String & source_path) { - return Poco::Path(source_path).parent().append(Poco::Path("detached")).toString() + '/'; + if (source_path.ends_with('/')) + return fs::path(source_path).parent_path().parent_path() / "detached/"; + return fs::path(source_path).parent_path() / "detached/"; } void DiskS3::onFreeze(const String & path) diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index 874a29f051d..9e314eb84c6 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace Poco { @@ -14,6 +15,8 @@ namespace zkutil class ZooKeeper; } +namespace fs = std::filesystem; + namespace DB { @@ -100,9 +103,9 @@ struct DDLTaskBase virtual ContextPtr makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper); - inline String getActiveNodePath() const { return entry_path + "/active/" + host_id_str; } - inline String getFinishedNodePath() const { return entry_path + "/finished/" + host_id_str; } - inline String getShardNodePath() const { return entry_path + "/shards/" + getShardID(); } + inline String getActiveNodePath() const { return fs::path(entry_path) / "active" / host_id_str; } + inline String getFinishedNodePath() const { return fs::path(entry_path) / "finished" / host_id_str; } + inline String getShardNodePath() const { return fs::path(entry_path) / "shards" / getShardID(); } static String getLogEntryName(UInt32 log_entry_number); static UInt32 getLogEntryNumber(const String & log_entry_name); diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 8a4b3b07692..3eb82048dec 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -542,7 +542,7 @@ void InterpreterSystemQuery::dropReplica(ASTSystemQuery & query) else if (!query.replica_zk_path.empty()) { getContext()->checkAccess(AccessType::SYSTEM_DROP_REPLICA); - auto remote_replica_path = query.replica_zk_path + "/replicas/" + query.replica; + String remote_replica_path = fs::path(query.replica_zk_path) / "replicas" / query.replica; /// This check is actually redundant, but it may prevent from some user mistakes for (auto & elem : DatabaseCatalog::instance().getDatabases()) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index bae81a56b30..e8835132f8f 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -958,13 +958,13 @@ void StorageDistributedDirectoryMonitor::markAsBroken(const std::string & file_p const auto last_path_separator_pos = file_path.rfind('/'); const auto & base_path = file_path.substr(0, last_path_separator_pos + 1); const auto & file_name = file_path.substr(last_path_separator_pos + 1); - const auto & broken_path = base_path + "broken/"; - const auto & broken_file_path = broken_path + file_name; + const String & broken_path = fs::path(base_path) / "broken/"; + const String & broken_file_path = fs::path(broken_path) / file_name; fs::create_directory(broken_path); auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path); - auto broken_dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path + "/broken/"); + auto broken_dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, fs::path(relative_path) / "broken/"); { std::lock_guard status_lock(status_mutex); diff --git a/src/Storages/MergeTree/LeaderElection.h b/src/Storages/MergeTree/LeaderElection.h index 7cdfd8e566d..2810385e9e5 100644 --- a/src/Storages/MergeTree/LeaderElection.h +++ b/src/Storages/MergeTree/LeaderElection.h @@ -83,7 +83,7 @@ private: void createNode() { shutdown_called = false; - node = EphemeralNodeHolder::createSequential(path + "/leader_election-", zookeeper, identifier); + node = EphemeralNodeHolder::createSequential(fs::path(path) / "leader_election-", zookeeper, identifier); std::string node_path = node->getPath(); node_name = node_path.substr(node_path.find_last_of('/') + 1); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1849b2efebd..c9ad3f10bd8 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3232,7 +3232,7 @@ void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, ContextPtr for (auto & [old_name, new_name] : renamed_parts.old_and_new_names) { const auto & [path, disk] = renamed_parts.old_part_name_to_path_and_disk[old_name]; - disk->removeRecursive(path + "detached/" + new_name + "/"); + disk->removeRecursive(fs::path(path) / "detached" / new_name / ""); LOG_DEBUG(log, "Dropped detached part {}", old_name); old_name.clear(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 23fe86947ef..64f800dc05a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -254,7 +254,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , zookeeper_name(extractZooKeeperName(zookeeper_path_)) , zookeeper_path(extractZooKeeperPath(zookeeper_path_)) , replica_name(replica_name_) - , replica_path(zookeeper_path + "/replicas/" + replica_name_) + , replica_path(fs::path(zookeeper_path) / "replicas" / replica_name_) , reader(*this) , writer(*this) , merger_mutator(*this, getContext()->getSettingsRef().background_pool_size) @@ -473,14 +473,14 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( /// Mutation maybe killed or whole replica was deleted. /// Wait event will unblock at this moment. Coordination::Stat exists_stat; - if (!getZooKeeper()->exists(zookeeper_path + "/mutations/" + mutation_id, &exists_stat, wait_event)) + if (!getZooKeeper()->exists(fs::path(zookeeper_path) / "mutations" / mutation_id, &exists_stat, wait_event)) { throw Exception(ErrorCodes::UNFINISHED, "Mutation {} was killed, manually removed or table was dropped", mutation_id); } auto zookeeper = getZooKeeper(); /// Replica could be inactive. - if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) + if (!zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "is_active")) { LOG_WARNING(log, "Replica {} is not active during mutation. Mutation will be done asynchronously when replica becomes active.", replica); @@ -488,7 +488,7 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( break; } - String mutation_pointer = zookeeper_path + "/replicas/" + replica + "/mutation_pointer"; + String mutation_pointer = fs::path(zookeeper_path) / "replicas" / replica / "mutation_pointer"; std::string mutation_pointer_value; /// Replica could be removed if (!zookeeper->tryGet(mutation_pointer, mutation_pointer_value, nullptr, wait_event)) @@ -512,7 +512,7 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( /// It maybe already removed from zk, but local in-memory mutations /// state was not updated. - if (!getZooKeeper()->exists(zookeeper_path + "/mutations/" + mutation_id)) + if (!getZooKeeper()->exists(fs::path(zookeeper_path) / "mutations" / mutation_id)) { throw Exception(ErrorCodes::UNFINISHED, "Mutation {} was killed, manually removed or table was dropped", mutation_id); } From 3439492700dcd0b974e29ec867d84acf6efb75da Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 11 May 2021 19:14:58 +0300 Subject: [PATCH 30/55] fs::last_write_time --- src/Common/Config/ConfigReloader.cpp | 5 ++++- src/Interpreters/ExternalLoaderXMLConfigRepository.cpp | 4 +++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index ad0ce7cc30e..b2aeb899e4d 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -170,7 +170,10 @@ struct ConfigReloader::FileWithTimestamp void ConfigReloader::FilesChangesTracker::addIfExists(const std::string & path_to_add) { if (!path_to_add.empty() && fs::exists(path_to_add)) - files.emplace(path_to_add, Poco::File(path_to_add).getLastModified().epochTime()); + { + fs::file_time_type fs_time = fs::last_write_time(path_to_add); + files.emplace(path_to_add, fs::file_time_type::clock::to_time_t(fs_time)); + } } bool ConfigReloader::FilesChangesTracker::isDifferOrNewerThan(const FilesChangesTracker & rhs) diff --git a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp index e1052817407..e4f3447c6df 100644 --- a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp +++ b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp @@ -21,7 +21,9 @@ ExternalLoaderXMLConfigRepository::ExternalLoaderXMLConfigRepository( Poco::Timestamp ExternalLoaderXMLConfigRepository::getUpdateTime(const std::string & definition_entity_name) { - return Poco::File(definition_entity_name).getLastModified(); + fs::file_time_type fs_time = fs::last_write_time(definition_entity_name); + auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); + return Poco::Timestamp(micro_sec.count()); } std::set ExternalLoaderXMLConfigRepository::getAllLoadablesDefinitionNames() From 81797393f1d03f299ca827b47a92e16942945750 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 12 May 2021 09:19:11 +0300 Subject: [PATCH 31/55] fs::last_write_time, fs::space --- src/Common/Config/ConfigProcessor.cpp | 6 +++--- src/Common/filesystemHelpers.cpp | 6 +----- src/Databases/DatabaseOnDisk.cpp | 8 +++++--- src/Interpreters/DatabaseCatalog.cpp | 3 ++- 4 files changed, 11 insertions(+), 12 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index d2b1d751422..d0b85cc7506 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -642,9 +642,9 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, } preprocessed_path = (fs::path(preprocessed_dir) / fs::path(new_path)).string(); - auto preprocessed_path_parent = Poco::Path(preprocessed_path).makeParent(); - if (!preprocessed_path_parent.toString().empty()) - Poco::File(preprocessed_path_parent).createDirectories(); + auto preprocessed_path_parent = fs::path(preprocessed_path).parent_path(); + if (!preprocessed_path_parent.string().empty()) + fs::create_directories(preprocessed_path_parent); } DOMWriter().writeNode(preprocessed_path, loaded_config.preprocessed_xml); LOG_DEBUG(log, "Saved preprocessed configuration to '{}'.", preprocessed_path); diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 835668dfa1c..b146392e1a6 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -40,12 +40,8 @@ struct statvfs getStatVFS(const String & path) bool enoughSpaceInDirectory(const std::string & path [[maybe_unused]], size_t data_size [[maybe_unused]]) { -#if POCO_VERSION >= 0x01090000 - auto free_space = Poco::File(path).freeSpace(); + auto free_space = fs::space(path).free; return data_size <= free_space; -#else - return true; -#endif } std::unique_ptr createTemporaryFile(const std::string & path) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 75788cb1a02..2ed6a0d9b6d 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -540,10 +540,12 @@ String DatabaseOnDisk::getObjectMetadataPath(const String & object_name) const time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_name) const { String table_metadata_path = getObjectMetadataPath(object_name); - Poco::File meta_file(table_metadata_path); - if (meta_file.exists()) - return meta_file.getLastModified().epochTime(); + if (fs::exists(table_metadata_path)) + { + fs::file_time_type fs_time = fs::last_write_time(table_metadata_path); + return fs::file_time_type::clock::to_time_t(fs_time); + } else return static_cast(0); } diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 6d6b8107f0c..45a803934b5 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -782,7 +782,8 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr } addUUIDMapping(table_id.uuid); - drop_time = Poco::File(dropped_metadata_path).getLastModified().epochTime(); + fs::file_time_type fs_time = fs::last_write_time(dropped_metadata_path); + drop_time = fs::file_time_type::clock::to_time_t(fs_time); } std::lock_guard lock(tables_marked_dropped_mutex); From 17432b1a4177397d173e7a5d41b42596628a02e1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 12 May 2021 17:32:22 +0300 Subject: [PATCH 32/55] Poco::moveTo to fs version --- src/Disks/S3/DiskS3.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index b15b7703557..4fbfc9141a6 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -1505,7 +1505,14 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio LOG_DEBUG(log, "Move directory to 'detached' {} -> {}", path, detached_path); - Poco::File(fs::path(metadata_path) / path).moveTo(fs::path(metadata_path) / detached_path); + fs::path from_path = fs::path(metadata_path) / path; + fs::path to_path = fs::path(metadata_path) / detached_path; + if (path.ends_with('/')) + to_path /= from_path.parent_path().filename(); + else + to_path /= from_path.filename(); + fs::copy(from_path, to_path, fs::copy_options::recursive | fs::copy_options::overwrite_existing); + fs::remove_all(from_path); } } From e466924898fe94d724712ad9e0def8a3365910bd Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 13 May 2021 02:01:35 +0300 Subject: [PATCH 33/55] fs::premissions --- src/Disks/DiskLocal.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 151d0d94597..bf0a8bac12f 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -299,7 +299,9 @@ void DiskLocal::createFile(const String & path) void DiskLocal::setReadOnly(const String & path) { - Poco::File(fs::path(disk_path) / path).setReadOnly(true); + fs::permissions(fs::path(disk_path) / path, + fs::perms::owner_read | fs::perms::group_read | fs::perms::others_read, + fs::perm_options::replace); } bool inline isSameDiskType(const IDisk & one, const IDisk & another) From b123ec135401844d306ceefd5d19bcda19c33517 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 13 May 2021 12:32:52 +0300 Subject: [PATCH 34/55] Poco::FileException, Poco::parentPath --- src/Common/Config/ConfigProcessor.cpp | 10 +++++----- src/Common/Exception.cpp | 25 ++++++++----------------- src/Disks/IDisk.h | 2 +- 3 files changed, 14 insertions(+), 23 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index d0b85cc7506..f85e22d28b6 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -623,11 +623,11 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, if (!loaded_config.configuration->has("path")) { // Will use current directory - auto parent_path = Poco::Path(loaded_config.config_path).makeParent(); - preprocessed_dir = parent_path.toString(); - Poco::Path poco_new_path(new_path); - poco_new_path.setBaseName(poco_new_path.getBaseName() + PREPROCESSED_SUFFIX); - new_path = poco_new_path.toString(); + fs::path parent_path = fs::path(loaded_config.config_path).parent_path(); + preprocessed_dir = parent_path.string(); + fs::path fs_new_path(new_path); + fs_new_path = fs_new_path.parent_path() / (fs_new_path.stem().string() + PREPROCESSED_SUFFIX + fs_new_path.extension().string()); + new_path = fs_new_path.string(); } else { diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index f17eaea8d7f..4da84e92f2a 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -21,6 +21,8 @@ # include #endif +namespace fs = std::filesystem; + namespace DB { @@ -164,7 +166,7 @@ void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_ } } -static void getNoSpaceLeftInfoMessage(std::filesystem::path path, std::string & msg) +static void getNoSpaceLeftInfoMessage(std::filesystem::path path, String & msg) { path = std::filesystem::absolute(path); /// It's possible to get ENOSPC for non existent file (e.g. if there are no free inodes and creat() fails) @@ -251,23 +253,12 @@ static std::string getExtraExceptionInfo(const std::exception & e) String msg; try { - /// TODO: this has to be adjusted for std::filesystem - if (const auto * file_exception = dynamic_cast(&e)) + if (const auto * file_exception = dynamic_cast(&e)) { - if (file_exception->code() == ENOSPC) - { - /// See Poco::FileImpl::handleLastErrorImpl(...) - constexpr const char * expected_error_message = "no space left on device: "; - if (startsWith(file_exception->message(), expected_error_message)) - { - String path = file_exception->message().substr(strlen(expected_error_message)); - getNoSpaceLeftInfoMessage(path, msg); - } - else - { - msg += "\nCannot print extra info for Poco::Exception"; - } - } + if (file_exception->code() == std::errc::no_space_on_device) + getNoSpaceLeftInfoMessage(file_exception->path1(), msg); + else + msg += "\nCannot print extra info for Poco::Exception"; } else if (const auto * errno_exception = dynamic_cast(&e)) { diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 62ab54e90c2..459991d6afd 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -301,7 +301,7 @@ inline String fullPath(const DiskPtr & disk, const String & path) /// Return parent path for the specified path. inline String parentPath(const String & path) { - return Poco::Path(path).parent().toString(); + return fs::path(path).parent_path() / ""; } /// Return file name for the specified path. From db849eb3e5ec110645d1561e38cd0dc102d4def2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 14 May 2021 10:06:38 +0300 Subject: [PATCH 35/55] Remaining poco --- src/Disks/IDisk.h | 2 ++ src/Interpreters/ExternalLoaderXMLConfigRepository.cpp | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 459991d6afd..2e83bf0ec26 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -301,6 +301,8 @@ inline String fullPath(const DiskPtr & disk, const String & path) /// Return parent path for the specified path. inline String parentPath(const String & path) { + if (path.ends_with('/')) + return fs::path(path).parent_path().parent_path() / ""; return fs::path(path).parent_path() / ""; } diff --git a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp index e4f3447c6df..4c6af554fae 100644 --- a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp +++ b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp @@ -40,8 +40,8 @@ std::set ExternalLoaderXMLConfigRepository::getAllLoadablesDefiniti if (pattern[0] != '/') { const auto app_config_path = main_config.getString("config-file", "config.xml"); - const auto config_dir = Poco::Path{app_config_path}.parent().toString(); - const auto absolute_path = config_dir + pattern; + const String config_dir = fs::path(app_config_path).parent_path(); + const String absolute_path = fs::path(config_dir) / pattern; Poco::Glob::glob(absolute_path, files, 0); if (!files.empty()) continue; From 018d52d71f7d54974f864356ea4671e26969126b Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 15 May 2021 09:20:26 +0300 Subject: [PATCH 36/55] Directory path --- src/Disks/IDisk.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 2e83bf0ec26..e299fffbf41 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -14,7 +14,6 @@ #include #include #include -#include #include "Poco/Util/AbstractConfiguration.h" namespace fs = std::filesystem; @@ -315,7 +314,7 @@ inline String fileName(const String & path) /// Return directory path for the specified path. inline String directoryPath(const String & path) { - return Poco::Path(path).setFileName("").toString(); + return fs::path(path).parent_path() / ""; } } From add5970b5f0fc092f8c5fbd213e2ce97aa72bf27 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 16 May 2021 03:12:13 +0300 Subject: [PATCH 37/55] fs::absolute --- .../Embedded/GeodataProviders/HierarchiesProvider.cpp | 9 ++++----- src/Storages/MergeTree/DataPartsExchange.cpp | 4 ++-- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp b/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp index 7257caa066f..3f35b9102a2 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp +++ b/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp @@ -30,13 +30,12 @@ RegionsHierarchiesDataProvider::RegionsHierarchiesDataProvider(const std::string void RegionsHierarchiesDataProvider::discoverFilesWithCustomHierarchies() { std::string basename = fs::path(path).stem(); + fs::path dir_path = fs::absolute(path).parent_path(); - Poco::Path dir_path = Poco::Path(path).absolute().parent(); - - Poco::DirectoryIterator dir_end; - for (Poco::DirectoryIterator dir_it(dir_path); dir_it != dir_end; ++dir_it) + fs::directory_iterator dir_end; + for (fs::directory_iterator dir_it(dir_path); dir_it != dir_end; ++dir_it) { - std::string candidate_basename = dir_it.path().getBaseName(); + std::string candidate_basename = dir_it->path().stem(); if (candidate_basename.starts_with(basename) && (candidate_basename.size() > basename.size() + 1) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 7f483fca681..69a92738777 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -652,8 +652,8 @@ void Fetcher::downloadBaseOrProjectionPartToDisk( /// File must be inside "absolute_part_path" directory. /// Otherwise malicious ClickHouse replica may force us to write to arbitrary path. - String absolute_file_path = Poco::Path(part_download_path + file_name).absolute().toString(); - if (!startsWith(absolute_file_path, Poco::Path(part_download_path).absolute().toString())) + String absolute_file_path = fs::absolute(fs::path(part_download_path) / file_name); + if (!startsWith(absolute_file_path, fs::absolute(part_download_path).string())) throw Exception("File path (" + absolute_file_path + ") doesn't appear to be inside part path (" + part_download_path + ")." " This may happen if we are trying to download part from malicious replica or logical error.", ErrorCodes::INSECURE_PATH); From 3b1bf2bae6c7a5c7be7c00e15c4f233916452150 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 16 May 2021 21:36:55 +0300 Subject: [PATCH 38/55] Poco::Path substitution --- src/Formats/FormatSchemaInfo.cpp | 55 +++++++++++++++--------- src/Storages/MergeTree/MergeTreeData.cpp | 4 +- 2 files changed, 36 insertions(+), 23 deletions(-) diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index 707f9babe8d..81c11eda0b4 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include namespace DB @@ -11,6 +13,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace fs = std::filesystem; namespace { @@ -34,55 +37,65 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & String default_file_extension = getFormatSchemaDefaultFileExtension(format); - Poco::Path path; + fs::path path; if (require_message) { size_t colon_pos = format_schema.find(':'); - if ((colon_pos == String::npos) || (colon_pos == 0) || (colon_pos == format_schema.length() - 1) - || path.assign(format_schema.substr(0, colon_pos)).makeFile().getFileName().empty()) + if ((colon_pos == String::npos) || (colon_pos == 0) || (colon_pos == format_schema.length() - 1)) { throw Exception( "Format schema requires the 'format_schema' setting to have the 'schema_file:message_name' format" + (default_file_extension.empty() ? "" : ", e.g. 'schema." + default_file_extension + ":Message'") + - ". Got '" + format_schema - + "'", - ErrorCodes::BAD_ARGUMENTS); + ". Got '" + format_schema + "'", ErrorCodes::BAD_ARGUMENTS); + } + else + { + path = fs::path(format_schema.substr(0, colon_pos)); + String filename = path.has_filename() ? path.filename() : path.parent_path().filename(); + if (filename.empty()) + throw Exception( + "Format schema requires the 'format_schema' setting to have the 'schema_file:message_name' format" + + (default_file_extension.empty() ? "" : ", e.g. 'schema." + default_file_extension + ":Message'") + + ". Got '" + format_schema + "'", ErrorCodes::BAD_ARGUMENTS); } - message_name = format_schema.substr(colon_pos + 1); } else - path.assign(format_schema).makeFile().getFileName(); + { + path = fs::path(format_schema); + if (!path.has_filename()) + path = path.parent_path() / ""; + } auto default_schema_directory = [&format_schema_path]() { - static const String str = Poco::Path(format_schema_path).makeAbsolute().makeDirectory().toString(); + static const String str = fs::absolute(format_schema_path) / ""; return str; }; - if (path.getExtension().empty() && !default_file_extension.empty()) - path.setExtension(default_file_extension); + if (!path.has_extension() && !default_file_extension.empty()) + path = path.parent_path() / (path.stem().string() + '.' + default_file_extension); - if (path.isAbsolute()) + if (path.is_absolute()) { if (is_server) - throw Exception("Absolute path in the 'format_schema' setting is prohibited: " + path.toString(), ErrorCodes::BAD_ARGUMENTS); - schema_path = path.getFileName(); - schema_directory = path.makeParent().toString(); + throw Exception("Absolute path in the 'format_schema' setting is prohibited: " + path.string(), ErrorCodes::BAD_ARGUMENTS); + schema_path = path.filename(); + schema_directory = path.parent_path() / ""; } - else if (path.depth() >= 1 && path.directory(0) == "..") + else if (Poco::Path(path.string()).depth() >= 1 && Poco::Path(path.string()).directory(0) == "..") { if (is_server) throw Exception( - "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.toString(), + "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.string(), ErrorCodes::BAD_ARGUMENTS); - path = Poco::Path(default_schema_directory()).resolve(path).toString(); - schema_path = path.getFileName(); - schema_directory = path.makeParent().toString(); + path = Poco::Path(default_schema_directory()).resolve(Poco::Path(path.string())).toString(); + schema_path = path.filename(); + schema_directory = path.parent_path() / ""; } else { - schema_path = path.toString(); + schema_path = path; schema_directory = default_schema_directory(); } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index fc752030b89..374fd473599 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3415,8 +3415,8 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const if (!containing_part.empty() && containing_part != name) // TODO maybe use PartsTemporaryRename here? - disk->moveDirectory(relative_data_path + source_dir + name, - relative_data_path + source_dir + "inactive_" + name); + disk->moveDirectory(fs::path(relative_data_path) / source_dir / name, + fs::path(relative_data_path) / source_dir / ("inactive_" + name)); else renamed_parts.addPart(name, "attaching_" + name); } From 2124113aa29ed8a25d0730deea43700e5eba380f Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 May 2021 01:06:09 +0300 Subject: [PATCH 39/55] Update programs/* --- programs/benchmark/Benchmark.cpp | 7 +++--- programs/client/Client.cpp | 9 +++++--- programs/copier/ClusterCopierApp.cpp | 12 ++++++---- programs/local/LocalServer.cpp | 12 +++++----- programs/server/Server.cpp | 34 ++++++++++++++++------------ 5 files changed, 42 insertions(+), 32 deletions(-) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 1d2b579db3a..498d1eecae2 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include @@ -36,7 +35,9 @@ #include #include #include +#include +namespace fs = std::filesystem; /** A tool for evaluating ClickHouse performance. * The tool emulates a case with fixed amount of simultaneously executing queries. @@ -119,8 +120,8 @@ public: int main(const std::vector &) override { - if (!json_path.empty() && Poco::File(json_path).exists()) /// Clear file with previous results - Poco::File(json_path).remove(); + if (!json_path.empty() && fs::exists(json_path)) /// Clear file with previous results + fs::remove(json_path); readQueries(); runBenchmark(); diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index aae2e0f1c59..0e4b577c40e 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -87,6 +87,8 @@ #include #include #include +#include +#include #if !defined(ARCADIA_BUILD) # include @@ -96,6 +98,7 @@ #pragma GCC optimize("-fno-var-tracking-assignments") #endif +namespace fs = std::filesystem; namespace DB { @@ -276,7 +279,7 @@ private: /// Set path for format schema files if (config().has("format_schema_path")) - context->setFormatSchemaPath(Poco::Path(config().getString("format_schema_path")).toString()); + context->setFormatSchemaPath(config().getString("format_schema_path")); /// Initialize query_id_formats if any if (config().has("query_id_formats")) @@ -633,8 +636,8 @@ private: history_file = home_path + "/.clickhouse-client-history"; } - if (!history_file.empty() && !Poco::File(history_file).exists()) - Poco::File(history_file).createFile(); + if (!history_file.empty() && !fs::exists(history_file)) + fs::createFile(history_file); LineReader::Patterns query_extenders = {"\\"}; LineReader::Patterns query_delimiters = {";", "\\G"}; diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index d3fff616b65..79bfb5b7411 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -5,7 +5,9 @@ #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -26,7 +28,7 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self) copy_fault_probability = std::max(std::min(config().getDouble("copy-fault-probability"), 1.0), 0.0); if (config().has("move-fault-probability")) move_fault_probability = std::max(std::min(config().getDouble("move-fault-probability"), 1.0), 0.0); - base_dir = (config().has("base-dir")) ? config().getString("base-dir") : Poco::Path::current(); + base_dir = (config().has("base-dir")) ? config().getString("base-dir") : fs::current_path().string(); if (config().has("experimental-use-sample-offset")) @@ -38,18 +40,18 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self) process_id = std::to_string(DateLUT::instance().toNumYYYYMMDDhhmmss(timestamp)) + "_" + std::to_string(curr_pid); host_id = escapeForFileName(getFQDNOrHostName()) + '#' + process_id; - process_path = Poco::Path(base_dir + "/clickhouse-copier_" + process_id).absolute().toString(); - Poco::File(process_path).createDirectories(); + process_path = fs::absolute(fs::path(base_dir) / ("clickhouse-copier_" + process_id)); + fs::create_directories(process_path); /// Override variables for BaseDaemon if (config().has("log-level")) config().setString("logger.level", config().getString("log-level")); if (config().has("base-dir") || !config().has("logger.log")) - config().setString("logger.log", process_path + "/log.log"); + config().setString("logger.log", fs::path(process_path) / "log.log"); if (config().has("base-dir") || !config().has("logger.errorlog")) - config().setString("logger.errorlog", process_path + "/log.err.log"); + config().setString("logger.errorlog", fs::path(process_path) / "log.err.log"); Base::initialize(self); } diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 043cc596e2b..c444f89e08a 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -42,9 +42,9 @@ #include #include #include - #include +namespace fs = std::filesystem; namespace DB { @@ -72,11 +72,11 @@ void LocalServer::initialize(Poco::Util::Application & self) Poco::Util::Application::initialize(self); /// Load config files if exists - if (config().has("config-file") || Poco::File("config.xml").exists()) + if (config().has("config-file") || fs::exists("config.xml")) { const auto config_path = config().getString("config-file", "config.xml"); ConfigProcessor config_processor(config_path, false, true); - config_processor.setConfigPath(Poco::Path(config_path).makeParent().toString()); + config_processor.setConfigPath(fs::path(config_path).parent_path()); auto loaded_config = config_processor.loadConfig(); config_processor.savePreprocessedConfig(loaded_config, loaded_config.configuration->getString("path", ".")); config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false); @@ -287,8 +287,8 @@ try status.emplace(path + "status", StatusFile::write_full_info); LOG_DEBUG(log, "Loading metadata from {}", path); - Poco::File(path + "data/").createDirectories(); - Poco::File(path + "metadata/").createDirectories(); + fs::create_directories(fs::path(path) / "data/"); + fs::create_directories(fs::path(path) / "metadata/"); loadMetadataSystem(global_context); attachSystemTables(global_context); loadMetadata(global_context); @@ -479,7 +479,7 @@ void LocalServer::setupUsers() { ConfigurationPtr users_config; - if (config().has("users_config") || config().has("config-file") || Poco::File("config.xml").exists()) + if (config().has("users_config") || config().has("config-file") || fs::exists("config.xml")) { const auto users_config_path = config().getString("users_config", config().getString("config-file", "config.xml")); ConfigProcessor config_processor(users_config_path); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index a36287a8051..8668f09ca06 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -74,6 +74,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) @@ -117,6 +118,8 @@ namespace CurrentMetrics extern const Metric MaxDDLEntryID; } +namespace fs = std::filesystem; + #if USE_JEMALLOC static bool jemallocOptionEnabled(const char *name) { @@ -183,7 +186,7 @@ void setupTmpPath(Poco::Logger * log, const std::string & path) { LOG_DEBUG(log, "Setting up {} to store temporary data in it", path); - Poco::File(path).createDirectories(); + fs::create_directories(path); /// Clearing old temporary files. Poco::DirectoryIterator dir_end; @@ -678,37 +681,38 @@ int Server::main(const std::vector & /*args*/) * Examples: do repair of local data; clone all replicated tables from replica. */ { - Poco::File(path + "flags/").createDirectories(); - global_context->setFlagsPath(path + "flags/"); + auto flags_path = fs::path(path) / "flags/"; + fs::create_directories(flags_path); + global_context->setFlagsPath(flags_path); } /** Directory with user provided files that are usable by 'file' table function. */ { - std::string user_files_path = config().getString("user_files_path", path + "user_files/"); + std::string user_files_path = config().getString("user_files_path", fs::path(path) / "user_files/"); global_context->setUserFilesPath(user_files_path); - Poco::File(user_files_path).createDirectories(); + fs::create_directories(user_files_path); } { - std::string dictionaries_lib_path = config().getString("dictionaries_lib_path", path + "dictionaries_lib/"); + std::string dictionaries_lib_path = config().getString("dictionaries_lib_path", fs::path(path) / "dictionaries_lib/"); global_context->setDictionariesLibPath(dictionaries_lib_path); - Poco::File(dictionaries_lib_path).createDirectories(); + fs::create_directories(dictionaries_lib_path); } /// top_level_domains_lists { - const std::string & top_level_domains_path = config().getString("top_level_domains_path", path + "top_level_domains/") + "/"; - TLDListsHolder::getInstance().parseConfig(top_level_domains_path, config()); + const std::string & top_level_domains_path = config().getString("top_level_domains_path", fs::path(path) / "top_level_domains/"); + TLDListsHolder::getInstance().parseConfig(fs::path(top_level_domains_path) / "", config()); } { - Poco::File(path + "data/").createDirectories(); - Poco::File(path + "metadata/").createDirectories(); + fs::create_directories(fs::path(path) / "data/"); + fs::create_directories(fs::path(path) / "metadata/"); /// Directory with metadata of tables, which was marked as dropped by Atomic database - Poco::File(path + "metadata_dropped/").createDirectories(); + fs::create_directories(fs::path(path) / "metadata_dropped/"); } if (config().has("interserver_http_port") && config().has("interserver_https_port")) @@ -891,9 +895,9 @@ int Server::main(const std::vector & /*args*/) #endif /// Set path for format schema files - auto format_schema_path = Poco::File(config().getString("format_schema_path", path + "format_schemas/")); - global_context->setFormatSchemaPath(format_schema_path.path()); - format_schema_path.createDirectories(); + fs::path format_schema_path(config().getString("format_schema_path", fs::path(path) / "format_schemas/")); + global_context->setFormatSchemaPath(format_schema_path); + fs::create_directories(format_schema_path); /// Check sanity of MergeTreeSettings on server startup global_context->getMergeTreeSettings().sanityCheck(settings); From 649dd23b8b7a5c97b39bf8f79a791d880436d540 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 May 2021 09:48:02 +0300 Subject: [PATCH 40/55] Poco::resolve --- src/Formats/FormatSchemaInfo.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index 81c11eda0b4..ae1eb422d27 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -89,7 +89,11 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & throw Exception( "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.string(), ErrorCodes::BAD_ARGUMENTS); - path = Poco::Path(default_schema_directory()).resolve(Poco::Path(path.string())).toString(); + fs::path default_schema_directory_path(default_schema_directory()); + if (default_schema_directory_path.is_absolute()) + path = default_schema_directory_path; + else + path /= default_schema_directory_path; schema_path = path.filename(); schema_directory = path.parent_path() / ""; } From 85cc7a89239656fb53aabf85614c896771f6a425 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 May 2021 17:44:10 +0300 Subject: [PATCH 41/55] Remove last presence of Poco::Path in src --- src/Formats/FormatSchemaInfo.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index ae1eb422d27..513d9738f62 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -1,9 +1,7 @@ #include -#include #include #include #include -#include namespace DB @@ -83,7 +81,7 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & schema_path = path.filename(); schema_directory = path.parent_path() / ""; } - else if (Poco::Path(path.string()).depth() >= 1 && Poco::Path(path.string()).directory(0) == "..") + else if (path.string().starts_with("..")) { if (is_server) throw Exception( From 5f083b779dde1072dc14e4729e04db45bda06c59 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 18 May 2021 13:11:12 +0300 Subject: [PATCH 42/55] A little better, fix checks --- programs/client/Client.cpp | 2 +- src/Common/createFile.cpp | 6 ++---- src/Common/createFile.h | 6 +++--- src/Databases/DatabaseOnDisk.cpp | 2 +- src/Databases/MySQL/DatabaseConnectionMySQL.cpp | 2 +- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Disks/DiskLocal.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Server/StaticRequestHandler.cpp | 7 ++++--- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/examples/remove_symlink_directory.cpp | 2 +- 11 files changed, 17 insertions(+), 18 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 407f70b36b6..58e3efe736d 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -637,7 +637,7 @@ private: } if (!history_file.empty() && !fs::exists(history_file)) - fs::createFile(history_file); + FS::createFile(history_file); LineReader::Patterns query_extenders = {"\\"}; LineReader::Patterns query_delimiters = {";", "\\G"}; diff --git a/src/Common/createFile.cpp b/src/Common/createFile.cpp index fc637f4d4f2..83ab9c36f4b 100644 --- a/src/Common/createFile.cpp +++ b/src/Common/createFile.cpp @@ -17,7 +17,7 @@ extern const int CANNOT_CREATE_FILE; } } -namespace std::filesystem +namespace FS { [[noreturn]] void handleLastError(const std::string & path) { @@ -37,7 +37,7 @@ namespace std::filesystem } /// Copy from Poco::createFile -bool createFile(const path & path) +bool createFile(const std::string & path) { int n = open(path.c_str(), O_WRONLY | O_CREAT | O_EXCL, S_IRUSR | S_IWUSR | S_IRGRP | S_IWGRP | S_IROTH | S_IWOTH); if (n != -1) @@ -45,8 +45,6 @@ bool createFile(const path & path) close(n); return true; } - if (n == -1 && errno == EEXIST) - return false; handleLastError(path); } } diff --git a/src/Common/createFile.h b/src/Common/createFile.h index 40ec74e6288..67c7023a455 100644 --- a/src/Common/createFile.h +++ b/src/Common/createFile.h @@ -1,7 +1,7 @@ #pragma once -#include +#include -namespace std::filesystem +namespace FS { -bool createFile(const path & path); +bool createFile(const std::string & path); } diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index a47a38cfa7c..8393f9e81c0 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -326,7 +326,7 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr, const String & table_nam fs::path detached_permanently_flag(getObjectMetadataPath(table_name) + detached_suffix); try { - fs::createFile(detached_permanently_flag); + FS::createFile(detached_permanently_flag); } catch (Exception & e) { diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index cd26aa9d02b..7047ed4ad53 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -436,7 +436,7 @@ void DatabaseConnectionMySQL::detachTablePermanently(ContextPtr, const String & try { table_iter->second.second->drop(); - fs::createFile(remove_flag); + FS::createFile(remove_flag); } catch (...) { diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 33ae26a9c40..5d75915ea91 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -258,7 +258,7 @@ void DatabasePostgreSQL::dropTable(ContextPtr, const String & table_name, bool / throw Exception(fmt::format("Table {}.{} is already dropped/detached", database_name, table_name), ErrorCodes::TABLE_IS_DROPPED); fs::path mark_table_removed = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix); - fs::createFile(mark_table_removed); + FS::createFile(mark_table_removed); if (cache_tables) cached_tables.erase(table_name); diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index bf0a8bac12f..b4880daaadd 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -294,7 +294,7 @@ void DiskLocal::truncateFile(const String & path, size_t size) void DiskLocal::createFile(const String & path) { - fs::createFile(fs::path(disk_path) / path); + FS::createFile(fs::path(disk_path) / path); } void DiskLocal::setReadOnly(const String & path) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 86b810d031e..8c872dcdec5 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -112,7 +112,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) /// Will write file with database metadata, if needed. String database_name_escaped = escapeForFileName(database_name); - fs::path metadata_path = fs::canonical(getContext()->getPath()); + fs::path metadata_path = fs::absolute(getContext()->getPath()); fs::path metadata_file_tmp_path = metadata_path / "metadata" / (database_name_escaped + ".sql.tmp"); fs::path metadata_file_path = metadata_path / "metadata" / (database_name_escaped + ".sql"); diff --git a/src/Server/StaticRequestHandler.cpp b/src/Server/StaticRequestHandler.cpp index 978271c1f04..4f730b7cc30 100644 --- a/src/Server/StaticRequestHandler.cpp +++ b/src/Server/StaticRequestHandler.cpp @@ -141,12 +141,13 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out) const auto & file_name = response_expression.substr(file_prefix.size(), response_expression.size() - file_prefix.size()); fs::path user_files_absolute_path = fs::absolute(fs::path(server.context()->getUserFilesPath())); - fs::path file_path = user_files_absolute_path / file_name; + /// Fixme: it does not work with fs::path(user_files_absolute_path) / file_name + String file_path = fs::absolute(user_files_absolute_path.string() + "/" + file_name); if (!fs::exists(file_path)) - throw Exception("Invalid file name " + file_path.string() + " for static HTTPHandler. ", ErrorCodes::INCORRECT_FILE_NAME); + throw Exception("Invalid file name " + file_path + " for static HTTPHandler. ", ErrorCodes::INCORRECT_FILE_NAME); - ReadBufferFromFile in(file_path.string()); + ReadBufferFromFile in(file_path); copyData(in, out); } else if (startsWith(response_expression, config_prefix)) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 57c6d3be9ec..48f8793a506 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4388,7 +4388,7 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( const String & with_name, ContextPtr local_context) { - String clickhouse_path = fs::canonical(local_context->getPath()); + String clickhouse_path = fs::absolute(local_context->getPath()); String default_shadow_path = fs::path(clickhouse_path) / "shadow/"; fs::create_directories(default_shadow_path); auto increment = Increment(fs::path(default_shadow_path) / "increment.txt").get(true); diff --git a/src/Storages/examples/remove_symlink_directory.cpp b/src/Storages/examples/remove_symlink_directory.cpp index 05fdc18be2f..a90083233e7 100644 --- a/src/Storages/examples/remove_symlink_directory.cpp +++ b/src/Storages/examples/remove_symlink_directory.cpp @@ -19,7 +19,7 @@ try { fs::path dir("./test_dir/"); fs::create_directories(dir); - fs::createFile("./test_dir/file"); + FS::createFile("./test_dir/file"); if (0 != symlink("./test_dir", "./test_link")) DB::throwFromErrnoWithPath("Cannot create symlink", "./test_link", DB::ErrorCodes::SYSTEM_ERROR); From 096d117f6801385fe4dc38f42e343206916dae28 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 22 May 2021 00:12:46 +0300 Subject: [PATCH 43/55] Review fixes --- src/Common/Config/ConfigProcessor.cpp | 11 +++++------ src/Common/ErrorCodes.cpp | 1 - src/Common/createFile.cpp | 4 ++-- src/Databases/DatabaseOnDisk.cpp | 2 -- src/Interpreters/DatabaseCatalog.cpp | 2 +- src/Interpreters/loadMetadata.cpp | 6 +++--- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 7 files changed, 12 insertions(+), 16 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 396334c6a8b..598f64e0d99 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -416,16 +416,15 @@ ConfigProcessor::Files ConfigProcessor::getConfigMergeFiles(const std::string & std::set merge_dirs; /// Add path_to_config/config_name.d dir - merge_dir_path = merge_dir_path.parent_path() / (merge_dir_path.stem().string() + ".d"); + merge_dir_path.replace_extension("d"); merge_dirs.insert(merge_dir_path); /// Add path_to_config/conf.d dir - merge_dir_path = merge_dir_path.parent_path() / "conf.d"; + merge_dir_path.replace_filename("conf.d"); merge_dirs.insert(merge_dir_path); for (const std::string & merge_dir_name : merge_dirs) { - fs::path merge_dir(merge_dir_name); - if (!fs::exists(merge_dir) || !is_directory(merge_dir)) + if (!fs::exists(merge_dir_name) || !fs::is_directory(merge_dir_name)) continue; for (fs::directory_iterator it(merge_dir_name); it != fs::directory_iterator(); ++it) @@ -635,7 +634,7 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, fs::path parent_path = fs::path(loaded_config.config_path).parent_path(); preprocessed_dir = parent_path.string(); fs::path fs_new_path(new_path); - fs_new_path = fs_new_path.parent_path() / (fs_new_path.stem().string() + PREPROCESSED_SUFFIX + fs_new_path.extension().string()); + fs_new_path.replace_filename(fs_new_path.stem().string() + PREPROCESSED_SUFFIX + fs_new_path.extension().string()); new_path = fs_new_path.string(); } else @@ -652,7 +651,7 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, preprocessed_path = (fs::path(preprocessed_dir) / fs::path(new_path)).string(); auto preprocessed_path_parent = fs::path(preprocessed_path).parent_path(); - if (!preprocessed_path_parent.string().empty()) + if (!preprocessed_path_parent.empty()) fs::create_directories(preprocessed_path_parent); } DOMWriter().writeNode(preprocessed_path, loaded_config.preprocessed_xml); diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index d3cd812ef64..dccc20d325b 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -553,7 +553,6 @@ M(583, ILLEGAL_PROJECTION) \ M(584, PROJECTION_NOT_USED) \ \ - M(996, OPERATION_NOT_PERMITTED) \ M(997, CANNOT_CREATE_FILE) \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ diff --git a/src/Common/createFile.cpp b/src/Common/createFile.cpp index 83ab9c36f4b..74f8f9cf980 100644 --- a/src/Common/createFile.cpp +++ b/src/Common/createFile.cpp @@ -11,7 +11,7 @@ namespace DB namespace ErrorCodes { extern const int FILE_ALREADY_EXISTS; -extern const int OPERATION_NOT_PERMITTED; +extern const int PATH_ACCESS_DENIED; extern const int NOT_ENOUGH_SPACE; extern const int CANNOT_CREATE_FILE; } @@ -26,7 +26,7 @@ namespace FS case EEXIST: throw DB::Exception(DB::ErrorCodes::FILE_ALREADY_EXISTS, "File {} already exist", path); case EPERM: - throw DB::Exception(DB::ErrorCodes::OPERATION_NOT_PERMITTED, "Not enough permissions to create file {}", path); + throw DB::Exception(DB::ErrorCodes::PATH_ACCESS_DENIED, "Not enough permissions to create file {}", path); case ENOSPC: throw DB::Exception(DB::ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space to create file {}", path); case ENAMETOOLONG: diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 8393f9e81c0..f50adf54c3f 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -19,8 +19,6 @@ #include #include #include -#include -#include #include namespace fs = std::filesystem; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index ca473b4aac6..0794cb52a49 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -356,7 +356,7 @@ DatabasePtr DatabaseCatalog::detachDatabase(const String & database_name, bool d /// Old ClickHouse versions did not store database.sql files fs::path database_metadata_file = fs::path(getContext()->getPath()) / "metadata" / (escapeForFileName(database_name) + ".sql"); if (fs::exists(database_metadata_file)) - fs::remove_all(database_metadata_file); + fs::remove(database_metadata_file); } return db; diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index c109c53af20..0a84cdca309 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -112,15 +112,15 @@ void loadMetadata(ContextPtr context, const String & default_database_name) if (!it->is_directory()) { /// TODO: DETACH DATABASE PERMANENTLY ? - if (endsWith(current_file, ".sql")) + if (fs::path(current_file).extension() == ".sql") { String db_name = current_file.substr(0, current_file.size() - 4); if (db_name != DatabaseCatalog::SYSTEM_DATABASE) - databases.emplace(unescapeForFileName(db_name), path + "/" + db_name); + databases.emplace(unescapeForFileName(db_name), fs::path(path) / db_name); } /// Temporary fails may be left from previous server runs. - if (endsWith(current_file, ".tmp")) + if (fs::path(current_file).extension() == ".tmp") { LOG_WARNING(log, "Removing temporary file {}", it->path().string()); try diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2a34ae3fd07..88c7edc948b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6532,7 +6532,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta ops.emplace_back(zkutil::makeCheckRequest(alter_partition_version_path, alter_partition_version_stat.version)); ops.emplace_back(zkutil::makeSetRequest(alter_partition_version_path, "", -1)); /// Just update version, because merges assignment relies on it - ops.emplace_back(zkutil::makeSetRequest(fs:path(dest_table_storage->zookeeper_path) / "log", "", -1)); + ops.emplace_back(zkutil::makeSetRequest(fs::path(dest_table_storage->zookeeper_path) / "log", "", -1)); ops.emplace_back(zkutil::makeCreateRequest(fs::path(dest_table_storage->zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); From 8bac10d24f3c76563a478f2c8b3492975dd65034 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 22 May 2021 21:24:13 +0300 Subject: [PATCH 44/55] Update base/* --- base/daemon/BaseDaemon.cpp | 36 +++++++++++++++++------------------- base/daemon/SentryWriter.cpp | 8 ++++---- base/loggers/Loggers.cpp | 17 +++++++++-------- 3 files changed, 30 insertions(+), 31 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 01e700ebba3..742d0958668 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -26,8 +26,6 @@ #include #include #include -#include -#include #include #include #include @@ -59,6 +57,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include @@ -70,6 +69,7 @@ #endif #include +namespace fs = std::filesystem; DB::PipeFDs signal_pipe; @@ -437,11 +437,11 @@ static void sanitizerDeathCallback() static std::string createDirectory(const std::string & file) { - auto path = Poco::Path(file).makeParent(); - if (path.toString().empty()) + fs::path path = fs::path(file).parent_path(); + if (path.empty()) return ""; - Poco::File(path).createDirectories(); - return path.toString(); + fs::create_directories(path); + return path; }; @@ -449,7 +449,7 @@ static bool tryCreateDirectories(Poco::Logger * logger, const std::string & path { try { - Poco::File(path).createDirectories(); + fs::create_directories(path); return true; } catch (...) @@ -470,7 +470,7 @@ void BaseDaemon::reloadConfiguration() */ config_path = config().getString("config-file", getDefaultConfigFileName()); DB::ConfigProcessor config_processor(config_path, false, true); - config_processor.setConfigPath(Poco::Path(config_path).makeParent().toString()); + config_processor.setConfigPath(fs::path(config_path).parent_path()); loaded_config = config_processor.loadConfig(/* allow_zk_includes = */ true); if (last_configuration != nullptr) @@ -524,18 +524,17 @@ std::string BaseDaemon::getDefaultConfigFileName() const void BaseDaemon::closeFDs() { #if defined(OS_FREEBSD) || defined(OS_DARWIN) - Poco::File proc_path{"/dev/fd"}; + fs::path proc_path{"/dev/fd"}; #else - Poco::File proc_path{"/proc/self/fd"}; + fs::path proc_path{"/proc/self/fd"}; #endif - if (proc_path.isDirectory()) /// Hooray, proc exists + if (fs::is_directory(proc_path)) /// Hooray, proc exists { - std::vector fds; /// in /proc/self/fd directory filenames are numeric file descriptors - proc_path.list(fds); - for (const auto & fd_str : fds) + fs::directory_iterator dir_end; + for (fs::directory_iterator dir_it(proc_path); dir_it != dir_end; ++dir_it) { - int fd = DB::parse(fd_str); + int fd = DB::parse(dir_it->path().filename()); if (fd > 2 && fd != signal_pipe.fds_rw[0] && fd != signal_pipe.fds_rw[1]) ::close(fd); } @@ -597,7 +596,7 @@ void BaseDaemon::initialize(Application & self) { /** When creating pid file and looking for config, will search for paths relative to the working path of the program when started. */ - std::string path = Poco::Path(config().getString("application.path")).setFileName("").toString(); + std::string path = fs::path(config().getString("application.path")).replace_filename(""); if (0 != chdir(path.c_str())) throw Poco::Exception("Cannot change directory to " + path); } @@ -645,7 +644,7 @@ void BaseDaemon::initialize(Application & self) std::string log_path = config().getString("logger.log", ""); if (!log_path.empty()) - log_path = Poco::Path(log_path).setFileName("").toString(); + log_path = fs::path(log_path).replace_filename(""); /** Redirect stdout, stderr to separate files in the log directory (or in the specified file). * Some libraries write to stderr in case of errors in debug mode, @@ -708,8 +707,7 @@ void BaseDaemon::initialize(Application & self) tryCreateDirectories(&logger(), core_path); - Poco::File cores = core_path; - if (!(cores.exists() && cores.isDirectory())) + if (!(fs::exists(core_path) && fs::is_directory(core_path))) { core_path = !log_path.empty() ? log_path : "/opt/"; tryCreateDirectories(&logger(), core_path); diff --git a/base/daemon/SentryWriter.cpp b/base/daemon/SentryWriter.cpp index 1028dc7d2dc..3719bb74c52 100644 --- a/base/daemon/SentryWriter.cpp +++ b/base/daemon/SentryWriter.cpp @@ -25,6 +25,7 @@ # include # include +namespace fs = std::filesystem; namespace { @@ -53,8 +54,7 @@ void setExtras() sentry_set_extra("physical_cpu_cores", sentry_value_new_int32(getNumberOfPhysicalCPUCores())); if (!server_data_path.empty()) - sentry_set_extra("disk_free_space", sentry_value_new_string(formatReadableSizeWithBinarySuffix( - Poco::File(server_data_path).freeSpace()).c_str())); + sentry_set_extra("disk_free_space", sentry_value_new_string(formatReadableSizeWithBinarySuffix(fs::space(server_data_path).free).c_str())); } void sentry_logger(sentry_level_e level, const char * message, va_list args, void *) @@ -110,12 +110,12 @@ void SentryWriter::initialize(Poco::Util::LayeredConfiguration & config) if (enabled) { server_data_path = config.getString("path", ""); - const std::filesystem::path & default_tmp_path = std::filesystem::path(config.getString("tmp_path", Poco::Path::temp())) / "sentry"; + const std::filesystem::path & default_tmp_path = fs::path(config.getString("tmp_path", fs::temp_directory_path())) / "sentry"; const std::string & endpoint = config.getString("send_crash_reports.endpoint"); const std::string & temp_folder_path = config.getString("send_crash_reports.tmp_path", default_tmp_path); - Poco::File(temp_folder_path).createDirectories(); + fs::create_directories(temp_folder_path); sentry_options_t * options = sentry_options_new(); /// will be freed by sentry_init or sentry_shutdown sentry_options_set_release(options, VERSION_STRING_SHORT); diff --git a/base/loggers/Loggers.cpp b/base/loggers/Loggers.cpp index 913deaf1eb8..11dfbdaeb71 100644 --- a/base/loggers/Loggers.cpp +++ b/base/loggers/Loggers.cpp @@ -6,10 +6,11 @@ #include "OwnFormattingChannel.h" #include "OwnPatternFormatter.h" #include -#include #include #include -#include +#include + +namespace fs = std::filesystem; namespace DB { @@ -20,11 +21,11 @@ namespace DB // TODO: move to libcommon static std::string createDirectory(const std::string & file) { - auto path = Poco::Path(file).makeParent(); - if (path.toString().empty()) + auto path = fs::path(file).parent_path(); + if (path.empty()) return ""; - Poco::File(path).createDirectories(); - return path.toString(); + fs::create_directories(path); + return path; }; void Loggers::setTextLog(std::shared_ptr log, int max_priority) @@ -70,7 +71,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log // Set up two channel chains. log_file = new Poco::FileChannel; - log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(log_path).absolute().toString()); + log_file->setProperty(Poco::FileChannel::PROP_PATH, fs::absolute(log_path)); log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M")); log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number"); log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config.getRawString("logger.compress", "true")); @@ -102,7 +103,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log std::cerr << "Logging errors to " << errorlog_path << std::endl; error_log_file = new Poco::FileChannel; - error_log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(errorlog_path).absolute().toString()); + error_log_file->setProperty(Poco::FileChannel::PROP_PATH, fs::absolute(errorlog_path)); error_log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M")); error_log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number"); error_log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config.getRawString("logger.compress", "true")); From 866b29fb5a0402e646bbd7e26a0a9780e6d2849e Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 23 May 2021 10:56:13 +0300 Subject: [PATCH 45/55] Return list fds with Poco, more canonical --- base/daemon/BaseDaemon.cpp | 8 +++++--- src/Databases/DatabaseAtomic.cpp | 6 +++--- .../Embedded/GeodataProviders/HierarchiesProvider.cpp | 2 +- src/Formats/FormatSchemaInfo.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 5 files changed, 11 insertions(+), 9 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 742d0958668..a2118cc7bb1 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -57,6 +57,7 @@ #include #include #include +#include #include #if !defined(ARCADIA_BUILD) @@ -530,11 +531,12 @@ void BaseDaemon::closeFDs() #endif if (fs::is_directory(proc_path)) /// Hooray, proc exists { + std::vector fds; /// in /proc/self/fd directory filenames are numeric file descriptors - fs::directory_iterator dir_end; - for (fs::directory_iterator dir_it(proc_path); dir_it != dir_end; ++dir_it) + Poco::File(proc_path.string()).list(fds); + for (const auto & fd_str : fds) { - int fd = DB::parse(dir_it->path().filename()); + int fd = DB::parse(fd_str); if (fd > 2 && fd != signal_pipe.fds_rw[0] && fd != signal_pipe.fds_rw[1]) ::close(fd); } diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index b69af842978..e4b7d6e3038 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -443,9 +443,9 @@ void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String & try { String link = path_to_table_symlinks + escapeForFileName(table_name); - fs::path data = fs::absolute(getContext()->getPath()).string() + actual_data_path; + fs::path data = fs::canonical(getContext()->getPath()) / actual_data_path; if (!if_data_path_exist || fs::exists(data)) - fs::create_symlink(data, link); + fs::create_directory_symlink(data, link); } catch (...) { @@ -481,7 +481,7 @@ void DatabaseAtomic::tryCreateMetadataSymlink() { try { - fs::create_symlink(metadata_path, path_to_metadata_symlink); + fs::create_directory_symlink(metadata_path, path_to_metadata_symlink); } catch (...) { diff --git a/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp b/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp index 3f35b9102a2..210459da0be 100644 --- a/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp +++ b/src/Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.cpp @@ -30,7 +30,7 @@ RegionsHierarchiesDataProvider::RegionsHierarchiesDataProvider(const std::string void RegionsHierarchiesDataProvider::discoverFilesWithCustomHierarchies() { std::string basename = fs::path(path).stem(); - fs::path dir_path = fs::absolute(path).parent_path(); + fs::path dir_path = fs::canonical(path).parent_path(); fs::directory_iterator dir_end; for (fs::directory_iterator dir_it(dir_path); dir_it != dir_end; ++dir_it) diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index 513d9738f62..f985cb2681f 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -67,7 +67,7 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & auto default_schema_directory = [&format_schema_path]() { - static const String str = fs::absolute(format_schema_path) / ""; + static const String str = fs::canonical(format_schema_path) / ""; return str; }; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 8c872dcdec5..86b810d031e 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -112,7 +112,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) /// Will write file with database metadata, if needed. String database_name_escaped = escapeForFileName(database_name); - fs::path metadata_path = fs::absolute(getContext()->getPath()); + fs::path metadata_path = fs::canonical(getContext()->getPath()); fs::path metadata_file_tmp_path = metadata_path / "metadata" / (database_name_escaped + ".sql.tmp"); fs::path metadata_file_path = metadata_path / "metadata" / (database_name_escaped + ".sql"); From 31107816bb42c0844df4959f01203b60ce4b517b Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 24 May 2021 19:03:09 +0300 Subject: [PATCH 46/55] fs::absolute to fs::canonical --- base/loggers/Loggers.cpp | 4 ++-- programs/copier/ClusterCopierApp.cpp | 2 +- programs/install/Install.cpp | 2 +- src/Disks/DiskLocal.cpp | 13 +++++-------- src/Server/StaticRequestHandler.cpp | 4 ++-- src/Storages/MergeTree/DataPartsExchange.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/StorageFile.cpp | 4 ++-- 8 files changed, 16 insertions(+), 19 deletions(-) diff --git a/base/loggers/Loggers.cpp b/base/loggers/Loggers.cpp index 11dfbdaeb71..80e62d0a6d6 100644 --- a/base/loggers/Loggers.cpp +++ b/base/loggers/Loggers.cpp @@ -71,7 +71,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log // Set up two channel chains. log_file = new Poco::FileChannel; - log_file->setProperty(Poco::FileChannel::PROP_PATH, fs::absolute(log_path)); + log_file->setProperty(Poco::FileChannel::PROP_PATH, fs::weakly_canonical(log_path)); log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M")); log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number"); log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config.getRawString("logger.compress", "true")); @@ -103,7 +103,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log std::cerr << "Logging errors to " << errorlog_path << std::endl; error_log_file = new Poco::FileChannel; - error_log_file->setProperty(Poco::FileChannel::PROP_PATH, fs::absolute(errorlog_path)); + error_log_file->setProperty(Poco::FileChannel::PROP_PATH, fs::weakly_canonical(errorlog_path)); error_log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M")); error_log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number"); error_log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config.getRawString("logger.compress", "true")); diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index 79bfb5b7411..8925ab63f99 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -40,7 +40,7 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self) process_id = std::to_string(DateLUT::instance().toNumYYYYMMDDhhmmss(timestamp)) + "_" + std::to_string(curr_pid); host_id = escapeForFileName(getFQDNOrHostName()) + '#' + process_id; - process_path = fs::absolute(fs::path(base_dir) / ("clickhouse-copier_" + process_id)); + process_path = fs::weakly_canonical(fs::path(base_dir) / ("clickhouse-copier_" + process_id)); fs::create_directories(process_path); /// Override variables for BaseDaemon diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 96d336673d0..1f2c1c30ae4 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -288,7 +288,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) bool is_symlink = fs::is_symlink(symlink_path); fs::path points_to; if (is_symlink) - points_to = fs::absolute(fs::read_symlink(symlink_path)); + points_to = fs::weakly_canonical(fs::read_symlink(symlink_path)); if (is_symlink && points_to == main_bin_path) { diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index b4880daaadd..51ed7fe577b 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -61,8 +61,8 @@ private: class DiskLocalDirectoryIterator : public IDiskDirectoryIterator { public: - explicit DiskLocalDirectoryIterator(const fs::path & disk_path_, const String & dir_path_) - : dir_path(dir_path_), entry(disk_path_ / dir_path_) + explicit DiskLocalDirectoryIterator(const String & disk_path_, const String & dir_path_) + : dir_path(dir_path_), entry(fs::path(disk_path_) / dir_path_) { } @@ -196,7 +196,7 @@ void DiskLocal::moveDirectory(const String & from_path, const String & to_path) DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path) { - return std::make_unique(fs::path(disk_path), path); + return std::make_unique(disk_path, path); } void DiskLocal::moveFile(const String & from_path, const String & to_path) @@ -314,12 +314,9 @@ void DiskLocal::copy(const String & from_path, const std::shared_ptr & to if (isSameDiskType(*this, *to_disk)) { fs::path to = fs::path(to_disk->getPath()) / to_path; - fs::path from; + fs::path from = fs::path(disk_path) / from_path; if (from_path.ends_with('/')) - from = fs::path(disk_path) / from_path.substr(0, from_path.size() - 1); - else - from = fs::path(disk_path) / from_path; - + from = from.parent_path(); if (fs::is_directory(from)) to /= from.filename(); diff --git a/src/Server/StaticRequestHandler.cpp b/src/Server/StaticRequestHandler.cpp index 4f730b7cc30..4746ab6d1e0 100644 --- a/src/Server/StaticRequestHandler.cpp +++ b/src/Server/StaticRequestHandler.cpp @@ -140,9 +140,9 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out) { const auto & file_name = response_expression.substr(file_prefix.size(), response_expression.size() - file_prefix.size()); - fs::path user_files_absolute_path = fs::absolute(fs::path(server.context()->getUserFilesPath())); + fs::path user_files_absolute_path = fs::canonical(fs::path(server.context()->getUserFilesPath())); /// Fixme: it does not work with fs::path(user_files_absolute_path) / file_name - String file_path = fs::absolute(user_files_absolute_path.string() + "/" + file_name); + String file_path = fs::canonical(user_files_absolute_path.string() + "/" + file_name); if (!fs::exists(file_path)) throw Exception("Invalid file name " + file_path + " for static HTTPHandler. ", ErrorCodes::INCORRECT_FILE_NAME); diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 69a92738777..47ed4a0762c 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -652,8 +652,8 @@ void Fetcher::downloadBaseOrProjectionPartToDisk( /// File must be inside "absolute_part_path" directory. /// Otherwise malicious ClickHouse replica may force us to write to arbitrary path. - String absolute_file_path = fs::absolute(fs::path(part_download_path) / file_name); - if (!startsWith(absolute_file_path, fs::absolute(part_download_path).string())) + String absolute_file_path = fs::weakly_canonical(fs::path(part_download_path) / file_name); + if (!startsWith(absolute_file_path, fs::weakly_canonical(part_download_path).string())) throw Exception("File path (" + absolute_file_path + ") doesn't appear to be inside part path (" + part_download_path + ")." " This may happen if we are trying to download part from malicious replica or logical error.", ErrorCodes::INSECURE_PATH); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 7cac7b9ffd7..d67d67768ef 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4406,7 +4406,7 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher( const String & with_name, ContextPtr local_context) { - String clickhouse_path = fs::absolute(local_context->getPath()); + String clickhouse_path = fs::canonical(local_context->getPath()); String default_shadow_path = fs::path(clickhouse_path) / "shadow/"; fs::create_directories(default_shadow_path); auto increment = Increment(fs::path(default_shadow_path) / "increment.txt").get(true); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 2dd3bb757f3..aac1b708567 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -129,13 +129,13 @@ void checkCreationIsAllowed(ContextPtr context_global, const std::string & db_di Strings StorageFile::getPathsList(const String & table_path, const String & user_files_path, ContextPtr context) { - fs::path user_files_absolute_path = fs::absolute(user_files_path); + fs::path user_files_absolute_path = fs::weakly_canonical(user_files_path); fs::path fs_table_path(table_path); if (fs_table_path.is_relative()) fs_table_path = user_files_absolute_path / fs_table_path; Strings paths; - const String path = fs::absolute(fs_table_path); + const String path = fs::weakly_canonical(fs_table_path); if (path.find_first_of("*?{") == std::string::npos) paths.push_back(path); else From 940eab692478b306d6f81ca352b4f19c4eab87d9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 25 May 2021 16:45:46 +0300 Subject: [PATCH 47/55] Review fixes --- src/Common/createFile.cpp | 52 ++++++++++++------- src/Common/createFile.h | 2 + src/Disks/DiskLocal.cpp | 26 ++-------- src/Disks/S3/DiskS3.cpp | 6 +-- src/Interpreters/loadMetadata.cpp | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 6 files changed, 43 insertions(+), 47 deletions(-) diff --git a/src/Common/createFile.cpp b/src/Common/createFile.cpp index 74f8f9cf980..64e344f1f22 100644 --- a/src/Common/createFile.cpp +++ b/src/Common/createFile.cpp @@ -10,31 +10,13 @@ namespace DB { namespace ErrorCodes { -extern const int FILE_ALREADY_EXISTS; extern const int PATH_ACCESS_DENIED; -extern const int NOT_ENOUGH_SPACE; extern const int CANNOT_CREATE_FILE; } } namespace FS { -[[noreturn]] void handleLastError(const std::string & path) -{ - switch (errno) - { - case EEXIST: - throw DB::Exception(DB::ErrorCodes::FILE_ALREADY_EXISTS, "File {} already exist", path); - case EPERM: - throw DB::Exception(DB::ErrorCodes::PATH_ACCESS_DENIED, "Not enough permissions to create file {}", path); - case ENOSPC: - throw DB::Exception(DB::ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space to create file {}", path); - case ENAMETOOLONG: - throw DB::Exception(DB::ErrorCodes::CANNOT_CREATE_FILE, "File name {} is too long"); - default: - throw DB::Exception(DB::ErrorCodes::CANNOT_CREATE_FILE, "Cannot create file {}. Error: {}", path, strerror(errno)); - } -} /// Copy from Poco::createFile bool createFile(const std::string & path) @@ -45,6 +27,38 @@ bool createFile(const std::string & path) close(n); return true; } - handleLastError(path); + DB::throwFromErrnoWithPath("Cannot create file: " + path, path, DB::ErrorCodes::CANNOT_CREATE_FILE); } + +bool canRead(const std::string & path) +{ + struct stat st; + if (stat(path.c_str(), &st) == 0) + { + if (st.st_uid == geteuid()) + return (st.st_mode & S_IRUSR) != 0; + else if (st.st_gid == getegid()) + return (st.st_mode & S_IRGRP) != 0; + else + return (st.st_mode & S_IROTH) != 0 || geteuid() == 0; + } + DB::throwFromErrnoWithPath("Cannot check read access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} + + +bool canWrite(const std::string & path) +{ + struct stat st; + if (stat(path.c_str(), &st) == 0) + { + if (st.st_uid == geteuid()) + return (st.st_mode & S_IWUSR) != 0; + else if (st.st_gid == getegid()) + return (st.st_mode & S_IWGRP) != 0; + else + return (st.st_mode & S_IWOTH) != 0 || geteuid() == 0; + } + DB::throwFromErrnoWithPath("Cannot check write access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} + } diff --git a/src/Common/createFile.h b/src/Common/createFile.h index 67c7023a455..f96d141d39e 100644 --- a/src/Common/createFile.h +++ b/src/Common/createFile.h @@ -4,4 +4,6 @@ namespace FS { bool createFile(const std::string & path); +bool canRead(const std::string & path); +bool canWrite(const std::string & path); } diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 51ed7fe577b..e31731a7edc 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -208,17 +208,7 @@ void DiskLocal::replaceFile(const String & from_path, const String & to_path) { fs::path from_file = fs::path(disk_path) / from_path; fs::path to_file = fs::path(disk_path) / to_path; - if (fs::exists(to_file)) - { - fs::path tmp_file(to_file.string() + ".old"); - fs::rename(to_file, tmp_file); - fs::rename(from_file, fs::path(disk_path) / to_path); - fs::remove(tmp_file); - } - else - { - fs::rename(from_file, to_file); - } + fs::rename(from_file, to_file); } std::unique_ptr @@ -300,8 +290,8 @@ void DiskLocal::createFile(const String & path) void DiskLocal::setReadOnly(const String & path) { fs::permissions(fs::path(disk_path) / path, - fs::perms::owner_read | fs::perms::group_read | fs::perms::others_read, - fs::perm_options::replace); + fs::perms::owner_write | fs::perms::group_write | fs::perms::others_write, + fs::perm_options::remove); } bool inline isSameDiskType(const IDisk & one, const IDisk & another) @@ -399,15 +389,7 @@ void registerDiskLocal(DiskFactory & factory) throw Exception("Disk path must end with /. Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); } - fs::path disk(path); - fs::perms p = fs::status(disk).permissions(); - bool is_readable = ((p & fs::perms::owner_read) != fs::perms::none) - | ((p & fs::perms::group_read) != fs::perms::none) - | ((p & fs::perms::others_read) != fs::perms::none); - bool is_writable = ((p & fs::perms::owner_write) != fs::perms::none) - | ((p & fs::perms::group_write) != fs::perms::none) - | ((p & fs::perms::others_write) != fs::perms::none); - if (!is_readable || !is_writable) + if (!FS::canRead(path) || !FS::canWrite(path)) throw Exception("There is no RW access to the disk " + name + " (" + path + ")", ErrorCodes::PATH_ACCESS_DENIED); bool has_space_ratio = config.has(config_prefix + ".keep_free_space_ratio"); diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 4fbfc9141a6..7dd3381cd51 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -1506,11 +1506,9 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio LOG_DEBUG(log, "Move directory to 'detached' {} -> {}", path, detached_path); fs::path from_path = fs::path(metadata_path) / path; - fs::path to_path = fs::path(metadata_path) / detached_path; + fs::path to_path = fs::path(metadata_path) / detached_path / from_path.filename(); if (path.ends_with('/')) - to_path /= from_path.parent_path().filename(); - else - to_path /= from_path.filename(); + to_path /= to_path.parent_path(); fs::copy(from_path, to_path, fs::copy_options::recursive | fs::copy_options::overwrite_existing); fs::remove_all(from_path); } diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 0a84cdca309..b60cd76e85a 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -114,7 +114,7 @@ void loadMetadata(ContextPtr context, const String & default_database_name) /// TODO: DETACH DATABASE PERMANENTLY ? if (fs::path(current_file).extension() == ".sql") { - String db_name = current_file.substr(0, current_file.size() - 4); + String db_name = fs::path(current_file).stem(); if (db_name != DatabaseCatalog::SYSTEM_DATABASE) databases.emplace(unescapeForFileName(db_name), fs::path(path) / db_name); } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 3a3ac90fab0..a76297f36a2 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -114,7 +114,7 @@ using RelativeSize = boost::rational; static std::string toString(const RelativeSize & x) { - return fs::path(ASTSampleRatio::toString(x.numerator())) / ASTSampleRatio::toString(x.denominator()); + return ASTSampleRatio::toString(x.numerator()) + "/" + ASTSampleRatio::toString(x.denominator()); } /// Converts sample size to an approximate number of rows (ex. `SAMPLE 1000000`) to relative value (ex. `SAMPLE 0.1`). From f9088b98de8b6880473fcac916a23fe46a70ae23 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 27 May 2021 00:02:24 +0300 Subject: [PATCH 48/55] Some disks update --- src/Disks/HDFS/DiskHDFS.cpp | 4 +- src/Disks/IDisk.h | 1 - src/Disks/IDiskRemote.cpp | 40 ++++++++++--------- src/Disks/IDiskRemote.h | 17 ++++---- src/Disks/ReadIndirectBufferFromRemoteFS.cpp | 1 + src/Disks/WriteIndirectBufferFromRemoteFS.cpp | 1 - src/Disks/WriteIndirectBufferFromRemoteFS.h | 2 +- 7 files changed, 34 insertions(+), 32 deletions(-) diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 0648fd9f08c..da6ccb024c7 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -172,8 +172,8 @@ void registerDiskHDFS(DiskFactory & factory) const String & config_prefix, ContextConstPtr context_) -> DiskPtr { - Poco::File disk{context_->getPath() + "disks/" + name}; - disk.createDirectories(); + fs::path disk = fs::path(context_->getPath()) / "disks" / name; + fs::create_directories(disk); String uri{config.getString(config_prefix + ".endpoint")}; diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 1cbc038f034..cb718605dd6 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -7,7 +7,6 @@ #include #include #include -#include "Disks/Executor.h" #include #include diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index bcb399f5d07..f32b046b5ec 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -179,9 +178,9 @@ void IDiskRemote::removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths { LOG_DEBUG(log, "Remove file by path: {}", backQuote(metadata_path + path)); - Poco::File file(metadata_path + path); + fs::path file(metadata_path + path); - if (!file.isFile()) + if (!fs::is_regular_file(file)) throw Exception(ErrorCodes::CANNOT_DELETE_DIRECTORY, "Path '{}' is a directory", path); try @@ -191,7 +190,7 @@ void IDiskRemote::removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths /// If there is no references - delete content from remote FS. if (metadata.ref_count == 0) { - file.remove(); + fs::remove(file); for (const auto & [remote_fs_object_path, _] : metadata.remote_fs_objects) fs_paths_keeper->addPath(remote_fs_root_path + remote_fs_object_path); } @@ -199,7 +198,7 @@ void IDiskRemote::removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths { --metadata.ref_count; metadata.save(); - file.remove(); + fs::remove(file); } } catch (const Exception & e) @@ -210,7 +209,7 @@ void IDiskRemote::removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths LOG_WARNING(log, "Metadata file {} can't be read by reason: {}. Removing it forcibly.", backQuote(path), e.nested() ? e.nested()->message() : e.message()); - file.remove(); + fs::remove(file); } else throw; @@ -222,8 +221,8 @@ void IDiskRemote::removeMetaRecursive(const String & path, RemoteFSPathKeeperPtr { checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks. - Poco::File file(metadata_path + path); - if (file.isFile()) + fs::path file = fs::path(metadata_path) / path; + if (fs::is_regular_file(file)) { removeMeta(path, fs_paths_keeper); } @@ -231,7 +230,7 @@ void IDiskRemote::removeMetaRecursive(const String & path, RemoteFSPathKeeperPtr { for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) removeMetaRecursive(it->path(), fs_paths_keeper); - file.remove(); + fs::remove(file); } } @@ -296,13 +295,13 @@ IDiskRemote::IDiskRemote( bool IDiskRemote::exists(const String & path) const { - return Poco::File(metadata_path + path).exists(); + return fs::exists(fs::path(metadata_path) / path); } bool IDiskRemote::isFile(const String & path) const { - return Poco::File(metadata_path + path).isFile(); + return fs::is_regular_file(fs::path(metadata_path) / path); } @@ -326,7 +325,7 @@ void IDiskRemote::moveFile(const String & from_path, const String & to_path) if (exists(to_path)) throw Exception("File already exists: " + to_path, ErrorCodes::FILE_ALREADY_EXISTS); - Poco::File(metadata_path + from_path).renameTo(metadata_path + to_path); + fs::rename(fs::path(metadata_path) / from_path, fs::path(metadata_path) / to_path); } @@ -347,7 +346,7 @@ void IDiskRemote::replaceFile(const String & from_path, const String & to_path) void IDiskRemote::removeFileIfExists(const String & path) { RemoteFSPathKeeperPtr fs_paths_keeper = createFSPathKeeper(); - if (Poco::File(metadata_path + path).exists()) + if (fs::exists(fs::path(metadata_path) / path)) { removeMeta(path, fs_paths_keeper); removeFromRemoteFS(fs_paths_keeper); @@ -385,19 +384,19 @@ void IDiskRemote::setReadOnly(const String & path) bool IDiskRemote::isDirectory(const String & path) const { - return Poco::File(metadata_path + path).isDirectory(); + return fs::is_directory(fs::path(metadata_path) / path); } void IDiskRemote::createDirectory(const String & path) { - Poco::File(metadata_path + path).createDirectory(); + fs::create_directory(fs::path(metadata_path) / path); } void IDiskRemote::createDirectories(const String & path) { - Poco::File(metadata_path + path).createDirectories(); + fs::create_directories(fs::path(metadata_path) / path); } @@ -411,7 +410,7 @@ void IDiskRemote::clearDirectory(const String & path) void IDiskRemote::removeDirectory(const String & path) { - Poco::File(metadata_path + path).remove(); + fs::remove(fs::path(metadata_path) / path); } @@ -430,13 +429,16 @@ void IDiskRemote::listFiles(const String & path, std::vector & file_name void IDiskRemote::setLastModified(const String & path, const Poco::Timestamp & timestamp) { - Poco::File(metadata_path + path).setLastModified(timestamp); + fs::last_write_time(fs::path(metadata_path) / path, + static_cast(std::chrono::microseconds(timestamp.epochMicroseconds()))); } Poco::Timestamp IDiskRemote::getLastModified(const String & path) { - return Poco::File(metadata_path + path).getLastModified(); + fs::file_time_type fs_time = fs::last_write_time(fs::path(metadata_path) / path); + auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); + return Poco::Timestamp(micro_sec.count()); } diff --git a/src/Disks/IDiskRemote.h b/src/Disks/IDiskRemote.h index b32258331a7..a90621443e4 100644 --- a/src/Disks/IDiskRemote.h +++ b/src/Disks/IDiskRemote.h @@ -4,11 +4,12 @@ #include #include "Disks/DiskFactory.h" #include "Disks/Executor.h" -#include #include #include #include +#include +namespace fs = std::filesystem; namespace DB { @@ -193,21 +194,21 @@ public: void next() override { ++iter; } - bool isValid() const override { return iter != Poco::DirectoryIterator(); } + bool isValid() const override { return iter != fs::directory_iterator(); } String path() const override { - if (iter->isDirectory()) - return folder_path + iter.name() + '/'; + if (fs::is_directory(iter->path())) + return folder_path / iter->path().filename().string() / ""; else - return folder_path + iter.name(); + return folder_path / iter->path().filename().string(); } - String name() const override { return iter.name(); } + String name() const override { return iter->path().filename(); } private: - Poco::DirectoryIterator iter; - String folder_path; + fs::directory_iterator iter; + fs::path folder_path; }; diff --git a/src/Disks/ReadIndirectBufferFromRemoteFS.cpp b/src/Disks/ReadIndirectBufferFromRemoteFS.cpp index 955986e5259..6d4764e4392 100644 --- a/src/Disks/ReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/ReadIndirectBufferFromRemoteFS.cpp @@ -21,6 +21,7 @@ ReadIndirectBufferFromRemoteFS::ReadIndirectBufferFromRemoteFS( { } + template off_t ReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) { diff --git a/src/Disks/WriteIndirectBufferFromRemoteFS.cpp b/src/Disks/WriteIndirectBufferFromRemoteFS.cpp index adc711608d7..6951b9fa92e 100644 --- a/src/Disks/WriteIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/WriteIndirectBufferFromRemoteFS.cpp @@ -8,7 +8,6 @@ namespace DB { -/// Stores data in S3/HDFS and adds the object key (S3 path) and object size to metadata file on local FS. template WriteIndirectBufferFromRemoteFS::WriteIndirectBufferFromRemoteFS( std::unique_ptr impl_, diff --git a/src/Disks/WriteIndirectBufferFromRemoteFS.h b/src/Disks/WriteIndirectBufferFromRemoteFS.h index cda7523e19e..ece7b9d5871 100644 --- a/src/Disks/WriteIndirectBufferFromRemoteFS.h +++ b/src/Disks/WriteIndirectBufferFromRemoteFS.h @@ -10,7 +10,7 @@ namespace DB { -/// Stores data in S3/HDFS and adds the object key (S3 path) and object size to metadata file on local FS. +/// Stores data in S3/HDFS and adds the object path and object size to metadata file on local FS. template class WriteIndirectBufferFromRemoteFS final : public WriteBufferFromFileDecorator { From f66c67a979cc303f9db46e9b9dae8ae450ac2d18 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 27 May 2021 10:35:36 +0300 Subject: [PATCH 49/55] Fixes --- base/daemon/BaseDaemon.cpp | 12 +++++++----- programs/server/Server.cpp | 12 ++++++------ src/Common/ErrorCodes.cpp | 2 +- src/Disks/S3/DiskS3.cpp | 6 ++++-- src/Formats/FormatSchemaInfo.cpp | 11 +++++++++-- 5 files changed, 27 insertions(+), 16 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index a2118cc7bb1..310e656dac2 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -531,12 +531,14 @@ void BaseDaemon::closeFDs() #endif if (fs::is_directory(proc_path)) /// Hooray, proc exists { - std::vector fds; - /// in /proc/self/fd directory filenames are numeric file descriptors - Poco::File(proc_path.string()).list(fds); - for (const auto & fd_str : fds) + /// in /proc/self/fd directory filenames are numeric file descriptors. + /// Iterate directory separately from closing fds to avoid closing iterated directory fd. + std::vector fds; + for (const auto & path : fs::directory_iterator(proc_path)) + fds.push_back(DB::parse(path.path().filename())); + + for (const auto & fd : fds) { - int fd = DB::parse(fd_str); if (fd > 2 && fd != signal_pipe.fds_rw[0] && fd != signal_pipe.fds_rw[1]) ::close(fd); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 820bb73c6ba..f169d793ee9 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -189,16 +189,16 @@ void setupTmpPath(Poco::Logger * log, const std::string & path) fs::create_directories(path); /// Clearing old temporary files. - Poco::DirectoryIterator dir_end; - for (Poco::DirectoryIterator it(path); it != dir_end; ++it) + fs::directory_iterator dir_end; + for (fs::directory_iterator it(path); it != dir_end; ++it) { - if (it->isFile() && startsWith(it.name(), "tmp")) + if (it->is_regular_file() && startsWith(it->path().filename(), "tmp")) { - LOG_DEBUG(log, "Removing old temporary file {}", it->path()); - it->remove(); + LOG_DEBUG(log, "Removing old temporary file {}", it->path().string()); + fs::remove(it->path()); } else - LOG_DEBUG(log, "Skipped file in temporary path {}", it->path()); + LOG_DEBUG(log, "Skipped file in temporary path {}", it->path().string()); } } diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 34e34418acb..d840830bf28 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -553,8 +553,8 @@ M(583, ILLEGAL_PROJECTION) \ M(584, PROJECTION_NOT_USED) \ M(585, CANNOT_PARSE_YAML) \ + M(586, CANNOT_CREATE_FILE) \ \ - M(997, CANNOT_CREATE_FILE) \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 7054987527d..89c2d20db9f 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -874,9 +874,11 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio LOG_DEBUG(log, "Move directory to 'detached' {} -> {}", path, detached_path); fs::path from_path = fs::path(metadata_path) / path; - fs::path to_path = fs::path(metadata_path) / detached_path / from_path.filename(); + fs::path to_path = fs::path(metadata_path) / detached_path; if (path.ends_with('/')) - to_path = to_path.parent_path(); + to_path /= from_path.parent_path().filename(); + else + to_path /= from_path.filename(); fs::copy(from_path, to_path, fs::copy_options::recursive | fs::copy_options::overwrite_existing); fs::remove_all(from_path); } diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index f985cb2681f..54e0961d161 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -74,6 +74,14 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & if (!path.has_extension() && !default_file_extension.empty()) path = path.parent_path() / (path.stem().string() + '.' + default_file_extension); + fs::path default_schema_directory_path(default_schema_directory()); + auto path_is_subdirectory_of = [](fs::path inner, const fs::path & outer) -> bool + { + while (inner != outer && inner != "/") + inner = inner.parent_path(); + return inner == outer; + }; + if (path.is_absolute()) { if (is_server) @@ -81,13 +89,12 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & schema_path = path.filename(); schema_directory = path.parent_path() / ""; } - else if (path.string().starts_with("..")) + else if (!path_is_subdirectory_of(path, default_schema_directory_path)) { if (is_server) throw Exception( "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.string(), ErrorCodes::BAD_ARGUMENTS); - fs::path default_schema_directory_path(default_schema_directory()); if (default_schema_directory_path.is_absolute()) path = default_schema_directory_path; else From 0d393c000617b955b521ef3d8f8b6bc7a1a864ca Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 27 May 2021 17:16:42 +0300 Subject: [PATCH 50/55] Fix tests --- programs/client/Client.cpp | 2 +- src/Formats/FormatSchemaInfo.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 2de95635d04..7dea4c9375b 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -279,7 +279,7 @@ private: /// Set path for format schema files if (config().has("format_schema_path")) - context->setFormatSchemaPath(config().getString("format_schema_path")); + context->setFormatSchemaPath(fs::absolute(config().getString("format_schema_path"))); /// Initialize query_id_formats if any if (config().has("query_id_formats")) diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index 54e0961d161..25ed6051382 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -89,7 +89,7 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & schema_path = path.filename(); schema_directory = path.parent_path() / ""; } - else if (!path_is_subdirectory_of(path, default_schema_directory_path)) + else if (path.has_parent_path() && !path_is_subdirectory_of(path, default_schema_directory_path)) { if (is_server) throw Exception( From 2a631aaf0835d17da3aa3e79fc9dc5f3e3e1b728 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 28 May 2021 21:17:16 +0300 Subject: [PATCH 51/55] Final fixes --- base/daemon/BaseDaemon.cpp | 1 - base/daemon/SentryWriter.cpp | 1 - programs/client/Client.cpp | 3 +- programs/copier/Internals.h | 1 - src/Bridge/IBridgeHelper.cpp | 1 - src/Bridge/LibraryBridgeHelper.cpp | 1 - src/Bridge/XDBCBridgeHelper.h | 2 -- src/Common/Config/ConfigProcessor.h | 3 -- src/Common/Config/ConfigReloader.cpp | 8 ++--- .../{createFile.cpp => FileSystemHelpers.cpp} | 33 ++++++++++++++++--- src/Common/FileSystemHelpers.h | 14 ++++++++ src/Common/FileUpdatesTracker.h | 5 ++- src/Common/createFile.h | 9 ----- src/Common/filesystemHelpers.cpp | 2 -- src/Databases/DatabaseFactory.cpp | 2 -- src/Databases/DatabaseOnDisk.cpp | 7 ++-- .../MySQL/DatabaseConnectionMySQL.cpp | 2 +- .../PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Dictionaries/DictionarySourceHelpers.h | 3 +- src/Dictionaries/FileDictionarySource.cpp | 10 ++---- src/Disks/DiskLocal.cpp | 9 ++--- src/Disks/IDiskRemote.cpp | 8 ++--- src/Disks/StoragePolicy.cpp | 2 -- src/Formats/FormatSchemaInfo.cpp | 11 ++----- src/Interpreters/DatabaseCatalog.cpp | 5 ++- .../ExternalLoaderXMLConfigRepository.cpp | 9 ++--- src/Interpreters/InterpreterDropQuery.cpp | 2 -- src/Server/HTTP/HTTPServerResponse.cpp | 4 +-- src/Server/HTTPHandler.cpp | 1 - src/Server/StaticRequestHandler.cpp | 5 ++- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 -- src/Storages/MergeTree/IMergeTreeReader.cpp | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreeDataPartChecksum.cpp | 1 - .../MergeTree/MergeTreeDataPartCompact.cpp | 1 - .../MergeTree/MergeTreeDataPartInMemory.cpp | 1 - .../MergeTree/MergeTreeDataPartWide.cpp | 1 - .../MergeTree/MergeTreeDataSelectExecutor.cpp | 1 - .../MergeTree/MergeTreeDataWriter.cpp | 1 - .../MergeTree/MergeTreeMarksLoader.cpp | 1 - .../MergeTree/MergeTreeMutationEntry.cpp | 3 -- .../MergeTree/MergeTreeReaderCompact.cpp | 1 - .../MergeTree/MergedBlockOutputStream.cpp | 1 - src/Storages/MergeTree/checkDataPart.cpp | 1 - src/Storages/StorageJoin.cpp | 2 -- src/Storages/StorageXDBC.cpp | 1 - .../examples/remove_symlink_directory.cpp | 2 +- 47 files changed, 74 insertions(+), 115 deletions(-) rename src/Common/{createFile.cpp => FileSystemHelpers.cpp} (66%) create mode 100644 src/Common/FileSystemHelpers.h delete mode 100644 src/Common/createFile.h diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 310e656dac2..6aa65942445 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -57,7 +57,6 @@ #include #include #include -#include #include #if !defined(ARCADIA_BUILD) diff --git a/base/daemon/SentryWriter.cpp b/base/daemon/SentryWriter.cpp index 3719bb74c52..3571c64edd6 100644 --- a/base/daemon/SentryWriter.cpp +++ b/base/daemon/SentryWriter.cpp @@ -1,6 +1,5 @@ #include -#include #include #include diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 7dea4c9375b..233134f0529 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -25,7 +25,6 @@ #include #include #include -#include #include #include #include @@ -88,7 +87,7 @@ #include #include #include -#include +#include #if !defined(ARCADIA_BUILD) # include diff --git a/programs/copier/Internals.h b/programs/copier/Internals.h index 7e45c0ea2ee..9e40d7ebd7b 100644 --- a/programs/copier/Internals.h +++ b/programs/copier/Internals.h @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Bridge/IBridgeHelper.cpp b/src/Bridge/IBridgeHelper.cpp index 93ca359c578..b6f3446d0a6 100644 --- a/src/Bridge/IBridgeHelper.cpp +++ b/src/Bridge/IBridgeHelper.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include diff --git a/src/Bridge/LibraryBridgeHelper.cpp b/src/Bridge/LibraryBridgeHelper.cpp index c589d0ce09e..7d378941d25 100644 --- a/src/Bridge/LibraryBridgeHelper.cpp +++ b/src/Bridge/LibraryBridgeHelper.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Bridge/XDBCBridgeHelper.h b/src/Bridge/XDBCBridgeHelper.h index 299df6ff888..b170a55e438 100644 --- a/src/Bridge/XDBCBridgeHelper.h +++ b/src/Bridge/XDBCBridgeHelper.h @@ -5,10 +5,8 @@ #include #include #include -#include #include #include -#include #include #include #include diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 5b16bc0cb1b..e13e66547a3 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -15,12 +15,9 @@ #include #include #include -#include -#include #include #include #include - #include diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index b2aeb899e4d..e39ec7c590a 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -1,11 +1,12 @@ #include "ConfigReloader.h" #include -#include #include #include #include "ConfigProcessor.h" #include +#include + namespace fs = std::filesystem; @@ -170,10 +171,7 @@ struct ConfigReloader::FileWithTimestamp void ConfigReloader::FilesChangesTracker::addIfExists(const std::string & path_to_add) { if (!path_to_add.empty() && fs::exists(path_to_add)) - { - fs::file_time_type fs_time = fs::last_write_time(path_to_add); - files.emplace(path_to_add, fs::file_time_type::clock::to_time_t(fs_time)); - } + files.emplace(path_to_add, FS::getModificationTime(path_to_add)); } bool ConfigReloader::FilesChangesTracker::isDifferOrNewerThan(const FilesChangesTracker & rhs) diff --git a/src/Common/createFile.cpp b/src/Common/FileSystemHelpers.cpp similarity index 66% rename from src/Common/createFile.cpp rename to src/Common/FileSystemHelpers.cpp index 64e344f1f22..7862d2ccdf5 100644 --- a/src/Common/createFile.cpp +++ b/src/Common/FileSystemHelpers.cpp @@ -1,10 +1,12 @@ -#include "createFile.h" +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include +#include +#include namespace DB { @@ -61,4 +63,25 @@ bool canWrite(const std::string & path) DB::throwFromErrnoWithPath("Cannot check write access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); } +time_t getModificationTime(const std::string & path) +{ + struct stat st; + if (stat(path.c_str(), &st) == 0) + return st.st_mtime; + DB::throwFromErrnoWithPath("Cannot check modification time for file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} + +Poco::Timestamp getModificationTimestamp(const std::string & path) +{ + return Poco::Timestamp::fromEpochTime(getModificationTime(path)); +} + +void setModificationTime(const std::string & path, time_t time) +{ + struct utimbuf tb; + tb.actime = time; + tb.modtime = time; + if (utime(path.c_str(), &tb) != 0) + DB::throwFromErrnoWithPath("Cannot set modification time for file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} } diff --git a/src/Common/FileSystemHelpers.h b/src/Common/FileSystemHelpers.h new file mode 100644 index 00000000000..bc11fd3acd1 --- /dev/null +++ b/src/Common/FileSystemHelpers.h @@ -0,0 +1,14 @@ +#pragma once +#include + +namespace FS +{ +bool createFile(const std::string & path); + +bool canRead(const std::string & path); +bool canWrite(const std::string & path); + +time_t getModificationTime(const std::string & path); +Poco::Timestamp getModificationTimestamp(const std::string & path); +void setModificationTime(const std::string & path, time_t time); +} diff --git a/src/Common/FileUpdatesTracker.h b/src/Common/FileUpdatesTracker.h index 471a62cf07a..5fcbbb0bf6c 100644 --- a/src/Common/FileUpdatesTracker.h +++ b/src/Common/FileUpdatesTracker.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace fs = std::filesystem; @@ -31,8 +32,6 @@ public: private: Poco::Timestamp getLastModificationTime() const { - fs::file_time_type fs_time = fs::last_write_time(path); - auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); - return Poco::Timestamp(micro_sec.count()); + return FS::getModificationTimestamp(path); } }; diff --git a/src/Common/createFile.h b/src/Common/createFile.h deleted file mode 100644 index f96d141d39e..00000000000 --- a/src/Common/createFile.h +++ /dev/null @@ -1,9 +0,0 @@ -#pragma once -#include - -namespace FS -{ -bool createFile(const std::string & path); -bool canRead(const std::string & path); -bool canWrite(const std::string & path); -} diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 39b8137f688..edf54961e16 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -6,8 +6,6 @@ # include #endif #include -#include -#include #include #include diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index d179adb18cf..e09860e79b6 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -11,8 +11,6 @@ #include #include #include -#include -#include #include #include #include diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index f50adf54c3f..406ca2a381a 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -19,7 +19,7 @@ #include #include #include -#include +#include namespace fs = std::filesystem; @@ -542,10 +542,7 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_n String table_metadata_path = getObjectMetadataPath(object_name); if (fs::exists(table_metadata_path)) - { - fs::file_time_type fs_time = fs::last_write_time(table_metadata_path); - return fs::file_time_type::clock::to_time_t(fs_time); - } + return FS::getModificationTime(table_metadata_path); else return static_cast(0); } diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index 8901a3f77a1..46beccb1ee6 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -25,7 +25,7 @@ # include # include # include -# include +# include namespace fs = std::filesystem; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 5d75915ea91..7080b3293d7 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include namespace fs = std::filesystem; diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index 1febf921e07..0b799fa7cce 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -3,14 +3,13 @@ #include #include - -#include #include #include #include #include #include + namespace DB { diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index fd0583a4078..a516d6f79ee 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -1,9 +1,5 @@ #include "FileDictionarySource.h" - #include - -#include - #include #include #include @@ -14,6 +10,8 @@ #include "DictionaryStructure.h" #include "registerDictionaries.h" #include "DictionarySourceHelpers.h" +#include + namespace fs = std::filesystem; @@ -70,9 +68,7 @@ std::string FileDictionarySource::toString() const Poco::Timestamp FileDictionarySource::getLastModification() const { - fs::file_time_type fs_time = fs::last_write_time(filepath); - auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); - return Poco::Timestamp(micro_sec.count()); + return FS::getModificationTimestamp(filepath); } diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index e31731a7edc..93f60a7a665 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -7,8 +7,7 @@ #include #include #include -#include -#include +#include #include #include @@ -260,14 +259,12 @@ void DiskLocal::listFiles(const String & path, std::vector & file_names) void DiskLocal::setLastModified(const String & path, const Poco::Timestamp & timestamp) { - fs::last_write_time(fs::path(disk_path) / path, static_cast(std::chrono::microseconds(timestamp.epochMicroseconds()))); + FS::setModificationTime(fs::path(disk_path) / path, timestamp.epochTime()); } Poco::Timestamp DiskLocal::getLastModified(const String & path) { - fs::file_time_type fs_time = fs::last_write_time(fs::path(disk_path) / path); - auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); - return Poco::Timestamp(micro_sec.count()); + return FS::getModificationTimestamp(fs::path(disk_path) / path); } void DiskLocal::createHardLink(const String & src_path, const String & dst_path) diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index f32b046b5ec..691e53ac553 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB @@ -429,16 +430,13 @@ void IDiskRemote::listFiles(const String & path, std::vector & file_name void IDiskRemote::setLastModified(const String & path, const Poco::Timestamp & timestamp) { - fs::last_write_time(fs::path(metadata_path) / path, - static_cast(std::chrono::microseconds(timestamp.epochMicroseconds()))); + FS::setModificationTime(fs::path(metadata_path) / path, timestamp.epochTime()); } Poco::Timestamp IDiskRemote::getLastModified(const String & path) { - fs::file_time_type fs_time = fs::last_write_time(fs::path(metadata_path) / path); - auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); - return Poco::Timestamp(micro_sec.count()); + return FS::getModificationTimestamp(fs::path(metadata_path) / path); } diff --git a/src/Disks/StoragePolicy.cpp b/src/Disks/StoragePolicy.cpp index cff2685ca24..efedff9929f 100644 --- a/src/Disks/StoragePolicy.cpp +++ b/src/Disks/StoragePolicy.cpp @@ -8,8 +8,6 @@ #include -#include - namespace { diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index 25ed6051382..bd1bd9367b2 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -74,14 +74,6 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & if (!path.has_extension() && !default_file_extension.empty()) path = path.parent_path() / (path.stem().string() + '.' + default_file_extension); - fs::path default_schema_directory_path(default_schema_directory()); - auto path_is_subdirectory_of = [](fs::path inner, const fs::path & outer) -> bool - { - while (inner != outer && inner != "/") - inner = inner.parent_path(); - return inner == outer; - }; - if (path.is_absolute()) { if (is_server) @@ -89,12 +81,13 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & schema_path = path.filename(); schema_directory = path.parent_path() / ""; } - else if (path.has_parent_path() && !path_is_subdirectory_of(path, default_schema_directory_path)) + else if (!fs::weakly_canonical(path).string().starts_with(fs::weakly_canonical(default_schema_directory()).string())) { if (is_server) throw Exception( "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.string(), ErrorCodes::BAD_ARGUMENTS); + fs::path default_schema_directory_path(default_schema_directory()); if (default_schema_directory_path.is_absolute()) path = default_schema_directory_path; else diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 0794cb52a49..fe8b4348820 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include @@ -18,6 +17,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include "config_core.h" @@ -782,8 +782,7 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr } addUUIDMapping(table_id.uuid); - fs::file_time_type fs_time = fs::last_write_time(dropped_metadata_path); - drop_time = fs::file_time_type::clock::to_time_t(fs_time); + drop_time = FS::getModificationTime(dropped_metadata_path); } std::lock_guard lock(tables_marked_dropped_mutex); diff --git a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp index 4c6af554fae..49c95516c83 100644 --- a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp +++ b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp @@ -3,12 +3,11 @@ #include #include #include - #include -#include -#include +#include #include + namespace fs = std::filesystem; namespace DB @@ -21,9 +20,7 @@ ExternalLoaderXMLConfigRepository::ExternalLoaderXMLConfigRepository( Poco::Timestamp ExternalLoaderXMLConfigRepository::getUpdateTime(const std::string & definition_entity_name) { - fs::file_time_type fs_time = fs::last_write_time(definition_entity_name); - auto micro_sec = std::chrono::duration_cast(fs_time.time_since_epoch()); - return Poco::Timestamp(micro_sec.count()); + return FS::getModificationTimestamp(definition_entity_name); } std::set ExternalLoaderXMLConfigRepository::getAllLoadablesDefinitionNames() diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 24cff90caea..a9d43c6e9cc 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -1,5 +1,3 @@ -#include - #include #include #include diff --git a/src/Server/HTTP/HTTPServerResponse.cpp b/src/Server/HTTP/HTTPServerResponse.cpp index db5cfb132e3..25e7604a515 100644 --- a/src/Server/HTTP/HTTPServerResponse.cpp +++ b/src/Server/HTTP/HTTPServerResponse.cpp @@ -1,11 +1,8 @@ #include - #include - #include #include #include -#include #include #include #include @@ -13,6 +10,7 @@ #include #include + namespace DB { diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 8aed5d20f74..907a778a07f 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -39,7 +39,6 @@ #include #include -#include #include #include #include diff --git a/src/Server/StaticRequestHandler.cpp b/src/Server/StaticRequestHandler.cpp index 4746ab6d1e0..e96f3a581a9 100644 --- a/src/Server/StaticRequestHandler.cpp +++ b/src/Server/StaticRequestHandler.cpp @@ -138,11 +138,10 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out) if (startsWith(response_expression, file_prefix)) { - const auto & file_name = response_expression.substr(file_prefix.size(), response_expression.size() - file_prefix.size()); + const auto & file_name = response_expression.substr(file_prefix.size() + 1, response_expression.size() - file_prefix.size()); fs::path user_files_absolute_path = fs::canonical(fs::path(server.context()->getUserFilesPath())); - /// Fixme: it does not work with fs::path(user_files_absolute_path) / file_name - String file_path = fs::canonical(user_files_absolute_path.string() + "/" + file_name); + String file_path = fs::weakly_canonical(user_files_absolute_path / file_name); if (!fs::exists(file_path)) throw Exception("Invalid file name " + file_path + " for static HTTPHandler. ", ErrorCodes::INCORRECT_FILE_NAME); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index a8a49680dd7..53640b41507 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -17,8 +17,6 @@ #include #include -#include - #include namespace zkutil diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 52d3e7ca9ab..14187564536 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1c9507d8cb1..fb144362b3c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3326,7 +3326,7 @@ MergeTreeData::getDetachedParts() const for (const auto & [path, disk] : getRelativeDataPathsWithDisks()) { - for (auto it = disk->iterateDirectory(path + MergeTreeData::DETACHED_DIR_NAME); it->isValid(); it->next()) + for (auto it = disk->iterateDirectory(fs::path(path) / MergeTreeData::DETACHED_DIR_NAME); it->isValid(); it->next()) { res.emplace_back(); auto & part = res.back(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index 8f5c4b6a848..1df97dc9241 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -7,7 +7,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 5c1a3b01804..7b1641a0537 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -2,7 +2,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 473f2e598a9..e929bfc6862 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 84378faa513..4eebd729dd8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -1,5 +1,4 @@ #include "MergeTreeDataPartWide.h" -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 4e11669b8d7..00bc4306e4b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -3,7 +3,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 3bf33d45802..0b05650b42c 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index c5a99b128e9..717179e5f26 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 49c4e93eb1d..2aefb3df2be 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -4,9 +4,6 @@ #include #include -#include -#include - #include diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index da28f75b57f..783a37cce60 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -2,7 +2,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index e8a790bce3f..6de01caa833 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -1,6 +1,5 @@ #include #include -#include #include diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 5de2b8707c3..8a234833da7 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index f0129e95495..7a448e1dde9 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -14,9 +14,7 @@ #include #include #include - #include /// toLower -#include #include #include diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index 88d54669889..9cffc32fda1 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/examples/remove_symlink_directory.cpp b/src/Storages/examples/remove_symlink_directory.cpp index a90083233e7..ae898edb49b 100644 --- a/src/Storages/examples/remove_symlink_directory.cpp +++ b/src/Storages/examples/remove_symlink_directory.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include namespace fs = std::filesystem; From 0a3589524ea5b5c412d5f55b9c27666fe5d14b99 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 29 May 2021 00:57:53 +0300 Subject: [PATCH 52/55] File movement --- programs/client/Client.cpp | 2 +- src/Common/Config/ConfigReloader.cpp | 2 +- src/Common/FileSystemHelpers.cpp | 87 ------------------- src/Common/FileSystemHelpers.h | 14 --- src/Common/FileUpdatesTracker.h | 2 +- src/Common/filesystemHelpers.cpp | 77 ++++++++++++++++ src/Common/filesystemHelpers.h | 12 +++ src/Databases/DatabaseOnDisk.cpp | 2 +- .../MySQL/DatabaseConnectionMySQL.cpp | 2 +- .../PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Dictionaries/FileDictionarySource.cpp | 2 +- src/Disks/DiskLocal.cpp | 2 +- src/Disks/IDiskRemote.cpp | 2 +- src/Interpreters/DatabaseCatalog.cpp | 2 +- .../ExternalLoaderXMLConfigRepository.cpp | 2 +- .../examples/remove_symlink_directory.cpp | 2 +- 16 files changed, 101 insertions(+), 113 deletions(-) delete mode 100644 src/Common/FileSystemHelpers.cpp delete mode 100644 src/Common/FileSystemHelpers.h diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 233134f0529..89899b86eb0 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -87,7 +87,7 @@ #include #include #include -#include +#include #if !defined(ARCADIA_BUILD) # include diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index e39ec7c590a..01bb4f556d6 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -5,7 +5,7 @@ #include #include "ConfigProcessor.h" #include -#include +#include namespace fs = std::filesystem; diff --git a/src/Common/FileSystemHelpers.cpp b/src/Common/FileSystemHelpers.cpp deleted file mode 100644 index 7862d2ccdf5..00000000000 --- a/src/Common/FileSystemHelpers.cpp +++ /dev/null @@ -1,87 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ -extern const int PATH_ACCESS_DENIED; -extern const int CANNOT_CREATE_FILE; -} -} - -namespace FS -{ - -/// Copy from Poco::createFile -bool createFile(const std::string & path) -{ - int n = open(path.c_str(), O_WRONLY | O_CREAT | O_EXCL, S_IRUSR | S_IWUSR | S_IRGRP | S_IWGRP | S_IROTH | S_IWOTH); - if (n != -1) - { - close(n); - return true; - } - DB::throwFromErrnoWithPath("Cannot create file: " + path, path, DB::ErrorCodes::CANNOT_CREATE_FILE); -} - -bool canRead(const std::string & path) -{ - struct stat st; - if (stat(path.c_str(), &st) == 0) - { - if (st.st_uid == geteuid()) - return (st.st_mode & S_IRUSR) != 0; - else if (st.st_gid == getegid()) - return (st.st_mode & S_IRGRP) != 0; - else - return (st.st_mode & S_IROTH) != 0 || geteuid() == 0; - } - DB::throwFromErrnoWithPath("Cannot check read access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); -} - - -bool canWrite(const std::string & path) -{ - struct stat st; - if (stat(path.c_str(), &st) == 0) - { - if (st.st_uid == geteuid()) - return (st.st_mode & S_IWUSR) != 0; - else if (st.st_gid == getegid()) - return (st.st_mode & S_IWGRP) != 0; - else - return (st.st_mode & S_IWOTH) != 0 || geteuid() == 0; - } - DB::throwFromErrnoWithPath("Cannot check write access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); -} - -time_t getModificationTime(const std::string & path) -{ - struct stat st; - if (stat(path.c_str(), &st) == 0) - return st.st_mtime; - DB::throwFromErrnoWithPath("Cannot check modification time for file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); -} - -Poco::Timestamp getModificationTimestamp(const std::string & path) -{ - return Poco::Timestamp::fromEpochTime(getModificationTime(path)); -} - -void setModificationTime(const std::string & path, time_t time) -{ - struct utimbuf tb; - tb.actime = time; - tb.modtime = time; - if (utime(path.c_str(), &tb) != 0) - DB::throwFromErrnoWithPath("Cannot set modification time for file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); -} -} diff --git a/src/Common/FileSystemHelpers.h b/src/Common/FileSystemHelpers.h deleted file mode 100644 index bc11fd3acd1..00000000000 --- a/src/Common/FileSystemHelpers.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once -#include - -namespace FS -{ -bool createFile(const std::string & path); - -bool canRead(const std::string & path); -bool canWrite(const std::string & path); - -time_t getModificationTime(const std::string & path); -Poco::Timestamp getModificationTimestamp(const std::string & path); -void setModificationTime(const std::string & path, time_t time); -} diff --git a/src/Common/FileUpdatesTracker.h b/src/Common/FileUpdatesTracker.h index 5fcbbb0bf6c..9fe37c2a75d 100644 --- a/src/Common/FileUpdatesTracker.h +++ b/src/Common/FileUpdatesTracker.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace fs = std::filesystem; diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index edf54961e16..d7c556d1c6a 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -7,7 +7,13 @@ #endif #include #include +#include #include +#include +#include +#include +#include +#include namespace fs = std::filesystem; @@ -21,6 +27,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int CANNOT_STATVFS; extern const int PATH_ACCESS_DENIED; + extern const int CANNOT_CREATE_FILE; } @@ -125,3 +132,73 @@ bool pathStartsWith(const String & path, const String & prefix_path) } } + + +/// Copied from Poco::File +namespace FS +{ + +bool createFile(const std::string & path) +{ + int n = open(path.c_str(), O_WRONLY | O_CREAT | O_EXCL, S_IRUSR | S_IWUSR | S_IRGRP | S_IWGRP | S_IROTH | S_IWOTH); + if (n != -1) + { + close(n); + return true; + } + DB::throwFromErrnoWithPath("Cannot create file: " + path, path, DB::ErrorCodes::CANNOT_CREATE_FILE); +} + +bool canRead(const std::string & path) +{ + struct stat st; + if (stat(path.c_str(), &st) == 0) + { + if (st.st_uid == geteuid()) + return (st.st_mode & S_IRUSR) != 0; + else if (st.st_gid == getegid()) + return (st.st_mode & S_IRGRP) != 0; + else + return (st.st_mode & S_IROTH) != 0 || geteuid() == 0; + } + DB::throwFromErrnoWithPath("Cannot check read access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} + + +bool canWrite(const std::string & path) +{ + struct stat st; + if (stat(path.c_str(), &st) == 0) + { + if (st.st_uid == geteuid()) + return (st.st_mode & S_IWUSR) != 0; + else if (st.st_gid == getegid()) + return (st.st_mode & S_IWGRP) != 0; + else + return (st.st_mode & S_IWOTH) != 0 || geteuid() == 0; + } + DB::throwFromErrnoWithPath("Cannot check write access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} + +time_t getModificationTime(const std::string & path) +{ + struct stat st; + if (stat(path.c_str(), &st) == 0) + return st.st_mtime; + DB::throwFromErrnoWithPath("Cannot check modification time for file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} + +Poco::Timestamp getModificationTimestamp(const std::string & path) +{ + return Poco::Timestamp::fromEpochTime(getModificationTime(path)); +} + +void setModificationTime(const std::string & path, time_t time) +{ + struct utimbuf tb; + tb.actime = time; + tb.modtime = time; + if (utime(path.c_str(), &tb) != 0) + DB::throwFromErrnoWithPath("Cannot set modification time for file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} +} diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index 0ffbf19643d..b7525a64fae 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -36,3 +36,15 @@ bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::p bool pathStartsWith(const String & path, const String & prefix_path); } + +namespace FS +{ +bool createFile(const std::string & path); + +bool canRead(const std::string & path); +bool canWrite(const std::string & path); + +time_t getModificationTime(const std::string & path); +Poco::Timestamp getModificationTimestamp(const std::string & path); +void setModificationTime(const std::string & path, time_t time); +} diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 406ca2a381a..e1b5f51a8fc 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -19,7 +19,7 @@ #include #include #include -#include +#include namespace fs = std::filesystem; diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index 46beccb1ee6..ccc1c058e74 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -25,7 +25,7 @@ # include # include # include -# include +# include namespace fs = std::filesystem; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 7080b3293d7..53d34fb07b1 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include namespace fs = std::filesystem; diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index a516d6f79ee..a22263575f4 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -10,7 +10,7 @@ #include "DictionaryStructure.h" #include "registerDictionaries.h" #include "DictionarySourceHelpers.h" -#include +#include namespace fs = std::filesystem; diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 93f60a7a665..cae166d2110 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index 691e53ac553..b30e9613ed8 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index fe8b4348820..17edc68dcc3 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -17,7 +17,7 @@ #include #include #include -#include +#include #if !defined(ARCADIA_BUILD) # include "config_core.h" diff --git a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp index 49c95516c83..00c31110ea4 100644 --- a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp +++ b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/examples/remove_symlink_directory.cpp b/src/Storages/examples/remove_symlink_directory.cpp index ae898edb49b..db436c0a608 100644 --- a/src/Storages/examples/remove_symlink_directory.cpp +++ b/src/Storages/examples/remove_symlink_directory.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include namespace fs = std::filesystem; From 69816e6effcde4ac55481d49a3319be33523956d Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 29 May 2021 08:44:07 +0300 Subject: [PATCH 53/55] Fix checks --- programs/client/Client.cpp | 2 +- src/Common/filesystemHelpers.cpp | 1 - src/Common/ya.make | 1 - src/Dictionaries/FileDictionarySource.cpp | 4 ---- src/Dictionaries/LibraryDictionarySource.cpp | 2 -- src/Disks/DiskLocal.cpp | 1 - src/Formats/FormatSchemaInfo.cpp | 8 ++++---- src/Server/StaticRequestHandler.cpp | 4 +++- .../examples/get_current_inserts_in_replicated.cpp | 3 ++- 9 files changed, 10 insertions(+), 16 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 89899b86eb0..cd8162e1af2 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -278,7 +278,7 @@ private: /// Set path for format schema files if (config().has("format_schema_path")) - context->setFormatSchemaPath(fs::absolute(config().getString("format_schema_path"))); + context->setFormatSchemaPath(fs::weakly_canonical(config().getString("format_schema_path"))); /// Initialize query_id_formats if any if (config().has("query_id_formats")) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index d7c556d1c6a..4855500b776 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include diff --git a/src/Common/ya.make b/src/Common/ya.make index c91f794e97c..57b60e9cce5 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -95,7 +95,6 @@ SRCS( ZooKeeper/ZooKeeperNodeCache.cpp checkStackSize.cpp clearPasswordFromCommandLine.cpp - createFile.cpp createHardLink.cpp escapeForFileName.cpp filesystemHelpers.cpp diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index a22263575f4..239c13e71c2 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -1,5 +1,4 @@ #include "FileDictionarySource.h" -#include #include #include #include @@ -10,11 +9,8 @@ #include "DictionaryStructure.h" #include "registerDictionaries.h" #include "DictionarySourceHelpers.h" -#include -namespace fs = std::filesystem; - namespace DB { static const UInt64 max_block_size = 8192; diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index 70022a9b721..0b8b52a2d67 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -7,8 +7,6 @@ #include #include #include -#include -#include #include #include diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index cae166d2110..768d1c2354a 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index bd1bd9367b2..1b406f639ea 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -81,12 +81,12 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & schema_path = path.filename(); schema_directory = path.parent_path() / ""; } - else if (!fs::weakly_canonical(path).string().starts_with(fs::weakly_canonical(default_schema_directory()).string())) + else if (path.has_parent_path() && !fs::weakly_canonical(path).string().starts_with(fs::weakly_canonical(default_schema_directory()).string())) { if (is_server) - throw Exception( - "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.string(), - ErrorCodes::BAD_ARGUMENTS); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: {} ({} not in {})", + path.string()); fs::path default_schema_directory_path(default_schema_directory()); if (default_schema_directory_path.is_absolute()) path = default_schema_directory_path; diff --git a/src/Server/StaticRequestHandler.cpp b/src/Server/StaticRequestHandler.cpp index e96f3a581a9..19b91ae9c42 100644 --- a/src/Server/StaticRequestHandler.cpp +++ b/src/Server/StaticRequestHandler.cpp @@ -138,7 +138,9 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out) if (startsWith(response_expression, file_prefix)) { - const auto & file_name = response_expression.substr(file_prefix.size() + 1, response_expression.size() - file_prefix.size()); + auto file_name = response_expression.substr(file_prefix.size(), response_expression.size() - file_prefix.size()); + if (file_name.starts_with('/')) + file_name = file_name.substr(1); fs::path user_files_absolute_path = fs::canonical(fs::path(server.context()->getUserFilesPath())); String file_path = fs::weakly_canonical(user_files_absolute_path / file_name); diff --git a/src/Storages/examples/get_current_inserts_in_replicated.cpp b/src/Storages/examples/get_current_inserts_in_replicated.cpp index 62dcce37d4e..56364997643 100644 --- a/src/Storages/examples/get_current_inserts_in_replicated.cpp +++ b/src/Storages/examples/get_current_inserts_in_replicated.cpp @@ -4,11 +4,12 @@ #include #include #include - #include +#include #include +namespace fs = std::filesystem; using namespace DB; From 054fe1cf2fd5ce1cacc074f19b22be9b6a080d66 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 May 2021 14:06:32 +0300 Subject: [PATCH 54/55] Fix --- src/Formats/FormatSchemaInfo.cpp | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index 1b406f639ea..2605c0bdf04 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -74,6 +74,7 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & if (!path.has_extension() && !default_file_extension.empty()) path = path.parent_path() / (path.stem().string() + '.' + default_file_extension); + fs::path default_schema_directory_path(default_schema_directory()); if (path.is_absolute()) { if (is_server) @@ -81,17 +82,13 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & schema_path = path.filename(); schema_directory = path.parent_path() / ""; } - else if (path.has_parent_path() && !fs::weakly_canonical(path).string().starts_with(fs::weakly_canonical(default_schema_directory()).string())) + else if (path.has_parent_path() && !fs::weakly_canonical(default_schema_directory_path / path).string().starts_with(fs::weakly_canonical(default_schema_directory_path).string())) { if (is_server) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: {} ({} not in {})", path.string()); - fs::path default_schema_directory_path(default_schema_directory()); - if (default_schema_directory_path.is_absolute()) - path = default_schema_directory_path; - else - path /= default_schema_directory_path; + path = default_schema_directory_path / path; schema_path = path.filename(); schema_directory = path.parent_path() / ""; } From 18988a60ad76b330f7665388a998be74f078465e Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 31 May 2021 18:03:45 +0300 Subject: [PATCH 55/55] Fix mutation wait --- src/Storages/StorageReplicatedMergeTree.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 319e04f2424..a83a2f36428 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -523,10 +523,11 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( if (wait_event->tryWait(1000)) continue; - /// Here we check mutation for errors or kill on local replica. If they happen on this replica + /// Here we check mutation for errors on local replica. If they happen on this replica /// they will happen on each replica, so we can check only in-memory info. auto mutation_status = queue.getIncompleteMutationsStatus(mutation_id); - if (!mutation_status || !mutation_status->latest_fail_reason.empty()) + /// If mutation status is empty, than local replica may just not loaded it into memory. + if (mutation_status && !mutation_status->latest_fail_reason.empty()) break; }