mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #3633 from yandex/fixed-bad-error-code
Fixed error introduced in #3553 and prevent it happening
This commit is contained in:
commit
86c1c2a343
@ -66,6 +66,7 @@ namespace ErrorCodes
|
||||
extern const int SUPPORT_IS_DISABLED;
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
|
||||
extern const int SYSTEM_ERROR;
|
||||
}
|
||||
|
||||
|
||||
@ -141,7 +142,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
{
|
||||
LOG_TRACE(log, "Will mlockall to prevent executable memory from being paged out. It may take a few seconds.");
|
||||
if (0 != mlockall(MCL_CURRENT))
|
||||
LOG_WARNING(log, "Failed mlockall: " + errnoToString());
|
||||
LOG_WARNING(log, "Failed mlockall: " + errnoToString(ErrorCodes::SYSTEM_ERROR));
|
||||
else
|
||||
LOG_TRACE(log, "The memory map of clickhouse executable has been mlock'ed");
|
||||
}
|
||||
|
@ -18,7 +18,15 @@
|
||||
|
||||
#include <common/Types.h>
|
||||
|
||||
#define SMALL_READ_WRITE_BUFFER_SIZE 16
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_OPEN_FILE;
|
||||
extern const int CANNOT_READ_ALL_DATA;
|
||||
extern const int ATTEMPT_TO_READ_AFTER_EOF;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/** Stores a number in the file.
|
||||
@ -26,6 +34,9 @@
|
||||
*/
|
||||
class CounterInFile
|
||||
{
|
||||
private:
|
||||
static inline constexpr size_t SMALL_READ_WRITE_BUFFER_SIZE = 16;
|
||||
|
||||
public:
|
||||
/// path - the name of the file, including the path
|
||||
CounterInFile(const std::string & path_) : path(path_) {}
|
||||
@ -56,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::throwFromErrno("Cannot open file " + 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::throwFromErrno("Cannot lock file " + path, DB::ErrorCodes::CANNOT_OPEN_FILE);
|
||||
|
||||
if (!file_doesnt_exists)
|
||||
{
|
||||
@ -130,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::throwFromErrno("Cannot open file " + path, DB::ErrorCodes::CANNOT_OPEN_FILE);
|
||||
|
||||
try
|
||||
{
|
||||
@ -178,6 +189,3 @@ private:
|
||||
std::string path;
|
||||
std::mutex mutex;
|
||||
};
|
||||
|
||||
|
||||
#undef SMALL_READ_WRITE_BUFFER_SIZE
|
||||
|
@ -397,6 +397,9 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_UPDATE_COLUMN = 420;
|
||||
extern const int CANNOT_ADD_DIFFERENT_AGGREGATE_STATES = 421;
|
||||
extern const int UNSUPPORTED_URI_SCHEME = 422;
|
||||
extern const int CANNOT_GETTIMEOFDAY = 423;
|
||||
extern const int CANNOT_LINK = 424;
|
||||
extern const int SYSTEM_ERROR = 425;
|
||||
|
||||
extern const int KEEPER_EXCEPTION = 999;
|
||||
extern const int POCO_EXCEPTION = 1000;
|
||||
|
@ -52,7 +52,7 @@ std::string errnoToString(int code, int e)
|
||||
|
||||
void throwFromErrno(const std::string & s, int code, int e)
|
||||
{
|
||||
throw ErrnoException(s + ", " + errnoToString(code, e));
|
||||
throw ErrnoException(s + ", " + errnoToString(code, e), code, e);
|
||||
}
|
||||
|
||||
void tryLogCurrentException(const char * log_name, const std::string & start_of_message)
|
||||
|
@ -42,11 +42,11 @@ private:
|
||||
class ErrnoException : public Exception
|
||||
{
|
||||
public:
|
||||
ErrnoException(const std::string & msg, int code = 0, int saved_errno_ = 0)
|
||||
ErrnoException(const std::string & msg, int code, int saved_errno_)
|
||||
: Exception(msg, code), saved_errno(saved_errno_) {}
|
||||
ErrnoException(const std::string & msg, const std::string & arg, int code = 0, int saved_errno_ = 0)
|
||||
ErrnoException(const std::string & msg, const std::string & arg, int code, int saved_errno_)
|
||||
: Exception(msg, arg, code), saved_errno(saved_errno_) {}
|
||||
ErrnoException(const std::string & msg, const Exception & exc, int code = 0, int saved_errno_ = 0)
|
||||
ErrnoException(const std::string & msg, const Exception & exc, int code, int saved_errno_)
|
||||
: Exception(msg, exc, code), saved_errno(saved_errno_) {}
|
||||
|
||||
int getErrno() const { return saved_errno; }
|
||||
@ -59,8 +59,8 @@ private:
|
||||
using Exceptions = std::vector<std::exception_ptr>;
|
||||
|
||||
|
||||
std::string errnoToString(int code = 0, int the_errno = errno);
|
||||
[[noreturn]] void throwFromErrno(const std::string & s, int code = 0, int the_errno = errno);
|
||||
std::string errnoToString(int code, int the_errno = errno);
|
||||
[[noreturn]] void throwFromErrno(const std::string & s, int code, int the_errno = errno);
|
||||
|
||||
|
||||
/** Try to write an exception to the log (and forget about it).
|
||||
|
@ -20,6 +20,14 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_OPEN_FILE;
|
||||
extern const int CANNOT_CLOSE_FILE;
|
||||
extern const int CANNOT_TRUNCATE_FILE;
|
||||
extern const int CANNOT_SEEK_THROUGH_FILE;
|
||||
}
|
||||
|
||||
|
||||
StatusFile::StatusFile(const std::string & path_)
|
||||
: path(path_)
|
||||
@ -43,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);
|
||||
throwFromErrno("Cannot open file " + path, ErrorCodes::CANNOT_OPEN_FILE);
|
||||
|
||||
try
|
||||
{
|
||||
@ -53,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.");
|
||||
else
|
||||
throwFromErrno("Cannot lock file " + path);
|
||||
throwFromErrno("Cannot lock file " + path, ErrorCodes::CANNOT_OPEN_FILE);
|
||||
}
|
||||
|
||||
if (0 != ftruncate(fd, 0))
|
||||
throwFromErrno("Cannot ftruncate " + path);
|
||||
throwFromErrno("Cannot ftruncate " + path, ErrorCodes::CANNOT_TRUNCATE_FILE);
|
||||
|
||||
if (0 != lseek(fd, 0, SEEK_SET))
|
||||
throwFromErrno("Cannot lseek " + path);
|
||||
throwFromErrno("Cannot lseek " + path, ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
|
||||
|
||||
/// Write information about current server instance to the file.
|
||||
{
|
||||
@ -82,10 +90,10 @@ StatusFile::StatusFile(const std::string & path_)
|
||||
StatusFile::~StatusFile()
|
||||
{
|
||||
if (0 != close(fd))
|
||||
LOG_ERROR(&Logger::get("StatusFile"), "Cannot close file " << path << ", " << errnoToString());
|
||||
LOG_ERROR(&Logger::get("StatusFile"), "Cannot close file " << path << ", " << errnoToString(ErrorCodes::CANNOT_CLOSE_FILE));
|
||||
|
||||
if (0 != unlink(path.c_str()))
|
||||
LOG_ERROR(&Logger::get("StatusFile"), "Cannot unlink file " << path << ", " << errnoToString());
|
||||
LOG_ERROR(&Logger::get("StatusFile"), "Cannot unlink file " << path << ", " << errnoToString(ErrorCodes::CANNOT_CLOSE_FILE));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -8,6 +8,12 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_STAT;
|
||||
extern const int CANNOT_LINK;
|
||||
}
|
||||
|
||||
void createHardLink(const String & source_path, const String & destination_path)
|
||||
{
|
||||
if (0 != link(source_path.c_str(), destination_path.c_str()))
|
||||
@ -20,16 +26,16 @@ 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);
|
||||
throwFromErrno("Cannot stat " + source_path, ErrorCodes::CANNOT_STAT);
|
||||
|
||||
if (0 != lstat(destination_path.c_str(), &destination_descr))
|
||||
throwFromErrno("Cannot stat " + destination_path);
|
||||
throwFromErrno("Cannot stat " + 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.", 0, link_errno);
|
||||
throwFromErrno("Destination file " + destination_path + " is already exist and have different inode.", ErrorCodes::CANNOT_LINK, link_errno);
|
||||
}
|
||||
else
|
||||
throwFromErrno("Cannot link " + source_path + " to " + destination_path);
|
||||
throwFromErrno("Cannot link " + source_path + " to " + destination_path, ErrorCodes::CANNOT_LINK);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -32,7 +32,7 @@ void setThreadName(const char * name)
|
||||
#else
|
||||
if (0 != prctl(PR_SET_NAME, name, 0, 0, 0))
|
||||
#endif
|
||||
DB::throwFromErrno("Cannot set thread name with prctl(PR_SET_NAME, ...)");
|
||||
DB::throwFromErrno("Cannot set thread name with prctl(PR_SET_NAME, ...)", DB::ErrorCodes::PTHREAD_ERROR);
|
||||
}
|
||||
|
||||
std::string getThreadName()
|
||||
@ -48,7 +48,7 @@ std::string getThreadName()
|
||||
// throw DB::Exception("Cannot get thread name with pthread_get_name_np()", DB::ErrorCodes::PTHREAD_ERROR);
|
||||
#else
|
||||
if (0 != prctl(PR_GET_NAME, name.data(), 0, 0, 0))
|
||||
DB::throwFromErrno("Cannot get thread name with prctl(PR_GET_NAME)");
|
||||
DB::throwFromErrno("Cannot get thread name with prctl(PR_GET_NAME)", DB::ErrorCodes::PTHREAD_ERROR);
|
||||
#endif
|
||||
|
||||
name.resize(std::strlen(name.data()));
|
||||
|
@ -25,6 +25,14 @@
|
||||
|
||||
using namespace DB;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int SYSTEM_ERROR;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/// Implementation of ArenaWithFreeLists, which contains a bug. Used to reproduce the bug.
|
||||
#if USE_BAD_ARENA
|
||||
@ -237,7 +245,7 @@ int main(int argc, char ** argv)
|
||||
|
||||
rusage resource_usage;
|
||||
if (0 != getrusage(RUSAGE_SELF, &resource_usage))
|
||||
throwFromErrno("Cannot getrusage");
|
||||
throwFromErrno("Cannot getrusage", ErrorCodes::SYSTEM_ERROR);
|
||||
|
||||
size_t allocated_bytes = resource_usage.ru_maxrss * 1024;
|
||||
std::cerr << "Current memory usage: " << allocated_bytes << " bytes.\n";
|
||||
|
@ -22,6 +22,15 @@ void f() { ++x; }
|
||||
void * g(void *) { f(); return {}; }
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int PTHREAD_ERROR;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
template <typename F>
|
||||
void test(size_t n, const char * name, F && kernel)
|
||||
{
|
||||
@ -80,9 +89,9 @@ int main(int argc, char ** argv)
|
||||
{
|
||||
pthread_t thread;
|
||||
if (pthread_create(&thread, nullptr, g, nullptr))
|
||||
DB::throwFromErrno("Cannot create thread.");
|
||||
DB::throwFromErrno("Cannot create thread.", DB::ErrorCodes::PTHREAD_ERROR);
|
||||
if (pthread_join(thread, nullptr))
|
||||
DB::throwFromErrno("Cannot join thread.");
|
||||
DB::throwFromErrno("Cannot join thread.", DB::ErrorCodes::PTHREAD_ERROR);
|
||||
});
|
||||
|
||||
test(n, "Create and destroy std::thread each iteration", []
|
||||
|
@ -54,6 +54,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int TIMEOUT_EXCEEDED;
|
||||
extern const int TYPE_MISMATCH;
|
||||
extern const int CANNOT_LINK;
|
||||
}
|
||||
|
||||
|
||||
@ -557,7 +558,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);
|
||||
throwFromErrno("Could not link " + block_file_path + " to " + first_file_tmp_path, ErrorCodes::CANNOT_LINK);
|
||||
}
|
||||
|
||||
/** remove the temporary file, enabling the OS to reclaim inode after all threads
|
||||
|
@ -79,6 +79,9 @@ namespace ErrorCodes
|
||||
extern const int TOO_MANY_PARTS;
|
||||
extern const int INCOMPATIBLE_COLUMNS;
|
||||
extern const int CANNOT_UPDATE_COLUMN;
|
||||
extern const int CANNOT_ALLOCATE_MEMORY;
|
||||
extern const int CANNOT_MUNMAP;
|
||||
extern const int CANNOT_MREMAP;
|
||||
}
|
||||
|
||||
|
||||
@ -477,7 +480,10 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
|
||||
/// Don't count the part as broken if there is not enough memory to load it.
|
||||
/// In fact, there can be many similar situations.
|
||||
/// But it is OK, because there is a safety guard against deleting too many parts.
|
||||
if (e.code() == ErrorCodes::MEMORY_LIMIT_EXCEEDED)
|
||||
if (e.code() == ErrorCodes::MEMORY_LIMIT_EXCEEDED
|
||||
|| e.code() == ErrorCodes::CANNOT_ALLOCATE_MEMORY
|
||||
|| e.code() == ErrorCodes::CANNOT_MUNMAP
|
||||
|| e.code() == ErrorCodes::CANNOT_MREMAP)
|
||||
throw;
|
||||
|
||||
broken = true;
|
||||
|
@ -5,6 +5,14 @@
|
||||
#include <Common/Exception.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int SYSTEM_ERROR;
|
||||
}
|
||||
}
|
||||
|
||||
int main(int, char **)
|
||||
try
|
||||
{
|
||||
@ -14,7 +22,7 @@ try
|
||||
Poco::File("./test_dir/file").createFile();
|
||||
|
||||
if (0 != symlink("./test_dir", "./test_link"))
|
||||
DB::throwFromErrno("Cannot create symlink");
|
||||
DB::throwFromErrno("Cannot create symlink", DB::ErrorCodes::SYSTEM_ERROR);
|
||||
|
||||
Poco::File link("./test_link");
|
||||
link.renameTo("./test_link2");
|
||||
|
@ -45,7 +45,7 @@ class BaseDaemon : public Poco::Util::ServerApplication
|
||||
friend class SignalListener;
|
||||
|
||||
public:
|
||||
static constexpr char DEFAULT_GRAPHITE_CONFIG_NAME[] = "graphite";
|
||||
static inline constexpr char DEFAULT_GRAPHITE_CONFIG_NAME[] = "graphite";
|
||||
|
||||
BaseDaemon();
|
||||
~BaseDaemon() override;
|
||||
|
@ -39,10 +39,8 @@
|
||||
#include <Poco/Observer.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Poco/AutoPtr.h>
|
||||
#include <Poco/SplitterChannel.h>
|
||||
#include <Poco/Ext/LevelFilterChannel.h>
|
||||
#include <Poco/Ext/ThreadNumber.h>
|
||||
#include <Poco/FormattingChannel.h>
|
||||
#include <Poco/PatternFormatter.h>
|
||||
#include <Poco/ConsoleChannel.h>
|
||||
#include <Poco/TaskManager.h>
|
||||
@ -71,20 +69,6 @@
|
||||
#include <Poco/Net/RemoteSyslogChannel.h>
|
||||
|
||||
|
||||
using Poco::Logger;
|
||||
using Poco::AutoPtr;
|
||||
using Poco::Observer;
|
||||
using Poco::FormattingChannel;
|
||||
using Poco::SplitterChannel;
|
||||
using Poco::ConsoleChannel;
|
||||
using Poco::FileChannel;
|
||||
using Poco::Path;
|
||||
using Poco::Message;
|
||||
using Poco::Util::AbstractConfiguration;
|
||||
|
||||
|
||||
constexpr char BaseDaemon::DEFAULT_GRAPHITE_CONFIG_NAME[];
|
||||
|
||||
/** For transferring information from signal handler to a separate thread.
|
||||
* If you need to do something serious in case of a signal (example: write a message to the log),
|
||||
* then sending information to a separate thread through pipe and doing all the stuff asynchronously
|
||||
@ -109,7 +93,7 @@ struct Pipe
|
||||
write_fd = -1;
|
||||
|
||||
if (0 != pipe(fds))
|
||||
DB::throwFromErrno("Cannot create pipe");
|
||||
DB::throwFromErrno("Cannot create pipe", 0);
|
||||
}
|
||||
|
||||
void close()
|
||||
@ -669,7 +653,7 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config)
|
||||
std::cerr << "Logging " << log_level << " to " << log_path << std::endl;
|
||||
|
||||
// Set up two channel chains.
|
||||
log_file = new FileChannel;
|
||||
log_file = new Poco::FileChannel;
|
||||
log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(log_path).absolute().toString());
|
||||
log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M"));
|
||||
log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number");
|
||||
@ -691,7 +675,7 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config)
|
||||
createDirectory(errorlog_path);
|
||||
std::cerr << "Logging errors to " << errorlog_path << std::endl;
|
||||
|
||||
error_log_file = new FileChannel;
|
||||
error_log_file = new Poco::FileChannel;
|
||||
error_log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(errorlog_path).absolute().toString());
|
||||
error_log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M"));
|
||||
error_log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number");
|
||||
@ -703,7 +687,7 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config)
|
||||
Poco::AutoPtr<OwnPatternFormatter> pf = new OwnPatternFormatter(this);
|
||||
|
||||
Poco::AutoPtr<DB::OwnFormattingChannel> errorlog = new DB::OwnFormattingChannel(pf, error_log_file);
|
||||
errorlog->setLevel(Message::PRIO_NOTICE);
|
||||
errorlog->setLevel(Poco::Message::PRIO_NOTICE);
|
||||
errorlog->open();
|
||||
split->addChannel(errorlog);
|
||||
}
|
||||
@ -767,12 +751,12 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config)
|
||||
Logger::root().setChannel(logger().getChannel());
|
||||
|
||||
// Explicitly specified log levels for specific loggers.
|
||||
AbstractConfiguration::Keys levels;
|
||||
Poco::Util::AbstractConfiguration::Keys levels;
|
||||
config.keys("logger.levels", levels);
|
||||
|
||||
if(!levels.empty())
|
||||
for(AbstractConfiguration::Keys::iterator it = levels.begin(); it != levels.end(); ++it)
|
||||
Logger::get(*it).setLevel(config.getString("logger.levels." + *it, "trace"));
|
||||
if (!levels.empty())
|
||||
for (const auto & level : levels)
|
||||
Logger::get(level).setLevel(config.getString("logger.levels." + level, "trace"));
|
||||
}
|
||||
|
||||
|
||||
@ -1077,7 +1061,7 @@ void BaseDaemon::logRevision() const
|
||||
/// Makes server shutdown if at least one Poco::Task have failed.
|
||||
void BaseDaemon::exitOnTaskError()
|
||||
{
|
||||
Observer<BaseDaemon, Poco::TaskFailedNotification> obs(*this, &BaseDaemon::handleNotification);
|
||||
Poco::Observer<BaseDaemon, Poco::TaskFailedNotification> obs(*this, &BaseDaemon::handleNotification);
|
||||
getTaskManager().addObserver(obs);
|
||||
}
|
||||
|
||||
@ -1085,7 +1069,7 @@ void BaseDaemon::exitOnTaskError()
|
||||
void BaseDaemon::handleNotification(Poco::TaskFailedNotification *_tfn)
|
||||
{
|
||||
task_failed = true;
|
||||
AutoPtr<Poco::TaskFailedNotification> fn(_tfn);
|
||||
Poco::AutoPtr<Poco::TaskFailedNotification> fn(_tfn);
|
||||
Logger *lg = &(logger());
|
||||
LOG_ERROR(lg, "Task '" << fn->task()->name() << "' failed. Daemon is shutting down. Reason - " << fn->reason().displayText());
|
||||
ServerApplication::terminate();
|
||||
|
@ -8,13 +8,18 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_GETTIMEOFDAY;
|
||||
}
|
||||
|
||||
ExtendedLogMessage ExtendedLogMessage::getFrom(const Poco::Message & base)
|
||||
{
|
||||
ExtendedLogMessage msg_ext(base);
|
||||
|
||||
::timeval tv;
|
||||
if (0 != gettimeofday(&tv, nullptr))
|
||||
DB::throwFromErrno("Cannot gettimeofday");
|
||||
DB::throwFromErrno("Cannot gettimeofday", ErrorCodes::CANNOT_GETTIMEOFDAY);
|
||||
|
||||
msg_ext.time_seconds = static_cast<UInt32>(tv.tv_sec);
|
||||
msg_ext.time_microseconds = static_cast<UInt32>(tv.tv_usec);
|
||||
|
@ -17,7 +17,16 @@
|
||||
#include <cstdlib>
|
||||
#include <port/clock.h>
|
||||
|
||||
using DB::throwFromErrno;
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_OPEN_FILE;
|
||||
extern const int CANNOT_CLOSE_FILE;
|
||||
extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR;
|
||||
extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
enum Mode
|
||||
@ -33,7 +42,9 @@ enum Mode
|
||||
|
||||
void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block_size, size_t count)
|
||||
{
|
||||
DB::Memory direct_buf(block_size, sysconf(_SC_PAGESIZE));
|
||||
using namespace DB;
|
||||
|
||||
Memory direct_buf(block_size, sysconf(_SC_PAGESIZE));
|
||||
std::vector<char> simple_buf(block_size);
|
||||
|
||||
char * buf;
|
||||
@ -60,12 +71,12 @@ void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block
|
||||
if (mode & MODE_READ)
|
||||
{
|
||||
if (static_cast<int>(block_size) != pread(fd, buf, block_size, offset))
|
||||
throwFromErrno("Cannot read");
|
||||
throwFromErrno("Cannot read", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (static_cast<int>(block_size) != pwrite(fd, buf, block_size, offset))
|
||||
throwFromErrno("Cannot write");
|
||||
throwFromErrno("Cannot write", ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -73,6 +84,8 @@ void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block
|
||||
|
||||
int mainImpl(int argc, char ** argv)
|
||||
{
|
||||
using namespace DB;
|
||||
|
||||
const char * file_name = 0;
|
||||
int mode = MODE_NONE;
|
||||
UInt64 min_offset = 0;
|
||||
@ -89,11 +102,11 @@ int mainImpl(int argc, char ** argv)
|
||||
}
|
||||
|
||||
file_name = argv[1];
|
||||
min_offset = DB::parse<UInt64>(argv[3]);
|
||||
max_offset = DB::parse<UInt64>(argv[4]);
|
||||
block_size = DB::parse<UInt64>(argv[5]);
|
||||
threads = DB::parse<UInt64>(argv[6]);
|
||||
count = DB::parse<UInt64>(argv[7]);
|
||||
min_offset = parse<UInt64>(argv[3]);
|
||||
max_offset = parse<UInt64>(argv[4]);
|
||||
block_size = parse<UInt64>(argv[5]);
|
||||
threads = parse<UInt64>(argv[6]);
|
||||
count = parse<UInt64>(argv[7]);
|
||||
|
||||
for (int i = 0; argv[2][i]; ++i)
|
||||
{
|
||||
@ -128,11 +141,11 @@ int mainImpl(int argc, char ** argv)
|
||||
int fd = open(file_name, ((mode & MODE_READ) ? O_RDONLY : O_WRONLY) | ((mode & MODE_SYNC) ? O_SYNC : 0));
|
||||
#endif
|
||||
if (-1 == fd)
|
||||
throwFromErrno("Cannot open file");
|
||||
throwFromErrno("Cannot open file", ErrorCodes::CANNOT_OPEN_FILE);
|
||||
#ifdef __APPLE__
|
||||
if (mode & MODE_DIRECT)
|
||||
if (fcntl(fd, F_NOCACHE, 1) == -1)
|
||||
throwFromErrno("Cannot open file");
|
||||
throwFromErrno("Cannot open file", ErrorCodes::CANNOT_CLOSE_FILE);
|
||||
#endif
|
||||
Stopwatch watch;
|
||||
|
||||
@ -145,7 +158,7 @@ int mainImpl(int argc, char ** argv)
|
||||
watch.stop();
|
||||
|
||||
if (0 != close(fd))
|
||||
throwFromErrno("Cannot close file");
|
||||
throwFromErrno("Cannot close file", ErrorCodes::CANNOT_CLOSE_FILE);
|
||||
|
||||
std::cout << std::fixed << std::setprecision(2)
|
||||
<< "Done " << count << " * " << threads << " ops";
|
||||
|
@ -29,7 +29,16 @@ int main(int argc, char ** argv) { return 0; }
|
||||
#include <sys/syscall.h>
|
||||
|
||||
|
||||
using DB::throwFromErrno;
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_OPEN_FILE;
|
||||
extern const int CANNOT_CLOSE_FILE;
|
||||
extern const int CANNOT_IO_SUBMIT;
|
||||
extern const int CANNOT_IO_GETEVENTS;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
enum Mode
|
||||
@ -41,11 +50,13 @@ enum Mode
|
||||
|
||||
void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block_size, size_t buffers_count, size_t count)
|
||||
{
|
||||
using namespace DB;
|
||||
|
||||
AIOContext ctx;
|
||||
|
||||
std::vector<DB::Memory> buffers(buffers_count);
|
||||
std::vector<Memory> buffers(buffers_count);
|
||||
for (size_t i = 0; i < buffers_count; ++i)
|
||||
buffers[i] = DB::Memory(block_size, sysconf(_SC_PAGESIZE));
|
||||
buffers[i] = Memory(block_size, sysconf(_SC_PAGESIZE));
|
||||
|
||||
drand48_data rand_data;
|
||||
timespec times;
|
||||
@ -109,13 +120,13 @@ void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block
|
||||
|
||||
/// Send queries.
|
||||
if (io_submit(ctx.ctx, query_cbs.size(), &query_cbs[0]) < 0)
|
||||
throwFromErrno("io_submit failed");
|
||||
throwFromErrno("io_submit failed", ErrorCodes::CANNOT_IO_SUBMIT);
|
||||
|
||||
/// Receive answers. If we have something else to send, then receive at least one answer (after that send them), otherwise wait all answers.
|
||||
memset(&events[0], 0, buffers_count * sizeof(events[0]));
|
||||
int evs = io_getevents(ctx.ctx, (blocks_sent < count ? 1 : in_progress), buffers_count, &events[0], nullptr);
|
||||
if (evs < 0)
|
||||
throwFromErrno("io_getevents failed");
|
||||
throwFromErrno("io_getevents failed", ErrorCodes::CANNOT_IO_GETEVENTS);
|
||||
|
||||
for (int i = 0; i < evs; ++i)
|
||||
{
|
||||
@ -131,6 +142,8 @@ void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block
|
||||
|
||||
int mainImpl(int argc, char ** argv)
|
||||
{
|
||||
using namespace DB;
|
||||
|
||||
const char * file_name = 0;
|
||||
int mode = MODE_READ;
|
||||
UInt64 min_offset = 0;
|
||||
@ -149,16 +162,16 @@ int mainImpl(int argc, char ** argv)
|
||||
file_name = argv[1];
|
||||
if (argv[2][0] == 'w')
|
||||
mode = MODE_WRITE;
|
||||
min_offset = DB::parse<UInt64>(argv[3]);
|
||||
max_offset = DB::parse<UInt64>(argv[4]);
|
||||
block_size = DB::parse<UInt64>(argv[5]);
|
||||
threads_count = DB::parse<UInt64>(argv[6]);
|
||||
buffers_count = DB::parse<UInt64>(argv[7]);
|
||||
count = DB::parse<UInt64>(argv[8]);
|
||||
min_offset = parse<UInt64>(argv[3]);
|
||||
max_offset = parse<UInt64>(argv[4]);
|
||||
block_size = parse<UInt64>(argv[5]);
|
||||
threads_count = parse<UInt64>(argv[6]);
|
||||
buffers_count = parse<UInt64>(argv[7]);
|
||||
count = parse<UInt64>(argv[8]);
|
||||
|
||||
int fd = open(file_name, ((mode == MODE_READ) ? O_RDONLY : O_WRONLY) | O_DIRECT);
|
||||
if (-1 == fd)
|
||||
throwFromErrno("Cannot open file");
|
||||
throwFromErrno("Cannot open file", ErrorCodes::CANNOT_OPEN_FILE);
|
||||
|
||||
ThreadPool pool(threads_count);
|
||||
|
||||
@ -171,7 +184,7 @@ int mainImpl(int argc, char ** argv)
|
||||
watch.stop();
|
||||
|
||||
if (0 != close(fd))
|
||||
throwFromErrno("Cannot close file");
|
||||
throwFromErrno("Cannot close file", ErrorCodes::CANNOT_CLOSE_FILE);
|
||||
|
||||
std::cout << std::fixed << std::setprecision(2)
|
||||
<< "Done " << count << " * " << threads_count << " ops";
|
||||
|
@ -20,7 +20,18 @@
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <port/clock.h>
|
||||
|
||||
using DB::throwFromErrno;
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_OPEN_FILE;
|
||||
extern const int CANNOT_CLOSE_FILE;
|
||||
extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR;
|
||||
extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR;
|
||||
extern const int CANNOT_FSYNC;
|
||||
extern const int SYSTEM_ERROR;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
enum Mode
|
||||
@ -32,6 +43,8 @@ enum Mode
|
||||
|
||||
int mainImpl(int argc, char ** argv)
|
||||
{
|
||||
using namespace DB;
|
||||
|
||||
const char * file_name = 0;
|
||||
Mode mode = MODE_READ;
|
||||
UInt64 min_offset = 0;
|
||||
@ -47,11 +60,11 @@ int mainImpl(int argc, char ** argv)
|
||||
}
|
||||
|
||||
file_name = argv[1];
|
||||
min_offset = DB::parse<UInt64>(argv[3]);
|
||||
max_offset = DB::parse<UInt64>(argv[4]);
|
||||
block_size = DB::parse<UInt64>(argv[5]);
|
||||
descriptors = DB::parse<UInt64>(argv[6]);
|
||||
count = DB::parse<UInt64>(argv[7]);
|
||||
min_offset = parse<UInt64>(argv[3]);
|
||||
max_offset = parse<UInt64>(argv[4]);
|
||||
block_size = parse<UInt64>(argv[5]);
|
||||
descriptors = parse<UInt64>(argv[6]);
|
||||
count = parse<UInt64>(argv[7]);
|
||||
|
||||
if (!strcmp(argv[2], "r"))
|
||||
mode = MODE_READ;
|
||||
@ -65,7 +78,7 @@ int mainImpl(int argc, char ** argv)
|
||||
{
|
||||
fds[i] = open(file_name, O_SYNC | ((mode == MODE_READ) ? O_RDONLY : O_WRONLY));
|
||||
if (-1 == fds[i])
|
||||
throwFromErrno("Cannot open file");
|
||||
throwFromErrno("Cannot open file", ErrorCodes::CANNOT_OPEN_FILE);
|
||||
}
|
||||
|
||||
std::vector<char> buf(block_size);
|
||||
@ -87,7 +100,7 @@ int mainImpl(int argc, char ** argv)
|
||||
while (ops < count)
|
||||
{
|
||||
if (poll(&polls[0], descriptors, -1) <= 0)
|
||||
throwFromErrno("poll failed");
|
||||
throwFromErrno("poll failed", ErrorCodes::SYSTEM_ERROR);
|
||||
for (size_t i = 0; i < descriptors; ++i)
|
||||
{
|
||||
if (!polls[i].revents)
|
||||
@ -109,12 +122,12 @@ int mainImpl(int argc, char ** argv)
|
||||
if (mode == MODE_READ)
|
||||
{
|
||||
if (static_cast<int>(block_size) != pread(fds[i], &buf[0], block_size, offset))
|
||||
throwFromErrno("Cannot read");
|
||||
throwFromErrno("Cannot read", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (static_cast<int>(block_size) != pwrite(fds[i], &buf[0], block_size, offset))
|
||||
throwFromErrno("Cannot write");
|
||||
throwFromErrno("Cannot write", ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -122,7 +135,7 @@ int mainImpl(int argc, char ** argv)
|
||||
for (size_t i = 0; i < descriptors; ++i)
|
||||
{
|
||||
if (fsync(fds[i]))
|
||||
throwFromErrno("Cannot fsync");
|
||||
throwFromErrno("Cannot fsync", ErrorCodes::CANNOT_FSYNC);
|
||||
}
|
||||
|
||||
watch.stop();
|
||||
@ -130,7 +143,7 @@ int mainImpl(int argc, char ** argv)
|
||||
for (size_t i = 0; i < descriptors; ++i)
|
||||
{
|
||||
if (0 != close(fds[i]))
|
||||
throwFromErrno("Cannot close file");
|
||||
throwFromErrno("Cannot close file", ErrorCodes::CANNOT_CLOSE_FILE);
|
||||
}
|
||||
|
||||
std::cout << std::fixed << std::setprecision(2)
|
||||
|
Loading…
Reference in New Issue
Block a user