mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 13:13:36 +00:00
Poco::Path to fs::path, less concatination
This commit is contained in:
parent
e26cb4bee0
commit
ab1a05a1f4
@ -5,7 +5,9 @@
|
||||
#include <Poco/Net/HTTPRequest.h>
|
||||
#include <Poco/Path.h>
|
||||
#include <Poco/URI.h>
|
||||
#include <filesystem>
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -87,10 +89,10 @@ std::unique_ptr<ShellCommand> 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<std::string> 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<ShellCommand> 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));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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<std::string> 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)
|
||||
{
|
||||
|
@ -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.
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -671,8 +671,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(
|
||||
auto & create = ast->as<ASTCreateQuery &>();
|
||||
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(
|
||||
|
@ -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))
|
||||
|
@ -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)
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Core/Types.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <Common/ZooKeeper/Types.h>
|
||||
#include <filesystem>
|
||||
|
||||
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);
|
||||
|
@ -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())
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user