Better text_log with ErrnoException

This commit is contained in:
Raúl Marín 2023-12-15 19:25:49 +01:00
parent 5b88c4ef05
commit b269f87f4c
78 changed files with 353 additions and 306 deletions

View File

@ -489,7 +489,7 @@ When using functions with response codes or `errno`, always check the result and
``` cpp
if (0 != close(fd))
throwFromErrno("Cannot close file " + file_name, ErrorCodes::CANNOT_CLOSE_FILE);
throw ErrnoException(ErrorCodes::CANNOT_CLOSE_FILE, "Cannot close file {}", file_name);
```
You can use assert to check invariant in code.

View File

@ -493,7 +493,7 @@ catch (const DB::Exception & e)
``` cpp
if (0 != close(fd))
throwFromErrno("Cannot close file " + file_name, ErrorCodes::CANNOT_CLOSE_FILE);
throw ErrnoException(ErrorCodes::CANNOT_CLOSE_FILE, "Cannot close file {}", file_name);
```
`assert` не используются.

View File

@ -485,7 +485,7 @@ catch (const DB::Exception & e)
``` cpp
if (0 != close(fd))
throwFromErrno("Cannot close file " + file_name, ErrorCodes::CANNOT_CLOSE_FILE);
throw ErrnoException(ErrorCodes::CANNOT_CLOSE_FILE, "Cannot close file {}", file_name);
```
`不要使用断言`

View File

@ -405,7 +405,7 @@ private:
|| sigaddset(&sig_set, SIGINT)
|| pthread_sigmask(SIG_BLOCK, &sig_set, nullptr))
{
throwFromErrno("Cannot block signal.", ErrorCodes::CANNOT_BLOCK_SIGNAL);
throw ErrnoException(ErrorCodes::CANNOT_BLOCK_SIGNAL, "Cannot block signal");
}
while (true)

View File

@ -328,7 +328,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
fs::create_symlink(binary_self_canonical_path, main_bin_path);
if (0 != chmod(binary_self_canonical_path.string().c_str(), S_IRUSR | S_IRGRP | S_IROTH | S_IXUSR | S_IXGRP | S_IXOTH))
throwFromErrno(fmt::format("Cannot chmod {}", binary_self_canonical_path.string()), ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot chmod {}", binary_self_canonical_path.string());
}
}
else
@ -361,7 +361,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
if (already_installed)
{
if (0 != chmod(main_bin_path.string().c_str(), S_IRUSR | S_IRGRP | S_IROTH | S_IXUSR | S_IXGRP | S_IXOTH))
throwFromErrno(fmt::format("Cannot chmod {}", main_bin_path.string()), ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot chmod {}", main_bin_path.string());
}
else
{
@ -395,7 +395,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
}
if (0 != chmod(destination.c_str(), S_IRUSR | S_IRGRP | S_IROTH | S_IXUSR | S_IXGRP | S_IXOTH))
throwFromErrno(fmt::format("Cannot chmod {}", main_bin_tmp_path.string()), ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot chmod {}", main_bin_tmp_path.string());
}
catch (const Exception & e)
{
@ -1122,7 +1122,7 @@ namespace
return 0;
}
else
throwFromErrno(fmt::format("Cannot obtain the status of pid {} with `kill`", pid), ErrorCodes::CANNOT_KILL);
throw ErrnoException(ErrorCodes::CANNOT_KILL, "Cannot obtain the status of pid {} with `kill`", pid);
}
if (!pid)
@ -1143,7 +1143,7 @@ namespace
if (0 == kill(pid, signal))
fmt::print("Sent {} signal to process with pid {}.\n", signal_name, pid);
else
throwFromErrno(fmt::format("Cannot send {} signal", signal_name), ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot send {} signal", signal_name);
size_t try_num = 0;
for (; try_num < max_tries; ++try_num)

View File

@ -1307,7 +1307,7 @@ try
/// stdin must be seekable
auto res = lseek(file->getFD(), 0, SEEK_SET);
if (-1 == res)
throwFromErrno("Input must be seekable file (it will be read twice).", ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
throw ErrnoException(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Input must be seekable file (it will be read twice)");
SingleReadBufferIterator read_buffer_iterator(std::move(file));
schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, false, context_const);
@ -1336,7 +1336,7 @@ try
/// stdin must be seekable
auto res = lseek(file_in.getFD(), 0, SEEK_SET);
if (-1 == res)
throwFromErrno("Input must be seekable file (it will be read twice).", ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
throw ErrnoException(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Input must be seekable file (it will be read twice)");
}
Obfuscator obfuscator(header, seed, markov_model_params);

View File

@ -56,7 +56,7 @@ void setUserAndGroup(std::string arg_uid, std::string arg_gid)
group * result{};
if (0 != getgrnam_r(arg_gid.data(), &entry, buf.get(), buf_size, &result))
throwFromErrno(fmt::format("Cannot do 'getgrnam_r' to obtain gid from group name ({})", arg_gid), ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot do 'getgrnam_r' to obtain gid from group name ({})", arg_gid);
if (!result)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Group {} is not found in the system", arg_gid);
@ -68,7 +68,7 @@ void setUserAndGroup(std::string arg_uid, std::string arg_gid)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Group has id 0, but dropping privileges to gid 0 does not make sense");
if (0 != setgid(gid))
throwFromErrno(fmt::format("Cannot do 'setgid' to user ({})", arg_gid), ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot do 'setgid' to user ({})", arg_gid);
}
if (!arg_uid.empty())
@ -81,7 +81,7 @@ void setUserAndGroup(std::string arg_uid, std::string arg_gid)
passwd * result{};
if (0 != getpwnam_r(arg_uid.data(), &entry, buf.get(), buf_size, &result))
throwFromErrno(fmt::format("Cannot do 'getpwnam_r' to obtain uid from user name ({})", arg_uid), ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot do 'getpwnam_r' to obtain uid from user name ({})", arg_uid);
if (!result)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "User {} is not found in the system", arg_uid);
@ -93,7 +93,7 @@ void setUserAndGroup(std::string arg_uid, std::string arg_gid)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "User has id 0, but dropping privileges to uid 0 does not make sense");
if (0 != setuid(uid))
throwFromErrno(fmt::format("Cannot do 'setuid' to user ({})", arg_uid), ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot do 'setuid' to user ({})", arg_uid);
}
}
@ -136,7 +136,7 @@ try
execvp(new_argv.front(), new_argv.data());
throwFromErrno("Cannot execvp", ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot execvp");
}
catch (...)
{

View File

@ -318,14 +318,14 @@ void ClientBase::setupSignalHandler()
sigemptyset(&new_act.sa_mask);
#else
if (sigemptyset(&new_act.sa_mask))
throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler");
#endif
if (sigaction(SIGINT, &new_act, nullptr))
throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler");
if (sigaction(SIGQUIT, &new_act, nullptr))
throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler");
}
@ -543,16 +543,16 @@ try
if (!pager.empty())
{
if (SIG_ERR == signal(SIGPIPE, SIG_IGN))
throwFromErrno("Cannot set signal handler for SIGPIPE.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler for SIGPIPE");
/// We need to reset signals that had been installed in the
/// setupSignalHandler() since terminal will send signals to both
/// processes and so signals will be delivered to the
/// clickhouse-client/local as well, which will be terminated when
/// signal will be delivered second time.
if (SIG_ERR == signal(SIGINT, SIG_IGN))
throwFromErrno("Cannot set signal handler for SIGINT.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler for SIGINT");
if (SIG_ERR == signal(SIGQUIT, SIG_IGN))
throwFromErrno("Cannot set signal handler for SIGQUIT.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler for SIGQUIT");
ShellCommand::Config config(pager);
config.pipe_stdin_only = true;
@ -1306,11 +1306,11 @@ void ClientBase::resetOutput()
pager_cmd->wait();
if (SIG_ERR == signal(SIGPIPE, SIG_DFL))
throwFromErrno("Cannot set signal handler for SIIGPIEP.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler for SIGPIPE");
if (SIG_ERR == signal(SIGINT, SIG_DFL))
throwFromErrno("Cannot set signal handler for SIGINT.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler for SIGINT");
if (SIG_ERR == signal(SIGQUIT, SIG_DFL))
throwFromErrno("Cannot set signal handler for SIGQUIT.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler for SIGQUIT");
setupSignalHandler();
}

View File

@ -18,9 +18,11 @@ void AlignedBuffer::alloc(size_t size, size_t alignment)
void * new_buf;
int res = ::posix_memalign(&new_buf, std::max(alignment, sizeof(void*)), size);
if (0 != res)
throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign), size: {}, alignment: {}.",
ReadableSize(size), ReadableSize(alignment)),
ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);
throw ErrnoException(
ErrorCodes::CANNOT_ALLOCATE_MEMORY,
"Cannot allocate memory (posix_memalign), size: {}, alignment: {}.",
ReadableSize(size),
ReadableSize(alignment));
buf = new_buf;
}

View File

