Update databases/*

This commit is contained in:
kssenii 2021-04-30 22:38:07 +03:00
parent 6e0cfdaeb5
commit e717b4c5c7
5 changed files with 30 additions and 26 deletions

View File

@ -12,8 +12,9 @@
#include <common/logger_useful.h>
#include <ext/scope_guard_safe.h>
#include <iomanip>
#include <Poco/File.h>
#include <filesystem>
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;

View File

@ -4,9 +4,9 @@
#include <Interpreters/Context.h>
#include <Parsers/ASTCreateQuery.h>
#include <Storages/IStorage.h>
#include <Poco/File.h>
#include <filesystem>
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 (...)
{

View File

@ -23,10 +23,10 @@
# include <Common/escapeForFileName.h>
# include <Common/parseAddress.h>
# include <Common/setThreadName.h>
# include <Poco/DirectoryIterator.h>
# include <filesystem>
# include <Poco/File.h>
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<std::mutex> 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);
}

View File

@ -13,9 +13,11 @@
# include <Databases/MySQL/MaterializeMySQLSyncThread.h>
# include <Parsers/ASTCreateQuery.h>
# include <Storages/StorageMaterializeMySQL.h>
# include <Poco/File.h>
# include <Poco/Logger.h>
# include <Common/setThreadName.h>
# include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -158,10 +160,10 @@ template<typename Base>
void DatabaseMaterializeMySQL<Base>::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_);
}

View File

@ -8,11 +8,13 @@
#include <Formats/MySQLBlockInputStream.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromFile.h>
#include <Poco/File.h>
#include <Common/quoteString.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -193,12 +195,11 @@ void commitMetadata(const std::function<void()> & 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);