ClickHouse/dbms/programs/server/Server.cpp

1037 lines
42 KiB
C++
Raw Normal View History

#include "Server.h"
#include <memory>
#include <sys/resource.h>
#include <sys/stat.h>
#include <sys/types.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>
2017-08-09 14:33:07 +00:00
#include <ext/scope_guard.h>
#include <common/logger_useful.h>
2019-07-25 22:35:47 +00:00
#include <common/phdr_cache.h>
2019-08-02 18:51:39 +00:00
#include <common/config_common.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/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 "config_core.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/TaskStatsInfoGetter.h>
#include <Common/ThreadStatus.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>
#include <Interpreters/DNSCacheUpdater.h>
2019-08-01 13:05:59 +00:00
#include <Interpreters/SystemLog.cpp>
#include <Interpreters/ExternalLoaderXMLConfigRepository.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>
#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>
2017-08-09 14:33:07 +00:00
#include "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>
2017-08-09 14:33:07 +00:00
#include "TCPHandlerFactory.h"
#include "Common/config_version.h"
#include <Common/SensitiveDataMasker.h>
#include "MySQLHandlerFactory.h"
2019-07-30 14:04:18 +00:00
2019-09-15 10:35:12 +00:00
#if defined(OS_LINUX)
#include <Common/hasLinuxCapability.h>
#include <sys/mman.h>
#endif
#if USE_POCO_NETSSL
#include <Poco/Net/Context.h>
#include <Poco/Net/SecureServerSocket.h>
#endif
namespace CurrentMetrics
{
extern const Metric Revision;
extern const Metric VersionInteger;
}
namespace
{
void setupTmpPath(Logger * log, const std::string & path)
{
LOG_DEBUG(log, "Setting up " << path << " to store temporary data in it");
Poco::File(path).createDirectories();
/// Clearing old temporary files.
Poco::DirectoryIterator dir_end;
for (Poco::DirectoryIterator it(path); it != dir_end; ++it)
{
if (it->isFile() && startsWith(it.name(), "tmp"))
{
LOG_DEBUG(log, "Removing old temporary file " << it->path());
it->remove();
}
else
LOG_DEBUG(log, "Skipped file in temporary path " << it->path());
}
}
}
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;
}
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);
}
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 helpFormatter(Server::options());
2019-02-02 15:37:54 +00:00
std::stringstream header;
header << commandName() << " [OPTION] [-- [ARG]...]\n";
header << "positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010";
helpFormatter.setHeader(header.str());
2019-02-02 13:17:55 +00:00
helpFormatter.format(std::cout);
return 0;
}
if (config().hasOption("version"))
{
std::cout << DBMS_NAME << " server version " << VERSION_STRING << VERSION_OFFICIAL << "." << std::endl;
return 0;
}
2019-02-02 13:17:55 +00:00
return Application::run();
}
void Server::initialize(Poco::Util::Application & self)
{
BaseDaemon::initialize(self);
logger().information("starting up");
}
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
}
2017-12-02 02:47:12 +00:00
int Server::main(const std::vector<std::string> & /*args*/)
{
Logger * log = &logger();
UseSSL use_ssl;
ThreadStatus thread_status;
2017-04-21 17:47:27 +00:00
registerFunctions();
registerAggregateFunctions();
registerTableFunctions();
registerStorages();
registerDictionaries();
2019-11-27 09:39:44 +00:00
registerDisks();
2017-04-21 17:47:27 +00:00
CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::get());
CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger());
/** Context contains all that query execution is dependent:
* settings, available functions, data types, aggregate functions, databases...
*/
global_context = std::make_unique<Context>(Context::createGlobal());
2019-07-08 02:14:32 +00:00
global_context->makeGlobalContext();
global_context->setApplicationType(Context::ApplicationType::SERVER);
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);
}
const auto memory_amount = getMemoryAmount();
2018-11-14 01:20:46 +00:00
#if defined(__linux__)
2019-08-11 20:37:53 +00:00
std::string executable_path = getExecutablePath();
if (executable_path.empty())
executable_path = "/usr/bin/clickhouse"; /// It is used for information messages.
2018-11-14 01:20:46 +00:00
/// After full config loaded
{
if (config().getBool("mlock_executable", false))
{
if (hasLinuxCapability(CAP_IPC_LOCK))
{
2018-11-14 01:20:46 +00:00
LOG_TRACE(log, "Will mlockall to prevent executable memory from being paged out. It may take a few seconds.");
if (0 != mlockall(MCL_CURRENT))
LOG_WARNING(log, "Failed mlockall: " + errnoToString(ErrorCodes::SYSTEM_ERROR));
else
2018-11-14 01:20:46 +00:00
LOG_TRACE(log, "The memory map of clickhouse executable has been mlock'ed");
}
else
{
2018-11-14 01:20:46 +00:00
LOG_INFO(log, "It looks like the process has no CAP_IPC_LOCK capability, binary mlock will be disabled."
" It could happen due to incorrect ClickHouse package installation."
2019-08-11 20:37:53 +00:00
" You could resolve the problem manually with 'sudo setcap cap_ipc_lock=+ep " << executable_path << "'."
2018-11-14 01:20:46 +00:00
" Note that it will not work on 'nosuid' mounted filesystems.");
}
}
}
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");
/// 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
{
LOG_WARNING(log, message);
}
}
global_context->setPath(path);
/// Create directories for 'path' and for default database, if not exist.
Poco::File(path + "data/" + default_database).createDirectories();
Poco::File(path + "metadata/" + default_database).createDirectories();
StatusFile status{path + "status"};
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.");
2019-06-20 07:17:21 +00:00
global_context->shutdown();
2019-06-20 07:17:21 +00:00
2019-12-26 14:09:36 +00:00
LOG_DEBUG(log, "Shut down storages.");
/** 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();
LOG_DEBUG(log, "Destroyed global context.");
});
/// 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)
{
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)
LOG_WARNING(log,
"Cannot set max number of file descriptors to " << rlim.rlim_cur
<< ". Try to specify max_open_files according to your system limits. error: "
<< strerror(errno));
else
LOG_DEBUG(log, "Set max number of file descriptors to " << rlim.rlim_cur << " (was " << old << ").");
}
}
static ServerErrorHandler error_handler;
Poco::ErrorHandler::set(&error_handler);
/// Initialize DateLUT early, to not interfere with running time of first query.
LOG_DEBUG(log, "Initializing DateLUT.");
DateLUT::instance();
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->disks)
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.
*/
{
Poco::File(path + "flags/").createDirectories();
global_context->setFlagsPath(path + "flags/");
}
/** Directory with user provided files that are usable by 'file' table function.
*/
{
std::string user_files_path = config().getString("user_files_path", path + "user_files/");
global_context->setUserFilesPath(user_files_path);
Poco::File(user_files_path).createDirectories();
}
{
std::string dictionaries_lib_path = config().getString("dictionaries_lib_path", path + "dictionaries_lib/");
global_context->setDictionariesLibPath(dictionaries_lib_path);
Poco::File(dictionaries_lib_path).createDirectories();
}
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();
LOG_DEBUG(log,
"Configuration parameter '" + String(host_tag) + "' doesn't exist or exists and empty. Will use '" + this_host
+ "' as replica 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);
}
}
if (config().has("interserver_http_credentials"))
{
String user = config().getString("interserver_http_credentials.user", "");
String password = config().getString("interserver_http_credentials.password", "");
if (user.empty())
2018-07-26 16:10:21 +00:00
throw Exception("Configuration parameter interserver_http_credentials user can't be empty", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
global_context->setInterserverCredentials(user, password);
}
if (config().has("macros"))
2018-03-13 23:44:23 +00:00
global_context->setMacros(std::make_unique<Macros>(config(), "macros"));
/// 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
}
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,
[&](ConfigurationPtr config)
{
// 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);
2018-03-13 23:44:23 +00:00
global_context->setMacros(std::make_unique<Macros>(*config, "macros"));
/// 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"));
},
/* already_loaded = */ true);
/// Initialize users config reloader.
std::string users_config_path = config().getString("users_config", config_path);
/// If path to users' config isn't absolute, try guess its root (current) dir.
/// At first, try to find it in dir of main config, after will use current dir.
if (users_config_path.empty() || users_config_path[0] != '/')
{
std::string config_dir = Poco::Path(config_path).parent().toString();
if (Poco::File(config_dir + users_config_path).exists())
users_config_path = config_dir + users_config_path;
}
auto users_config_reloader = std::make_unique<ConfigReloader>(users_config_path,
include_from_path,
config().getString("path", ""),
zkutil::ZooKeeperNodeCache([&] { return global_context->getZooKeeper(); }),
std::make_shared<Poco::Event>(),
[&](ConfigurationPtr config) { global_context->setUsersConfig(config); },
/* already_loaded = */ false);
/// Reload config in SYSTEM RELOAD CONFIG query.
2018-05-07 02:01:11 +00:00
global_context->setConfigReloadCallback([&]()
{
main_config_reloader->reload();
users_config_reloader->reload();
});
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 " << formatReadableSizeWithBinarySuffix(uncompressed_cache_size)
<< " because the system has low amount of memory");
}
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());
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)
LOG_ERROR(log, "Too low mark cache size will lead to severe performance degradation.");
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 " << formatReadableSizeWithBinarySuffix(uncompressed_cache_size)
<< " because the system has low amount of memory");
}
2019-02-09 22:38:26 +00:00
global_context->setMarkCache(mark_cache_size);
2018-09-06 19:13:49 +00:00
#if USE_EMBEDDED_COMPILER
size_t compiled_expression_cache_size = config().getUInt64("compiled_expression_cache_size", 500);
if (compiled_expression_cache_size)
2018-09-05 12:42:37 +00:00
global_context->setCompiledExpressionCache(compiled_expression_cache_size);
2018-09-06 19:13:49 +00:00
#endif
/// Set path for format schema files
auto format_schema_path = Poco::File(config().getString("format_schema_path", path + "format_schemas/"));
global_context->setFormatSchemaPath(format_schema_path.path());
format_schema_path.createDirectories();
2019-02-19 20:51:44 +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);
/// After attaching system databases we can initialize system log.
global_context->initializeSystemLogs();
/// After the system database is created, attach virtual system tables (in addition to query_log and part_log)
attachSystemTablesServer(*global_context->getDatabase("system"), has_zookeeper);
/// Then, load remaining databases
loadMetadata(*global_context);
}
catch (...)
{
tryLogCurrentException(log, "Caught exception while loading metadata");
throw;
}
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
///
#if USE_UNWIND && !WITH_COVERAGE && !defined(SANITIZER)
2019-08-02 18:51:39 +00:00
/// QueryProfiler cannot work reliably with any other libunwind or without PHDR cache.
2019-07-25 22:35:47 +00:00
if (hasPHDRCache())
global_context->initializeTraceCollector();
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
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they cannot work without bundled unwind (stack unwinding) library.");
#endif
#if WITH_COVERAGE
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage.");
#endif
#if defined(SANITIZER)
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they cannot work under sanitizers"
" when two different stack unwinding methods will interfere with each other.");
#endif
if (!hasPHDRCache())
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created"
" (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe).");
global_context->setCurrentDatabase(default_database);
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/");
global_context->setDDLWorker(std::make_unique<DDLWorker>(ddl_zookeeper_path, *global_context, &config(), "distributed_ddl"));
}
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();
}
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(__linux__)
2018-08-21 21:05:30 +00:00
if (!TaskStatsInfoGetter::checkPermissions())
{
2018-08-31 22:41:42 +00:00
LOG_INFO(log, "It looks like the process has no CAP_NET_ADMIN capability, 'taskstats' performance statistics will be disabled."
2019-07-06 18:02:28 +00:00
" It could happen due to incorrect ClickHouse package installation."
2019-08-11 20:37:53 +00:00
" You could resolve the problem manually with 'sudo setcap cap_net_admin=+ep " << executable_path << "'."
2019-07-06 18:02:28 +00:00
" 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.");
}
if (!hasLinuxCapability(CAP_SYS_NICE))
{
LOG_INFO(log, "It looks like the process has no CAP_SYS_NICE capability, the setting 'os_thread_nice' will have no effect."
" It could happen due to incorrect ClickHouse package installation."
2019-08-11 20:37:53 +00:00
" You could resolve the problem manually with 'sudo setcap cap_sys_nice=+ep " << executable_path << "'."
2019-07-06 18:02:28 +00:00
" Note that it will not work on 'nosuid' mounted filesystems.");
}
#else
LOG_INFO(log, "TaskStats is not implemented for this OS. IO accounting will be disabled.");
#endif
{
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;
2018-07-10 18:39:32 +00:00
http_params->setTimeout(settings.http_receive_timeout);
http_params->setKeepAliveTimeout(keep_alive_timeout);
std::vector<std::unique_ptr<Poco::Net::TCPServer>> servers;
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;
}
2017-09-06 02:42:44 +00:00
auto make_socket_address = [&](const std::string & host, UInt16 port)
{
Poco::Net::SocketAddress socket_address;
try
{
socket_address = Poco::Net::SocketAddress(host, port);
}
catch (const Poco::Net::DNSException & e)
{
2018-03-12 15:48:55 +00:00
const auto code = e.code();
if (code == EAI_FAMILY
#if defined(EAI_ADDRFAMILY)
2018-03-12 15:48:55 +00:00
|| code == EAI_ADDRFAMILY
#endif
)
{
LOG_ERROR(log,
2018-03-12 15:48:55 +00:00
"Cannot resolve listen_host (" << host << "), error " << e.code() << ": " << e.message() << ". "
2017-05-04 04:01:19 +00:00
"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>");
}
throw;
}
return socket_address;
};
auto socket_bind_listen = [&](auto & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = 0)
{
auto address = make_socket_address(host, port);
#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
2018-03-30 12:42:06 +00:00
socket.bind(address, /* reuseAddress = */ true);
else
#endif
2018-03-30 12:42:06 +00:00
#if POCO_VERSION < 0x01080000
socket.bind(address, /* reuseAddress = */ true);
#else
socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ config().getBool("listen_reuse_port", false));
2018-03-30 12:42:06 +00:00
#endif
socket.listen(/* backlog = */ config().getUInt("listen_backlog", 64));
return address;
};
2019-11-26 20:27:24 +00:00
/// This object will periodically calculate some metrics.
AsynchronousMetrics async_metrics(*global_context);
attachSystemTablesAsync(*global_context->getDatabase("system"), async_metrics);
for (const auto & listen_host : listen_hosts)
{
auto create_server = [&](const char * port_name, auto && func)
{
/// 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_ERROR(log, message
<< ". 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>");
}
else
{
throw Exception{message, ErrorCodes::NETWORK_ERROR};
}
}
};
/// HTTP
create_server("http_port", [&](UInt16 port)
{
Poco::Net::ServerSocket socket;
auto address = socket_bind_listen(socket, listen_host, port);
socket.setReceiveTimeout(settings.http_receive_timeout);
socket.setSendTimeout(settings.http_send_timeout);
2019-12-03 01:14:01 +00:00
auto handler_factory = createDefaultHandlerFatory<HTTPHandler>(*this, "HTTPHandler-factory");
2019-11-24 16:27:00 +00:00
if (config().has("prometheus") && config().getInt("prometheus.port", 0) == 0)
2020-01-11 09:50:41 +00:00
handler_factory->addHandler<PrometheusHandlerFactory>(async_metrics);
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
2019-12-03 01:14:01 +00:00
handler_factory,
server_pool,
socket,
http_params));
2019-12-15 22:53:52 +00:00
LOG_INFO(log, "Listening for http://" + address.toString());
});
/// HTTPS
2019-08-06 14:03:41 +00:00
create_server("https_port", [&](UInt16 port)
{
#if USE_POCO_NETSSL
Poco::Net::SecureServerSocket socket;
auto address = socket_bind_listen(socket, listen_host, port, /* secure = */ true);
socket.setReceiveTimeout(settings.http_receive_timeout);
socket.setSendTimeout(settings.http_send_timeout);
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
2019-11-24 20:03:09 +00:00
createDefaultHandlerFatory<HTTPHandler>(*this, "HTTPSHandler-factory"),
server_pool,
socket,
http_params));
2019-12-15 22:53:52 +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
create_server("tcp_port", [&](UInt16 port)
{
Poco::Net::ServerSocket socket;
auto address = socket_bind_listen(socket, listen_host, port);
socket.setReceiveTimeout(settings.receive_timeout);
socket.setSendTimeout(settings.send_timeout);
servers.emplace_back(std::make_unique<Poco::Net::TCPServer>(
new TCPHandlerFactory(*this),
server_pool,
socket,
new Poco::Net::TCPServerParams));
LOG_INFO(log, "Listening for connections with native protocol (tcp): " + address.toString());
});
/// TCP with SSL
2019-08-06 14:04:51 +00:00
create_server("tcp_port_secure", [&](UInt16 port)
{
#if USE_POCO_NETSSL
Poco::Net::SecureServerSocket socket;
auto address = socket_bind_listen(socket, listen_host, port, /* secure = */ true);
socket.setReceiveTimeout(settings.receive_timeout);
socket.setSendTimeout(settings.send_timeout);
servers.emplace_back(std::make_unique<Poco::Net::TCPServer>(
new TCPHandlerFactory(*this, /* secure= */ true),
server_pool,
socket,
new Poco::Net::TCPServerParams));
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
create_server("interserver_http_port", [&](UInt16 port)
{
Poco::Net::ServerSocket socket;
auto address = socket_bind_listen(socket, listen_host, port);
socket.setReceiveTimeout(settings.http_receive_timeout);
socket.setSendTimeout(settings.http_send_timeout);
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
2019-11-24 20:03:09 +00:00
createDefaultHandlerFatory<InterserverIOHTTPHandler>(*this, "InterserverIOHTTPHandler-factory"),
server_pool,
socket,
http_params));
2019-12-15 22:53:52 +00:00
LOG_INFO(log, "Listening for replica communication (interserver): http://" + address.toString());
});
2019-08-06 14:03:41 +00:00
create_server("interserver_https_port", [&](UInt16 port)
{
#if USE_POCO_NETSSL
Poco::Net::SecureServerSocket socket;
auto address = socket_bind_listen(socket, listen_host, port, /* secure = */ true);
socket.setReceiveTimeout(settings.http_receive_timeout);
socket.setSendTimeout(settings.http_send_timeout);
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
2019-11-24 20:03:09 +00:00
createDefaultHandlerFatory<InterserverIOHTTPHandler>(*this, "InterserverIOHTTPHandler-factory"),
server_pool,
socket,
http_params));
2019-12-15 22:53:52 +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
});
2019-08-06 14:03:41 +00:00
create_server("mysql_port", [&](UInt16 port)
{
Poco::Net::ServerSocket socket;
auto address = socket_bind_listen(socket, listen_host, port, /* secure = */ true);
socket.setReceiveTimeout(Poco::Timespan());
socket.setSendTimeout(settings.send_timeout);
servers.emplace_back(std::make_unique<Poco::Net::TCPServer>(
new MySQLHandlerFactory(*this),
server_pool,
socket,
new Poco::Net::TCPServerParams));
LOG_INFO(log, "Listening for MySQL compatibility protocol: " + address.toString());
});
2019-11-24 16:27:00 +00:00
/// Prometheus (if defined and not setup yet with http_port)
create_server("prometheus.port", [&](UInt16 port)
{
Poco::Net::ServerSocket socket;
auto address = socket_bind_listen(socket, listen_host, port);
socket.setReceiveTimeout(settings.http_receive_timeout);
socket.setSendTimeout(settings.http_send_timeout);
2019-12-03 01:14:01 +00:00
auto handler_factory = new HTTPRequestHandlerFactoryMain(*this, "PrometheusHandler-factory");
2020-01-11 09:50:41 +00:00
handler_factory->addHandler<PrometheusHandlerFactory>(async_metrics);
2019-11-24 16:27:00 +00:00
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
2019-12-03 01:14:01 +00:00
handler_factory,
2019-11-24 16:27:00 +00:00
server_pool,
socket,
http_params));
2019-12-15 22:53:52 +00:00
LOG_INFO(log, "Listening for Prometheus: http://" + address.toString());
2019-11-24 16:27:00 +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);
for (auto & server : servers)
server->start();
{
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();
users_config_reloader->start();
if (dns_cache_updater)
dns_cache_updater->start();
{
std::stringstream message;
2019-02-09 22:38:26 +00:00
message << "Available RAM: " << formatReadableSizeWithBinarySuffix(memory_amount) << ";"
<< " physical cores: " << getNumberOfPhysicalCPUCores() << ";"
// on ARM processors it can show only enabled at current moment cores
2019-02-09 22:38:26 +00:00
<< " logical cores: " << std::thread::hardware_concurrency() << ".";
LOG_INFO(log, message.str());
}
LOG_INFO(log, "Ready for connections.");
SCOPE_EXIT({
LOG_DEBUG(log, "Received termination signal.");
LOG_DEBUG(log, "Waiting for current connections to close.");
is_cancelled = true;
int current_connections = 0;
for (auto & server : servers)
{
server->stop();
current_connections += server->currentConnections();
}
2019-07-30 23:12:04 +00:00
LOG_INFO(log,
"Closed all listening sockets."
<< (current_connections ? " Waiting for " + toString(current_connections) + " outstanding connections." : ""));
2019-07-30 23:12:04 +00:00
/// Killing remaining queries.
global_context->getProcessList().killAllQueries();
if (current_connections)
{
const int sleep_max_ms = 1000 * config().getInt("shutdown_wait_unfinished", 5);
const int sleep_one_ms = 100;
int sleep_current_ms = 0;
while (sleep_current_ms < sleep_max_ms)
{
current_connections = 0;
for (auto & server : servers)
current_connections += server->currentConnections();
if (!current_connections)
break;
sleep_current_ms += sleep_one_ms;
std::this_thread::sleep_for(std::chrono::milliseconds(sleep_one_ms));
}
}
2019-07-30 23:12:04 +00:00
LOG_INFO(
log, "Closed connections." << (current_connections ? " But " + toString(current_connections) + " remains."
2017-05-05 21:25:53 +00:00
" Tip: To increase wait time add to config: <shutdown_wait_unfinished>60</shutdown_wait_unfinished>" : ""));
dns_cache_updater.reset();
main_config_reloader.reset();
users_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();
2019-07-30 23:12:04 +00:00
LOG_INFO(log, "Will shutdown forcefully.");
_exit(Application::EXIT_OK);
}
});
/// try to load dictionaries immediately, throw on error and die
ext::scope_guard dictionaries_xmls, models_xmls;
try
{
if (!config().getBool("dictionaries_lazy_load", true))
{
global_context->tryCreateEmbeddedDictionaries();
global_context->getExternalDictionariesLoader().enableAlwaysLoadEverything(true);
}
dictionaries_xmls = global_context->getExternalDictionariesLoader().addConfigRepository(
std::make_unique<ExternalLoaderXMLConfigRepository>(config(), "dictionaries_config"));
models_xmls = global_context->getExternalModelsLoader().addConfigRepository(
std::make_unique<ExternalLoaderXMLConfigRepository>(config(), "models_config"));
}
catch (...)
{
LOG_ERROR(log, "Caught exception while loading dictionaries.");
throw;
}
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));
}
SessionCleaner session_cleaner(*global_context);
waitForTerminationRequest();
}
return Application::EXIT_OK;
2012-03-09 03:06:09 +00:00
}
}
2019-12-15 06:34:43 +00:00
#pragma GCC diagnostic ignored "-Wunused-function"
#pragma GCC diagnostic ignored "-Wmissing-declarations"
2017-12-02 02:47:12 +00:00
int mainEntryClickHouseServer(int argc, char ** argv)
{
DB::Server app;
try
{
return app.run(argc, argv);
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(true) << "\n";
auto code = DB::getCurrentExceptionCode();
return code ? code : 1;
}
}