@ -118,8 +118,11 @@ public:
void * new_buf = ::realloc(buf, new_size);
if (nullptr == new_buf)
{
DB::throwFromErrno(
fmt::format("Allocator: Cannot realloc from {} to {}.", ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
throw DB::ErrnoException(
DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY,
"Allocator: Cannot realloc from {} to {}",
ReadableSize(old_size),
ReadableSize(new_size));
}
buf = new_buf;
@ -164,7 +167,7 @@ private:
buf = ::malloc(size);
if (nullptr == buf)
DB::throwFromErrno(fmt::format("Allocator: Cannot malloc {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
throw DB::ErrnoException(DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Allocator: Cannot malloc {}.", ReadableSize(size));
}
else
{
@ -172,8 +175,8 @@ private:
int res = posix_memalign(&buf, alignment, size);
if (0 != res)
DB::throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign) {}.", ReadableSize(size)),
DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);
throw DB::ErrnoException(
DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Cannot allocate memory (posix_memalign) {}.", ReadableSize(size));
if constexpr (clear_memory)
memset(buf, 0, size);

View File

@ -179,13 +179,13 @@ private:
{
ptr = mmap(address_hint, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
if (MAP_FAILED == ptr)
DB::throwFromErrno(fmt::format("Allocator: Cannot mmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
throw ErrnoException(DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Allocator: Cannot mmap {}.", ReadableSize(size));
}
~Chunk()
{
if (ptr && 0 != munmap(ptr, size))
DB::throwFromErrno(fmt::format("Allocator: Cannot munmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_MUNMAP);
throw ErrnoException(DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Allocator: Cannot munmap {}.", ReadableSize(size));
}
Chunk(Chunk && other) noexcept : ptr(other.ptr), size(other.size)

View File

@ -797,7 +797,7 @@ void AsynchronousMetrics::update(TimePoint update_time)
int64_t hz = sysconf(_SC_CLK_TCK);
if (-1 == hz)
throwFromErrno("Cannot call 'sysconf' to obtain system HZ", ErrorCodes::CANNOT_SYSCONF);
throw ErrnoException(ErrorCodes::CANNOT_SYSCONF, "Cannot call 'sysconf' to obtain system HZ");
double multiplier = 1.0 / hz / (std::chrono::duration_cast<std::chrono::nanoseconds>(time_after_previous_update).count() / 1e9);
size_t num_cpus = 0;

View File

@ -69,13 +69,13 @@ public:
int fd = ::open(path.c_str(), O_RDWR | O_CREAT | O_CLOEXEC, 0666);
if (-1 == fd)
DB::throwFromErrnoWithPath("Cannot open file " + path, path, DB::ErrorCodes::CANNOT_OPEN_FILE);
DB::ErrnoException::throwFromPath(DB::ErrorCodes::CANNOT_OPEN_FILE, path, "Cannot open file {}", path);
try
{
int flock_ret = flock(fd, LOCK_EX);
if (-1 == flock_ret)
DB::throwFromErrnoWithPath("Cannot lock file " + path, path, DB::ErrorCodes::CANNOT_OPEN_FILE);
DB::ErrnoException::throwFromPath(DB::ErrorCodes::CANNOT_OPEN_FILE, path, "Cannot lock file {}", path);
if (!file_doesnt_exists)
{
@ -145,7 +145,7 @@ public:
int fd = ::open(path.c_str(), O_RDWR | O_CREAT | O_CLOEXEC, 0666);
if (-1 == fd)
DB::throwFromErrnoWithPath("Cannot open file " + path, path, DB::ErrorCodes::CANNOT_OPEN_FILE);
DB::ErrnoException::throwFromPath(DB::ErrorCodes::CANNOT_OPEN_FILE, path, "Cannot open file {}", path);
try
{

View File

@ -19,7 +19,7 @@ Epoll::Epoll() : events_count(0)
{
epoll_fd = epoll_create1(0);
if (epoll_fd == -1)
throwFromErrno("Cannot open epoll descriptor", DB::ErrorCodes::EPOLL_ERROR);
throw DB::ErrnoException(DB::ErrorCodes::EPOLL_ERROR, "Cannot open epoll descriptor");
}
Epoll::Epoll(Epoll && other) noexcept : epoll_fd(other.epoll_fd), events_count(other.events_count.load())
@ -47,7 +47,7 @@ void Epoll::add(int fd, void * ptr, uint32_t events)
++events_count;
if (epoll_ctl(epoll_fd, EPOLL_CTL_ADD, fd, &event) == -1)
throwFromErrno("Cannot add new descriptor to epoll", DB::ErrorCodes::EPOLL_ERROR);
throw DB::ErrnoException(DB::ErrorCodes::EPOLL_ERROR, "Cannot add new descriptor to epoll");
}
void Epoll::remove(int fd)
@ -55,7 +55,7 @@ void Epoll::remove(int fd)
--events_count;
if (epoll_ctl(epoll_fd, EPOLL_CTL_DEL, fd, nullptr) == -1)
throwFromErrno("Cannot remove descriptor from epoll", DB::ErrorCodes::EPOLL_ERROR);
throw DB::ErrnoException(DB::ErrorCodes::EPOLL_ERROR, "Cannot remove descriptor from epoll");
}
size_t Epoll::getManyReady(int max_events, epoll_event * events_out, int timeout) const
@ -82,7 +82,7 @@ size_t Epoll::getManyReady(int max_events, epoll_event * events_out, int timeout
continue;
}
else
throwFromErrno("Error in epoll_wait", DB::ErrorCodes::EPOLL_ERROR);
throw DB::ErrnoException(DB::ErrorCodes::EPOLL_ERROR, "Error in epoll_wait");
}
else
break;

View File

@ -21,7 +21,7 @@ EventFD::EventFD()
{
fd = eventfd(0 /* initval */, 0 /* flags */);
if (fd == -1)
throwFromErrno("Cannot create eventfd", ErrorCodes::CANNOT_PIPE);
throw ErrnoException(ErrorCodes::CANNOT_PIPE, "Cannot create eventfd");
}
uint64_t EventFD::read() const
@ -33,7 +33,7 @@ uint64_t EventFD::read() const
break;
if (errno != EINTR)
throwFromErrno("Cannot read from eventfd", ErrorCodes::CANNOT_READ_FROM_SOCKET);
throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot read from eventfd");
}
return buf;
@ -47,7 +47,7 @@ bool EventFD::write(uint64_t increase) const
return false;
if (errno != EINTR)
throwFromErrno("Cannot write to eventfd", ErrorCodes::CANNOT_WRITE_TO_SOCKET);
throw ErrnoException(ErrorCodes::CANNOT_WRITE_TO_SOCKET, "Cannot write to eventfd");
}
return true;

View File

@ -1,25 +1,24 @@
#include "Exception.h"
#include <algorithm>
#include <cstring>
#include <cxxabi.h>
#include <cstdlib>
#include <Poco/String.h>
#include <Common/logger_useful.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <cstring>
#include <filesystem>
#include <cxxabi.h>
#include <IO/Operators.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <base/demangle.h>
#include <base/errnoToString.h>
#include <Common/formatReadable.h>
#include <Common/filesystemHelpers.h>
#include <Poco/String.h>
#include <Common/ErrorCodes.h>
#include <Common/LockMemoryExceptionInThread.h>
#include <Common/MemorySanitizer.h>
#include <Common/SensitiveDataMasker.h>
#include <Common/LockMemoryExceptionInThread.h>
#include <filesystem>
#include <Common/filesystemHelpers.h>
#include <Common/formatReadable.h>
#include <Common/logger_useful.h>
#include <Common/config_version.h>
@ -212,17 +211,6 @@ Exception::FramePointers Exception::getStackFramePointers() const
thread_local bool Exception::enable_job_stack_trace = false;
thread_local std::vector<StackTrace::FramePointers> Exception::thread_frame_pointers = {};
void throwFromErrno(const std::string & s, int code, int the_errno)
{
throw ErrnoException(s + ", " + errnoToString(the_errno), code, the_errno);
}
void throwFromErrnoWithPath(const std::string & s, const std::string & path, int code, int the_errno)
{
throw ErrnoException(s + ", " + errnoToString(the_errno), code, the_errno, path);
}
static void tryLogCurrentExceptionImpl(Poco::Logger * logger, const std::string & start_of_message)
{
try

View File

@ -7,9 +7,10 @@
#include <Poco/Exception.h>
#include <base/defines.h>
#include <base/errnoToString.h>
#include <base/scope_guard.h>
#include <Common/StackTrace.h>
#include <Common/LoggingFormatStringHelpers.h>
#include <Common/StackTrace.h>
#include <fmt/format.h>
@ -173,12 +174,61 @@ std::string getExceptionStackTraceString(const std::exception & e);
std::string getExceptionStackTraceString(std::exception_ptr e);
/// Contains an additional member `saved_errno`. See the throwFromErrno function.
/// Contains an additional member `saved_errno`
class ErrnoException : public Exception
{
public:
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(std::string && msg, int code, int with_errno) : Exception(msg, code), saved_errno(with_errno)
{
capture_thread_frame_pointers = thread_frame_pointers;
addMessage(", {}", errnoToString(saved_errno));
}
/// Message must be a compile-time constant
template <typename T>
requires std::is_convertible_v<T, String>
ErrnoException(int code, T && message) : Exception(message, code), saved_errno(errno)
{
capture_thread_frame_pointers = thread_frame_pointers;
addMessage(", {}", errnoToString(saved_errno));
}
// Format message with fmt::format, like the logging functions.
template <typename... Args>
ErrnoException(int code, FormatStringHelper<Args...> fmt, Args &&... args)
: Exception(fmt::format(fmt.fmt_str, std::forward<Args>(args)...), code), saved_errno(errno)
{
capture_thread_frame_pointers = thread_frame_pointers;
message_format_string = fmt.message_format_string;
addMessage(", {}", errnoToString(saved_errno));
}
template <typename... Args>
[[noreturn]] static void throwWithErrno(int code, int with_errno, FormatStringHelper<Args...> fmt, Args &&... args)
{
auto e = ErrnoException(fmt::format(fmt.fmt_str, std::forward<Args>(args)...), code, with_errno);
e.message_format_string = fmt.message_format_string;
throw e;
}
template <typename... Args>
[[noreturn]] static void throwFromPath(int code, const std::string & path, FormatStringHelper<Args...> fmt, Args &&... args)
{
auto e = ErrnoException(fmt::format(fmt.fmt_str, std::forward<Args>(args)...), code, errno);
e.message_format_string = fmt.message_format_string;
e.path = path;
throw e;
}
template <typename... Args>
[[noreturn]] static void
throwFromPathWithErrno(int code, const std::string & path, int with_errno, FormatStringHelper<Args...> fmt, Args &&... args)
{
auto e = ErrnoException(fmt::format(fmt.fmt_str, std::forward<Args>(args)...), code, with_errno);
e.message_format_string = fmt.message_format_string;
e.path = path;
throw e;
}
ErrnoException * clone() const override { return new ErrnoException(*this); }
void rethrow() const override { throw *this; } // NOLINT
@ -188,7 +238,7 @@ public:
private:
int saved_errno;
std::optional<std::string> path;
std::optional<std::string> path{};
const char * name() const noexcept override { return "DB::ErrnoException"; }
const char * className() const noexcept override { return "DB::ErrnoException"; }
@ -233,13 +283,6 @@ private:
using Exceptions = std::vector<std::exception_ptr>;
[[noreturn]] void throwFromErrno(const std::string & s, int code, int the_errno = errno);
/// Useful to produce some extra information about available space and inodes on device
[[noreturn]] void throwFromErrnoWithPath(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).
* Can be used in destructors in the catch-all block.
*/

View File

@ -46,14 +46,14 @@ public:
void * vp = ::mmap(nullptr, num_bytes, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
if (MAP_FAILED == vp)
DB::throwFromErrno(fmt::format("FiberStack: Cannot mmap {}.", ReadableSize(num_bytes)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
throw DB::ErrnoException(DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "FiberStack: Cannot mmap {}.", ReadableSize(num_bytes));
/// TODO: make reports on illegal guard page access more clear.
/// Currently we will see segfault and almost random stacktrace.
if (-1 == ::mprotect(vp, page_size, PROT_NONE))
{
::munmap(vp, num_bytes);
DB::throwFromErrno("FiberStack: cannot protect guard page", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
throw DB::ErrnoException(DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "FiberStack: cannot protect guard page");
}
/// Do not count guard page in memory usage.

View File

@ -58,9 +58,8 @@ private:
public:
InterruptListener() : active(false)
{
if (sigemptyset(&sig_set)
|| sigaddset(&sig_set, SIGINT))
throwFromErrno("Cannot manipulate with signal set.", ErrorCodes::CANNOT_MANIPULATE_SIGSET);
if (sigemptyset(&sig_set) || sigaddset(&sig_set, SIGINT))
throw ErrnoException(ErrorCodes::CANNOT_MANIPULATE_SIGSET, "Cannot manipulate with signal set");
block();
}
@ -82,7 +81,7 @@ public:
if (errno == EAGAIN)
return false;
else
throwFromErrno("Cannot poll signal (sigtimedwait).", ErrorCodes::CANNOT_WAIT_FOR_SIGNAL);
throw ErrnoException(ErrorCodes::CANNOT_WAIT_FOR_SIGNAL, "Cannot poll signal (sigtimedwait)");
}
return true;
@ -93,7 +92,7 @@ public:
if (!active)
{
if (pthread_sigmask(SIG_BLOCK, &sig_set, nullptr))
throwFromErrno("Cannot block signal.", ErrorCodes::CANNOT_BLOCK_SIGNAL);
throw ErrnoException(ErrorCodes::CANNOT_BLOCK_SIGNAL, "Cannot block signal");
active = true;
}
@ -105,7 +104,7 @@ public:
if (active)
{
if (pthread_sigmask(SIG_UNBLOCK, &sig_set, nullptr))
throwFromErrno("Cannot unblock signal.", ErrorCodes::CANNOT_UNBLOCK_SIGNAL);
throw ErrnoException(ErrorCodes::CANNOT_UNBLOCK_SIGNAL, "Cannot unblock signal");
active = false;
}

View File

@ -39,7 +39,8 @@ MemoryStatisticsOS::MemoryStatisticsOS()
fd = ::open(filename, O_RDONLY | O_CLOEXEC);
if (-1 == fd)
throwFromErrno("Cannot open file " + std::string(filename), errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE);
ErrnoException::throwFromPath(
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE, filename, "Cannot open file {}", filename);
}
MemoryStatisticsOS::~MemoryStatisticsOS()
@ -48,9 +49,8 @@ MemoryStatisticsOS::~MemoryStatisticsOS()
{
try
{
throwFromErrno(
"File descriptor for \"" + std::string(filename) + "\" could not be closed. "
"Something seems to have gone wrong. Inspect errno.", ErrorCodes::CANNOT_CLOSE_FILE);
ErrnoException::throwFromPath(
ErrorCodes::CANNOT_CLOSE_FILE, filename, "File descriptor for '{}' could not be closed", filename);
}
catch (const ErrnoException &)
{
@ -77,7 +77,7 @@ MemoryStatisticsOS::Data MemoryStatisticsOS::get() const
if (errno == EINTR)
continue;
throwFromErrno("Cannot read from file " + std::string(filename), ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR, filename, "Cannot read from file {}", filename);
}
assert(res >= 0);
@ -136,7 +136,7 @@ MemoryStatisticsOS::Data MemoryStatisticsOS::get() const
size_t len = sizeof(struct kinfo_proc);
if (-1 == ::sysctl(mib, 4, &kp, &len, nullptr, 0))
throwFromErrno("Cannot sysctl(kern.proc.pid." + std::to_string(self) + ")", ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot sysctl(kern.proc.pid.{})", std::to_string(self));
if (sizeof(struct kinfo_proc) != len)
throw DB::Exception(DB::ErrorCodes::SYSTEM_ERROR, "Kernel returns structure of {} bytes instead of expected {}",

View File

@ -117,7 +117,7 @@ struct NetlinkMessage
if (errno == EAGAIN)
continue;
else
throwFromErrno("Can't send a Netlink command", ErrorCodes::NETLINK_ERROR);
throw ErrnoException(ErrorCodes::NETLINK_ERROR, "Can't send a Netlink command");
}
if (bytes_sent > request_size)
@ -255,7 +255,7 @@ NetlinkMetricsProvider::NetlinkMetricsProvider()
{
netlink_socket_fd = ::socket(PF_NETLINK, SOCK_RAW, NETLINK_GENERIC);
if (netlink_socket_fd < 0)
throwFromErrno("Can't create PF_NETLINK socket", ErrorCodes::NETLINK_ERROR);
throw ErrnoException(ErrorCodes::NETLINK_ERROR, "Can't create PF_NETLINK socket");
try
{
@ -267,7 +267,7 @@ NetlinkMetricsProvider::NetlinkMetricsProvider()
tv.tv_usec = 50000;
if (0 != ::setsockopt(netlink_socket_fd, SOL_SOCKET, SO_RCVTIMEO, reinterpret_cast<const char *>(&tv), sizeof(tv)))
throwFromErrno("Can't set timeout on PF_NETLINK socket", ErrorCodes::NETLINK_ERROR);
throw ErrnoException(ErrorCodes::NETLINK_ERROR, "Can't set timeout on PF_NETLINK socket");
union
{
@ -277,7 +277,7 @@ NetlinkMetricsProvider::NetlinkMetricsProvider()
addr.nl_family = AF_NETLINK;
if (::bind(netlink_socket_fd, &sockaddr, sizeof(addr)) < 0)
throwFromErrno("Can't bind PF_NETLINK socket", ErrorCodes::NETLINK_ERROR);
throw ErrnoException(ErrorCodes::NETLINK_ERROR, "Can't bind PF_NETLINK socket");
taskstats_family_id = getFamilyId(netlink_socket_fd);
}

View File

@ -209,7 +209,7 @@ protected:
{
size_t length = right_rounded_down - left_rounded_up;
if (0 != mprotect(left_rounded_up, length, prot))
throwFromErrno("Cannot mprotect memory region", ErrorCodes::CANNOT_MPROTECT);
throw ErrnoException(ErrorCodes::CANNOT_MPROTECT, "Cannot mprotect memory region");
}
}

View File

@ -29,14 +29,14 @@ void LazyPipeFDs::open()
#ifndef OS_DARWIN
if (0 != pipe2(fds_rw, O_CLOEXEC))
throwFromErrno("Cannot create pipe", ErrorCodes::CANNOT_PIPE);
throw ErrnoException(ErrorCodes::CANNOT_PIPE, "Cannot create pipe");
#else
if (0 != pipe(fds_rw))
throwFromErrno("Cannot create pipe", ErrorCodes::CANNOT_PIPE);
throw ErrnoException(ErrorCodes::CANNOT_PIPE, "Cannot create pipe");
if (0 != fcntl(fds_rw[0], F_SETFD, FD_CLOEXEC))
throwFromErrno("Cannot setup auto-close on exec for read end of pipe", ErrorCodes::CANNOT_FCNTL);
throw ErrnoException(ErrorCodes::CANNOT_FCNTL, "Cannot setup auto-close on exec for read end of pipe");
if (0 != fcntl(fds_rw[1], F_SETFD, FD_CLOEXEC))
throwFromErrno("Cannot setup auto-close on exec for write end of pipe", ErrorCodes::CANNOT_FCNTL);
throw ErrnoException(ErrorCodes::CANNOT_FCNTL, "Cannot setup auto-close on exec for write end of pipe");
#endif
}
@ -47,7 +47,7 @@ void LazyPipeFDs::close()
if (fd < 0)
continue;
if (0 != ::close(fd))
throwFromErrno("Cannot close pipe", ErrorCodes::CANNOT_PIPE);
throw ErrnoException(ErrorCodes::CANNOT_PIPE, "Cannot close pipe");
fd = -1;
}
}
@ -74,18 +74,18 @@ void LazyPipeFDs::setNonBlockingWrite()
{
int flags = fcntl(fds_rw[1], F_GETFL, 0);
if (-1 == flags)
throwFromErrno("Cannot get file status flags of pipe", ErrorCodes::CANNOT_FCNTL);
throw ErrnoException(ErrorCodes::CANNOT_FCNTL, "Cannot get file status flags of pipe");
if (-1 == fcntl(fds_rw[1], F_SETFL, flags | O_NONBLOCK))
throwFromErrno("Cannot set non-blocking mode of pipe", ErrorCodes::CANNOT_FCNTL);
throw ErrnoException(ErrorCodes::CANNOT_FCNTL, "Cannot set non-blocking mode of pipe");
}
void LazyPipeFDs::setNonBlockingRead()
{
int flags = fcntl(fds_rw[0], F_GETFL, 0);
if (-1 == flags)
throwFromErrno("Cannot get file status flags of pipe", ErrorCodes::CANNOT_FCNTL);
throw ErrnoException(ErrorCodes::CANNOT_FCNTL, "Cannot get file status flags of pipe");
if (-1 == fcntl(fds_rw[0], F_SETFL, flags | O_NONBLOCK))
throwFromErrno("Cannot set non-blocking mode of pipe", ErrorCodes::CANNOT_FCNTL);
throw ErrnoException(ErrorCodes::CANNOT_FCNTL, "Cannot set non-blocking mode of pipe");
}
void LazyPipeFDs::setNonBlockingReadWrite()
@ -110,13 +110,13 @@ void LazyPipeFDs::tryIncreaseSize(int desired_size)
/// It will work nevertheless.
}
else
throwFromErrno("Cannot get pipe capacity", ErrorCodes::CANNOT_FCNTL);
throw ErrnoException(ErrorCodes::CANNOT_FCNTL, "Cannot get pipe capacity");
}
else
{
for (errno = 0; errno != EPERM && pipe_size < desired_size; pipe_size *= 2)
if (-1 == fcntl(fds_rw[1], F_SETPIPE_SZ, pipe_size * 2) && errno != EPERM)
throwFromErrno("Cannot increase pipe capacity to " + std::to_string(pipe_size * 2), ErrorCodes::CANNOT_FCNTL);
throw ErrnoException(ErrorCodes::CANNOT_FCNTL, "Cannot increase pipe capacity to {}", pipe_size * 2);
LOG_TRACE(log, "Pipe capacity is {}", ReadableSize(std::min(pipe_size, desired_size)));
}

View File

@ -37,18 +37,15 @@ namespace
{
[[noreturn]] inline void throwWithFailedToOpenFile(const std::string & filename)
{
throwFromErrno(
"Cannot open file " + filename,
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE);
ErrnoException::throwFromPath(
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE, filename, "Cannot open file {}", filename);
}
inline void emitErrorMsgWithFailedToCloseFile(const std::string & filename)
{
try
{
throwFromErrno(
"File descriptor for \"" + filename + "\" could not be closed. "
"Something seems to have gone wrong. Inspect errno.", ErrorCodes::CANNOT_CLOSE_FILE);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_CLOSE_FILE, filename, "File descriptor for {} could not be closed", filename);
}
catch (const ErrnoException &)
{
@ -69,9 +66,7 @@ ssize_t readFromFD(const int fd, const char * filename, char * buf, size_t buf_s
if (errno == EINTR)
continue;
throwFromErrno(
"Cannot read from file " + std::string(filename),
ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR, filename, "Cannot read from file {}", filename);
}
assert(res >= 0);

View File

@ -141,7 +141,7 @@ void Timer::createIfNecessary(UInt64 thread_id, int clock_type, int pause_signal
/// Also, it cannot be created if the server has too many threads.
throwFromErrno("Failed to create thread timer", ErrorCodes::CANNOT_CREATE_TIMER);
throw ErrnoException(ErrorCodes::CANNOT_CREATE_TIMER, "Failed to create thread timer");
}
timer_id.emplace(local_timer_id);
CurrentMetrics::add(CurrentMetrics::CreatedTimersInQueryProfiler);
@ -164,7 +164,7 @@ void Timer::set(UInt32 period)
struct itimerspec timer_spec = {.it_interval = interval, .it_value = offset};
if (timer_settime(*timer_id, 0, &timer_spec, nullptr))
throwFromErrno("Failed to set thread timer period", ErrorCodes::CANNOT_SET_TIMER_PERIOD);
throw ErrnoException(ErrorCodes::CANNOT_SET_TIMER_PERIOD, "Failed to set thread timer period");
CurrentMetrics::add(CurrentMetrics::ActiveTimersInQueryProfiler);
}
@ -238,13 +238,13 @@ QueryProfilerBase<ProfilerImpl>::QueryProfilerBase(UInt64 thread_id, int clock_t
sa.sa_flags = SA_SIGINFO | SA_RESTART;
if (sigemptyset(&sa.sa_mask))
throwFromErrno("Failed to clean signal mask for query profiler", ErrorCodes::CANNOT_MANIPULATE_SIGSET);
throw ErrnoException(ErrorCodes::CANNOT_MANIPULATE_SIGSET, "Failed to clean signal mask for query profiler");
if (sigaddset(&sa.sa_mask, pause_signal))
throwFromErrno("Failed to add signal to mask for query profiler", ErrorCodes::CANNOT_MANIPULATE_SIGSET);
throw ErrnoException(ErrorCodes::CANNOT_MANIPULATE_SIGSET, "Failed to add signal to mask for query profiler");
if (sigaction(pause_signal, &sa, nullptr))
throwFromErrno("Failed to setup signal handler for query profiler", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
throw ErrnoException(ErrorCodes::CANNOT_MANIPULATE_SIGSET, "Failed to setup signal handler for query profiler");
try
{

View File

@ -145,7 +145,7 @@ std::unique_ptr<ShellCommand> ShellCommand::executeImpl(
#endif
if (!real_vfork)
throwFromErrno("Cannot find symbol vfork in myself", ErrorCodes::CANNOT_DLSYM);
throw ErrnoException(ErrorCodes::CANNOT_DLSYM, "Cannot find symbol vfork in myself");
PipeFDs pipe_stdin;
PipeFDs pipe_stdout;
@ -163,7 +163,7 @@ std::unique_ptr<ShellCommand> ShellCommand::executeImpl(
pid_t pid = reinterpret_cast<pid_t(*)()>(real_vfork)();
if (pid == -1)
throwFromErrno("Cannot vfork", ErrorCodes::CANNOT_FORK);
throw ErrnoException(ErrorCodes::CANNOT_FORK, "Cannot vfork");
if (0 == pid)
{
@ -305,7 +305,7 @@ int ShellCommand::tryWait()
while (waitpid(pid, &status, 0) < 0)
{
if (errno != EINTR)
throwFromErrno("Cannot waitpid", ErrorCodes::CANNOT_WAITPID);
throw ErrnoException(ErrorCodes::CANNOT_WAITPID, "Cannot waitpid");
}
LOG_TRACE(getLogger(), "Wait for shell command pid {} completed with status {}", pid, status);

View File

@ -64,7 +64,7 @@ StatusFile::StatusFile(std::string path_, FillFunction fill_)
fd = ::open(path.c_str(), O_WRONLY | O_CREAT | O_CLOEXEC, 0666);
if (-1 == fd)
throwFromErrnoWithPath("Cannot open file " + path, path, ErrorCodes::CANNOT_OPEN_FILE);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_OPEN_FILE, path, "Cannot open file {}", path);
try
{
@ -74,14 +74,14 @@ StatusFile::StatusFile(std::string path_, FillFunction fill_)
if (errno == EWOULDBLOCK)
throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Cannot lock file {}. Another server instance in same directory is already running.", path);
else
throwFromErrnoWithPath("Cannot lock file " + path, path, ErrorCodes::CANNOT_OPEN_FILE);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_OPEN_FILE, path, "Cannot lock file {}", path);
}
if (0 != ftruncate(fd, 0))
throwFromErrnoWithPath("Cannot ftruncate " + path, path, ErrorCodes::CANNOT_TRUNCATE_FILE);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_TRUNCATE_FILE, path, "Cannot ftruncate file {}", path);
if (0 != lseek(fd, 0, SEEK_SET))
throwFromErrnoWithPath("Cannot lseek " + path, path, ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, path, "Cannot lseek file {}", path);
/// Write information about current server instance to the file.
WriteBufferFromFileDescriptor out(fd, 1024);

View File

@ -19,12 +19,12 @@ uint16_t getTerminalWidth()
if (isatty(STDIN_FILENO))
{
if (ioctl(STDIN_FILENO, TIOCGWINSZ, &terminal_size))
DB::throwFromErrno("Cannot obtain terminal window size (ioctl TIOCGWINSZ)", DB::ErrorCodes::SYSTEM_ERROR);
throw DB::ErrnoException(DB::ErrorCodes::SYSTEM_ERROR, "Cannot obtain terminal window size (ioctl TIOCGWINSZ)");
}
else if (isatty(STDERR_FILENO))
{
if (ioctl(STDERR_FILENO, TIOCGWINSZ, &terminal_size))
DB::throwFromErrno("Cannot obtain terminal window size (ioctl TIOCGWINSZ)", DB::ErrorCodes::SYSTEM_ERROR);
throw DB::ErrnoException(DB::ErrorCodes::SYSTEM_ERROR, "Cannot obtain terminal window size (ioctl TIOCGWINSZ)");
}
/// Default - 0.
return terminal_size.ws_col;

View File

@ -258,10 +258,10 @@ void ThreadFuzzer::setup() const
#if defined(OS_LINUX)
if (sigemptyset(&sa.sa_mask))
throwFromErrno("Failed to clean signal mask for thread fuzzer", ErrorCodes::CANNOT_MANIPULATE_SIGSET);
throw ErrnoException(ErrorCodes::CANNOT_MANIPULATE_SIGSET, "Failed to clean signal mask for thread fuzzer");
if (sigaddset(&sa.sa_mask, SIGPROF))
throwFromErrno("Failed to add signal to mask for thread fuzzer", ErrorCodes::CANNOT_MANIPULATE_SIGSET);
throw ErrnoException(ErrorCodes::CANNOT_MANIPULATE_SIGSET, "Failed to add signal to mask for thread fuzzer");
#else
// the two following functions always return 0 under mac
sigemptyset(&sa.sa_mask);
@ -269,7 +269,7 @@ void ThreadFuzzer::setup() const
#endif
if (sigaction(SIGPROF, &sa, nullptr))
throwFromErrno("Failed to setup signal handler for thread fuzzer", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
throw ErrnoException(ErrorCodes::CANNOT_MANIPULATE_SIGSET, "Failed to setup signal handler for thread fuzzer");
static constexpr UInt32 timer_precision = 1000000;
@ -280,7 +280,7 @@ void ThreadFuzzer::setup() const
struct itimerval timer = {.it_interval = interval, .it_value = interval};
if (0 != setitimer(ITIMER_PROF, &timer, nullptr))
throwFromErrno("Failed to create profiling timer", ErrorCodes::CANNOT_CREATE_TIMER);
throw ErrnoException(ErrorCodes::CANNOT_CREATE_TIMER, "Failed to create profiling timer");
}

View File

@ -24,7 +24,7 @@ TimerDescriptor::TimerDescriptor(int clockid, int flags)
throw Exception(ErrorCodes::CANNOT_CREATE_TIMER, "Cannot create timer_fd descriptor");
if (-1 == fcntl(timer_fd, F_SETFL, O_NONBLOCK))
throwFromErrno("Cannot set O_NONBLOCK for timer_fd", ErrorCodes::CANNOT_FCNTL);
throw ErrnoException(ErrorCodes::CANNOT_FCNTL, "Cannot set O_NONBLOCK for timer_fd");
}
TimerDescriptor::TimerDescriptor(TimerDescriptor && other) noexcept : timer_fd(other.timer_fd)
@ -57,7 +57,7 @@ void TimerDescriptor::reset() const
spec.it_value.tv_nsec = 0;
if (-1 == timerfd_settime(timer_fd, 0 /*relative timer */, &spec, nullptr))
throwFromErrno("Cannot reset timer_fd", ErrorCodes::CANNOT_SET_TIMER_PERIOD);
throw ErrnoException(ErrorCodes::CANNOT_SET_TIMER_PERIOD, "Cannot reset timer_fd");
/// Drain socket.
/// It may be possible that alarm happened and socket is readable.
@ -78,7 +78,7 @@ void TimerDescriptor::drain() const
break;
if (errno != EINTR)
throwFromErrno("Cannot drain timer_fd", ErrorCodes::CANNOT_READ_FROM_SOCKET);
throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot drain timer_fd");
}
}
}
@ -94,7 +94,7 @@ void TimerDescriptor::setRelative(uint64_t usec) const
spec.it_value.tv_nsec = (usec % TIMER_PRECISION) * 1'000;
if (-1 == timerfd_settime(timer_fd, 0 /*relative timer */, &spec, nullptr))
throwFromErrno("Cannot set time for timer_fd", ErrorCodes::CANNOT_SET_TIMER_PERIOD);
throw ErrnoException(ErrorCodes::CANNOT_SET_TIMER_PERIOD, "Cannot set time for timer_fd");
}
void TimerDescriptor::setRelative(Poco::Timespan timespan) const

View File

@ -31,7 +31,8 @@ namespace
const auto error = getpwuid_r(user_id, &passwd_entry, buffer.data(), buffer_size, &result);
if (error)
throwFromErrno("Failed to find user name for " + std::to_string(user_id), ErrorCodes::FAILED_TO_GETPWUID, error);
ErrnoException::throwWithErrno(
ErrorCodes::FAILED_TO_GETPWUID, error, "Failed to find user name for {}", std::to_string(user_id));
else if (result)
return result->pw_name;
return std::to_string(user_id);

View File

@ -87,10 +87,12 @@ static bool renameat2(const std::string & old_path, const std::string & new_path
return false;
if (errno == EEXIST)
throwFromErrno(fmt::format("Cannot rename {} to {} because the second path already exists", old_path, new_path), ErrorCodes::ATOMIC_RENAME_FAIL);
throw ErrnoException(
ErrorCodes::ATOMIC_RENAME_FAIL, "Cannot rename {} to {} because the second path already exists", old_path, new_path);
if (errno == ENOENT)
throwFromErrno(fmt::format("Paths cannot be exchanged because {} or {} does not exist", old_path, new_path), ErrorCodes::ATOMIC_RENAME_FAIL);
throwFromErrnoWithPath(fmt::format("Cannot rename {} to {}", old_path, new_path), new_path, ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(
ErrorCodes::ATOMIC_RENAME_FAIL, "Paths cannot be exchanged because {} or {} does not exist", old_path, new_path);
ErrnoException::throwFromPath(ErrorCodes::SYSTEM_ERROR, new_path, "Cannot rename {} to {}", old_path, new_path);
}
bool supportsAtomicRename()
@ -139,11 +141,12 @@ static bool renameat2(const std::string & old_path, const std::string & new_path
if (errnum == ENOTSUP || errnum == EINVAL)
return false;
if (errnum == EEXIST)
throwFromErrno(fmt::format("Cannot rename {} to {} because the second path already exists", old_path, new_path), ErrorCodes::ATOMIC_RENAME_FAIL);
throw ErrnoException(
ErrorCodes::ATOMIC_RENAME_FAIL, "Cannot rename {} to {} because the second path already exists", old_path, new_path);
if (errnum == ENOENT)
throwFromErrno(fmt::format("Paths cannot be exchanged because {} or {} does not exist", old_path, new_path), ErrorCodes::ATOMIC_RENAME_FAIL);
throwFromErrnoWithPath(
fmt::format("Cannot rename {} to {}: {}", old_path, new_path, strerror(errnum)), new_path, ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(
ErrorCodes::ATOMIC_RENAME_FAIL, "Paths cannot be exchanged because {} or {} does not exist", old_path, new_path);
ErrnoException::throwFromPath(ErrorCodes::SYSTEM_ERROR, new_path, "Cannot rename {} to {}", old_path, new_path);
}

View File

@ -54,7 +54,7 @@ static size_t getStackSize(void ** out_address)
# if defined(OS_FREEBSD) || defined(OS_SUNOS)
pthread_attr_init(&attr);
if (0 != pthread_attr_get_np(pthread_self(), &attr))
throwFromErrno("Cannot pthread_attr_get_np", ErrorCodes::CANNOT_PTHREAD_ATTR);
throw ErrnoException(ErrorCodes::CANNOT_PTHREAD_ATTR, "Cannot pthread_attr_get_np");
# else
if (0 != pthread_getattr_np(pthread_self(), &attr))
{
@ -64,14 +64,14 @@ static size_t getStackSize(void ** out_address)
return 0;
}
else
throwFromErrno("Cannot pthread_getattr_np", ErrorCodes::CANNOT_PTHREAD_ATTR);
throw ErrnoException(ErrorCodes::CANNOT_PTHREAD_ATTR, "Cannot pthread_getattr_np");
}
# endif
SCOPE_EXIT({ pthread_attr_destroy(&attr); });
if (0 != pthread_attr_getstack(&attr, &address, &size))
throwFromErrno("Cannot pthread_getattr_np", ErrorCodes::CANNOT_PTHREAD_ATTR);
throw ErrnoException(ErrorCodes::CANNOT_PTHREAD_ATTR, "Cannot pthread_attr_getstack");
#ifdef USE_MUSL
/// Adjust stack size for the main thread under musl.

View File

@ -26,19 +26,21 @@ void createHardLink(const String & source_path, const String & destination_path)
struct stat destination_descr;
if (0 != lstat(source_path.c_str(), &source_descr))
throwFromErrnoWithPath("Cannot stat " + source_path, source_path, ErrorCodes::CANNOT_STAT);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_STAT, source_path, "Cannot stat {}", source_path);
if (0 != lstat(destination_path.c_str(), &destination_descr))
throwFromErrnoWithPath("Cannot stat " + destination_path, destination_path, ErrorCodes::CANNOT_STAT);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_STAT, destination_path, "Cannot stat {}", destination_path);
if (source_descr.st_ino != destination_descr.st_ino)
throwFromErrnoWithPath(
"Destination file " + destination_path + " is already exist and have different inode.",
destination_path, ErrorCodes::CANNOT_LINK, link_errno);
ErrnoException::throwFromPathWithErrno(
ErrorCodes::CANNOT_STAT,
destination_path,
link_errno,
"Destination file {} already exists and has a different inode",
destination_path);
}
else
throwFromErrnoWithPath("Cannot link " + source_path + " to " + destination_path, destination_path,
ErrorCodes::CANNOT_LINK);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_LINK, destination_path, "Cannot link {} to {}", source_path, destination_path);
}
}

View File

@ -248,7 +248,7 @@ int main(int argc, char ** argv)
rusage resource_usage;
if (0 != getrusage(RUSAGE_SELF, &resource_usage))
throwFromErrno("Cannot getrusage", ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot getrusage");
size_t allocated_bytes = resource_usage.ru_maxrss * 1024;
std::cerr << "Current memory usage: " << allocated_bytes << " bytes.\n";

View File

@ -82,9 +82,9 @@ int main(int argc, char ** argv)
{
pthread_t thread;
if (pthread_create(&thread, nullptr, g, nullptr))
DB::throwFromErrno("Cannot create thread.", DB::ErrorCodes::PTHREAD_ERROR);
throw DB::ErrnoException(DB::ErrorCodes::PTHREAD_ERROR, "Cannot create thread");
if (pthread_join(thread, nullptr))
DB::throwFromErrno("Cannot join thread.", DB::ErrorCodes::PTHREAD_ERROR);
throw DB::ErrnoException(DB::ErrorCodes::PTHREAD_ERROR, "Cannot join thread");
});
test(n, "Create and destroy std::thread each iteration", []

View File

@ -49,7 +49,7 @@ struct statvfs getStatVFS(const String & path)
{
if (errno == EINTR)
continue;
throwFromErrnoWithPath("Could not calculate available disk space (statvfs)", path, ErrorCodes::CANNOT_STATVFS);
DB::ErrnoException::throwFromPath(DB::ErrorCodes::CANNOT_STATVFS, path, "Could not calculate available disk space (statvfs)");
}
return fs;
}
@ -79,7 +79,7 @@ String getBlockDeviceId([[maybe_unused]] const String & path)
#if defined(OS_LINUX)
struct stat sb;
if (lstat(path.c_str(), &sb))
throwFromErrnoWithPath("Cannot lstat " + path, path, ErrorCodes::CANNOT_STAT);
DB::ErrnoException::throwFromPath(DB::ErrorCodes::CANNOT_STAT, path, "Cannot lstat {}", path);
WriteBufferFromOwnString ss;
ss << major(sb.st_dev) << ":" << minor(sb.st_dev);
return ss.str();
@ -164,7 +164,7 @@ std::filesystem::path getMountPoint(std::filesystem::path absolute_path)
{
struct stat st;
if (stat(p.c_str(), &st)) /// NOTE: man stat does not list EINTR as possible error
throwFromErrnoWithPath("Cannot stat " + p.string(), p.string(), ErrorCodes::SYSTEM_ERROR);
DB::ErrnoException::throwFromPath(DB::ErrorCodes::SYSTEM_ERROR, p.string(), "Cannot stat {}", p.string());
return st.st_dev;
};
@ -250,10 +250,8 @@ size_t getSizeFromFileDescriptor(int fd, const String & file_name)
int res = fstat(fd, &buf);
if (-1 == res)
{
throwFromErrnoWithPath(
"Cannot execute fstat" + (file_name.empty() ? "" : " file: " + file_name),
file_name,
ErrorCodes::CANNOT_FSTAT);
DB::ErrnoException::throwFromPath(
DB::ErrorCodes::CANNOT_FSTAT, file_name, "Cannot execute fstat{}", file_name.empty() ? "" : " file: " + file_name);
}
return buf.st_size;
}
@ -263,10 +261,7 @@ Int64 getINodeNumberFromPath(const String & path)
struct stat file_stat;
if (stat(path.data(), &file_stat))
{
throwFromErrnoWithPath(
"Cannot execute stat for file " + path,
path,
ErrorCodes::CANNOT_STAT);
DB::ErrnoException::throwFromPath(DB::ErrorCodes::CANNOT_STAT, path, "Cannot execute stat for file {}", path);
}
return file_stat.st_ino;
}
@ -302,7 +297,7 @@ bool createFile(const std::string & path)
close(n);
return true;
}
DB::throwFromErrnoWithPath("Cannot create file: " + path, path, DB::ErrorCodes::CANNOT_CREATE_FILE);
DB::ErrnoException::throwFromPath(DB::ErrorCodes::CANNOT_CREATE_FILE, path, "Cannot create file: {}", path);
}
bool exists(const std::string & path)
@ -317,7 +312,7 @@ bool canRead(const std::string & path)
return true;
if (errno == EACCES)
return false;
DB::throwFromErrnoWithPath("Cannot check read access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED);
DB::ErrnoException::throwFromPath(DB::ErrorCodes::PATH_ACCESS_DENIED, path, "Cannot check read access to file: {}", path);
}
bool canWrite(const std::string & path)
@ -327,7 +322,7 @@ bool canWrite(const std::string & path)
return true;
if (errno == EACCES)
return false;
DB::throwFromErrnoWithPath("Cannot check write access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED);
DB::ErrnoException::throwFromPath(DB::ErrorCodes::PATH_ACCESS_DENIED, path, "Cannot check write access to file: {}", path);
}
bool canExecute(const std::string & path)
@ -337,7 +332,7 @@ bool canExecute(const std::string & path)
return true;
if (errno == EACCES)
return false;
DB::throwFromErrnoWithPath("Cannot check write access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED);
DB::ErrnoException::throwFromPath(DB::ErrorCodes::PATH_ACCESS_DENIED, path, "Cannot check execute access to file: {}", path);
}
time_t getModificationTime(const std::string & path)
@ -369,7 +364,7 @@ void setModificationTime(const std::string & path, time_t time)
tb.actime = time;
tb.modtime = time;
if (utime(path.c_str(), &tb) != 0)
DB::throwFromErrnoWithPath("Cannot set modification time for file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED);
DB::ErrnoException::throwFromPath(DB::ErrorCodes::PATH_ACCESS_DENIED, path, "Cannot set modification time to file: {}", path);
}
bool isSymlink(const fs::path & path)

View File

@ -27,7 +27,7 @@ static __user_cap_data_struct getCapabilities()
/// Avoid dependency on 'libcap'.
if (0 != syscall(SYS_capget, &request, &response))
throwFromErrno("Cannot do 'capget' syscall", ErrorCodes::NETLINK_ERROR);
throw ErrnoException(ErrorCodes::NETLINK_ERROR, "Cannot do 'capget' syscall");
return response;
}

