refactor throwFromErrno

This commit is contained in:
Alexander Tokmakov 2019-08-06 21:54:06 +03:00
parent c3039196da
commit aefc664819
19 changed files with 51 additions and 59 deletions

View File

@ -67,13 +67,13 @@ public:
int fd = ::open(path.c_str(), O_RDWR | O_CREAT, 0666);
if (-1 == fd)
DB::throwFromErrno("Cannot open file " + path, DB::ErrorCodes::CANNOT_OPEN_FILE);
DB::throwFromErrno("Cannot open file " + path, path, DB::ErrorCodes::CANNOT_OPEN_FILE);
try
{
int flock_ret = flock(fd, LOCK_EX);
if (-1 == flock_ret)
DB::throwFromErrno("Cannot lock file " + path, DB::ErrorCodes::CANNOT_OPEN_FILE);
DB::throwFromErrno("Cannot lock file " + path, path, DB::ErrorCodes::CANNOT_OPEN_FILE);
if (!file_doesnt_exists)
{
@ -141,7 +141,7 @@ public:
int fd = ::open(path.c_str(), O_RDWR | O_CREAT, 0666);
if (-1 == fd)
DB::throwFromErrno("Cannot open file " + path, DB::ErrorCodes::CANNOT_OPEN_FILE);
DB::throwFromErrno("Cannot open file " + path, path, DB::ErrorCodes::CANNOT_OPEN_FILE);
try
{

View File

@ -55,6 +55,11 @@ void throwFromErrno(const std::string & s, int code, int e)
throw ErrnoException(s + ", " + errnoToString(code, e), code, e);
}
void throwFromErrno(const std::string & s, const std::string & path, int code, int the_errno)
{
throw ErrnoException(s + ", " + errnoToString(code, the_errno), code, the_errno, path);
}
void tryLogCurrentException(const char * log_name, const std::string & start_of_message)
{
tryLogCurrentException(&Logger::get(log_name), start_of_message);
@ -73,15 +78,12 @@ void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_
void getNoSpaceLeftInfoMessage(std::filesystem::path path, std::string & msg)
{
path = std::filesystem::absolute(path);
/// It's possible to get ENOSPC for non existent file (e.g. if there are no free inodes and creat() fails)
/// So try to get info for existent parent directory.
while (!std::filesystem::exists(path) && path.has_relative_path())
path = path.parent_path();
/// Most likely path is invalid
if (!path.has_relative_path())
return;
auto fs = DiskSpaceMonitor::getStatVFS(path);
msg += "\nTotal space: " + formatReadableSizeWithBinarySuffix(fs.f_blocks * fs.f_bsize)
+ "\nAvailable space: " + formatReadableSizeWithBinarySuffix(fs.f_bavail * fs.f_bsize)
@ -105,22 +107,8 @@ std::string getExtraExceptionInfo(const std::exception & e)
}
else if (auto errno_exception = dynamic_cast<const DB::ErrnoException *>(&e))
{
if (errno_exception->getErrno() == ENOSPC)
{
/// Try to extract path from text exception message. Most likely the exception was thrown by
/// DB::throwFromErrno("Some message" + filename, ...);
/// We suppose "Some message " does not contain '/' and filename is an absolute path starts with '/'.
/// throwFromErrno appends ", errno: ..." to the first argument.
/// It's ugly hack which may not work correctly. However, getEnospcInfoMessage(...) checks if path exists.
size_t likely_path_begin = errno_exception->message().find('/');
size_t likely_path_end = errno_exception->message().find(", errno: ", likely_path_begin);
if (likely_path_end != std::string::npos)
{
std::string supposed_to_be_path = errno_exception->message().substr(likely_path_begin,
likely_path_end - likely_path_begin);
getNoSpaceLeftInfoMessage(supposed_to_be_path, msg);
}
}
if (errno_exception->getErrno() == ENOSPC && errno_exception->getPath())
getNoSpaceLeftInfoMessage(errno_exception->getPath().value(), msg);
}
}
catch (...) {

View File

@ -52,16 +52,18 @@ private:
class ErrnoException : public Exception
{
public:
ErrnoException(const std::string & msg, int code, int saved_errno_)
: Exception(msg, code), saved_errno(saved_errno_) {}
ErrnoException(const std::string & msg, int code, int saved_errno_, const std::optional<std::string> & path_ = {})
: Exception(msg, code), saved_errno(saved_errno_), path(path_) {}
ErrnoException * clone() const override { return new ErrnoException(*this); }
void rethrow() const override { throw *this; }
int getErrno() const { return saved_errno; }
const std::optional<std::string> getPath() const { return path; }
private:
int saved_errno;
std::optional<std::string> path;
const char * name() const throw() override { return "DB::ErrnoException"; }
const char * className() const throw() override { return "DB::ErrnoException"; }
@ -73,6 +75,7 @@ using Exceptions = std::vector<std::exception_ptr>;
std::string errnoToString(int code, int the_errno = errno);
[[noreturn]] void throwFromErrno(const std::string & s, int code, int the_errno = errno);
[[noreturn]] void throwFromErrno(const std::string & s, const std::string & path, int code, int the_errno = errno);
/** Try to write an exception to the log (and forget about it).

View File

@ -51,7 +51,7 @@ StatusFile::StatusFile(const std::string & path_)
fd = ::open(path.c_str(), O_WRONLY | O_CREAT, 0666);
if (-1 == fd)
throwFromErrno("Cannot open file " + path, ErrorCodes::CANNOT_OPEN_FILE);
throwFromErrno("Cannot open file " + path, path, ErrorCodes::CANNOT_OPEN_FILE);
try
{
@ -61,14 +61,14 @@ StatusFile::StatusFile(const std::string & path_)
if (errno == EWOULDBLOCK)
throw Exception("Cannot lock file " + path + ". Another server instance in same directory is already running.", ErrorCodes::CANNOT_OPEN_FILE);
else
throwFromErrno("Cannot lock file " + path, ErrorCodes::CANNOT_OPEN_FILE);
throwFromErrno("Cannot lock file " + path, path, ErrorCodes::CANNOT_OPEN_FILE);
}
if (0 != ftruncate(fd, 0))
throwFromErrno("Cannot ftruncate " + path, ErrorCodes::CANNOT_TRUNCATE_FILE);
throwFromErrno("Cannot ftruncate " + path, path, ErrorCodes::CANNOT_TRUNCATE_FILE);
if (0 != lseek(fd, 0, SEEK_SET))
throwFromErrno("Cannot lseek " + path, ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
throwFromErrno("Cannot lseek " + path, path, ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
/// Write information about current server instance to the file.
{

View File

@ -26,16 +26,17 @@ void createHardLink(const String & source_path, const String & destination_path)
struct stat destination_descr;
if (0 != lstat(source_path.c_str(), &source_descr))
throwFromErrno("Cannot stat " + source_path, ErrorCodes::CANNOT_STAT);
throwFromErrno("Cannot stat " + source_path, source_path, ErrorCodes::CANNOT_STAT);
if (0 != lstat(destination_path.c_str(), &destination_descr))
throwFromErrno("Cannot stat " + destination_path, ErrorCodes::CANNOT_STAT);
throwFromErrno("Cannot stat " + destination_path, destination_path, ErrorCodes::CANNOT_STAT);
if (source_descr.st_ino != destination_descr.st_ino)
throwFromErrno("Destination file " + destination_path + " is already exist and have different inode.", ErrorCodes::CANNOT_LINK, link_errno);
throwFromErrno("Destination file " + destination_path + " is already exist and have different inode.",
destination_path, ErrorCodes::CANNOT_LINK, link_errno);
}
else
throwFromErrno("Cannot link " + source_path + " to " + destination_path, ErrorCodes::CANNOT_LINK);
throwFromErrno("Cannot link " + source_path + " to " + destination_path, destination_path, ErrorCodes::CANNOT_LINK);
}
}

View File

@ -29,7 +29,7 @@ void MMapReadBufferFromFile::open(const std::string & file_name)
fd = ::open(file_name.c_str(), O_RDONLY);
if (-1 == fd)
throwFromErrno("Cannot open file " + file_name, errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE);
throwFromErrno("Cannot open file " + file_name, file_name, errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE);
}

View File

@ -54,7 +54,7 @@ ReadBufferAIO::ReadBufferAIO(const std::string & filename_, size_t buffer_size_,
if (fd == -1)
{
auto error_code = (errno == ENOENT) ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE;
throwFromErrno("Cannot open file " + filename, error_code);
throwFromErrno("Cannot open file " + filename, filename, error_code);
}
}

View File

@ -41,12 +41,12 @@ ReadBufferFromFile::ReadBufferFromFile(
fd = ::open(file_name.c_str(), flags == -1 ? O_RDONLY : flags);
if (-1 == fd)
throwFromErrno("Cannot open file " + file_name, errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE);
throwFromErrno("Cannot open file " + file_name, file_name, errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE);
#ifdef __APPLE__
if (o_direct)
{
if (fcntl(fd, F_NOCACHE, 1) == -1)
throwFromErrno("Cannot set F_NOCACHE on file " + file_name, ErrorCodes::CANNOT_OPEN_FILE);
throwFromErrno("Cannot set F_NOCACHE on file " + file_name, file_name, ErrorCodes::CANNOT_OPEN_FILE);
}
#endif
}

View File

@ -61,7 +61,7 @@ bool ReadBufferFromFileDescriptor::nextImpl()
if (-1 == res && errno != EINTR)
{
ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorReadFailed);
throwFromErrno("Cannot read from file " + getFileName(), ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR);
throwFromErrno("Cannot read from file " + getFileName(), getFileName(), ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR);
}
if (res > 0)
@ -124,7 +124,7 @@ off_t ReadBufferFromFileDescriptor::doSeek(off_t offset, int whence)
pos = working_buffer.end();
off_t res = ::lseek(fd, new_pos, SEEK_SET);
if (-1 == res)
throwFromErrno("Cannot seek through file " + getFileName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
throwFromErrno("Cannot seek through file " + getFileName(), getFileName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
pos_in_file = new_pos;
watch.stop();

View File

@ -62,7 +62,7 @@ WriteBufferAIO::WriteBufferAIO(const std::string & filename_, size_t buffer_size
if (fd == -1)
{
auto error_code = (errno == ENOENT) ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE;
throwFromErrno("Cannot open file " + filename, error_code);
throwFromErrno("Cannot open file " + filename, filename, error_code);
}
}
@ -96,7 +96,7 @@ void WriteBufferAIO::sync()
/// Ask OS to flush data to disk.
int res = ::fsync(fd);
if (res == -1)
throwFromErrno("Cannot fsync " + getFileName(), ErrorCodes::CANNOT_FSYNC);
throwFromErrno("Cannot fsync " + getFileName(), getFileName(), ErrorCodes::CANNOT_FSYNC);
}
void WriteBufferAIO::nextImpl()
@ -173,7 +173,7 @@ void WriteBufferAIO::doTruncate(off_t length)
int res = ::ftruncate(fd, length);
if (res == -1)
throwFromErrno("Cannot truncate file " + filename, ErrorCodes::CANNOT_TRUNCATE_FILE);
throwFromErrno("Cannot truncate file " + filename, filename, ErrorCodes::CANNOT_TRUNCATE_FILE);
}
void WriteBufferAIO::flush()
@ -427,7 +427,7 @@ void WriteBufferAIO::finalize()
/// Truncate the file to remove unnecessary zeros from it.
int res = ::ftruncate(fd, max_pos_in_file);
if (res == -1)
throwFromErrno("Cannot truncate file " + filename, ErrorCodes::CANNOT_TRUNCATE_FILE);
throwFromErrno("Cannot truncate file " + filename, filename, ErrorCodes::CANNOT_TRUNCATE_FILE);
}
}

View File

@ -44,13 +44,13 @@ WriteBufferFromFile::WriteBufferFromFile(
fd = ::open(file_name.c_str(), flags == -1 ? O_WRONLY | O_TRUNC | O_CREAT : flags, mode);
if (-1 == fd)
throwFromErrno("Cannot open file " + file_name, errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE);
throwFromErrno("Cannot open file " + file_name, file_name, errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE);
#ifdef __APPLE__
if (o_direct)
{
if (fcntl(fd, F_NOCACHE, 1) == -1)
throwFromErrno("Cannot set F_NOCACHE on file " + file_name, ErrorCodes::CANNOT_OPEN_FILE);
throwFromErrno("Cannot set F_NOCACHE on file " + file_name, file_name, ErrorCodes::CANNOT_OPEN_FILE);
}
#endif
}

View File

@ -56,7 +56,7 @@ void WriteBufferFromFileDescriptor::nextImpl()
if ((-1 == res || 0 == res) && errno != EINTR)
{
ProfileEvents::increment(ProfileEvents::WriteBufferFromFileDescriptorWriteFailed);
throwFromErrno("Cannot write to file " + getFileName(), ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR);
throwFromErrno("Cannot write to file " + getFileName(), getFileName(), ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR);
}
if (res > 0)
@ -111,7 +111,7 @@ void WriteBufferFromFileDescriptor::sync()
/// Request OS to sync data with storage medium.
int res = fsync(fd);
if (-1 == res)
throwFromErrno("Cannot fsync " + getFileName(), ErrorCodes::CANNOT_FSYNC);
throwFromErrno("Cannot fsync " + getFileName(), getFileName(), ErrorCodes::CANNOT_FSYNC);
}
@ -119,7 +119,7 @@ off_t WriteBufferFromFileDescriptor::doSeek(off_t offset, int whence)
{
off_t res = lseek(fd, offset, whence);
if (-1 == res)
throwFromErrno("Cannot seek through file " + getFileName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
throwFromErrno("Cannot seek through file " + getFileName(), getFileName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
return res;
}
@ -128,7 +128,7 @@ void WriteBufferFromFileDescriptor::doTruncate(off_t length)
{
int res = ftruncate(fd, length);
if (-1 == res)
throwFromErrno("Cannot truncate file " + getFileName(), ErrorCodes::CANNOT_TRUNCATE_FILE);
throwFromErrno("Cannot truncate file " + getFileName(), getFileName(), ErrorCodes::CANNOT_TRUNCATE_FILE);
}
}

View File

@ -39,7 +39,7 @@ public:
off_t res = lseek(fd, 0, SEEK_SET);
if (-1 == res)
throwFromErrno("Cannot reread temporary file " + file_name, ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
throwFromErrno("Cannot reread temporary file " + file_name, file_name, ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
return std::make_shared<ReadBufferFromTemporaryWriteBuffer>(fd, file_name, std::move(origin->tmp_file));
}

View File

@ -565,7 +565,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std::
}
if (link(first_file_tmp_path.data(), block_file_path.data()))
throwFromErrno("Could not link " + block_file_path + " to " + first_file_tmp_path, ErrorCodes::CANNOT_LINK);
throwFromErrno("Could not link " + block_file_path + " to " + first_file_tmp_path, block_file_path, ErrorCodes::CANNOT_LINK);
}
/** remove the temporary file, enabling the OS to reclaim inode after all threads

View File

@ -109,7 +109,7 @@ public:
{
struct statvfs fs;
if (statvfs(path.c_str(), &fs) != 0)
throwFromErrno("Could not calculate available disk space (statvfs)", ErrorCodes::CANNOT_STATVFS);
throwFromErrno("Could not calculate available disk space (statvfs)", path, ErrorCodes::CANNOT_STATVFS);
return fs;
}
@ -166,7 +166,7 @@ public:
{
struct stat st;
if (stat(p.c_str(), &st))
throwFromErrno("Cannot stat " + p.string(), errno);
throwFromErrno("Cannot stat " + p.string(), p.string(), ErrorCodes::SYSTEM_ERROR);
return st.st_dev;
};

View File

@ -413,7 +413,7 @@ void MergeTreeDataPart::remove() const
{
String path_to_remove = to + "/" + file;
if (0 != unlink(path_to_remove.c_str()))
throwFromErrno("Cannot unlink file " + path_to_remove, ErrorCodes::CANNOT_UNLINK);
throwFromErrno("Cannot unlink file " + path_to_remove, path_to_remove, ErrorCodes::CANNOT_UNLINK);
}
#if !__clang__
#pragma GCC diagnostic pop
@ -423,11 +423,11 @@ void MergeTreeDataPart::remove() const
{
String path_to_remove = to + "/" + file;
if (0 != unlink(path_to_remove.c_str()))
throwFromErrno("Cannot unlink file " + path_to_remove, ErrorCodes::CANNOT_UNLINK);
throwFromErrno("Cannot unlink file " + path_to_remove, path_to_remove, ErrorCodes::CANNOT_UNLINK);
}
if (0 != rmdir(to.c_str()))
throwFromErrno("Cannot rmdir file " + to, ErrorCodes::CANNOT_UNLINK);
throwFromErrno("Cannot rmdir file " + to, to, ErrorCodes::CANNOT_UNLINK);
}
catch (...)
{

View File

@ -214,7 +214,7 @@ StorageStripeLog::StorageStripeLog(
{
/// create files if they do not exist
if (0 != mkdir(full_path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST)
throwFromErrno("Cannot create directory " + full_path, ErrorCodes::CANNOT_CREATE_DIRECTORY);
throwFromErrno("Cannot create directory " + full_path, full_path, ErrorCodes::CANNOT_CREATE_DIRECTORY);
}
}

View File

@ -343,7 +343,7 @@ StorageTinyLog::StorageTinyLog(
{
/// create files if they do not exist
if (0 != mkdir(full_path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST)
throwFromErrno("Cannot create directory " + full_path, ErrorCodes::CANNOT_CREATE_DIRECTORY);
throwFromErrno("Cannot create directory " + full_path, full_path, ErrorCodes::CANNOT_CREATE_DIRECTORY);
}
for (const auto & col : getColumns().getAllPhysical())

View File

@ -22,7 +22,7 @@ try
Poco::File("./test_dir/file").createFile();
if (0 != symlink("./test_dir", "./test_link"))
DB::throwFromErrno("Cannot create symlink", DB::ErrorCodes::SYSTEM_ERROR);
DB::throwFromErrno("Cannot create symlink", "./test_link", DB::ErrorCodes::SYSTEM_ERROR);
Poco::File link("./test_link");
link.renameTo("./test_link2");