From ab1a05a1f4b0f2176469c510dac2c125c8fac815 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 9 May 2021 14:59:49 +0300 Subject: [PATCH] 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); }