View File

@ -28,9 +28,7 @@ struct NetworkInterfaces : public boost::noncopyable
NetworkInterfaces()
{
if (getifaddrs(&ifaddr) == -1)
{
throwFromErrno("Cannot getifaddrs", ErrorCodes::SYSTEM_ERROR);
}
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot getifaddrs");
}
bool hasAddress(const Poco::Net::IPAddress & address) const

View File

@ -24,7 +24,7 @@ DB::UInt64 randomSeed()
{
struct timespec times;
if (clock_gettime(CLOCK_MONOTONIC, &times))
DB::throwFromErrno("Cannot clock_gettime.", DB::ErrorCodes::CANNOT_CLOCK_GETTIME);
throw DB::ErrnoException(DB::ErrorCodes::CANNOT_CLOCK_GETTIME, "Cannot clock_gettime");
/// Not cryptographically secure as time, pid and stack address can be predictable.

View File

@ -120,7 +120,7 @@ __attribute__((__noinline__)) void remapToHugeStep1(void * begin, size_t size)
void * scratch = mmap(nullptr, size, PROT_READ | PROT_WRITE | PROT_EXEC, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
if (MAP_FAILED == scratch)
throwFromErrno(fmt::format("Cannot mmap {} bytes", size), ErrorCodes::CANNOT_ALLOCATE_MEMORY);
throw ErrnoException(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Cannot mmap {} bytes", size);
memcpy(scratch, begin, size);

View File

@ -44,7 +44,7 @@ void setThreadName(const char * name)
if (0 != prctl(PR_SET_NAME, name, 0, 0, 0))
#endif
if (errno != ENOSYS && errno != EPERM) /// It's ok if the syscall is unsupported or not allowed in some environments.
DB::throwFromErrno("Cannot set thread name with prctl(PR_SET_NAME, ...)", DB::ErrorCodes::PTHREAD_ERROR);
throw DB::ErrnoException(DB::ErrorCodes::PTHREAD_ERROR, "Cannot set thread name with prctl(PR_SET_NAME, ...)");
memcpy(thread_name, name, std::min<size_t>(1 + strlen(name), THREAD_NAME_SIZE - 1));
}
@ -64,7 +64,7 @@ const char * getThreadName()
#else
if (0 != prctl(PR_GET_NAME, thread_name, 0, 0, 0))
if (errno != ENOSYS && errno != EPERM) /// It's ok if the syscall is unsupported or not allowed in some environments.
DB::throwFromErrno("Cannot get thread name with prctl(PR_GET_NAME)", DB::ErrorCodes::PTHREAD_ERROR);
throw DB::ErrnoException(DB::ErrorCodes::PTHREAD_ERROR, "Cannot get thread name with prctl(PR_GET_NAME)");
#endif
return thread_name;

View File

@ -92,10 +92,10 @@ PipeFDs signal_pipe;
static void call_default_signal_handler(int sig)
{
if (SIG_ERR == signal(sig, SIG_DFL))
throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler");
if (0 != raise(sig))
throwFromErrno("Cannot send signal.", ErrorCodes::CANNOT_SEND_SIGNAL);
throw ErrnoException(ErrorCodes::CANNOT_SEND_SIGNAL, "Cannot send signal");
}
static const size_t signal_pipe_buf_size =
@ -659,7 +659,17 @@ BaseDaemon::~BaseDaemon()
/// Reset signals to SIG_DFL to avoid trying to write to the signal_pipe that will be closed after.
for (int sig : handled_signals)
if (SIG_ERR == signal(sig, SIG_DFL))
throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
{
try
{
throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler");
}
catch (ErrnoException &)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
signal_pipe.close();
}
@ -1129,7 +1139,7 @@ void BaseDaemon::setupWatchdog()
pid = fork();
if (-1 == pid)
throwFromErrno("Cannot fork", ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot fork");
if (0 == pid)
{
@ -1225,7 +1235,7 @@ void BaseDaemon::setupWatchdog()
if (SIG_ERR == signal(sig, SIG_IGN))
{
char * signal_description = strsignal(sig); // NOLINT(concurrency-mt-unsafe)
throwFromErrno(fmt::format("Cannot ignore {}", signal_description), ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot ignore {}", signal_description);
}
}
}
@ -1315,7 +1325,7 @@ void systemdNotify(const std::string_view & command)
int s = socket(AF_UNIX, SOCK_DGRAM | SOCK_CLOEXEC, 0);
if (s == -1)
throwFromErrno("Can't create UNIX socket for systemd notify.", ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Can't create UNIX socket for systemd notify");
SCOPE_EXIT({ close(s); });
@ -1351,7 +1361,7 @@ void systemdNotify(const std::string_view & command)
if (errno == EINTR)
continue;
else
throwFromErrno("Failed to notify systemd, sendto returned error.", ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Failed to notify systemd, sendto returned error");
}
else
sent_bytes_total += sent_bytes;

View File

@ -675,8 +675,11 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(
if (errno == ENOENT && !throw_on_error)
return nullptr;
throwFromErrnoWithPath("Cannot open file " + metadata_file_path, metadata_file_path,
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE);
ErrnoException::throwFromPath(
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE,
metadata_file_path,
"Cannot open file {}",
metadata_file_path);
}
ReadBufferFromFile in(metadata_file_fd, metadata_file_path, METADATA_FILE_BUFFER_SIZE);

View File

@ -481,7 +481,7 @@ public:
if (file.fd == -1)
{
auto error_code = (errno == ENOENT) ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE;
throwFromErrnoWithPath("Cannot open file " + file_path, file_path, error_code);
ErrnoException::throwFromPath(error_code, file_path, "Cannot open file {}", file_path);
}
allocateSizeForNextPartition();
@ -490,7 +490,8 @@ public:
void allocateSizeForNextPartition()
{
if (preallocateDiskSpace(file.fd, current_blocks_size * block_size, block_size * file_blocks_size) < 0)
throwFromErrnoWithPath("Cannot preallocate space for the file " + file_path, file_path, ErrorCodes::CANNOT_ALLOCATE_MEMORY);
ErrnoException::throwFromPath(
ErrorCodes::CANNOT_ALLOCATE_MEMORY, file_path, "Cannot preallocate space for the file {}", file_path);
current_blocks_size += file_blocks_size;
}
@ -552,11 +553,11 @@ public:
Stopwatch watch;
#if defined(OS_DARWIN)
if (::fsync(file.fd) < 0)
throwFromErrnoWithPath("Cannot fsync " + file_path, file_path, ErrorCodes::CANNOT_FSYNC);
#else
ErrnoException::throwFromPath(ErrorCodes::CANNOT_FSYNC, file_path, "Cannot fsync {}", file_path);
# else
if (::fdatasync(file.fd) < 0)
throwFromErrnoWithPath("Cannot fdatasync " + file_path, file_path, ErrorCodes::CANNOT_FSYNC);
#endif
ErrnoException::throwFromPath(ErrorCodes::CANNOT_FSYNC, file_path, "Cannot fdatasync {}", file_path);
# endif
ProfileEvents::increment(ProfileEvents::FileSyncElapsedMicroseconds, watch.elapsedMicroseconds());
current_block_index += buffer_size_in_blocks;
@ -598,13 +599,13 @@ public:
while (io_submit(aio_context.ctx, 1, &request_ptr) != 1)
{
if (errno != EINTR)
throwFromErrno("io_submit: Failed to submit a request for asynchronous IO", ErrorCodes::CANNOT_IO_SUBMIT);
throw ErrnoException(ErrorCodes::CANNOT_IO_SUBMIT, "io_submit: Failed to submit a request for asynchronous IO");
}
while (io_getevents(aio_context.ctx, 1, 1, &event, nullptr) != 1)
{
if (errno != EINTR)
throwFromErrno("io_getevents: Failed to get an event for asynchronous IO", ErrorCodes::CANNOT_IO_GETEVENTS);
throw ErrnoException(ErrorCodes::CANNOT_IO_GETEVENTS, "io_getevents: Failed to get an event for asynchronous IO");
}
auto read_bytes = eventResult(event);
@ -692,7 +693,7 @@ public:
while (to_pop < to_push && (popped = io_getevents(aio_context.ctx, to_push - to_pop, to_push - to_pop, &events[to_pop], nullptr)) <= 0)
{
if (errno != EINTR)
throwFromErrno("io_getevents: Failed to get an event for asynchronous IO", ErrorCodes::CANNOT_IO_GETEVENTS);
throw ErrnoException(ErrorCodes::CANNOT_IO_GETEVENTS, "io_getevents: Failed to get an event for asynchronous IO");
}
for (size_t i = to_pop; i < to_pop + popped; ++i)
@ -743,7 +744,7 @@ public:
while (new_tasks_count > 0 && (pushed = io_submit(aio_context.ctx, new_tasks_count, &pointers[to_push])) <= 0)
{
if (errno != EINTR)
throwFromErrno("io_submit: Failed to submit a request for asynchronous IO", ErrorCodes::CANNOT_IO_SUBMIT);
throw ErrnoException(ErrorCodes::CANNOT_IO_SUBMIT, "io_submit: Failed to submit a request for asynchronous IO");
}
to_push += pushed;

View File

@ -359,21 +359,21 @@ void DiskLocal::removeFile(const String & path)
{
auto fs_path = fs::path(disk_path) / path;
if (0 != unlink(fs_path.c_str()))
throwFromErrnoWithPath("Cannot unlink file " + fs_path.string(), fs_path, ErrorCodes::CANNOT_UNLINK);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_UNLINK, fs_path, "Cannot unlink file {}", fs_path);
}
void DiskLocal::removeFileIfExists(const String & path)
{
auto fs_path = fs::path(disk_path) / path;
if (0 != unlink(fs_path.c_str()) && errno != ENOENT)
throwFromErrnoWithPath("Cannot unlink file " + fs_path.string(), fs_path, ErrorCodes::CANNOT_UNLINK);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_UNLINK, fs_path, "Cannot unlink file {}", fs_path);
}
void DiskLocal::removeDirectory(const String & path)
{
auto fs_path = fs::path(disk_path) / path;
if (0 != rmdir(fs_path.c_str()))
throwFromErrnoWithPath("Cannot rmdir " + fs_path.string(), fs_path, ErrorCodes::CANNOT_RMDIR);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_RMDIR, fs_path, "Cannot rmdir {}", fs_path);
}
void DiskLocal::removeRecursive(const String & path)
@ -412,7 +412,7 @@ void DiskLocal::truncateFile(const String & path, size_t size)
{
int res = truncate((fs::path(disk_path) / path).string().data(), size);
if (-1 == res)
throwFromErrnoWithPath("Cannot truncate file " + path, path, ErrorCodes::CANNOT_TRUNCATE_FILE);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_TRUNCATE_FILE, path, "Cannot truncate {}", path);
}
void DiskLocal::createFile(const String & path)
@ -709,7 +709,7 @@ struct stat DiskLocal::stat(const String & path) const
auto full_path = fs::path(disk_path) / path;
if (::stat(full_path.string().c_str(), &st) == 0)
return st;
DB::throwFromErrnoWithPath("Cannot stat file: " + path, path, DB::ErrorCodes::CANNOT_STAT);
DB::ErrnoException::throwFromPath(DB::ErrorCodes::CANNOT_STAT, path, "Cannot stat file: {}", path);
}
void DiskLocal::chmod(const String & path, mode_t mode)
@ -717,7 +717,7 @@ void DiskLocal::chmod(const String & path, mode_t mode)
auto full_path = fs::path(disk_path) / path;
if (::chmod(full_path.string().c_str(), mode) == 0)
return;
DB::throwFromErrnoWithPath("Cannot chmod file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED);
DB::ErrnoException::throwFromPath(DB::ErrorCodes::PATH_ACCESS_DENIED, path, "Cannot chmod file: {}", path);
}
void registerDiskLocal(DiskFactory & factory, bool global_skip_access_check)

