Poco::createFile to fs::createFile

This commit is contained in:
kssenii 2021-05-08 00:53:44 +03:00
parent c3eab60b8b
commit 35f999bf04
12 changed files with 131 additions and 58 deletions

View File

@ -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) \

View File

@ -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<const Poco::FileException *>(&e))
{
if (file_exception->code() == ENOSPC)

52
src/Common/createFile.cpp Normal file
View File

@ -0,0 +1,52 @@
#include "createFile.h"
#include <Common/ErrorCodes.h>
#include <Common/Exception.h>
#include <unistd.h>
#include <sys/types.h>
#include <sys/stat.h>
#include <fcntl.h>
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);
}
}

7
src/Common/createFile.h Normal file
View File

@ -0,0 +1,7 @@
#pragma once
#include <filesystem>
namespace std::filesystem
{
bool createFile(const path & path);
}

View File

@ -14,14 +14,14 @@
#include <Storages/StorageFactory.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/escapeForFileName.h>
#include <common/logger_useful.h>
#include <Poco/DirectoryIterator.h>
#include <Databases/DatabaseOrdinary.h>
#include <Databases/DatabaseAtomic.h>
#include <Common/assert_cast.h>
#include <filesystem>
#include <Poco/File.h>
#include <Poco/Path.h>
#include <Common/createFile.h>
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<std::pair<String, bool>> 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

View File

@ -24,7 +24,7 @@
# include <Common/parseAddress.h>
# include <Common/setThreadName.h>
# include <filesystem>
# include <Poco/File.h>
# include <Common/createFile.h>
namespace fs = std::filesystem;
@ -408,27 +408,25 @@ void DatabaseConnectionMySQL::detachTablePermanently(ContextPtr, const String &
{
std::lock_guard<std::mutex> 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 (...)
{

View File

@ -14,6 +14,7 @@
#include <Common/escapeForFileName.h>
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
#include <Common/quoteString.h>
#include <Common/createFile.h>
#include <filesystem>
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);

View File

@ -8,6 +8,7 @@
#include <Common/quoteString.h>
#include <IO/createReadBufferFromFileBase.h>
#include <Poco/File.h>
#include <Common/createFile.h>
#include <fstream>
#include <unistd.h>
@ -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)

View File

@ -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())

View File

@ -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;
}
}

View File

@ -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;

View File

@ -1,9 +1,10 @@
#include <unistd.h>
#include <iostream>
#include <Poco/File.h>
#include <Poco/Path.h>
#include <Common/Exception.h>
#include <Common/createFile.h>
#include <filesystem>
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 (...)