ClickHouse/programs/server/Server.cpp

1484 lines
60 KiB
C++
Raw Normal View History

#include "Server.h"
#include <memory>
#include <sys/resource.h>
#include <sys/stat.h>
#include <sys/types.h>
2020-08-08 04:44:04 +00:00
#include <sys/wait.h>
#include <errno.h>
#include <pwd.h>
#include <unistd.h>
#include <Poco/Version.h>
#include <Poco/DirectoryIterator.h>
2017-08-09 14:33:07 +00:00
#include <Poco/Net/HTTPServer.h>
#include <Poco/Net/NetException.h>
2019-02-02 14:05:27 +00:00
#include <Poco/Util/HelpFormatter.h>
#include <Poco/Environment.h>
2021-06-15 19:55:21 +00:00
#include <common/scope_guard.h>
2021-04-25 03:06:38 +00:00
#include <common/defines.h>
#include <common/logger_useful.h>
2019-07-25 22:35:47 +00:00
#include <common/phdr_cache.h>
2015-09-29 19:19:54 +00:00
#include <common/ErrorHandlers.h>
2017-08-09 14:33:07 +00:00
#include <common/getMemoryAmount.h>
#include <common/errnoToString.h>
#include <common/coverage.h>
2017-08-09 14:33:07 +00:00
#include <Common/ClickHouseRevision.h>
2018-04-19 13:56:14 +00:00
#include <Common/DNSResolver.h>
2017-08-09 14:33:07 +00:00
#include <Common/CurrentMetrics.h>
#include <Common/Macros.h>
#include <Common/StringUtils/StringUtils.h>
2017-08-09 14:33:07 +00:00
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/ZooKeeperNodeCache.h>
#include <common/getFQDNOrHostName.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
2019-08-11 20:37:53 +00:00
#include <Common/getExecutablePath.h>
#include <Common/ThreadProfileEvents.h>
#include <Common/ThreadStatus.h>
2020-09-17 12:53:52 +00:00
#include <Common/getMappedArea.h>
#include <Common/remapExecutable.h>
#include <Common/TLDListsHolder.h>
#include <IO/HTTPCommon.h>
#include <IO/UseSSL.h>
#include <Interpreters/AsynchronousMetrics.h>
2017-08-09 14:33:07 +00:00
#include <Interpreters/DDLWorker.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/ExternalModelsLoader.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/loadMetadata.h>
2020-02-03 12:54:36 +00:00
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/DNSCacheUpdater.h>
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
#include <Interpreters/InterserverCredentials.h>
#include <Interpreters/JIT/CompiledExpressionCache.h>
#include <Access/AccessControlManager.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/System/attachSystemTables.h>
2017-08-09 14:33:07 +00:00
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <Functions/registerFunctions.h>
#include <TableFunctions/registerTableFunctions.h>
2020-10-29 03:39:43 +00:00
#include <Formats/registerFormats.h>
#include <Storages/registerStorages.h>
#include <Dictionaries/registerDictionaries.h>
2019-11-27 09:39:44 +00:00
#include <Disks/registerDisks.h>
#include <Common/Config/ConfigReloader.h>
#include <Server/HTTPHandlerFactory.h>
2016-01-17 13:34:36 +00:00
#include "MetricsTransmitter.h"
2018-06-05 20:09:51 +00:00
#include <Common/StatusFile.h>
#include <Server/TCPHandlerFactory.h>
#include <Common/SensitiveDataMasker.h>
2020-03-01 21:58:50 +00:00
#include <Common/ThreadFuzzer.h>
#include <Common/getHashOfLoadedBinary.h>
#include <Common/Elf.h>
#include <Server/MySQLHandlerFactory.h>
2020-05-30 20:02:11 +00:00
#include <Server/PostgreSQLHandlerFactory.h>
2020-10-08 00:23:10 +00:00
#include <Server/ProtocolServerAdapter.h>
#include <Server/HTTP/HTTPServer.h>
2021-05-16 22:06:09 +00:00
#include <filesystem>
2019-07-30 14:04:18 +00:00
#if !defined(ARCADIA_BUILD)
2020-05-18 10:26:23 +00:00
# include "config_core.h"
# include "Common/config_version.h"
# if USE_OPENCL
2020-05-18 12:50:23 +00:00
# include "Common/BitonicSort.h" // Y_IGNORE
2020-05-18 10:26:23 +00:00
# endif
#endif
2019-09-15 10:35:12 +00:00
#if defined(OS_LINUX)
# include <sys/mman.h>
# include <sys/ptrace.h>
# include <Common/hasLinuxCapability.h>
2021-04-25 03:06:38 +00:00
# include <unistd.h>
# include <sys/syscall.h>
#endif
#if USE_SSL
# include <Poco/Net/Context.h>
# include <Poco/Net/SecureServerSocket.h>
#endif
2020-10-11 02:19:01 +00:00
#if USE_GRPC
# include <Server/GRPCServer.h>
#endif
2021-02-01 13:18:17 +00:00
#if USE_NURAFT
2021-03-29 08:24:56 +00:00
# include <Server/KeeperTCPHandlerFactory.h>
2021-02-01 13:18:17 +00:00
#endif
2020-10-11 02:19:01 +00:00
#if USE_JEMALLOC
# include <jemalloc/jemalloc.h>
#endif
namespace CurrentMetrics
{
extern const Metric Revision;
extern const Metric VersionInteger;
2020-04-19 21:43:06 +00:00
extern const Metric MemoryTracking;
2021-02-15 10:26:34 +00:00
extern const Metric MaxDDLEntryID;
}
2021-05-16 22:06:09 +00:00
namespace fs = std::filesystem;
#if USE_JEMALLOC
static bool jemallocOptionEnabled(const char *name)
{
bool value;
size_t size = sizeof(value);
if (mallctl(name, reinterpret_cast<void *>(&value), &size, /* newp= */ nullptr, /* newlen= */ 0))
throw Poco::SystemException("mallctl() failed");
return value;
}
#else
static bool jemallocOptionEnabled(const char *) { return 0; }
#endif
2020-09-17 12:39:37 +00:00
int mainEntryClickHouseServer(int argc, char ** argv)
{
DB::Server app;
if (jemallocOptionEnabled("opt.background_thread"))
{
LOG_ERROR(&app.logger(),
"jemalloc.background_thread was requested, "
"however ClickHouse uses percpu_arena and background_thread most likely will not give any benefits, "
"and also background_thread is not compatible with ClickHouse watchdog "
"(that can be disabled with CLICKHOUSE_WATCHDOG_ENABLE=0)");
}
/// Do not fork separate process from watchdog if we attached to terminal.
/// Otherwise it breaks gdb usage.
/// Can be overridden by environment variable (cannot use server config at this moment).
if (argc > 0)
{
const char * env_watchdog = getenv("CLICKHOUSE_WATCHDOG_ENABLE");
if (env_watchdog)
{
if (0 == strcmp(env_watchdog, "1"))
app.shouldSetupWatchdog(argv[0]);
/// Other values disable watchdog explicitly.
}
else if (!isatty(STDIN_FILENO) && !isatty(STDOUT_FILENO) && !isatty(STDERR_FILENO))
app.shouldSetupWatchdog(argv[0]);
}
2020-09-17 12:39:37 +00:00
try
{
return app.run(argc, argv);
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(true) << "\n";
auto code = DB::getCurrentExceptionCode();
return code ? code : 1;
}
}
namespace
{
2020-05-30 21:57:37 +00:00
void setupTmpPath(Poco::Logger * log, const std::string & path)
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG(log, "Setting up {} to store temporary data in it", path);
2021-05-16 22:06:09 +00:00
fs::create_directories(path);
/// Clearing old temporary files.
2021-05-27 07:35:36 +00:00
fs::directory_iterator dir_end;
for (fs::directory_iterator it(path); it != dir_end; ++it)
{
2021-05-27 07:35:36 +00:00
if (it->is_regular_file() && startsWith(it->path().filename(), "tmp"))
{
2021-05-27 07:35:36 +00:00
LOG_DEBUG(log, "Removing old temporary file {}", it->path().string());
fs::remove(it->path());
}
else
2021-05-27 07:35:36 +00:00
LOG_DEBUG(log, "Skipped file in temporary path {}", it->path().string());
}
}
int waitServersToFinish(std::vector<DB::ProtocolServerAdapter> & servers, size_t seconds_to_wait)
{
const int sleep_max_ms = 1000 * seconds_to_wait;
const int sleep_one_ms = 100;
int sleep_current_ms = 0;
int current_connections = 0;
for (;;)
{
current_connections = 0;
for (auto & server : servers)
{
server.stop();
current_connections += server.currentConnections();
}
if (!current_connections)
break;
sleep_current_ms += sleep_one_ms;
if (sleep_current_ms < sleep_max_ms)
std::this_thread::sleep_for(std::chrono::milliseconds(sleep_one_ms));
else
break;
}
return current_connections;
}
}
2012-03-09 03:06:09 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int SUPPORT_IS_DISABLED;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
extern const int INVALID_CONFIG_PARAMETER;
extern const int SYSTEM_ERROR;
extern const int FAILED_TO_GETPWUID;
extern const int MISMATCHING_USERS_FOR_PROCESS_AND_DATA;
extern const int NETWORK_ERROR;
extern const int CORRUPTED_DATA;
}
static std::string getCanonicalPath(std::string && path)
2012-03-09 03:06:09 +00:00
{
Poco::trimInPlace(path);
if (path.empty())
throw Exception("path configuration parameter is empty", ErrorCodes::INVALID_CONFIG_PARAMETER);
if (path.back() != '/')
path += '/';
return std::move(path);
}
2018-12-10 17:42:33 +00:00
static std::string getUserName(uid_t user_id)
{
/// Try to convert user id into user name.
auto buffer_size = sysconf(_SC_GETPW_R_SIZE_MAX);
if (buffer_size <= 0)
buffer_size = 1024;
std::string buffer;
buffer.reserve(buffer_size);
struct passwd passwd_entry;
struct passwd * result = nullptr;
const auto error = getpwuid_r(user_id, &passwd_entry, buffer.data(), buffer_size, &result);
if (error)
throwFromErrno("Failed to find user name for " + toString(user_id), ErrorCodes::FAILED_TO_GETPWUID, error);
else if (result)
return result->pw_name;
return toString(user_id);
}
2020-11-11 13:07:06 +00:00
Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port, Poco::Logger * log)
{
Poco::Net::SocketAddress socket_address;
try
{
socket_address = Poco::Net::SocketAddress(host, port);
}
catch (const Poco::Net::DNSException & e)
{
const auto code = e.code();
if (code == EAI_FAMILY
#if defined(EAI_ADDRFAMILY)
|| code == EAI_ADDRFAMILY
#endif
)
2020-11-11 13:55:28 +00:00
{
LOG_ERROR(log, "Cannot resolve listen_host ({}), error {}: {}. "
"If it is an IPv6 address and your host has disabled IPv6, then consider to "
"specify IPv4 address to listen in <listen_host> element of configuration "
"file. Example: <listen_host>0.0.0.0</listen_host>",
host, e.code(), e.message());
}
throw;
}
return socket_address;
2020-11-11 13:07:06 +00:00
}
2020-11-23 20:22:04 +00:00
Poco::Net::SocketAddress Server::socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure) const
2020-11-11 13:07:06 +00:00
{
auto address = makeSocketAddress(host, port, &logger());
#if !defined(POCO_CLICKHOUSE_PATCH) || POCO_VERSION < 0x01090100
if (secure)
/// Bug in old (<1.9.1) poco, listen() after bind() with reusePort param will fail because have no implementation in SecureServerSocketImpl
/// https://github.com/pocoproject/poco/pull/2257
socket.bind(address, /* reuseAddress = */ true);
else
#endif
#if POCO_VERSION < 0x01080000
socket.bind(address, /* reuseAddress = */ true);
#else
socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ config().getBool("listen_reuse_port", false));
#endif
/// If caller requests any available port from the OS, discover it after binding.
if (port == 0)
{
address = socket.address();
LOG_DEBUG(&logger(), "Requested any available port (port == 0), actual port is {:d}", address.port());
}
2020-11-11 13:07:06 +00:00
socket.listen(/* backlog = */ config().getUInt("listen_backlog", 64));
return address;
}
2020-11-23 20:22:04 +00:00
void Server::createServer(const std::string & listen_host, const char * port_name, bool listen_try, CreateServerFunc && func) const
2020-11-11 13:07:06 +00:00
{
/// For testing purposes, user may omit tcp_port or http_port or https_port in configuration file.
if (!config().has(port_name))
return;
auto port = config().getInt(port_name);
try
{
func(port);
}
catch (const Poco::Exception &)
{
std::string message = "Listen [" + listen_host + "]:" + std::to_string(port) + " failed: " + getCurrentExceptionMessage(false);
if (listen_try)
{
LOG_WARNING(&logger(), "{}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to "
"specify not disabled IPv4 or IPv6 address to listen in <listen_host> element of configuration "
"file. Example for disabled IPv6: <listen_host>0.0.0.0</listen_host> ."
" Example for disabled IPv4: <listen_host>::</listen_host>",
message);
}
else
{
throw Exception{message, ErrorCodes::NETWORK_ERROR};
}
}
}
void Server::uninitialize()
{
logger().information("shutting down");
BaseDaemon::uninitialize();
}
2019-02-02 13:17:55 +00:00
int Server::run()
{
if (config().hasOption("help"))
{
Poco::Util::HelpFormatter help_formatter(Server::options());
2020-11-10 18:22:26 +00:00
auto header_str = fmt::format("{} [OPTION] [-- [ARG]...]\n"
"positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010",
commandName());
help_formatter.setHeader(header_str);
help_formatter.format(std::cout);
2019-02-02 13:17:55 +00:00
return 0;
}
if (config().hasOption("version"))
{
std::cout << DBMS_NAME << " server version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl;
return 0;
}
2020-03-18 02:02:24 +00:00
return Application::run(); // NOLINT
2019-02-02 13:17:55 +00:00
}
void Server::initialize(Poco::Util::Application & self)
{
BaseDaemon::initialize(self);
logger().information("starting up");
2021-06-22 23:02:57 +00:00
LOG_INFO(&logger(), "OS name: {}, version: {}, architecture: {}",
Poco::Environment::osName(),
Poco::Environment::osVersion(),
Poco::Environment::osArchitecture());
}
std::string Server::getDefaultCorePath() const
{
return getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH)) + "cores";
}
2019-08-03 11:02:40 +00:00
void Server::defineOptions(Poco::Util::OptionSet & options)
2019-02-02 13:17:55 +00:00
{
2019-08-03 11:02:40 +00:00
options.addOption(
2019-02-02 13:17:55 +00:00
Poco::Util::Option("help", "h", "show help and exit")
.required(false)
.repeatable(false)
.binding("help"));
2019-08-03 11:02:40 +00:00
options.addOption(
2019-02-04 12:49:54 +00:00
Poco::Util::Option("version", "V", "show version and exit")
.required(false)
.repeatable(false)
.binding("version"));
2019-08-03 11:02:40 +00:00
BaseDaemon::defineOptions(options);
2019-02-02 13:17:55 +00:00
}
2020-06-04 19:30:30 +00:00
void checkForUsersNotInMainConfig(
const Poco::Util::AbstractConfiguration & config,
const std::string & config_path,
const std::string & users_config_path,
Poco::Logger * log)
{
if (config.getBool("skip_check_for_incorrect_settings", false))
return;
if (config.has("users") || config.has("profiles") || config.has("quotas"))
{
/// We cannot throw exception here, because we have support for obsolete 'conf.d' directory
/// (that does not correspond to config.d or users.d) but substitute configuration to both of them.
LOG_ERROR(log, "The <users>, <profiles> and <quotas> elements should be located in users config file: {} not in main config {}."
" Also note that you should place configuration changes to the appropriate *.d directory like 'users.d'.",
users_config_path, config_path);
}
}
2020-06-04 19:30:30 +00:00
2021-04-25 03:06:38 +00:00
[[noreturn]] void forceShutdown()
{
#if defined(THREAD_SANITIZER) && defined(OS_LINUX)
/// Thread sanitizer tries to do something on exit that we don't need if we want to exit immediately,
/// while connection handling threads are still run.
(void)syscall(SYS_exit_group, 0);
__builtin_unreachable();
#else
_exit(0);
#endif
}
2017-12-02 02:47:12 +00:00
int Server::main(const std::vector<std::string> & /*args*/)
{
2020-05-30 21:57:37 +00:00
Poco::Logger * log = &logger();
2020-08-16 11:52:55 +00:00
UseSSL use_ssl;
MainThreadStatus::getInstance();
2017-04-21 17:47:27 +00:00
registerFunctions();
registerAggregateFunctions();
registerTableFunctions();
registerStorages();
registerDictionaries();
2019-11-27 09:39:44 +00:00
registerDisks();
2020-10-29 03:39:43 +00:00
registerFormats();
2017-04-21 17:47:27 +00:00
2020-09-17 12:15:05 +00:00
CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision());
CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger());
2020-03-01 21:58:50 +00:00
if (ThreadFuzzer::instance().isEffective())
2020-05-23 22:24:01 +00:00
LOG_WARNING(log, "ThreadFuzzer is enabled. Application will run slowly and unstable.");
2020-03-01 21:58:50 +00:00
#if !defined(NDEBUG) || !defined(__OPTIMIZE__)
LOG_WARNING(log, "Server was built in debug mode. It will work slowly.");
#endif
2020-10-26 02:18:36 +00:00
#if defined(SANITIZER)
LOG_WARNING(log, "Server was built with sanitizer. It will work slowly.");
#endif
/** Context contains all that query execution is dependent:
2020-07-08 10:40:02 +00:00
* settings, available functions, data types, aggregate functions, databases, ...
*/
2020-04-16 14:37:38 +00:00
auto shared_context = Context::createShared();
global_context = Context::createGlobal(shared_context.get());
2019-07-08 02:14:32 +00:00
global_context->makeGlobalContext();
global_context->setApplicationType(Context::ApplicationType::SERVER);
// Initialize global thread pool. Do it before we fetch configs from zookeeper
// nodes (`from_zk`), because ZooKeeper interface uses the pool. We will
// ignore `max_thread_pool_size` in configs we fetch from ZK, but oh well.
GlobalThreadPool::initialize(config().getUInt("max_thread_pool_size", 10000));
bool has_zookeeper = config().has("zookeeper");
zkutil::ZooKeeperNodeCache main_config_zk_node_cache([&] { return global_context->getZooKeeper(); });
zkutil::EventPtr main_config_zk_changed_event = std::make_shared<Poco::Event>();
if (loaded_config.has_zk_includes)
{
auto old_configuration = loaded_config.configuration;
ConfigProcessor config_processor(config_path);
loaded_config = config_processor.loadConfigWithZooKeeperIncludes(
main_config_zk_node_cache, main_config_zk_changed_event, /* fallback_to_preprocessed = */ true);
config_processor.savePreprocessedConfig(loaded_config, config().getString("path", DBMS_DEFAULT_PATH));
config().removeConfiguration(old_configuration.get());
config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false);
}
Settings::checkNoSettingNamesAtTopLevel(config(), config_path);
2020-06-04 19:30:30 +00:00
const auto memory_amount = getMemoryAmount();
2018-11-14 01:20:46 +00:00
#if defined(OS_LINUX)
2019-08-11 20:37:53 +00:00
std::string executable_path = getExecutablePath();
if (!executable_path.empty())
{
/// Integrity check based on checksum of the executable code.
/// Note: it is not intended to protect from malicious party,
/// because the reference checksum can be easily modified as well.
/// And we don't involve asymmetric encryption with PKI yet.
/// It's only intended to protect from faulty hardware.
/// Note: it is only based on machine code.
/// But there are other sections of the binary (e.g. exception handling tables)
/// that are interpreted (not executed) but can alter the behaviour of the program as well.
String calculated_binary_hash = getHashOfLoadedBinaryHex();
if (stored_binary_hash.empty())
{
LOG_WARNING(log, "Calculated checksum of the binary: {}."
" There is no information about the reference checksum.", calculated_binary_hash);
}
else if (calculated_binary_hash == stored_binary_hash)
{
LOG_INFO(log, "Calculated checksum of the binary: {}, integrity check passed.", calculated_binary_hash);
}
else
{
/// If program is run under debugger, ptrace will fail.
if (ptrace(PTRACE_TRACEME, 0, nullptr, nullptr) == -1)
{
/// Program is run under debugger. Modification of it's binary image is ok for breakpoints.
LOG_WARNING(log, "Server is run under debugger and its binary image is modified (most likely with breakpoints).",
calculated_binary_hash);
}
else
{
throw Exception(ErrorCodes::CORRUPTED_DATA,
"Calculated checksum of the ClickHouse binary ({0}) does not correspond"
" to the reference checksum stored in the binary ({1})."
" It may indicate one of the following:"
" - the file {2} was changed just after startup;"
" - the file {2} is damaged on disk due to faulty hardware;"
" - the loaded executable is damaged in memory due to faulty hardware;"
" - the file {2} was intentionally modified;"
" - logical error in code."
, calculated_binary_hash, stored_binary_hash, executable_path);
}
}
}
else
2019-08-11 20:37:53 +00:00
executable_path = "/usr/bin/clickhouse"; /// It is used for information messages.
2018-11-14 01:20:46 +00:00
/// After full config loaded
{
2020-09-14 18:08:09 +00:00
if (config().getBool("remap_executable", false))
{
LOG_DEBUG(log, "Will remap executable in memory.");
remapExecutable();
LOG_DEBUG(log, "The code in memory has been successfully remapped.");
2020-09-14 18:08:09 +00:00
}
2020-09-14 18:08:09 +00:00
if (config().getBool("mlock_executable", false))
{
if (hasLinuxCapability(CAP_IPC_LOCK))
{
2020-09-25 02:03:58 +00:00
try
{
/// Get the memory area with (current) code segment.
/// It's better to lock only the code segment instead of calling "mlockall",
/// because otherwise debug info will be also locked in memory, and it can be huge.
auto [addr, len] = getMappedArea(reinterpret_cast<void *>(mainEntryClickHouseServer));
LOG_TRACE(log, "Will do mlock to prevent executable memory from being paged out. It may take a few seconds.");
if (0 != mlock(addr, len))
LOG_WARNING(log, "Failed mlock: {}", errnoToString(ErrorCodes::SYSTEM_ERROR));
else
LOG_TRACE(log, "The memory map of clickhouse executable has been mlock'ed, total {}", ReadableSize(len));
}
catch (...)
{
LOG_WARNING(log, "Cannot mlock: {}", getCurrentExceptionMessage(false));
}
}
else
{
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "It looks like the process has no CAP_IPC_LOCK capability, binary mlock will be disabled."
2020-05-23 22:21:29 +00:00
" It could happen due to incorrect ClickHouse package installation."
" You could resolve the problem manually with 'sudo setcap cap_ipc_lock=+ep {}'."
" Note that it will not work on 'nosuid' mounted filesystems.", executable_path);
2020-09-14 18:08:09 +00:00
}
}
}
2018-11-14 01:20:46 +00:00
#endif
global_context->setRemoteHostFilter(config());
std::string path = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH));
std::string default_database = config().getString("default_database", "default");
2020-08-08 01:52:28 +00:00
/// Check that the process user id matches the owner of the data.
const auto effective_user_id = geteuid();
struct stat statbuf;
if (stat(path.c_str(), &statbuf) == 0 && effective_user_id != statbuf.st_uid)
{
const auto effective_user = getUserName(effective_user_id);
const auto data_owner = getUserName(statbuf.st_uid);
std::string message = "Effective user of the process (" + effective_user +
") does not match the owner of the data (" + data_owner + ").";
if (effective_user_id == 0)
{
message += " Run under 'sudo -u " + data_owner + "'.";
throw Exception(message, ErrorCodes::MISMATCHING_USERS_FOR_PROCESS_AND_DATA);
}
else
{
2020-05-23 22:24:01 +00:00
LOG_WARNING(log, message);
}
}
global_context->setPath(path);
2020-07-04 13:54:24 +00:00
StatusFile status{path + "status", StatusFile::write_full_info};
/// Try to increase limit on number of open files.
{
rlimit rlim;
if (getrlimit(RLIMIT_NOFILE, &rlim))
throw Poco::Exception("Cannot getrlimit");
if (rlim.rlim_cur == rlim.rlim_max)
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG(log, "rlimit on number of file descriptors is {}", rlim.rlim_cur);
}
else
{
rlim_t old = rlim.rlim_cur;
rlim.rlim_cur = config().getUInt("max_open_files", rlim.rlim_max);
int rc = setrlimit(RLIMIT_NOFILE, &rlim);
if (rc != 0)
2020-05-23 22:24:01 +00:00
LOG_WARNING(log, "Cannot set max number of file descriptors to {}. Try to specify max_open_files according to your system limits. error: {}", rlim.rlim_cur, strerror(errno));
else
2020-05-23 22:24:01 +00:00
LOG_DEBUG(log, "Set max number of file descriptors to {} (was {}).", rlim.rlim_cur, old);
}
}
static ServerErrorHandler error_handler;
Poco::ErrorHandler::set(&error_handler);
/// Initialize DateLUT early, to not interfere with running time of first query.
2020-05-23 22:24:01 +00:00
LOG_DEBUG(log, "Initializing DateLUT.");
DateLUT::instance();
2020-05-23 22:24:01 +00:00
LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone());
/// Storage with temporary data for processing of heavy queries.
{
std::string tmp_path = config().getString("tmp_path", path + "tmp/");
std::string tmp_policy = config().getString("tmp_policy", "");
const VolumePtr & volume = global_context->setTemporaryStorage(tmp_path, tmp_policy);
for (const DiskPtr & disk : volume->getDisks())
setupTmpPath(log, disk->getPath());
}
/** Directory with 'flags': files indicating temporary settings for the server set by system administrator.
* Flags may be cleared automatically after being applied by the server.
* Examples: do repair of local data; clone all replicated tables from replica.
*/
{
2021-05-16 22:06:09 +00:00
auto flags_path = fs::path(path) / "flags/";
fs::create_directories(flags_path);
global_context->setFlagsPath(flags_path);
}
/** Directory with user provided files that are usable by 'file' table function.
*/
{
2021-05-16 22:06:09 +00:00
std::string user_files_path = config().getString("user_files_path", fs::path(path) / "user_files/");
global_context->setUserFilesPath(user_files_path);
2021-05-16 22:06:09 +00:00
fs::create_directories(user_files_path);
}
{
2021-05-16 22:06:09 +00:00
std::string dictionaries_lib_path = config().getString("dictionaries_lib_path", fs::path(path) / "dictionaries_lib/");
global_context->setDictionariesLibPath(dictionaries_lib_path);
2021-05-16 22:06:09 +00:00
fs::create_directories(dictionaries_lib_path);
}
/// top_level_domains_lists
{
2021-05-16 22:06:09 +00:00
const std::string & top_level_domains_path = config().getString("top_level_domains_path", fs::path(path) / "top_level_domains/");
TLDListsHolder::getInstance().parseConfig(fs::path(top_level_domains_path) / "", config());
}
2020-03-19 21:14:52 +00:00
{
2021-05-16 22:06:09 +00:00
fs::create_directories(fs::path(path) / "data/");
fs::create_directories(fs::path(path) / "metadata/");
2020-08-12 20:40:13 +00:00
2020-03-19 21:14:52 +00:00
/// Directory with metadata of tables, which was marked as dropped by Atomic database
2021-05-16 22:06:09 +00:00
fs::create_directories(fs::path(path) / "metadata_dropped/");
2020-03-19 21:14:52 +00:00
}
if (config().has("interserver_http_port") && config().has("interserver_https_port"))
throw Exception("Both http and https interserver ports are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
static const auto interserver_tags =
{
std::make_tuple("interserver_http_host", "interserver_http_port", "http"),
std::make_tuple("interserver_https_host", "interserver_https_port", "https")
};
for (auto [host_tag, port_tag, scheme] : interserver_tags)
{
if (config().has(port_tag))
{
String this_host = config().getString(host_tag, "");
if (this_host.empty())
{
this_host = getFQDNOrHostName();
2020-05-23 22:24:01 +00:00
LOG_DEBUG(log, "Configuration parameter '{}' doesn't exist or exists and empty. Will use '{}' as replica host.",
2020-05-23 22:21:29 +00:00
host_tag, this_host);
}
String port_str = config().getString(port_tag);
int port = parse<int>(port_str);
if (port < 0 || port > 0xFFFF)
throw Exception("Out of range '" + String(port_tag) + "': " + toString(port), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
global_context->setInterserverIOAddress(this_host, port);
global_context->setInterserverScheme(scheme);
}
}
2021-04-07 13:52:11 +00:00
LOG_DEBUG(log, "Initiailizing interserver credentials.");
global_context->updateInterserverCredentials(config());
if (config().has("macros"))
2020-09-26 19:18:28 +00:00
global_context->setMacros(std::make_unique<Macros>(config(), "macros", log));
/// Initialize main config reloader.
std::string include_from_path = config().getString("include_from", "/etc/metrika.xml");
2019-06-20 07:17:21 +00:00
if (config().has("query_masking_rules"))
{
SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
2019-06-20 07:17:21 +00:00
}
2020-06-04 19:30:30 +00:00
auto main_config_reloader = std::make_unique<ConfigReloader>(
config_path,
include_from_path,
config().getString("path", ""),
std::move(main_config_zk_node_cache),
main_config_zk_changed_event,
2021-02-19 16:22:47 +00:00
[&](ConfigurationPtr config, bool initial_loading)
{
Settings::checkNoSettingNamesAtTopLevel(*config, config_path);
2020-06-04 19:30:30 +00:00
/// Limit on total memory usage
size_t max_server_memory_usage = config->getUInt64("max_server_memory_usage", 0);
double max_server_memory_usage_to_ram_ratio = config->getDouble("max_server_memory_usage_to_ram_ratio", 0.9);
size_t default_max_server_memory_usage = memory_amount * max_server_memory_usage_to_ram_ratio;
if (max_server_memory_usage == 0)
{
max_server_memory_usage = default_max_server_memory_usage;
LOG_INFO(log, "Setting max_server_memory_usage was set to {}"
" ({} available * {:.2f} max_server_memory_usage_to_ram_ratio)",
formatReadableSizeWithBinarySuffix(max_server_memory_usage),
formatReadableSizeWithBinarySuffix(memory_amount),
max_server_memory_usage_to_ram_ratio);
}
else if (max_server_memory_usage > default_max_server_memory_usage)
{
max_server_memory_usage = default_max_server_memory_usage;
LOG_INFO(log, "Setting max_server_memory_usage was lowered to {}"
" because the system has low amount of memory. The amount was"
" calculated as {} available"
" * {:.2f} max_server_memory_usage_to_ram_ratio",
formatReadableSizeWithBinarySuffix(max_server_memory_usage),
formatReadableSizeWithBinarySuffix(memory_amount),
max_server_memory_usage_to_ram_ratio);
}
total_memory_tracker.setHardLimit(max_server_memory_usage);
total_memory_tracker.setDescription("(total)");
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
// FIXME logging-related things need synchronization -- see the 'Logger * log' saved
// in a lot of places. For now, disable updating log configuration without server restart.
//setTextLog(global_context->getTextLog());
//buildLoggers(*config, logger());
global_context->setClustersConfig(config);
2020-09-26 19:18:28 +00:00
global_context->setMacros(std::make_unique<Macros>(*config, "macros", log));
global_context->setExternalAuthenticatorsConfig(*config);
global_context->setExternalModelsConfig(config);
/// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default)
if (config->has("max_table_size_to_drop"))
global_context->setMaxTableSizeToDrop(config->getUInt64("max_table_size_to_drop"));
if (config->has("max_partition_size_to_drop"))
global_context->setMaxPartitionSizeToDrop(config->getUInt64("max_partition_size_to_drop"));
2021-02-18 14:13:23 +00:00
if (!initial_loading)
{
/// We do not load ZooKeeper configuration on the first config loading
/// because TestKeeper server is not started yet.
if (config->has("zookeeper"))
global_context->reloadZooKeeperIfChanged(config);
2021-02-18 14:13:23 +00:00
global_context->reloadAuxiliaryZooKeepersConfigIfChanged(config);
}
global_context->updateStorageConfiguration(*config);
global_context->updateInterserverCredentials(*config);
},
2021-02-18 14:13:23 +00:00
/* already_loaded = */ false); /// Reload it right now (initial loading)
auto & access_control = global_context->getAccessControlManager();
if (config().has("custom_settings_prefixes"))
access_control.setCustomSettingsPrefixes(config().getString("custom_settings_prefixes"));
/// Initialize access storages.
access_control.addStoragesFromMainConfig(config(), config_path, [&] { return global_context->getZooKeeper(); });
/// Reload config in SYSTEM RELOAD CONFIG query.
2018-05-07 02:01:11 +00:00
global_context->setConfigReloadCallback([&]()
{
main_config_reloader->reload();
access_control.reloadUsersConfigs();
});
2017-08-09 15:34:09 +00:00
/// Limit on total number of concurrently executed queries.
global_context->getProcessList().setMaxSize(config().getInt("max_concurrent_queries", 0));
2019-02-09 22:38:26 +00:00
/// Set up caches.
/// Lower cache size on low-memory systems.
double cache_size_to_ram_max_ratio = config().getDouble("cache_size_to_ram_max_ratio", 0.5);
size_t max_cache_size = memory_amount * cache_size_to_ram_max_ratio;
/// Size of cache for uncompressed blocks. Zero means disabled.
size_t uncompressed_cache_size = config().getUInt64("uncompressed_cache_size", 0);
2019-02-09 22:38:26 +00:00
if (uncompressed_cache_size > max_cache_size)
{
2019-02-09 22:38:26 +00:00
uncompressed_cache_size = max_cache_size;
LOG_INFO(log, "Uncompressed cache size was lowered to {} because the system has low amount of memory",
formatReadableSizeWithBinarySuffix(uncompressed_cache_size));
}
2019-02-09 22:38:26 +00:00
global_context->setUncompressedCache(uncompressed_cache_size);
/// Load global settings from default_profile and system_profile.
global_context->setDefaultProfiles(config());
const Settings & settings = global_context->getSettingsRef();
2019-02-09 22:38:26 +00:00
/// Size of cache for marks (index of MergeTree family of tables). It is mandatory.
size_t mark_cache_size = config().getUInt64("mark_cache_size");
2019-02-09 22:38:26 +00:00
if (!mark_cache_size)
2020-05-23 22:24:01 +00:00
LOG_ERROR(log, "Too low mark cache size will lead to severe performance degradation.");
2019-02-09 22:38:26 +00:00
if (mark_cache_size > max_cache_size)
{
2019-02-09 22:38:26 +00:00
mark_cache_size = max_cache_size;
LOG_INFO(log, "Mark cache size was lowered to {} because the system has low amount of memory",
formatReadableSizeWithBinarySuffix(mark_cache_size));
}
2019-02-09 22:38:26 +00:00
global_context->setMarkCache(mark_cache_size);
/// A cache for mmapped files.
size_t mmap_cache_size = config().getUInt64("mmap_cache_size", 1000); /// The choice of default is arbitrary.
if (mmap_cache_size)
2021-03-28 19:24:28 +00:00
global_context->setMMappedFileCache(mmap_cache_size);
2018-09-06 19:13:49 +00:00
#if USE_EMBEDDED_COMPILER
2021-05-04 20:32:43 +00:00
constexpr size_t compiled_expression_cache_size_default = 1024 * 1024 * 1024;
size_t compiled_expression_cache_size = config().getUInt64("compiled_expression_cache_size", compiled_expression_cache_size_default);
2021-03-05 09:54:17 +00:00
CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_size);
2018-09-06 19:13:49 +00:00
#endif
/// Set path for format schema files
2021-05-16 22:06:09 +00:00
fs::path format_schema_path(config().getString("format_schema_path", fs::path(path) / "format_schemas/"));
global_context->setFormatSchemaPath(format_schema_path);
fs::create_directories(format_schema_path);
2020-07-30 19:08:13 +00:00
/// Check sanity of MergeTreeSettings on server startup
global_context->getMergeTreeSettings().sanityCheck(settings);
2020-08-10 11:23:58 +00:00
global_context->getReplicatedMergeTreeSettings().sanityCheck(settings);
2020-07-30 19:08:13 +00:00
2020-11-11 13:07:06 +00:00
Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0);
Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024));
Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
http_params->setTimeout(settings.http_receive_timeout);
http_params->setKeepAliveTimeout(keep_alive_timeout);
2020-12-21 23:03:08 +00:00
auto servers_to_start_before_tables = std::make_shared<std::vector<ProtocolServerAdapter>>();
2020-11-11 13:07:06 +00:00
std::vector<std::string> listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host");
bool listen_try = config().getBool("listen_try", false);
if (listen_hosts.empty())
{
listen_hosts.emplace_back("::1");
listen_hosts.emplace_back("127.0.0.1");
listen_try = true;
}
2021-03-29 08:24:56 +00:00
if (config().has("keeper_server"))
2020-11-11 13:07:06 +00:00
{
2021-02-01 13:18:17 +00:00
#if USE_NURAFT
2021-02-01 14:14:59 +00:00
/// Initialize test keeper RAFT. Do nothing if no nu_keeper_server in config.
2021-03-29 08:24:56 +00:00
global_context->initializeKeeperStorageDispatcher();
2021-02-01 13:18:17 +00:00
for (const auto & listen_host : listen_hosts)
2020-11-11 13:07:06 +00:00
{
2021-03-29 08:24:56 +00:00
/// TCP Keeper
const char * port_name = "keeper_server.tcp_port";
2021-02-01 13:18:17 +00:00
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
{
Poco::Net::ServerSocket socket;
auto address = socketBindListen(socket, listen_host, port);
socket.setReceiveTimeout(settings.receive_timeout);
socket.setSendTimeout(settings.send_timeout);
servers_to_start_before_tables->emplace_back(
port_name,
std::make_unique<Poco::Net::TCPServer>(
new KeeperTCPHandlerFactory(*this, false), server_pool, socket, new Poco::Net::TCPServerParams));
2021-02-01 13:18:17 +00:00
2021-03-29 08:24:56 +00:00
LOG_INFO(log, "Listening for connections to Keeper (tcp): {}", address.toString());
2021-02-01 13:18:17 +00:00
});
const char * secure_port_name = "keeper_server.tcp_port_secure";
createServer(listen_host, secure_port_name, listen_try, [&](UInt16 port)
{
#if USE_SSL
Poco::Net::SecureServerSocket socket;
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
socket.setReceiveTimeout(settings.receive_timeout);
socket.setSendTimeout(settings.send_timeout);
servers_to_start_before_tables->emplace_back(
secure_port_name,
std::make_unique<Poco::Net::TCPServer>(
new KeeperTCPHandlerFactory(*this, true), server_pool, socket, new Poco::Net::TCPServerParams));
LOG_INFO(log, "Listening for connections to Keeper with secure protocol (tcp_secure): {}", address.toString());
#else
UNUSED(port);
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
ErrorCodes::SUPPORT_IS_DISABLED};
#endif
});
2021-02-01 13:18:17 +00:00
}
#else
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "ClickHouse server built without NuRaft library. Cannot use internal coordination.");
#endif
2020-11-11 13:07:06 +00:00
}
2020-12-21 21:47:10 +00:00
for (auto & server : *servers_to_start_before_tables)
2020-11-25 08:18:15 +00:00
server.start();
2020-11-11 13:07:06 +00:00
2020-11-23 10:55:00 +00:00
SCOPE_EXIT({
/** Ask to cancel background jobs all table engines,
* and also query_log.
* It is important to do early, not in destructor of Context, because
* table engines could use Context on destroy.
*/
LOG_INFO(log, "Shutting down storages.");
global_context->shutdown();
LOG_DEBUG(log, "Shut down storages.");
2020-12-21 21:47:10 +00:00
if (!servers_to_start_before_tables->empty())
{
LOG_DEBUG(log, "Waiting for current connections to servers for tables to finish.");
int current_connections = 0;
2020-12-21 21:47:10 +00:00
for (auto & server : *servers_to_start_before_tables)
{
server.stop();
current_connections += server.currentConnections();
}
if (current_connections)
LOG_INFO(log, "Closed all listening sockets. Waiting for {} outstanding connections.", current_connections);
else
LOG_INFO(log, "Closed all listening sockets.");
if (current_connections > 0)
2020-12-21 21:47:10 +00:00
current_connections = waitServersToFinish(*servers_to_start_before_tables, config().getInt("shutdown_wait_unfinished", 5));
if (current_connections)
LOG_INFO(log, "Closed connections to servers for tables. But {} remain. Probably some tables of other users cannot finish their connections after context shutdown.", current_connections);
else
LOG_INFO(log, "Closed connections to servers for tables.");
2021-01-27 17:54:25 +00:00
2021-03-29 08:24:56 +00:00
global_context->shutdownKeeperStorageDispatcher();
}
2020-11-23 10:55:00 +00:00
Fix SIGSEGV by waiting servers thread pool It is easy to reproduce with shutdown_wait_unfinished=0: ================================================================= ==13442==ERROR: AddressSanitizer: heap-use-after-free on address 0x611000210f30 at pc 0x00000a8e55a0 bp 0x7fff2b83e270 sp 0x7fff2b83e268 WRITE of size 8 at 0x611000210f30 thread T2 (TCPHandler) 0 0xa8e559f in long std::__1::__cxx_atomic_fetch_add<long>(std::__1::__cxx_atomic_base_impl<long>*, long, std::__1::memory_order) obj-x86_64-linux-gnu/../contrib/libcxx/include/atomic:1050:12 1 0xa8e559f in std::__1::__atomic_base<long, true>::fetch_add(long, std::__1::memory_order) obj-x86_64-linux-gnu/../contrib/libcxx/include/atomic:1719:17 2 0xa8e559f in MemoryTracker::alloc(long) obj-x86_64-linux-gnu/../src/Common/MemoryTracker.cpp:146:35 3 0xa8e510c in MemoryTracker::alloc(long) obj-x86_64-linux-gnu/../src/Common/MemoryTracker.cpp 4 0xa90b474 in DB::ThreadStatus::~ThreadStatus() obj-x86_64-linux-gnu/../src/Common/ThreadStatus.cpp:92:28 5 0x1f90ee83 in DB::TCPHandler::runImpl() obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:450:1 6 0x1f92dcac in DB::TCPHandler::run() obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:1492:9 7 0x25bdc2fe in Poco::Net::TCPServerConnection::start() obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerConnection.cpp:43:3 8 0x25bdce1b in Poco::Net::TCPServerDispatcher::run() obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerDispatcher.cpp:113:19 9 0x25e9c784 in Poco::PooledThread::run() obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/ThreadPool.cpp:199:14 10 0x25e96cd6 in Poco::ThreadImpl::runnableEntry(void*) obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Thread_POSIX.cpp:345:27 11 0x7ffff7f723e8 in start_thread (/usr/lib/libpthread.so.0+0x93e8) 12 0x7ffff7ea0292 in clone (/usr/lib/libc.so.6+0x100292) 0x611000210f30 is located 112 bytes inside of 216-byte region [0x611000210ec0,0x611000210f98) freed by thread T0 here: 0 0xa845d02 in operator delete(void*, unsigned long) (/src/ch/tmp/upstream/clickhouse-asan+0xa845d02) 1 0x1d38328c in void std::__1::__libcpp_operator_delete<void*, unsigned long>(void*, unsigned long) obj-x86_64-linux-gnu/../contrib/libcxx/include/new:245:3 2 0x1d38328c in void std::__1::__do_deallocate_handle_size<>(void*, unsigned long) obj-x86_64-linux-gnu/../contrib/libcxx/include/new:271:10 3 0x1d38328c in std::__1::__libcpp_deallocate(void*, unsigned long, unsigned long) obj-x86_64-linux-gnu/../contrib/libcxx/include/new:285:14 4 0x1d38328c in std::__1::allocator<std::__1::__hash_node<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, void*> >::deallocate(std::__1::__hash_node<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, void*>*, unsigned long) obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:849:13 5 0x1d38328c in std::__1::allocator_traits<std::__1::allocator<std::__1::__hash_node<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, void*> > >::deallocate(std::__1::allocator<std::__1::__hash_node<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, void*> >&, std::__1::__hash_node<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, void*>*, unsigned long) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:476:14 6 0x1d38328c in std::__1::__hash_table<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, std::__1::__unordered_map_hasher<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, std::__1::hash<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, std::__1::equal_to<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, true>, std::__1::__unordered_map_equal<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, std::__1::equal_to<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, std::__1::hash<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, true>, std::__1::allocator<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser> > >::__deallocate_node(std::__1::__hash_node_base<std::__1::__hash_node<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, void*>*>*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__hash_table:1581:9 7 0x1d38328c in std::__1::__hash_table<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, std::__1::__unordered_map_hasher<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, std::__1::hash<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, std::__1::equal_to<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, true>, std::__1::__unordered_map_equal<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, std::__1::equal_to<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, std::__1::hash<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, true>, std::__1::allocator<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser> > >::~__hash_table() obj-x86_64-linux-gnu/../contrib/libcxx/include/__hash_table:1519:5 8 0x1d38328c in std::__1::unordered_map<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser, std::__1::hash<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, std::__1::equal_to<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, std::__1::allocator<std::__1::pair<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const, DB::ProcessListForUser> > >::~unordered_map() obj-x86_64-linux-gnu/../contrib/libcxx/include/unordered_map:1044:5 9 0x1d38328c in DB::ProcessList::~ProcessList() obj-x86_64-linux-gnu/../src/Interpreters/ProcessList.h:263:7 10 0x1d38169c in DB::ContextShared::~ContextShared() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:417:5 11 0x1d32f3e5 in std::__1::default_delete<DB::ContextShared>::operator()(DB::ContextShared*) const obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:1397:5 12 0x1d32f3e5 in std::__1::unique_ptr<DB::ContextShared, std::__1::default_delete<DB::ContextShared> >::reset(DB::ContextShared*) obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:1658:7 13 0x1d32f3e5 in DB::SharedContextHolder::reset() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:485:44 14 0xa8863d4 in DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:880:5 15 0xa8863d4 in ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 16 0xa8863d4 in ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 17 0xa86d889 in DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1379:1 18 0x25c0c8b5 in Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 19 0xa85070d in DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:340:25 20 0x25c49eb7 in Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 21 0xa84cd11 in mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:132:20 22 0xa848c3a in main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 23 0x7ffff7dc8151 in __libc_start_main (/usr/lib/libc.so.6+0x28151) previously allocated by thread T2 (TCPHandler) here: 0 0xa84509d in operator new(unsigned long) (/src/ch/tmp/upstream/clickhouse-asan+0xa84509d) 1 0x1e2a7aa6 in void* std::__1::__libcpp_operator_new<unsigned long>(unsigned long) obj-x86_64-linux-gnu/../contrib/libcxx/include/new:235:10 2 0x1e2a7aa6 in std::__1::__libcpp_allocate(unsigned long, unsigned long) obj-x86_64-linux-gnu/../contrib/libcxx/include/new:261:10 3 0x1e2a7aa6 in std::__1::allocator<std::__1::__hash_node<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, void*> >::allocate(unsigned long) obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:840:38 4 0x1e2a7aa6 in std::__1::allocator_traits<std::__1::allocator<std::__1::__hash_node<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, void*> > >::allocate(std::__1::allocator<std::__1::__hash_node<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, void*> >&, unsigned long) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:468:21 5 0x1e2a7aa6 in std::__1::unique_ptr<std::__1::__hash_node<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, void*>, std::__1::__hash_node_destructor<std::__1::allocator<std::__1::__hash_node<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, void*> > > > std::__1::__hash_table<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, std::__1::__unordered_map_hasher<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, std::__1::hash<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, std::__1::equal_to<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, true>, std::__1::__unordered_map_equal<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, std::__1::equal_to<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, std::__1::hash<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, true>, std::__1::allocator<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser> > >::__construct_node_hash<std::__1::piecewise_construct_t const&, std::__1::tuple<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&>, std::__1::tuple<> >(unsigned long, std::__1::piecewise_construct_t const&, std::__1::tuple<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&>&&, std::__1::tuple<>&&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__hash_table:2472:23 6 0x1e2a7aa6 in std::__1::pair<std::__1::__hash_iterator<std::__1::__hash_node<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, void*>*>, bool> std::__1::__hash_table<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, std::__1::__unordered_map_hasher<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, std::__1::hash<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, std::__1::equal_to<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, true>, std::__1::__unordered_map_equal<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser>, std::__1::equal_to<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, std::__1::hash<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, true>, std::__1::allocator<std::__1::__hash_value_type<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser> > >::__emplace_unique_key_args<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::piecewise_construct_t const&, std::__1::tuple<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&>, std::__1::tuple<> >(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::piecewise_construct_t const&, std::__1::tuple<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&>&&, std::__1::tuple<>&&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__hash_table:2093:29 7 0x1e29c13d in std::__1::unordered_map<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, DB::ProcessListForUser, std::__1::hash<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, std::__1::equal_to<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > >, std::__1::allocator<std::__1::pair<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const, DB::ProcessListForUser> > >::operator[](std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&) obj-x86_64-linux-gnu/../contrib/libcxx/include/unordered_map:1740:21 8 0x1e29c13d in DB::ProcessList::insert(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, DB::IAST const*, DB::Context&) obj-x86_64-linux-gnu/../src/Interpreters/ProcessList.cpp:183:50 9 0x1e5a3a58 in DB::executeQueryImpl(char const*, char const*, DB::Context&, bool, DB::QueryProcessingStage::Enum, bool, DB::ReadBuffer*) obj-x86_64-linux-gnu/../src/Interpreters/executeQuery.cpp:486:59 10 0x1e5a153e in DB::executeQuery(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum, bool) obj-x86_64-linux-gnu/../src/Interpreters/executeQuery.cpp:904:30 11 0x1f909bdc in DB::TCPHandler::runImpl() obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:289:24 12 0x1f92dcac in DB::TCPHandler::run() obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:1492:9 13 0x25bdc2fe in Poco::Net::TCPServerConnection::start() obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerConnection.cpp:43:3 14 0x25bdce1b in Poco::Net::TCPServerDispatcher::run() obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerDispatcher.cpp:113:19 15 0x25e9c784 in Poco::PooledThread::run() obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/ThreadPool.cpp:199:14 16 0x25e96cd6 in Poco::ThreadImpl::runnableEntry(void*) obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Thread_POSIX.cpp:345:27 17 0x7ffff7f723e8 in start_thread (/usr/lib/libpthread.so.0+0x93e8) Thread T2 (TCPHandler) created by T0 here: 0 0xa7ffe0a in pthread_create (/src/ch/tmp/upstream/clickhouse-asan+0xa7ffe0a) 1 0x25e9606f in Poco::ThreadImpl::startImpl(Poco::SharedPtr<Poco::Runnable, Poco::ReferenceCounter, Poco::ReleasePolicy<Poco::Runnable> >) obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Thread_POSIX.cpp:202:6 2 0x25e98eea in Poco::Thread::start(Poco::Runnable&) obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Thread.cpp:128:2 3 0x25e9cd28 in Poco::PooledThread::start() obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/ThreadPool.cpp:85:10 4 0x25e9cd28 in Poco::ThreadPool::ThreadPool(int, int, int, int) obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/ThreadPool.cpp:252:12 5 0xa865aff in DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:831:22 6 0x25c0c8b5 in Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 7 0xa85070d in DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:340:25 8 0x25c49eb7 in Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 9 0xa84cd11 in mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:132:20 10 0xa848c3a in main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 11 0x7ffff7dc8151 in __libc_start_main (/usr/lib/libc.so.6+0x28151) SUMMARY: AddressSanitizer: heap-use-after-free obj-x86_64-linux-gnu/../contrib/libcxx/include/atomic:1050:12 in long std::__1::__cxx_atomic_fetch_add<long>(std::__1::__cxx_atomic_base_impl<long>*, long, std::__1::memory_order) Shadow bytes around the buggy address: 0x0c228003a190: fd fd fd fd fd fd fd fd fd fd fd fd fd fd fd fd 0x0c228003a1a0: fd fd fd fd fd fd fa fa fa fa fa fa fa fa fa fa 0x0c228003a1b0: fd fd fd fd fd fd fd fd fd fd fd fd fd fd fd fd 0x0c228003a1c0: fd fd fd fd fd fd fd fd fd fd fd fd fd fd fa fa 0x0c228003a1d0: fa fa fa fa fa fa fa fa fd fd fd fd fd fd fd fd =>0x0c228003a1e0: fd fd fd fd fd fd[fd]fd fd fd fd fd fd fd fd fd 0x0c228003a1f0: fd fd fd fa fa fa fa fa fa fa fa fa fa fa fa fa 0x0c228003a200: fd fd fd fd fd fd fd fd fd fd fd fd fd fd fd fd 0x0c228003a210: fd fd fd fd fd fd fd fd fd fa fa fa fa fa fa fa 0x0c228003a220: fa fa fa fa fa fa fa fa fd fd fd fd fd fd fd fd 0x0c228003a230: fd fd fd fd fd fd fd fd fd fd fd fd fd fd fd fd Shadow byte legend (one shadow byte represents 8 application bytes): Addressable: 00 Partially addressable: 01 02 03 04 05 06 07 Heap left redzone: fa Freed heap region: fd Stack left redzone: f1 Stack mid redzone: f2 Stack right redzone: f3 Stack after return: f5 Stack use after scope: f8 Global redzone: f9 Global init order: f6 Poisoned by user: f7 Container overflow: fc Array cookie: ac Intra object redzone: bb ASan internal: fe Left alloca redzone: ca Right alloca redzone: cb Shadow gap: cc ==13442==ABORTING 2021.02.20 16:39:50.861426 [ 13443 ] {} <Trace> BaseDaemon: Received signal -3 2021.02.20 16:39:50.861668 [ 14989 ] {} <Fatal> BaseDaemon: ######################################## 2021.02.20 16:39:50.861749 [ 14989 ] {} <Fatal> BaseDaemon: (version 21.3.1.6073 (official build), build id: AC8A516D2F60B8505FA128074527EC2C86198E64) (from thread 13874) (no query) Received signal Unknown signal (-3) 2021.02.20 16:39:50.861810 [ 14989 ] {} <Fatal> BaseDaemon: Sanitizer trap. 2021.02.20 16:39:50.861880 [ 14989 ] {} <Fatal> BaseDaemon: Stack trace: 0xa8e94a7 0xad25b1b 0xa831a16 0xa819444 0xa81aefe 0xa81bb4b 0xa8e55a0 0xa8e510d 0xa90b475 0x1f90ee84 0x1f92dcad 0x25bdc2ff 0x25bdce1c 0x25e9c785 0x25e96cd7 0x7ffff7f723e9 0x7ffff7ea0293 2021.02.20 16:39:50.903643 [ 14989 ] {} <Fatal> BaseDaemon: 0.1. inlined from ./obj-x86_64-linux-gnu/../src/Common/StackTrace.cpp:298: StackTrace::tryCapture() 2021.02.20 16:39:50.903708 [ 14989 ] {} <Fatal> BaseDaemon: 0. ../src/Common/StackTrace.cpp:259: StackTrace::StackTrace() @ 0xa8e94a7 in /src/ch/tmp/upstream/clickhouse-asan 2021.02.20 16:39:51.041733 [ 14989 ] {} <Fatal> BaseDaemon: 1.1. inlined from ./obj-x86_64-linux-gnu/../src/Common/CurrentThread.h:78: DB::CurrentThread::getQueryId() 2021.02.20 16:39:51.041768 [ 14989 ] {} <Fatal> BaseDaemon: 1. ../base/daemon/BaseDaemon.cpp:381: sanitizerDeathCallback() @ 0xad25b1b in /src/ch/tmp/upstream/clickhouse-asan 2021.02.20 16:39:52.551623 [ 13442 ] {} <Information> Application: shutting down 2021.02.20 16:39:52.551696 [ 13442 ] {} <Debug> Application: Uninitializing subsystem: Logging Subsystem 2021.02.20 16:39:52.551792 [ 13443 ] {} <Trace> BaseDaemon: Received signal -2 2021.02.20 16:39:52.551831 [ 13443 ] {} <Information> BaseDaemon: Stop SignalListener thread
2021-02-20 13:46:55 +00:00
/// Wait server pool to avoid use-after-free of destroyed context in the handlers
server_pool.joinAll();
2020-11-23 10:55:00 +00:00
/** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available.
* At this moment, no one could own shared part of Context.
*/
global_context.reset();
shared_context.reset();
LOG_DEBUG(log, "Destroyed global context.");
});
2020-09-25 11:27:00 +00:00
/// Set current database name before loading tables and databases because
/// system logs may copy global context.
global_context->setCurrentDatabaseNameInGlobalContext(default_database);
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Loading metadata from {}", path);
2019-07-30 14:04:18 +00:00
2018-12-10 23:21:03 +00:00
try
{
loadMetadataSystem(global_context);
2018-12-10 23:21:03 +00:00
/// After attaching system databases we can initialize system log.
global_context->initializeSystemLogs();
2020-09-25 11:27:00 +00:00
auto & database_catalog = DatabaseCatalog::instance();
2018-12-10 23:21:03 +00:00
/// After the system database is created, attach virtual system tables (in addition to query_log and part_log)
2020-09-25 11:27:00 +00:00
attachSystemTablesServer(*database_catalog.getSystemDatabase(), has_zookeeper);
/// We load temporary database first, because projections need it.
database_catalog.initializeAndLoadTemporaryDatabase();
2018-12-10 23:21:03 +00:00
/// Then, load remaining databases
loadMetadata(global_context, default_database);
2020-09-25 11:27:00 +00:00
database_catalog.loadDatabases();
/// After loading validate that default database exists
database_catalog.assertDatabaseExists(default_database);
2018-12-10 23:21:03 +00:00
}
catch (...)
{
tryLogCurrentException(log, "Caught exception while loading metadata");
throw;
}
2020-05-23 22:24:01 +00:00
LOG_DEBUG(log, "Loaded metadata.");
2019-02-03 21:30:45 +00:00
/// Init trace collector only after trace_log system table was created
/// Disable it if we collect test coverage information, because it will work extremely slow.
2019-09-01 16:21:54 +00:00
///
/// It also cannot work with sanitizers.
/// Sanitizers are using quick "frame walking" stack unwinding (this implies -fno-omit-frame-pointer)
2020-01-11 09:50:41 +00:00
/// And they do unwinding frequently (on every malloc/free, thread/mutex operations, etc).
/// They change %rbp during unwinding and it confuses libunwind if signal comes during sanitizer unwinding
2019-09-01 16:21:54 +00:00
/// and query profiler decide to unwind stack with libunwind at this moment.
///
/// Symptoms: you'll get silent Segmentation Fault - without sanitizer message and without usual ClickHouse diagnostics.
///
/// Look at compiler-rt/lib/sanitizer_common/sanitizer_stacktrace.h
///
2021-04-04 01:52:41 +00:00
#if USE_UNWIND && !WITH_COVERAGE && !defined(SANITIZER) && defined(__x86_64__)
2020-04-22 17:52:21 +00:00
/// Profilers cannot work reliably with any other libunwind or without PHDR cache.
2019-07-25 22:35:47 +00:00
if (hasPHDRCache())
2020-04-22 17:52:21 +00:00
{
2019-07-25 22:35:47 +00:00
global_context->initializeTraceCollector();
2020-04-22 17:52:21 +00:00
/// Set up server-wide memory profiler (for total memory tracker).
UInt64 total_memory_profiler_step = config().getUInt64("total_memory_profiler_step", 0);
if (total_memory_profiler_step)
{
total_memory_tracker.setOrRaiseProfilerLimit(total_memory_profiler_step);
total_memory_tracker.setProfilerStep(total_memory_profiler_step);
}
2020-04-30 13:25:17 +00:00
double total_memory_tracker_sample_probability = config().getDouble("total_memory_tracker_sample_probability", 0);
if (total_memory_tracker_sample_probability)
{
total_memory_tracker.setSampleProbability(total_memory_tracker_sample_probability);
}
2020-04-22 17:52:21 +00:00
}
2019-08-02 18:51:39 +00:00
#endif
2019-02-03 21:30:45 +00:00
/// Describe multiple reasons when query profiler cannot work.
#if !USE_UNWIND
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they cannot work without bundled unwind (stack unwinding) library.");
#endif
#if WITH_COVERAGE
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage.");
#endif
#if defined(SANITIZER)
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they cannot work under sanitizers"
2020-05-23 22:21:29 +00:00
" when two different stack unwinding methods will interfere with each other.");
#endif
2021-04-04 01:52:41 +00:00
#if !defined(__x86_64__)
LOG_INFO(log, "Query Profiler is only tested on x86_64. It also known to not work under qemu-user.");
#endif
if (!hasPHDRCache())
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created"
2020-05-23 22:21:29 +00:00
" (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe).");
std::unique_ptr<DNSCacheUpdater> dns_cache_updater;
if (config().has("disable_internal_dns_cache") && config().getInt("disable_internal_dns_cache"))
{
/// Disable DNS caching at all
2018-04-19 13:56:14 +00:00
DNSResolver::instance().setDisableCacheFlag();
LOG_DEBUG(log, "DNS caching disabled");
}
else
{
/// Initialize a watcher periodically updating DNS cache
dns_cache_updater = std::make_unique<DNSCacheUpdater>(global_context, config().getInt("dns_cache_update_period", 15));
}
#if defined(OS_LINUX)
if (!TasksStatsCounters::checkIfAvailable())
{
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "It looks like this system does not have procfs mounted at /proc location,"
2020-05-23 22:21:29 +00:00
" neither clickhouse-server process has CAP_NET_ADMIN capability."
" 'taskstats' performance statistics will be disabled."
" It could happen due to incorrect ClickHouse package installation."
" You can try to resolve the problem manually with 'sudo setcap cap_net_admin=+ep {}'."
" Note that it will not work on 'nosuid' mounted filesystems."
" It also doesn't work if you run clickhouse-server inside network namespace as it happens in some containers.",
executable_path);
2019-07-06 18:02:28 +00:00
}
if (!hasLinuxCapability(CAP_SYS_NICE))
{
2020-06-27 12:56:06 +00:00
LOG_INFO(log, "It looks like the process has no CAP_SYS_NICE capability, the setting 'os_thread_priority' will have no effect."
2020-05-23 22:21:29 +00:00
" It could happen due to incorrect ClickHouse package installation."
" You could resolve the problem manually with 'sudo setcap cap_sys_nice=+ep {}'."
" Note that it will not work on 'nosuid' mounted filesystems.",
executable_path);
}
#else
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "TaskStats is not implemented for this OS. IO accounting will be disabled.");
#endif
2020-12-21 23:03:08 +00:00
auto servers = std::make_shared<std::vector<ProtocolServerAdapter>>();
{
2019-11-26 20:27:24 +00:00
/// This object will periodically calculate some metrics.
2020-12-17 13:47:03 +00:00
AsynchronousMetrics async_metrics(
2021-07-04 22:41:09 +00:00
global_context, config().getUInt("asynchronous_metrics_update_period_s", 1), servers_to_start_before_tables, servers);
attachSystemTablesAsync(*DatabaseCatalog::instance().getSystemDatabase(), async_metrics);
2019-11-26 20:27:24 +00:00
for (const auto & listen_host : listen_hosts)
{
/// HTTP
2020-12-17 13:47:03 +00:00
const char * port_name = "http_port";
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
{
Poco::Net::ServerSocket socket;
2020-11-11 13:07:06 +00:00
auto address = socketBindListen(socket, listen_host, port);
socket.setReceiveTimeout(settings.http_receive_timeout);
socket.setSendTimeout(settings.http_send_timeout);
servers->emplace_back(
port_name,
std::make_unique<HTTPServer>(
context(), createHandlerFactory(*this, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params));
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Listening for http://{}", address.toString());
});
/// HTTPS
2020-12-17 13:47:03 +00:00
port_name = "https_port";
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
{
#if USE_SSL
Poco::Net::SecureServerSocket socket;
2020-11-11 13:07:06 +00:00
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
socket.setReceiveTimeout(settings.http_receive_timeout);
socket.setSendTimeout(settings.http_send_timeout);
servers->emplace_back(
port_name,
std::make_unique<HTTPServer>(
context(), createHandlerFactory(*this, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params));
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Listening for https://{}", address.toString());
2017-09-28 19:43:31 +00:00
#else
2019-08-06 14:03:41 +00:00
UNUSED(port);
throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.",
ErrorCodes::SUPPORT_IS_DISABLED};
2017-09-28 19:43:31 +00:00
#endif
});
/// TCP
2020-12-17 13:47:03 +00:00
port_name = "tcp_port";
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
{
Poco::Net::ServerSocket socket;
2020-11-11 13:07:06 +00:00
auto address = socketBindListen(socket, listen_host, port);
socket.setReceiveTimeout(settings.receive_timeout);
socket.setSendTimeout(settings.send_timeout);
2020-12-21 21:47:10 +00:00
servers->emplace_back(port_name, std::make_unique<Poco::Net::TCPServer>(
2020-12-02 21:05:51 +00:00
new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ false),
server_pool,
socket,
new Poco::Net::TCPServerParams));
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Listening for connections with native protocol (tcp): {}", address.toString());
});
2020-12-02 21:05:51 +00:00
/// TCP with PROXY protocol, see https://github.com/wolfeidau/proxyv2/blob/master/docs/proxy-protocol.txt
2020-12-17 13:47:03 +00:00
port_name = "tcp_with_proxy_port";
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
2020-12-02 21:05:51 +00:00
{
Poco::Net::ServerSocket socket;
2020-12-08 14:49:18 +00:00
auto address = socketBindListen(socket, listen_host, port);
2020-12-02 21:05:51 +00:00
socket.setReceiveTimeout(settings.receive_timeout);
socket.setSendTimeout(settings.send_timeout);
2020-12-21 21:47:10 +00:00
servers->emplace_back(port_name, std::make_unique<Poco::Net::TCPServer>(
2020-12-02 21:05:51 +00:00
new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ true),
server_pool,
socket,
new Poco::Net::TCPServerParams));
LOG_INFO(log, "Listening for connections with native protocol (tcp) with PROXY: {}", address.toString());
});
/// TCP with SSL
2020-12-17 13:47:03 +00:00
port_name = "tcp_port_secure";
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
{
#if USE_SSL
Poco::Net::SecureServerSocket socket;
2020-11-11 13:07:06 +00:00
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
socket.setReceiveTimeout(settings.receive_timeout);
socket.setSendTimeout(settings.send_timeout);
2020-12-21 21:47:10 +00:00
servers->emplace_back(port_name, std::make_unique<Poco::Net::TCPServer>(
2020-12-02 21:05:51 +00:00
new TCPHandlerFactory(*this, /* secure */ true, /* proxy protocol */ false),
server_pool,
socket,
new Poco::Net::TCPServerParams));
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Listening for connections with secure native protocol (tcp_secure): {}", address.toString());
#else
2019-08-06 14:03:41 +00:00
UNUSED(port);
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
ErrorCodes::SUPPORT_IS_DISABLED};
#endif
});
2019-03-16 02:08:21 +00:00
/// Interserver IO HTTP
2020-12-17 13:47:03 +00:00
port_name = "interserver_http_port";
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
{
Poco::Net::ServerSocket socket;
2020-11-11 13:07:06 +00:00
auto address = socketBindListen(socket, listen_host, port);
socket.setReceiveTimeout(settings.http_receive_timeout);
socket.setSendTimeout(settings.http_send_timeout);
servers->emplace_back(
port_name,
std::make_unique<HTTPServer>(
context(),
createHandlerFactory(*this, async_metrics, "InterserverIOHTTPHandler-factory"),
server_pool,
socket,
http_params));
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Listening for replica communication (interserver): http://{}", address.toString());
});
2020-12-17 13:47:03 +00:00
port_name = "interserver_https_port";
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
{
#if USE_SSL
Poco::Net::SecureServerSocket socket;
2020-11-11 13:07:06 +00:00
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
socket.setReceiveTimeout(settings.http_receive_timeout);
socket.setSendTimeout(settings.http_send_timeout);
servers->emplace_back(
port_name,
std::make_unique<HTTPServer>(
context(),
createHandlerFactory(*this, async_metrics, "InterserverIOHTTPSHandler-factory"),
server_pool,
socket,
http_params));
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Listening for secure replica communication (interserver): https://{}", address.toString());
#else
2019-08-06 14:03:41 +00:00
UNUSED(port);
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
ErrorCodes::SUPPORT_IS_DISABLED};
#endif
});
2020-12-17 13:47:03 +00:00
port_name = "mysql_port";
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
{
Poco::Net::ServerSocket socket;
2020-11-11 13:07:06 +00:00
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
socket.setReceiveTimeout(Poco::Timespan());
socket.setSendTimeout(settings.send_timeout);
2020-12-21 21:47:10 +00:00
servers->emplace_back(port_name, std::make_unique<Poco::Net::TCPServer>(
new MySQLHandlerFactory(*this),
server_pool,
socket,
new Poco::Net::TCPServerParams));
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Listening for MySQL compatibility protocol: {}", address.toString());
});
2019-11-24 16:27:00 +00:00
2020-12-17 13:47:03 +00:00
port_name = "postgresql_port";
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
2020-05-30 20:02:11 +00:00
{
Poco::Net::ServerSocket socket;
2020-11-11 13:07:06 +00:00
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
2020-05-30 20:02:11 +00:00
socket.setReceiveTimeout(Poco::Timespan());
socket.setSendTimeout(settings.send_timeout);
2020-12-21 21:47:10 +00:00
servers->emplace_back(port_name, std::make_unique<Poco::Net::TCPServer>(
2020-05-30 20:02:11 +00:00
new PostgreSQLHandlerFactory(*this),
server_pool,
socket,
new Poco::Net::TCPServerParams));
LOG_INFO(log, "Listening for PostgreSQL compatibility protocol: " + address.toString());
});
2020-10-11 02:19:01 +00:00
#if USE_GRPC
2020-12-17 13:47:03 +00:00
port_name = "grpc_port";
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
2020-09-21 22:12:55 +00:00
{
Poco::Net::SocketAddress server_address(listen_host, port);
2020-12-21 21:47:10 +00:00
servers->emplace_back(port_name, std::make_unique<GRPCServer>(*this, makeSocketAddress(listen_host, port, log)));
2020-09-21 22:12:55 +00:00
LOG_INFO(log, "Listening for gRPC protocol: " + server_address.toString());
});
2020-10-11 02:19:01 +00:00
#endif
2020-09-21 22:12:55 +00:00
2019-11-24 16:27:00 +00:00
/// Prometheus (if defined and not setup yet with http_port)
2020-12-17 13:47:03 +00:00
port_name = "prometheus.port";
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
2019-11-24 16:27:00 +00:00
{
Poco::Net::ServerSocket socket;
2020-11-11 13:07:06 +00:00
auto address = socketBindListen(socket, listen_host, port);
2019-11-24 16:27:00 +00:00
socket.setReceiveTimeout(settings.http_receive_timeout);
socket.setSendTimeout(settings.http_send_timeout);
servers->emplace_back(
port_name,
std::make_unique<HTTPServer>(
context(),
createHandlerFactory(*this, async_metrics, "PrometheusHandler-factory"),
server_pool,
socket,
http_params));
2019-11-24 16:27:00 +00:00
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Listening for Prometheus: http://{}", address.toString());
2019-11-24 16:27:00 +00:00
});
}
2020-12-21 21:47:10 +00:00
if (servers->empty())
throw Exception("No servers started (add valid listen_host and 'tcp_port' or 'http_port' to configuration file.)",
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
2020-12-17 13:47:03 +00:00
/// Must be done after initialization of `servers`, because async_metrics will access `servers` variable from its thread.
async_metrics.start();
global_context->enableNamedSessions();
{
String level_str = config().getString("text_log.level", "");
int level = level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(level_str);
setTextLog(global_context->getTextLog(), level);
}
buildLoggers(config(), logger());
main_config_reloader->start();
access_control.startPeriodicReloadingUsersConfigs();
if (dns_cache_updater)
dns_cache_updater->start();
{
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Available RAM: {}; physical cores: {}; logical cores: {}.",
2020-05-23 22:21:29 +00:00
formatReadableSizeWithBinarySuffix(memory_amount),
getNumberOfPhysicalCPUCores(), // on ARM processors it can show only enabled at current moment cores
std::thread::hardware_concurrency());
}
/// try to load dictionaries immediately, throw on error and die
try
{
global_context->loadDictionaries(config());
}
catch (...)
{
LOG_ERROR(log, "Caught exception while loading dictionaries.");
throw;
}
if (has_zookeeper && config().has("distributed_ddl"))
{
/// DDL worker should be started after all tables were loaded
String ddl_zookeeper_path = config().getString("distributed_ddl.path", "/clickhouse/task_queue/ddl/");
int pool_size = config().getInt("distributed_ddl.pool_size", 1);
if (pool_size < 1)
throw Exception("distributed_ddl.pool_size should be greater then 0", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
global_context->setDDLWorker(std::make_unique<DDLWorker>(pool_size, ddl_zookeeper_path, global_context, &config(),
"distributed_ddl", "DDLWorker", &CurrentMetrics::MaxDDLEntryID));
}
for (auto & server : *servers)
server.start();
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Ready for connections.");
SCOPE_EXIT({
2020-05-23 22:24:01 +00:00
LOG_DEBUG(log, "Received termination signal.");
LOG_DEBUG(log, "Waiting for current connections to close.");
is_cancelled = true;
int current_connections = 0;
2020-12-21 21:47:10 +00:00
for (auto & server : *servers)
{
2020-12-09 08:58:41 +00:00
server.stop();
current_connections += server.currentConnections();
}
2020-05-23 22:21:29 +00:00
if (current_connections)
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Closed all listening sockets. Waiting for {} outstanding connections.", current_connections);
2020-05-23 22:21:29 +00:00
else
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Closed all listening sockets.");
2019-07-30 23:12:04 +00:00
/// Killing remaining queries.
global_context->getProcessList().killAllQueries();
if (current_connections)
2020-12-21 21:47:10 +00:00
current_connections = waitServersToFinish(*servers, config().getInt("shutdown_wait_unfinished", 5));
2020-05-23 22:21:29 +00:00
if (current_connections)
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Closed connections. But {} remain."
2020-05-23 22:21:29 +00:00
" Tip: To increase wait time add to config: <shutdown_wait_unfinished>60</shutdown_wait_unfinished>", current_connections);
else
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Closed connections.");
dns_cache_updater.reset();
main_config_reloader.reset();
2019-07-30 23:12:04 +00:00
if (current_connections)
{
/// There is no better way to force connections to close in Poco.
/// Otherwise connection handlers will continue to live
/// (they are effectively dangling objects, but they use global thread pool
/// and global thread pool destructor will wait for threads, preventing server shutdown).
/// Dump coverage here, because std::atexit callback would not be called.
dumpCoverageReportIfPossible();
2020-05-23 22:24:01 +00:00
LOG_INFO(log, "Will shutdown forcefully.");
2021-04-25 03:06:38 +00:00
forceShutdown();
2019-07-30 23:12:04 +00:00
}
});
std::vector<std::unique_ptr<MetricsTransmitter>> metrics_transmitters;
for (const auto & graphite_key : DB::getMultipleKeysFromConfig(config(), "", "graphite"))
{
metrics_transmitters.emplace_back(std::make_unique<MetricsTransmitter>(
2019-07-04 19:08:37 +00:00
global_context->getConfigRef(), graphite_key, async_metrics));
}
waitForTerminationRequest();
}
return Application::EXIT_OK;
2012-03-09 03:06:09 +00:00
}
}