View File

@ -77,7 +77,7 @@ IOUringReader::IOUringReader(uint32_t entries_)
int ret = io_uring_queue_init_params(entries_, &ring, &params);
if (ret < 0)
throwFromErrno("Failed initializing io_uring", ErrorCodes::IO_URING_INIT_FAILED, -ret);
ErrnoException::throwWithErrno(ErrorCodes::IO_URING_INIT_FAILED, -ret, "Failed initializing io_uring");
cq_entries = params.cq_entries;
ring_completion_monitor = std::make_unique<ThreadFromGlobalPool>([this] { monitorRing(); });

View File

@ -175,9 +175,8 @@ std::future<IAsynchronousReader::Result> ThreadPoolReader::submit(Request reques
else
{
ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorReadFailed);
promise.set_exception(std::make_exception_ptr(ErrnoException(
fmt::format("Cannot read from file {}, {}", fd, errnoToString()),
ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR, errno)));
promise.set_exception(std::make_exception_ptr(
ErrnoException(ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR, "Cannot read from file {}", fd)));
return future;
}
}
@ -233,7 +232,7 @@ std::future<IAsynchronousReader::Result> ThreadPoolReader::submit(Request reques
if (-1 == res && errno != EINTR)
{
ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorReadFailed);
throwFromErrno(fmt::format("Cannot read from file {}", fd), ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR);
throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR, "Cannot read from file {}", fd);
}
bytes_read += res;

