mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-12 17:32:32 +00:00
Less Poco
This commit is contained in:
parent
5068b163b8
commit
140bf7e2ea
@ -5,7 +5,9 @@
|
||||
#include <common/logger_useful.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include "ConfigProcessor.h"
|
||||
#include <filesystem>
|
||||
|
||||
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());
|
||||
}
|
||||
|
||||
|
@ -408,7 +408,7 @@ void DatabaseConnectionMySQL::detachTablePermanently(ContextPtr, const String &
|
||||
{
|
||||
std::lock_guard<std::mutex> 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",
|
||||
|
@ -309,7 +309,7 @@ bool inline isSameDiskType(const IDisk & one, const IDisk & another)
|
||||
void DiskLocal::copy(const String & from_path, const std::shared_ptr<IDisk> & 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.
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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<IDisk> s3disk = std::make_shared<DiskS3>(
|
||||
name,
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user