ClickHouse/libs/libdaemon/src/BaseDaemon.cpp

895 lines
25 KiB
C++
Raw Normal View History

#include <daemon/BaseDaemon.h>
#include <Common/Config/ConfigProcessor.h>
#include <sys/stat.h>
#include <sys/types.h>
#include <sys/time.h>
2019-01-19 21:05:20 +00:00
#include <fcntl.h>
#include <errno.h>
#include <string.h>
#include <signal.h>
#include <cxxabi.h>
#include <execinfo.h>
#include <unistd.h>
#include <typeinfo>
#include <common/logger_useful.h>
#include <common/ErrorHandlers.h>
2019-07-01 22:11:11 +00:00
#include <common/StackTrace.h>
#include <sys/time.h>
#include <sys/resource.h>
#include <iostream>
#include <fstream>
#include <sstream>
#include <memory>
#include <Poco/Observer.h>
#include <Poco/AutoPtr.h>
#include <common/getThreadNumber.h>
#include <Poco/PatternFormatter.h>
#include <Poco/TaskManager.h>
#include <Poco/File.h>
#include <Poco/Path.h>
#include <Poco/Message.h>
#include <Poco/Util/Application.h>
#include <Poco/Exception.h>
#include <Poco/ErrorHandler.h>
#include <Poco/Condition.h>
#include <Poco/SyslogChannel.h>
#include <Poco/DirectoryIterator.h>
#include <Common/Exception.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Common/ClickHouseRevision.h>
2018-07-17 18:22:32 +00:00
#include <Common/config_version.h>
#include <common/argsToConfig.h>
2019-03-27 15:42:24 +00:00
#ifdef __APPLE__
// ucontext is not available without _XOPEN_SOURCE
2019-07-02 06:37:57 +00:00
#define _XOPEN_SOURCE 700
2019-03-27 15:42:24 +00:00
#endif
#include <ucontext.h>
2018-01-17 18:06:39 +00:00
/** 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
* - is probably the only safe method for doing it.
* (Because it's only safe to use reentrant functions in signal handlers.)
*/
struct Pipe
{
union
{
int fds[2];
struct
{
int read_fd;
int write_fd;
};
};
Pipe()
{
read_fd = -1;
write_fd = -1;
if (0 != pipe(fds))
DB::throwFromErrno("Cannot create pipe", 0);
}
void close()
{
if (-1 != read_fd)
{
::close(read_fd);
read_fd = -1;
}
if (-1 != write_fd)
{
::close(write_fd);
write_fd = -1;
}
}
~Pipe()
{
close();
}
};
Pipe signal_pipe;
2018-01-17 18:06:39 +00:00
/** Reset signal handler to the default and send signal to itself.
* It's called from user signal handler to write core dump.
*/
static void call_default_signal_handler(int sig)
{
signal(sig, SIG_DFL);
2018-08-20 23:16:50 +00:00
raise(sig);
}
using ThreadNumber = decltype(getThreadNumber());
2019-07-01 22:11:11 +00:00
static const size_t buf_size = sizeof(int) + sizeof(siginfo_t) + sizeof(ucontext_t) + sizeof(StackTrace) + sizeof(ThreadNumber);
using signal_function = void(int, siginfo_t*, void*);
static void writeSignalIDtoSignalPipe(int sig)
{
char buf[buf_size];
DB::WriteBufferFromFileDescriptor out(signal_pipe.write_fd, buf_size, buf);
DB::writeBinary(sig, out);
out.next();
}
2018-01-17 18:06:39 +00:00
/** Signal handler for HUP / USR1 */
static void closeLogsSignalHandler(int sig, siginfo_t * info, void * context)
{
writeSignalIDtoSignalPipe(sig);
}
static void terminateRequestedSignalHandler(int sig, siginfo_t * info, void * context)
{
writeSignalIDtoSignalPipe(sig);
}
Prevent infinite recursion in faultSignalHandler (#CLICKHOUSE-3304) (#1288) * Prevent infinite recursion in faultSignalHandler (#CLICKHOUSE-3304) 21 0x00000000012a0cbd in faultSignalHandler (sig=<optimized out>, info=0x7ffcacfd5730, context=0x7ffcacfd5600) at /home/robot-metrika-test/jenkins/workspace/clickhouse-packages-build@2/sources/libs/libdaemon/src/BaseDaemon.cpp:166 22 <signal handler called> 23 0x00007fc770f90c37 in raise () from /lib/x86_64-linux-gnu/libc.so.6 24 0x00007fc770f94028 in abort () from /lib/x86_64-linux-gnu/libc.so.6 25 0x00000000012a0d5f in terminate_handler () at /home/robot-metrika-test/jenkins/workspace/clickhouse-packages-build@2/sources/libs/libdaemon/src/BaseDaemon.cpp:404 26 0x00000000037c8836 in __cxxabiv1::__terminate(void (*)()) () 27 0x000000000382d379 in __cxa_call_terminate () 28 0x00000000037d475d in __gxx_personality_v0 () 29 0x00000000038331a3 in _Unwind_RaiseException_Phase2 () 30 0x00000000038339c7 in _Unwind_Resume () 31 0x00000000012a0cbd in faultSignalHandler (sig=<optimized out>, info=0x7ffcacfd6ef0, context=0x7ffcacfd6dc0) at /home/robot-metrika-test/jenkins/workspace/clickhouse-packages-build@2/sources/libs/libdaemon/src/BaseDaemon.cpp:166 32 <signal handler called> 33 0x00007fc770f90c37 in raise () from /lib/x86_64-linux-gnu/libc.so.6 34 0x00007fc770f94028 in abort () from /lib/x86_64-linux-gnu/libc.so.6 35 0x00000000012a0d5f in terminate_handler () at /home/robot-metrika-test/jenkins/workspace/clickhouse-packages-build@2/sources/libs/libdaemon/src/BaseDaemon.cpp:404 36 0x00000000037c8836 in __cxxabiv1::__terminate(void (*)()) () 37 0x000000000382d379 in __cxa_call_terminate () 38 0x00000000037d475d in __gxx_personality_v0 () 39 0x00000000038331a3 in _Unwind_RaiseException_Phase2 () 40 0x00000000038339c7 in _Unwind_Resume () 41 0x00000000012a0cbd in faultSignalHandler (sig=<optimized out>, info=0x7ffcacfd86b0, context=0x7ffcacfd8580) at /home/robot-metrika-test/jenkins/workspace/clickhouse-packages-build@2/sources/libs/libdaemon/src/BaseDaemon.cpp:166 * Requested changes
2017-09-30 11:04:56 +00:00
thread_local bool already_signal_handled = false;
2018-01-17 18:06:39 +00:00
/** Handler for "fault" signals. Send data about fault to separate thread to write into log.
*/
static void faultSignalHandler(int sig, siginfo_t * info, void * context)
{
Prevent infinite recursion in faultSignalHandler (#CLICKHOUSE-3304) (#1288) * Prevent infinite recursion in faultSignalHandler (#CLICKHOUSE-3304) 21 0x00000000012a0cbd in faultSignalHandler (sig=<optimized out>, info=0x7ffcacfd5730, context=0x7ffcacfd5600) at /home/robot-metrika-test/jenkins/workspace/clickhouse-packages-build@2/sources/libs/libdaemon/src/BaseDaemon.cpp:166 22 <signal handler called> 23 0x00007fc770f90c37 in raise () from /lib/x86_64-linux-gnu/libc.so.6 24 0x00007fc770f94028 in abort () from /lib/x86_64-linux-gnu/libc.so.6 25 0x00000000012a0d5f in terminate_handler () at /home/robot-metrika-test/jenkins/workspace/clickhouse-packages-build@2/sources/libs/libdaemon/src/BaseDaemon.cpp:404 26 0x00000000037c8836 in __cxxabiv1::__terminate(void (*)()) () 27 0x000000000382d379 in __cxa_call_terminate () 28 0x00000000037d475d in __gxx_personality_v0 () 29 0x00000000038331a3 in _Unwind_RaiseException_Phase2 () 30 0x00000000038339c7 in _Unwind_Resume () 31 0x00000000012a0cbd in faultSignalHandler (sig=<optimized out>, info=0x7ffcacfd6ef0, context=0x7ffcacfd6dc0) at /home/robot-metrika-test/jenkins/workspace/clickhouse-packages-build@2/sources/libs/libdaemon/src/BaseDaemon.cpp:166 32 <signal handler called> 33 0x00007fc770f90c37 in raise () from /lib/x86_64-linux-gnu/libc.so.6 34 0x00007fc770f94028 in abort () from /lib/x86_64-linux-gnu/libc.so.6 35 0x00000000012a0d5f in terminate_handler () at /home/robot-metrika-test/jenkins/workspace/clickhouse-packages-build@2/sources/libs/libdaemon/src/BaseDaemon.cpp:404 36 0x00000000037c8836 in __cxxabiv1::__terminate(void (*)()) () 37 0x000000000382d379 in __cxa_call_terminate () 38 0x00000000037d475d in __gxx_personality_v0 () 39 0x00000000038331a3 in _Unwind_RaiseException_Phase2 () 40 0x00000000038339c7 in _Unwind_Resume () 41 0x00000000012a0cbd in faultSignalHandler (sig=<optimized out>, info=0x7ffcacfd86b0, context=0x7ffcacfd8580) at /home/robot-metrika-test/jenkins/workspace/clickhouse-packages-build@2/sources/libs/libdaemon/src/BaseDaemon.cpp:166 * Requested changes
2017-09-30 11:04:56 +00:00
if (already_signal_handled)
return;
already_signal_handled = true;
char buf[buf_size];
DB::WriteBufferFromFileDescriptor out(signal_pipe.write_fd, buf_size, buf);
2019-06-28 18:06:38 +00:00
const ucontext_t signal_context = *reinterpret_cast<ucontext_t *>(context);
2019-07-02 00:19:56 +00:00
const StackTrace stack_trace(signal_context);
2019-03-27 20:37:39 +00:00
DB::writeBinary(sig, out);
DB::writePODBinary(*info, out);
2019-06-28 18:06:38 +00:00
DB::writePODBinary(signal_context, out);
2019-07-02 00:19:56 +00:00
DB::writePODBinary(stack_trace, out);
DB::writeBinary(getThreadNumber(), out);
out.next();
2018-01-17 18:06:39 +00:00
/// The time that is usually enough for separate thread to print info into log.
::sleep(10);
call_default_signal_handler(sig);
}
2018-01-17 18:06:39 +00:00
/** The thread that read info about signal or std::terminate from pipe.
* On HUP / USR1, close log files (for new files to be opened later).
* On information about std::terminate, write it to log.
* On other signals, write info to log.
*/
class SignalListener : public Poco::Runnable
{
public:
enum Signals : int
{
StdTerminate = -1,
StopThread = -2
};
explicit SignalListener(BaseDaemon & daemon_)
: log(&Logger::get("BaseDaemon"))
, daemon(daemon_)
{
}
void run()
{
char buf[buf_size];
DB::ReadBufferFromFileDescriptor in(signal_pipe.read_fd, buf_size, buf);
while (!in.eof())
{
int sig = 0;
DB::readBinary(sig, in);
if (sig == Signals::StopThread)
{
LOG_INFO(log, "Stop SignalListener thread");
break;
}
else if (sig == SIGHUP || sig == SIGUSR1)
{
LOG_DEBUG(log, "Received signal to close logs.");
BaseDaemon::instance().closeLogs(BaseDaemon::instance().logger());
LOG_INFO(log, "Opened new log file after received signal.");
}
else if (sig == Signals::StdTerminate)
{
ThreadNumber thread_num;
std::string message;
DB::readBinary(thread_num, in);
DB::readBinary(message, in);
onTerminate(message, thread_num);
}
else if (sig == SIGINT ||
sig == SIGQUIT ||
sig == SIGTERM)
{
daemon.handleSignal(sig);
}
else
{
siginfo_t info;
ucontext_t context;
2019-07-02 00:19:56 +00:00
StackTrace stack_trace(NoCapture{});
ThreadNumber thread_num;
DB::readPODBinary(info, in);
DB::readPODBinary(context, in);
2019-07-02 00:19:56 +00:00
DB::readPODBinary(stack_trace, in);
DB::readBinary(thread_num, in);
2019-07-02 00:19:56 +00:00
onFault(sig, info, context, stack_trace, thread_num);
}
}
}
private:
Logger * log;
BaseDaemon & daemon;
private:
void onTerminate(const std::string & message, ThreadNumber thread_num) const
{
LOG_ERROR(log, "(version " << VERSION_STRING << VERSION_OFFICIAL << ") (from thread " << thread_num << ") " << message);
}
2019-07-02 00:19:56 +00:00
void onFault(int sig, siginfo_t & info, ucontext_t & context, const StackTrace & stack_trace, ThreadNumber thread_num) const
{
LOG_ERROR(log, "########################################");
LOG_ERROR(log, "(version " << VERSION_STRING << VERSION_OFFICIAL << ") (from thread " << thread_num << ") "
<< "Received signal " << strsignal(sig) << " (" << sig << ")" << ".");
2019-06-28 18:06:38 +00:00
LOG_ERROR(log, signalToErrorMessage(sig, info, context));
2019-07-02 00:19:56 +00:00
LOG_ERROR(log, stack_trace.toString());
}
};
2018-01-17 18:06:39 +00:00
/** To use with std::set_terminate.
* Collects slightly more info than __gnu_cxx::__verbose_terminate_handler,
* and send it to pipe. Other thread will read this info from pipe and asynchronously write it to log.
* Look at libstdc++-v3/libsupc++/vterminate.cc for example.
*/
static void terminate_handler()
{
static thread_local bool terminating = false;
if (terminating)
{
abort();
return; /// Just for convenience.
}
terminating = true;
std::string log_message;
if (std::current_exception())
log_message = "Terminate called for uncaught exception:\n" + DB::getCurrentExceptionMessage(true);
else
log_message = "Terminate called without an active exception";
static const size_t buf_size = 1024;
if (log_message.size() > buf_size - 16)
log_message.resize(buf_size - 16);
char buf[buf_size];
DB::WriteBufferFromFileDescriptor out(signal_pipe.write_fd, buf_size, buf);
DB::writeBinary(static_cast<int>(SignalListener::StdTerminate), out);
DB::writeBinary(getThreadNumber(), out);
DB::writeBinary(log_message, out);
out.next();
abort();
}
2018-02-14 20:41:46 +00:00
static std::string createDirectory(const std::string & file)
{
2018-02-14 20:41:46 +00:00
auto path = Poco::Path(file).makeParent();
if (path.toString().empty())
return "";
Poco::File(path).createDirectories();
return path.toString();
};
static bool tryCreateDirectories(Poco::Logger * logger, const std::string & path)
{
try
{
Poco::File(path).createDirectories();
return true;
}
catch (...)
{
LOG_WARNING(logger, __PRETTY_FUNCTION__ << ": when creating " << path << ", " << DB::getCurrentExceptionMessage(true));
}
return false;
}
void BaseDaemon::reloadConfiguration()
{
2018-01-17 18:06:39 +00:00
/** If the program is not run in daemon mode and 'config-file' is not specified,
* then we use config from 'config.xml' file in current directory,
* but will log to console (or use parameters --log-file, --errorlog-file from command line)
* instead of using files specified in config.xml.
* (It's convenient to log in console when you start server without any command line parameters.)
*/
config_path = config().getString("config-file", "config.xml");
DB::ConfigProcessor config_processor(config_path, false, true);
config_processor.setConfigPath(Poco::Path(config_path).makeParent().toString());
loaded_config = config_processor.loadConfig(/* allow_zk_includes = */ true);
if (last_configuration != nullptr)
config().removeConfiguration(last_configuration);
last_configuration = loaded_config.configuration.duplicate();
config().add(last_configuration, PRIO_DEFAULT, false);
}
2019-02-02 12:26:07 +00:00
BaseDaemon::BaseDaemon()
{
2019-02-02 14:23:48 +00:00
checkRequiredInstructions();
2019-02-02 12:26:07 +00:00
}
BaseDaemon::~BaseDaemon()
{
writeSignalIDtoSignalPipe(SignalListener::StopThread);
signal_listener_thread.join();
signal_pipe.close();
}
2019-02-02 17:07:20 +00:00
enum class InstructionFail
{
2019-02-02 12:26:07 +00:00
NONE = 0,
SSE3 = 1,
SSSE3 = 2,
SSE4_1 = 3,
SSE4_2 = 4,
AVX = 5,
AVX2 = 6,
AVX512 = 7
2019-02-02 12:26:07 +00:00
};
2019-02-02 17:07:20 +00:00
static std::string instructionFailToString(InstructionFail fail)
{
switch(fail)
{
case InstructionFail::NONE:
return "NONE";
case InstructionFail::SSE3:
return "SSE3";
case InstructionFail::SSSE3:
return "SSSE3";
case InstructionFail::SSE4_1:
return "SSE4.1";
case InstructionFail::SSE4_2:
return "SSE4.2";
case InstructionFail::AVX:
return "AVX";
case InstructionFail::AVX2:
return "AVX2";
case InstructionFail::AVX512:
return "AVX512";
}
2019-02-02 14:23:48 +00:00
__builtin_unreachable();
}
2019-02-02 12:26:07 +00:00
static sigjmp_buf jmpbuf;
2019-02-02 14:23:48 +00:00
static void sigIllCheckHandler(int sig, siginfo_t * info, void * context)
2019-02-02 12:26:07 +00:00
{
siglongjmp(jmpbuf, 1);
}
/// Check if necessary sse extensions are available by trying to execute some sse instructions.
/// If instruction is unavailable, SIGILL will be sent by kernel.
2019-02-02 14:23:48 +00:00
static void checkRequiredInstructions(volatile InstructionFail & fail)
{
#if __SSE3__
2019-02-02 13:54:30 +00:00
fail = InstructionFail::SSE3;
__asm__ volatile ("addsubpd %%xmm0, %%xmm0" : : : "xmm0");
#endif
#if __SSSE3__
2019-02-02 13:54:30 +00:00
fail = InstructionFail::SSSE3;
__asm__ volatile ("pabsw %%xmm0, %%xmm0" : : : "xmm0");
#endif
2019-02-02 12:26:07 +00:00
#if __SSE4_1__
2019-02-02 13:54:30 +00:00
fail = InstructionFail::SSE4_1;
__asm__ volatile ("pmaxud %%xmm0, %%xmm0" : : : "xmm0");
#endif
#if __SSE4_2__
2019-02-02 13:54:30 +00:00
fail = InstructionFail::SSE4_2;
__asm__ volatile ("pcmpgtq %%xmm0, %%xmm0" : : : "xmm0");
#endif
#if __AVX__
2019-02-02 13:54:30 +00:00
fail = InstructionFail::AVX;
2019-02-27 20:04:35 +00:00
__asm__ volatile ("vaddpd %%ymm0, %%ymm0, %%ymm0" : : : "ymm0");
#endif
#if __AVX2__
2019-02-02 13:54:30 +00:00
fail = InstructionFail::AVX2;
2019-03-07 16:38:39 +00:00
__asm__ volatile ("vpabsw %%ymm0, %%ymm0" : : : "ymm0");
#endif
#if __AVX512__
2019-02-02 13:54:30 +00:00
fail = InstructionFail::AVX512;
2019-03-07 16:38:39 +00:00
__asm__ volatile ("vpabsw %%zmm0, %%zmm0" : : : "zmm0");
#endif
2019-02-02 13:54:30 +00:00
fail = InstructionFail::NONE;
2019-02-02 12:26:07 +00:00
}
2019-02-02 14:23:48 +00:00
void BaseDaemon::checkRequiredInstructions()
2019-02-02 12:26:07 +00:00
{
2019-02-02 13:54:30 +00:00
struct sigaction sa{};
2019-02-02 14:23:48 +00:00
struct sigaction sa_old{};
sa.sa_sigaction = sigIllCheckHandler;
2019-02-02 12:26:07 +00:00
sa.sa_flags = SA_SIGINFO;
auto signal = SIGILL;
2019-02-02 17:07:20 +00:00
if (sigemptyset(&sa.sa_mask) != 0
|| sigaddset(&sa.sa_mask, signal) != 0
|| sigaction(signal, &sa, &sa_old) != 0)
{
2019-02-02 13:07:02 +00:00
std::cerr << "Can not set signal handler\n";
2019-02-02 12:26:07 +00:00
exit(1);
}
volatile InstructionFail fail = InstructionFail::NONE;
2019-02-02 17:07:20 +00:00
if (sigsetjmp(jmpbuf, 1))
{
std::cerr << "Instruction check fail. There is no " << instructionFailToString(fail) << " instruction set\n";
2019-02-02 12:26:07 +00:00
exit(1);
}
2019-02-02 14:23:48 +00:00
::checkRequiredInstructions(fail);
2019-02-02 12:26:07 +00:00
2019-02-02 17:07:20 +00:00
if (sigaction(signal, &sa_old, nullptr))
{
2019-02-02 13:07:02 +00:00
std::cerr << "Can not set signal handler\n";
2019-02-02 12:26:07 +00:00
exit(1);
}
}
void BaseDaemon::terminate()
{
getTaskManager().cancelAll();
2018-08-20 23:16:50 +00:00
if (::raise(SIGTERM) != 0)
throw Poco::SystemException("cannot terminate process");
}
void BaseDaemon::kill()
{
pid.clear();
2018-08-20 23:16:50 +00:00
if (::raise(SIGKILL) != 0)
throw Poco::SystemException("cannot kill process");
}
void BaseDaemon::sleep(double seconds)
{
wakeup_event.reset();
wakeup_event.tryWait(seconds * 1000);
}
void BaseDaemon::wakeup()
{
wakeup_event.set();
}
2017-02-14 12:12:23 +00:00
std::string BaseDaemon::getDefaultCorePath() const
{
return "/opt/cores/";
}
void BaseDaemon::closeFDs()
{
#if defined(__FreeBSD__) || (defined(__APPLE__) && defined(__MACH__))
Poco::File proc_path{"/dev/fd"};
#else
Poco::File proc_path{"/proc/self/fd"};
#endif
if (proc_path.isDirectory()) /// Hooray, proc exists
{
2019-01-22 14:37:28 +00:00
std::vector<std::string> fds;
/// in /proc/self/fd directory filenames are numeric file descriptors
proc_path.list(fds);
for (const auto & fd_str : fds)
{
2019-01-22 16:28:05 +00:00
int fd = DB::parse<int>(fd_str);
if (fd > 2 && fd != signal_pipe.read_fd && fd != signal_pipe.write_fd)
::close(fd);
}
}
else
{
2019-01-22 16:28:05 +00:00
int max_fd = -1;
#ifdef _SC_OPEN_MAX
max_fd = sysconf(_SC_OPEN_MAX);
if (max_fd == -1)
#endif
max_fd = 256; /// bad fallback
2019-01-22 16:28:05 +00:00
for (int fd = 3; fd < max_fd; ++fd)
if (fd != signal_pipe.read_fd && fd != signal_pipe.write_fd)
::close(fd);
}
}
void BaseDaemon::initialize(Application & self)
{
closeFDs();
task_manager.reset(new Poco::TaskManager);
ServerApplication::initialize(self);
/// now highest priority (lowest value) is PRIO_APPLICATION = -100, we want higher!
argsToConfig(argv(), config(), PRIO_APPLICATION - 100);
2018-05-15 16:22:00 +00:00
bool is_daemon = config().getBool("application.runAsDaemon", false);
if (is_daemon)
{
2018-01-17 18:06:39 +00:00
/** When creating pid file and looking for config, will search for paths relative to the working path of the program when started.
*/
std::string path = Poco::Path(config().getString("application.path")).setFileName("").toString();
if (0 != chdir(path.c_str()))
throw Poco::Exception("Cannot change directory to " + path);
}
reloadConfiguration();
/// This must be done before creation of any files (including logs).
mode_t umask_num = 0027;
if (config().has("umask"))
{
std::string umask_str = config().getString("umask");
std::stringstream stream;
stream << umask_str;
stream >> std::oct >> umask_num;
}
umask(umask_num);
DB::ConfigProcessor(config_path).savePreprocessedConfig(loaded_config, "");
2018-01-17 18:06:39 +00:00
/// Write core dump on crash.
{
struct rlimit rlim;
if (getrlimit(RLIMIT_CORE, &rlim))
throw Poco::Exception("Cannot getrlimit");
2018-01-17 18:06:39 +00:00
/// 1 GiB by default. If more - it writes to disk too long.
rlim.rlim_cur = config().getUInt64("core_dump.size_limit", 1024 * 1024 * 1024);
if (rlim.rlim_cur && setrlimit(RLIMIT_CORE, &rlim))
{
2018-01-17 18:06:39 +00:00
/// It doesn't work under address/thread sanitizer. http://lists.llvm.org/pipermail/llvm-bugs/2013-April/027880.html
std::cerr << "Cannot set max size of core file to " + std::to_string(rlim.rlim_cur) << std::endl;
}
}
/// This must be done before any usage of DateLUT. In particular, before any logging.
if (config().has("timezone"))
{
if (0 != setenv("TZ", config().getString("timezone").data(), 1))
throw Poco::Exception("Cannot setenv TZ variable");
tzset();
}
std::string log_path = config().getString("logger.log", "");
if (!log_path.empty())
log_path = Poco::Path(log_path).setFileName("").toString();
/** Redirect stdout, stderr to separate files in the log directory (or in the specified file).
* Some libraries write to stderr in case of errors in debug mode,
* and this output makes sense even if the program is run in daemon mode.
* We have to do it before buildLoggers, for errors on logger initialization will be written to these files.
* If logger.stderr is specified then stderr will be forcibly redirected to that file.
*/
if ((!log_path.empty() && is_daemon) || config().has("logger.stderr"))
{
std::string stderr_path = config().getString("logger.stderr", log_path + "/stderr.log");
if (!freopen(stderr_path.c_str(), "a+", stderr))
throw Poco::OpenFileException("Cannot attach stderr to " + stderr_path);
}
if ((!log_path.empty() && is_daemon) || config().has("logger.stdout"))
{
std::string stdout_path = config().getString("logger.stdout", log_path + "/stdout.log");
if (!freopen(stdout_path.c_str(), "a+", stdout))
throw Poco::OpenFileException("Cannot attach stdout to " + stdout_path);
}
/// Create pid file.
if (config().has("pid"))
pid.seed(config().getString("pid"));
/// Change path for logging.
if (!log_path.empty())
{
std::string path = createDirectory(log_path);
if (is_daemon
&& chdir(path.c_str()) != 0)
throw Poco::Exception("Cannot change directory to " + path);
}
else
{
if (is_daemon
&& chdir("/tmp") != 0)
throw Poco::Exception("Cannot change directory to /tmp");
}
buildLoggers(config(), logger());
if (is_daemon)
{
2018-01-17 18:06:39 +00:00
/** Change working directory to the directory to write core dumps.
* We have to do it after buildLoggers, because there is the case when config files was in current directory.
*/
std::string core_path = config().getString("core_path", "");
if (core_path.empty())
core_path = getDefaultCorePath();
tryCreateDirectories(&logger(), core_path);
Poco::File cores = core_path;
if (!(cores.exists() && cores.isDirectory()))
{
core_path = !log_path.empty() ? log_path : "/opt/";
tryCreateDirectories(&logger(), core_path);
}
if (0 != chdir(core_path.c_str()))
throw Poco::Exception("Cannot change directory to " + core_path);
}
initializeTerminationAndSignalProcessing();
logRevision();
for (const auto & key : DB::getMultipleKeysFromConfig(config(), "", "graphite"))
{
graphite_writers.emplace(key, std::make_unique<GraphiteWriter>(key));
}
}
void BaseDaemon::initializeTerminationAndSignalProcessing()
{
std::set_terminate(terminate_handler);
/// We want to avoid SIGPIPE when working with sockets and pipes, and just handle return value/errno instead.
{
sigset_t sig_set;
if (sigemptyset(&sig_set) || sigaddset(&sig_set, SIGPIPE) || pthread_sigmask(SIG_BLOCK, &sig_set, nullptr))
throw Poco::Exception("Cannot block signal.");
}
2018-01-17 18:06:39 +00:00
/// Setup signal handlers.
auto add_signal_handler =
[](const std::vector<int> & signals, signal_function handler)
{
struct sigaction sa;
memset(&sa, 0, sizeof(sa));
sa.sa_sigaction = handler;
sa.sa_flags = SA_SIGINFO;
{
if (sigemptyset(&sa.sa_mask))
throw Poco::Exception("Cannot set signal handler.");
for (auto signal : signals)
if (sigaddset(&sa.sa_mask, signal))
throw Poco::Exception("Cannot set signal handler.");
for (auto signal : signals)
2018-08-20 23:16:50 +00:00
if (sigaction(signal, &sa, nullptr))
throw Poco::Exception("Cannot set signal handler.");
}
};
add_signal_handler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE}, faultSignalHandler);
add_signal_handler({SIGHUP, SIGUSR1}, closeLogsSignalHandler);
add_signal_handler({SIGINT, SIGQUIT, SIGTERM}, terminateRequestedSignalHandler);
2018-01-17 18:06:39 +00:00
/// Set up Poco ErrorHandler for Poco Threads.
static KillingErrorHandler killing_error_handler;
Poco::ErrorHandler::set(&killing_error_handler);
signal_listener.reset(new SignalListener(*this));
signal_listener_thread.start(*signal_listener);
}
2016-02-10 07:54:22 +00:00
void BaseDaemon::logRevision() const
{
Logger::root().information("Starting " + std::string{VERSION_FULL} + " with revision " + std::to_string(ClickHouseRevision::get()));
}
2018-01-17 18:06:39 +00:00
/// Makes server shutdown if at least one Poco::Task have failed.
void BaseDaemon::exitOnTaskError()
{
Poco::Observer<BaseDaemon, Poco::TaskFailedNotification> obs(*this, &BaseDaemon::handleNotification);
getTaskManager().addObserver(obs);
}
2018-01-17 18:06:39 +00:00
/// Used for exitOnTaskError()
void BaseDaemon::handleNotification(Poco::TaskFailedNotification *_tfn)
{
task_failed = true;
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();
}
void BaseDaemon::defineOptions(Poco::Util::OptionSet& _options)
{
Poco::Util::ServerApplication::defineOptions (_options);
_options.addOption(
2018-01-17 16:39:56 +00:00
Poco::Util::Option("config-file", "C", "load configuration from a given file")
.required(false)
.repeatable(false)
.argument("<file>")
.binding("config-file"));
_options.addOption(
2018-01-17 16:39:56 +00:00
Poco::Util::Option("log-file", "L", "use given log file")
.required(false)
.repeatable(false)
.argument("<file>")
.binding("logger.log"));
_options.addOption(
2018-01-17 16:39:56 +00:00
Poco::Util::Option("errorlog-file", "E", "use given log file for errors only")
.required(false)
.repeatable(false)
.argument("<file>")
.binding("logger.errorlog"));
_options.addOption(
2018-01-17 16:39:56 +00:00
Poco::Util::Option("pid-file", "P", "use given pidfile")
.required(false)
.repeatable(false)
.argument("<file>")
.binding("pid"));
}
bool isPidRunning(pid_t pid)
{
if (getpgid(pid) >= 0)
return 1;
return 0;
}
void BaseDaemon::PID::seed(const std::string & file_)
{
2018-01-17 18:06:39 +00:00
file = Poco::Path(file_).absolute().toString();
Poco::File poco_file(file);
if (poco_file.exists())
{
pid_t pid_read = 0;
{
std::ifstream in(file);
if (in.good())
{
in >> pid_read;
if (pid_read && isPidRunning(pid_read))
throw Poco::Exception("Pid file exists and program running with pid = " + std::to_string(pid_read) + ", should not start daemon.");
}
}
std::cerr << "Old pid file exists (with pid = " << pid_read << "), removing." << std::endl;
poco_file.remove();
}
int fd = open(file.c_str(),
O_CREAT | O_EXCL | O_WRONLY,
S_IRUSR | S_IWUSR | S_IRGRP | S_IWGRP | S_IROTH | S_IWOTH);
if (-1 == fd)
{
file.clear();
if (EEXIST == errno)
throw Poco::Exception("Pid file exists, should not start daemon.");
throw Poco::CreateFileException("Cannot create pid file.");
}
try
{
std::stringstream s;
s << getpid();
if (static_cast<ssize_t>(s.str().size()) != write(fd, s.str().c_str(), s.str().size()))
throw Poco::Exception("Cannot write to pid file.");
}
catch (...)
{
close(fd);
throw;
}
close(fd);
}
void BaseDaemon::PID::clear()
{
if (!file.empty())
{
Poco::File(file).remove();
file.clear();
}
}
void BaseDaemon::handleSignal(int signal_id)
{
if (signal_id == SIGINT ||
signal_id == SIGQUIT ||
signal_id == SIGTERM)
{
std::unique_lock<std::mutex> lock(signal_handler_mutex);
{
++terminate_signals_counter;
sigint_signals_counter += signal_id == SIGINT;
signal_event.notify_all();
}
onInterruptSignals(signal_id);
}
else
throw DB::Exception(std::string("Unsupported signal: ") + strsignal(signal_id), 0);
}
void BaseDaemon::onInterruptSignals(int signal_id)
{
is_cancelled = true;
LOG_INFO(&logger(), "Received termination signal (" << strsignal(signal_id) << ")");
if (sigint_signals_counter >= 2)
{
LOG_INFO(&logger(), "Received second signal Interrupt. Immediately terminate.");
kill();
}
}
void BaseDaemon::waitForTerminationRequest()
{
std::unique_lock<std::mutex> lock(signal_handler_mutex);
signal_event.wait(lock, [this](){ return terminate_signals_counter > 0; });
}