View File

@ -29,8 +29,7 @@ public:
off_t res = lseek(fd, 0, SEEK_SET);
if (-1 == res)
throwFromErrnoWithPath("Cannot reread temporary file " + file_name, file_name,
ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, file_name, "Cannot reread temporary file {}", file_name);
return std::make_unique<ReadBufferFromTemporaryWriteBuffer>(fd, file_name, std::move(origin->tmp_file));
}

View File

@ -31,8 +31,8 @@ LocalDirectorySyncGuard::LocalDirectorySyncGuard(const String & full_path)
: fd(::open(full_path.c_str(), O_DIRECTORY))
{
if (-1 == fd)
throwFromErrnoWithPath("Cannot open file " + full_path, full_path,
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE);
ErrnoException::throwFromPath(
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE, full_path, "Cannot open file {}", full_path);
}
LocalDirectorySyncGuard::~LocalDirectorySyncGuard()
@ -45,7 +45,7 @@ LocalDirectorySyncGuard::~LocalDirectorySyncGuard()
#if defined(OS_DARWIN)
if (fcntl(fd, F_FULLFSYNC, 0))
throwFromErrno("Cannot fcntl(F_FULLFSYNC)", ErrorCodes::CANNOT_FSYNC);
throw ErrnoException(ErrorCodes::CANNOT_FSYNC, "Cannot fcntl(F_FULLFSYNC)");
#else
if (-1 == ::fdatasync(fd))
throw Exception(ErrorCodes::CANNOT_FSYNC, "Cannot fdatasync");

View File

@ -141,7 +141,7 @@ void LocalObjectStorage::removeObject(const StoredObject & object)
return;
if (0 != unlink(object.remote_path.data()))
throwFromErrnoWithPath("Cannot unlink file " + object.remote_path, object.remote_path, ErrorCodes::CANNOT_UNLINK);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_UNLINK, object.remote_path, "Cannot unlink file {}", object.remote_path);
}
void LocalObjectStorage::removeObjects(const StoredObjects & objects)

View File

@ -30,7 +30,7 @@ Field nowSubsecond(UInt32 scale)
timespec spec{};
if (clock_gettime(CLOCK_REALTIME, &spec))
throwFromErrno("Cannot clock_gettime.", ErrorCodes::CANNOT_CLOCK_GETTIME);
throw ErrnoException(ErrorCodes::CANNOT_CLOCK_GETTIME, "Cannot clock_gettime");
DecimalUtils::DecimalComponents<DateTime64> components{spec.tv_sec, spec.tv_nsec};

View File

@ -165,7 +165,7 @@ public:
std::uniform_int_distribution<intptr_t>(0x100000000000UL, 0x700000000000UL)(thread_local_rng));
void * map = mmap(hint, 4096, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
if (MAP_FAILED == map)
throwFromErrno("Allocator: Cannot mmap", ErrorCodes::CANNOT_ALLOCATE_MEMORY);
throw ErrnoException(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Allocator: Cannot mmap");
maps.push_back(map);
}
}

View File

@ -46,7 +46,7 @@ AIOContext::AIOContext(unsigned int nr_events)
{
ctx = 0;
if (io_setup(nr_events, &ctx) < 0)
DB::throwFromErrno("io_setup failed", DB::ErrorCodes::CANNOT_IOSETUP);
throw DB::ErrnoException(DB::ErrorCodes::CANNOT_IOSETUP, "io_setup failed");
}
AIOContext::~AIOContext()
@ -137,7 +137,7 @@ AIOContext::AIOContext(unsigned int)
{
ctx = io_setup();
if (ctx < 0)
DB::throwFromErrno("io_setup failed", DB::ErrorCodes::CANNOT_IOSETUP);
throw ErrnoException(DB::ErrorCodes::CANNOT_IOSETUP, "io_setup failed");
}
AIOContext::~AIOContext()

View File

@ -46,13 +46,17 @@ AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile(
fd = ::open(file_name.c_str(), flags == -1 ? O_RDONLY | O_CLOEXEC : flags | O_CLOEXEC);
if (-1 == fd)
throwFromErrnoWithPath("Cannot open file " + file_name, file_name,
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE);
ErrnoException::throwFromPath(
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE, file_name, "Cannot open file {}", file_name);
#ifdef OS_DARWIN
if (o_direct)
{
if (fcntl(fd, F_NOCACHE, 1) == -1)
throwFromErrnoWithPath("Cannot set F_NOCACHE on file " + file_name, file_name, ErrorCodes::CANNOT_OPEN_FILE);
ErrnoException::throwFromPath(
errno == ENOENT ? ErrorCodes::CANNOT_OPEN_FILE : ErrorCodes::CANNOT_OPEN_FILE,
file_name,
"Cannot set F_NOCACHE on file {}",
file_name);
}
#endif
}

View File

@ -29,8 +29,8 @@ void MMapReadBufferFromFile::open()
fd = ::open(file_name.c_str(), O_RDONLY | O_CLOEXEC);
if (-1 == fd)
throwFromErrnoWithPath("Cannot open file " + file_name, file_name,
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE);
ErrnoException::throwFromPath(
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE, file_name, "Cannot open file {}", file_name);
}

View File

@ -30,8 +30,8 @@ void MMappedFile::open()
fd = ::open(file_name.c_str(), O_RDONLY | O_CLOEXEC);
if (-1 == fd)
throwFromErrnoWithPath("Cannot open file " + file_name, file_name,
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE);
ErrnoException::throwFromPath(
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE, file_name, "Cannot open file {}", file_name);
}

View File

@ -28,7 +28,7 @@ static size_t getFileSize(int fd)
{
struct stat stat_res {};
if (0 != fstat(fd, &stat_res))
throwFromErrno("MMappedFileDescriptor: Cannot fstat.", ErrorCodes::CANNOT_STAT);
throw ErrnoException(ErrorCodes::CANNOT_STAT, "MMappedFileDescriptor: Cannot fstat");
off_t file_size = stat_res.st_size;
@ -63,8 +63,7 @@ void MMappedFileDescriptor::set(int fd_, size_t offset_, size_t length_)
void * buf = mmap(nullptr, length, PROT_READ, MAP_PRIVATE, fd, offset);
if (MAP_FAILED == buf)
throwFromErrno(fmt::format("MMappedFileDescriptor: Cannot mmap {}.", ReadableSize(length)),
ErrorCodes::CANNOT_ALLOCATE_MEMORY);
throw ErrnoException(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "MMappedFileDescriptor: Cannot mmap {}", ReadableSize(length));
data = static_cast<char *>(buf);
@ -88,8 +87,7 @@ void MMappedFileDescriptor::finish()
return;
if (0 != munmap(data, length))
throwFromErrno(fmt::format("MMappedFileDescriptor: Cannot munmap {}.", ReadableSize(length)),
ErrorCodes::CANNOT_MUNMAP);
throw ErrnoException(ErrorCodes::CANNOT_MUNMAP, "MMappedFileDescriptor: Cannot munmap {}", ReadableSize(length));
length = 0;

View File

@ -30,8 +30,8 @@ void OpenedFile::open() const
fd = ::open(file_name.c_str(), (flags == -1 ? 0 : flags) | O_RDONLY | O_CLOEXEC);
if (-1 == fd)
throwFromErrnoWithPath("Cannot open file " + file_name, file_name,
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE);
DB::ErrnoException::throwFromPath(
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE, file_name, "Cannot open file {}", file_name);
}
int OpenedFile::getFD() const

View File

@ -45,13 +45,17 @@ ReadBufferFromFile::ReadBufferFromFile(
fd = ::open(file_name.c_str(), flags == -1 ? O_RDONLY | O_CLOEXEC : flags | O_CLOEXEC);
if (-1 == fd)
throwFromErrnoWithPath("Cannot open file " + file_name, file_name,
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE);
ErrnoException::throwFromPath(
errno == ENOENT ? ErrorCodes::FILE_DOESNT_EXIST : ErrorCodes::CANNOT_OPEN_FILE, file_name, "Cannot open file {}", file_name);
#ifdef OS_DARWIN
if (o_direct)
{
if (fcntl(fd, F_NOCACHE, 1) == -1)
throwFromErrnoWithPath("Cannot set F_NOCACHE on file " + file_name, file_name, ErrorCodes::CANNOT_OPEN_FILE);
ErrnoException::throwFromPath(
errno == ENOENT ? ErrorCodes::CANNOT_OPEN_FILE : ErrorCodes::CANNOT_OPEN_FILE,
file_name,
"Cannot set F_NOCACHE on file {}",
file_name);
}
#endif
}

View File

@ -80,7 +80,8 @@ size_t ReadBufferFromFileDescriptor::readImpl(char * to, size_t min_bytes, size_
if (-1 == res && errno != EINTR)
{
ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorReadFailed);
throwFromErrnoWithPath("Cannot read from file: " + getFileName(), getFileName(), ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR);
ErrnoException::throwFromPath(
ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR, getFileName(), "Cannot read from file {}", getFileName());
}
if (res > 0)
@ -145,7 +146,7 @@ void ReadBufferFromFileDescriptor::prefetch(Priority)
/// Ask OS to prefetch data into page cache.
if (0 != posix_fadvise(fd, file_offset_of_buffer_end, internal_buffer.size(), POSIX_FADV_WILLNEED))
throwFromErrno("Cannot posix_fadvise", ErrorCodes::CANNOT_ADVISE);
throw ErrnoException(ErrorCodes::CANNOT_ADVISE, "Cannot posix_fadvise");
#endif
}
@ -208,8 +209,12 @@ off_t ReadBufferFromFileDescriptor::seek(off_t offset, int whence)
off_t res = ::lseek(fd, seek_pos, SEEK_SET);
if (-1 == res)
throwFromErrnoWithPath(fmt::format("Cannot seek through file {} at offset {}", getFileName(), seek_pos), getFileName(),
ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
ErrnoException::throwFromPath(
ErrorCodes::CANNOT_SEEK_THROUGH_FILE,
getFileName(),
"Cannot seek through file {} at offset {}",
getFileName(),
seek_pos);
/// Also note that seeking past the file size is not allowed.
if (res != seek_pos)
@ -237,8 +242,8 @@ void ReadBufferFromFileDescriptor::rewind()
ProfileEvents::increment(ProfileEvents::Seek);
off_t res = ::lseek(fd, 0, SEEK_SET);
if (-1 == res)
throwFromErrnoWithPath("Cannot seek through file " + getFileName(), getFileName(),
ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
ErrnoException::throwFromPath(
ErrorCodes::CANNOT_SEEK_THROUGH_FILE, getFileName(), "Cannot seek through file {}", getFileName());
}
/// In case of pread, the ProfileEvents::Seek is not accounted, but it's Ok.

View File

@ -43,7 +43,7 @@ std::future<IAsynchronousReader::Result> SynchronousReader::submit(Request reque
#if defined(POSIX_FADV_WILLNEED)
int fd = assert_cast<const LocalFileDescriptor &>(*request.descriptor).fd;
if (0 != posix_fadvise(fd, request.offset, request.size, POSIX_FADV_WILLNEED))
throwFromErrno("Cannot posix_fadvise", ErrorCodes::CANNOT_ADVISE);
throw ErrnoException(ErrorCodes::CANNOT_ADVISE, "Cannot posix_fadvise");
#endif
return std::async(std::launch::deferred, [request, this]
@ -73,7 +73,7 @@ IAsynchronousReader::Result SynchronousReader::execute(Request request)
if (-1 == res && errno != EINTR)
{
ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorReadFailed);
throwFromErrno(fmt::format("Cannot read from file {}", fd), ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR);
throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR, "Cannot read from file {}", fd);
}
if (res > 0)

View File

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

View File

@ -69,8 +69,8 @@ void WriteBufferFromFileDescriptor::nextImpl()
String error_file_name = file_name;
if (error_file_name.empty())
error_file_name = "(fd = " + toString(fd) + ")";
throwFromErrnoWithPath("Cannot write to file " + error_file_name, error_file_name,
ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR);
ErrnoException::throwFromPath(
ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR, error_file_name, "Cannot write to file {}", error_file_name);
}
if (res > 0)
@ -137,7 +137,7 @@ void WriteBufferFromFileDescriptor::sync()
ProfileEvents::increment(ProfileEvents::FileSyncElapsedMicroseconds, watch.elapsedMicroseconds());
if (-1 == res)
throwFromErrnoWithPath("Cannot fsync " + getFileName(), getFileName(), ErrorCodes::CANNOT_FSYNC);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR, getFileName(), "Cannot fsync {}", getFileName());
}
@ -145,8 +145,7 @@ off_t WriteBufferFromFileDescriptor::seek(off_t offset, int whence) // NOLINT
{
off_t res = lseek(fd, offset, whence);
if (-1 == res)
throwFromErrnoWithPath("Cannot seek through file " + getFileName(), getFileName(),
ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, getFileName(), "Cannot seek through {}", getFileName());
return res;
}
@ -154,7 +153,7 @@ void WriteBufferFromFileDescriptor::truncate(off_t length) // NOLINT
{
int res = ftruncate(fd, length);
if (-1 == res)
throwFromErrnoWithPath("Cannot truncate file " + getFileName(), getFileName(), ErrorCodes::CANNOT_TRUNCATE_FILE);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_TRUNCATE_FILE, getFileName(), "Cannot truncate file {}", getFileName());
}
@ -163,7 +162,7 @@ off_t WriteBufferFromFileDescriptor::size() const
struct stat buf;
int res = fstat(fd, &buf);
if (-1 == res)
throwFromErrnoWithPath("Cannot execute fstat " + getFileName(), getFileName(), ErrorCodes::CANNOT_FSTAT);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_FSTAT, getFileName(), "Cannot execute fstat {}", getFileName());
return buf.st_size;
}

View File

@ -289,7 +289,7 @@ BlockIO InterpreterSystemQuery::execute()
{
getContext()->checkAccess(AccessType::SYSTEM_SHUTDOWN);
if (kill(0, SIGTERM))
throwFromErrno("System call kill(0, SIGTERM) failed", ErrorCodes::CANNOT_KILL);
throw ErrnoException(ErrorCodes::CANNOT_KILL, "System call kill(0, SIGTERM) failed");
break;
}
case Type::KILL:

View File

@ -153,7 +153,7 @@ public:
{
int res = mprotect(block.base(), block.blockSize(), protection_flags | PROT_READ);
if (res != 0)
throwFromErrno("Cannot mprotect memory region", ErrorCodes::CANNOT_MPROTECT);
throw ErrnoException(ErrorCodes::CANNOT_MPROTECT, "Cannot mprotect memory region");
llvm::sys::Memory::InvalidateInstructionCache(block.base(), block.blockSize());
invalidate_cache = false;
@ -161,7 +161,7 @@ public:
# endif
int res = mprotect(block.base(), block.blockSize(), protection_flags);
if (res != 0)
throwFromErrno("Cannot mprotect memory region", ErrorCodes::CANNOT_MPROTECT);
throw ErrnoException(ErrorCodes::CANNOT_MPROTECT, "Cannot mprotect memory region");
if (invalidate_cache)
llvm::sys::Memory::InvalidateInstructionCache(block.base(), block.blockSize());
@ -232,10 +232,12 @@ private:
int res = posix_memalign(&buf, page_size, allocate_size);
if (res != 0)
throwFromErrno(
fmt::format("Cannot allocate memory (posix_memalign) alignment {} size {}.", page_size, ReadableSize(allocate_size)),
ErrnoException::throwWithErrno(
ErrorCodes::CANNOT_ALLOCATE_MEMORY,
res);
res,
"Cannot allocate memory (posix_memalign) alignment {} size {}",
page_size,
ReadableSize(allocate_size));
page_blocks.emplace_back(buf, pages_to_allocate_size, page_size);
page_blocks_allocated_size.emplace_back(0);

View File

@ -221,7 +221,7 @@ void ThreadStatus::applyQuerySettings()
LOG_TRACE(log, "Setting nice to {}", new_os_thread_priority);
if (0 != setpriority(PRIO_PROCESS, static_cast<unsigned>(thread_id), new_os_thread_priority))
throwFromErrno("Cannot 'setpriority'", ErrorCodes::CANNOT_SET_THREAD_PRIORITY);
throw ErrnoException(ErrorCodes::CANNOT_SET_THREAD_PRIORITY, "Cannot 'setpriority'");
os_thread_priority = new_os_thread_priority;
}

View File

@ -19,7 +19,7 @@ ExtendedLogMessage ExtendedLogMessage::getFrom(const Poco::Message & base)
::timeval tv;
if (0 != gettimeofday(&tv, nullptr))
DB::throwFromErrno("Cannot gettimeofday", ErrorCodes::CANNOT_GETTIMEOFDAY);
throw ErrnoException(ErrorCodes::CANNOT_GETTIMEOFDAY, "Cannot gettimeofday");
msg_ext.time_seconds = static_cast<UInt32>(tv.tv_sec);
msg_ext.time_microseconds = static_cast<UInt32>(tv.tv_usec);

View File

@ -25,7 +25,7 @@ namespace ErrorCodes
PollingQueue::PollingQueue()
{
if (-1 == pipe2(pipe_fd, O_NONBLOCK))
throwFromErrno("Cannot create pipe", ErrorCodes::CANNOT_OPEN_FILE);
throw ErrnoException(ErrorCodes::CANNOT_OPEN_FILE, "Cannot create pipe");
epoll.add(pipe_fd[0], pipe_fd);
}
@ -111,7 +111,7 @@ void PollingQueue::finish()
break;
if (errno != EINTR)
throwFromErrno("Cannot write to pipe", ErrorCodes::CANNOT_READ_FROM_SOCKET);
throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot write to pipe");
}
}

View File

@ -44,7 +44,7 @@ static void makeFdNonBlocking(int fd)
{
bool result = tryMakeFdNonBlocking(fd);
if (!result)
throwFromErrno("Cannot set non-blocking mode of pipe", ErrorCodes::CANNOT_FCNTL);
throw ErrnoException(ErrorCodes::CANNOT_FCNTL, "Cannot set non-blocking mode of pipe");
}
static bool tryMakeFdBlocking(int fd)
@ -63,7 +63,7 @@ static void makeFdBlocking(int fd)
{
bool result = tryMakeFdBlocking(fd);
if (!result)
throwFromErrno("Cannot set blocking mode of pipe", ErrorCodes::CANNOT_FCNTL);
throw ErrnoException(ErrorCodes::CANNOT_FCNTL, "Cannot set blocking mode of pipe");
}
static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_milliseconds)
@ -78,7 +78,7 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond
if (res < 0)
{
if (errno != EINTR)
throwFromErrno("Cannot poll", ErrorCodes::CANNOT_POLL);
throw ErrnoException(ErrorCodes::CANNOT_POLL, "Cannot poll");
const auto elapsed = watch.elapsedMilliseconds();
if (timeout_milliseconds <= elapsed)
@ -177,7 +177,7 @@ public:
ssize_t res = ::read(stdout_fd, internal_buffer.begin(), internal_buffer.size());
if (-1 == res && errno != EINTR)
throwFromErrno("Cannot read from pipe", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR);
throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR, "Cannot read from pipe");
if (res == 0)
break;
@ -261,7 +261,7 @@ public:
ssize_t res = ::write(fd, working_buffer.begin() + bytes_written, offset() - bytes_written);
if ((-1 == res || 0 == res) && errno != EINTR)
throwFromErrno("Cannot write into pipe", ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR);
throw ErrnoException(ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR, "Cannot write into pipe");
if (res > 0)
bytes_written += res;

View File

@ -22,7 +22,7 @@ RemoteQueryExecutorReadContext::RemoteQueryExecutorReadContext(RemoteQueryExecut
: AsyncTaskExecutor(std::make_unique<Task>(*this)), executor(executor_), suspend_when_query_sent(suspend_when_query_sent_)
{
if (-1 == pipe2(pipe_fd, O_NONBLOCK))
throwFromErrno("Cannot create pipe", ErrorCodes::CANNOT_OPEN_FILE);
throw ErrnoException(ErrorCodes::CANNOT_OPEN_FILE, "Cannot create pipe");
epoll.add(pipe_fd[0]);
epoll.add(timer.getDescriptor());
@ -132,7 +132,7 @@ void RemoteQueryExecutorReadContext::cancelBefore()
break;
if (errno != EINTR)
throwFromErrno("Cannot write to pipe", ErrorCodes::CANNOT_READ_FROM_SOCKET);
throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_SOCKET, "Cannot write to pipe");
}
}

View File

@ -83,7 +83,7 @@ struct SocketInterruptablePollWrapper
#if defined(POCO_HAVE_FD_EPOLL)
epollfd = epoll_create(2);
if (epollfd < 0)
throwFromErrno("Cannot epoll_create", ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot epoll_create");
socket_event.events = EPOLLIN | EPOLLERR | EPOLLPRI;
socket_event.data.fd = sockfd;
@ -92,7 +92,7 @@ struct SocketInterruptablePollWrapper
int err = ::close(epollfd);
chassert(!err || errno == EINTR);
throwFromErrno("Cannot insert socket into epoll queue", ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot epoll_create");
}
pipe_event.events = EPOLLIN | EPOLLERR | EPOLLPRI;
pipe_event.data.fd = pipe.fds_rw[0];
@ -101,7 +101,7 @@ struct SocketInterruptablePollWrapper
int err = ::close(epollfd);
chassert(!err || errno == EINTR);
throwFromErrno("Cannot insert socket into epoll queue", ErrorCodes::SYSTEM_ERROR);
throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot insert socket into epoll queue");
}
#endif
}

View File

@ -67,10 +67,8 @@ void MySQLHandlerFactory::readRSAKeys()
FILE * fp = fopen(certificate_file.data(), "r");
if (fp == nullptr)
throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Cannot open certificate file: {}.", certificate_file);
SCOPE_EXIT(
if (0 != fclose(fp))
throwFromErrno("Cannot close file with the certificate in MySQLHandlerFactory", ErrorCodes::CANNOT_CLOSE_FILE);
);
SCOPE_EXIT(if (0 != fclose(fp)) throw ErrnoException(
ErrorCodes::CANNOT_CLOSE_FILE, "Cannot close file with the certificate in MySQLHandlerFactory"););
X509 * x509 = PEM_read_X509(fp, nullptr, nullptr, nullptr);
SCOPE_EXIT(X509_free(x509));
@ -93,10 +91,8 @@ void MySQLHandlerFactory::readRSAKeys()
FILE * fp = fopen(private_key_file.data(), "r");
if (fp == nullptr)
throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Cannot open private key file {}.", private_key_file);
SCOPE_EXIT(
if (0 != fclose(fp))
throwFromErrno("Cannot close file with the certificate in MySQLHandlerFactory", ErrorCodes::CANNOT_CLOSE_FILE);
);
SCOPE_EXIT(if (0 != fclose(fp)) throw ErrnoException(
ErrorCodes::CANNOT_CLOSE_FILE, "Cannot close file with the certificate in MySQLHandlerFactory"););
private_key.reset(PEM_read_RSAPrivateKey(fp, nullptr, nullptr, nullptr));
if (!private_key)

View File

@ -36,7 +36,7 @@ DirectoryWatcherBase::DirectoryWatcherBase(
fd = inotify_init();
if (fd == -1)
throwFromErrno("Cannot initialize inotify", ErrorCodes::IO_SETUP_ERROR);
throw ErrnoException(ErrorCodes::IO_SETUP_ERROR, "Cannot initialize inotify");
watch_task = getContext()->getSchedulePool().createTask("directory_watch", [this] { watchFunc(); });
start();
@ -60,7 +60,7 @@ void DirectoryWatcherBase::watchFunc()
if (wd == -1)
{
owner.onError(Exception(ErrorCodes::IO_SETUP_ERROR, "Watch directory {} failed", path));
throwFromErrnoWithPath("Watch directory {} failed", path, ErrorCodes::IO_SETUP_ERROR);
ErrnoException::throwFromPath(ErrorCodes::IO_SETUP_ERROR, path, "Watch directory {} failed", path);
}
std::string buffer;

View File

@ -95,8 +95,7 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl
{
int result = hdfsSync(fs.get(), fout);
if (result < 0)
throwFromErrno("Cannot HDFS sync" + hdfs_uri + " " + std::string(hdfsGetLastError()),
ErrorCodes::CANNOT_FSYNC);
throw ErrnoException(ErrorCodes::CANNOT_FSYNC, "Cannot HDFS sync {} {}", hdfs_uri, std::string(hdfsGetLastError()));
}
};

View File

@ -299,13 +299,13 @@ struct stat getFileStat(const String & current_path, bool use_table_fd, int tabl
{
/// Check if file descriptor allows random reads (and reading it twice).
if (0 != fstat(table_fd, &file_stat))
throwFromErrno("Cannot stat table file descriptor, inside " + storage_name, ErrorCodes::CANNOT_STAT);
throw ErrnoException(ErrorCodes::CANNOT_STAT, "Cannot stat table file descriptor, inside {}", storage_name);
}
else
{
/// Check if file descriptor allows random reads (and reading it twice).
if (0 != stat(current_path.c_str(), &file_stat))
throwFromErrno("Cannot stat file " + current_path, ErrorCodes::CANNOT_STAT);
throw ErrnoException(ErrorCodes::CANNOT_STAT, "Cannot stat file {}", current_path);
}
return file_stat;
@ -813,7 +813,7 @@ StorageFile::StorageFile(int table_fd_, CommonArguments args)
struct stat buf;
int res = fstat(table_fd_, &buf);
if (-1 == res)
throwFromErrno("Cannot execute fstat", res, ErrorCodes::CANNOT_FSTAT);
throw ErrnoException(ErrorCodes::CANNOT_FSTAT, "Cannot execute fstat");
total_bytes_to_read = buf.st_size;
if (args.getContext()->getApplicationType() == Context::ApplicationType::SERVER)
@ -1793,7 +1793,7 @@ void StorageFile::truncate(
if (use_table_fd)
{
if (0 != ::ftruncate(table_fd, 0))
throwFromErrno("Cannot truncate file at fd " + toString(table_fd), ErrorCodes::CANNOT_TRUNCATE_FILE);
throw ErrnoException(ErrorCodes::CANNOT_TRUNCATE_FILE, "Cannot truncate file at fd {}", toString(table_fd));
}
else
{
@ -1803,7 +1803,7 @@ void StorageFile::truncate(
continue;
if (0 != ::truncate(path.c_str(), 0))
throwFromErrnoWithPath("Cannot truncate file " + path, path, ErrorCodes::CANNOT_TRUNCATE_FILE);
ErrnoException::throwFromPath(ErrorCodes::CANNOT_TRUNCATE_FILE, path, "Cannot truncate file at {}", path);
}
}
}

View File

@ -133,7 +133,7 @@ bool wait(int timeout_ms)
continue;
}
throwFromErrno("Cannot poll pipe", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR);
throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR, "Cannot poll pipe");
}
if (poll_res == 0)
return false;
@ -146,7 +146,7 @@ bool wait(int timeout_ms)
if (errno == EINTR)
continue;
throwFromErrno("Cannot read from pipe", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR);
throw ErrnoException(ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR, "Cannot read from pipe");
}
if (read_res == sizeof(notification_num))
@ -296,7 +296,7 @@ protected:
if (ESRCH == errno)
continue;
throwFromErrno("Cannot send signal with sigqueue", ErrorCodes::CANNOT_SIGQUEUE);
throw ErrnoException(ErrorCodes::CANNOT_SIGQUEUE, "Cannot send signal with sigqueue");
}
/// Just in case we will wait for pipe with timeout. In case signal didn't get processed.
@ -402,13 +402,13 @@ StorageSystemStackTrace::StorageSystemStackTrace(const StorageID & table_id_)
sa.sa_flags = SA_SIGINFO;
if (sigemptyset(&sa.sa_mask))
throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_MANIPULATE_SIGSET);
throw ErrnoException(ErrorCodes::CANNOT_MANIPULATE_SIGSET, "Cannot set signal handler");
if (sigaddset(&sa.sa_mask, sig))
throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_MANIPULATE_SIGSET);
throw ErrnoException(ErrorCodes::CANNOT_MANIPULATE_SIGSET, "Cannot set signal handler");
if (sigaction(sig, &sa, nullptr))
throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler");
}

View File

@ -174,7 +174,7 @@ void Runner::thread(std::vector<std::shared_ptr<Coordination::ZooKeeper>> zookee
|| sigaddset(&sig_set, SIGINT)
|| pthread_sigmask(SIG_BLOCK, &sig_set, nullptr))
{
DB::throwFromErrno("Cannot block signal.", DB::ErrorCodes::CANNOT_BLOCK_SIGNAL);
throw DB::ErrnoException(DB::ErrorCodes::CANNOT_BLOCK_SIGNAL, "Cannot block signal");
}
while (true)