2017-03-03 00:24:56 +00:00
# include "Server.h"
2014-04-15 16:39:56 +00:00
2017-03-03 18:59:42 +00:00
# include <memory>
2017-03-03 00:24:56 +00:00
# include <sys/resource.h>
2018-12-09 22:15:59 +00:00
# include <sys/stat.h>
# include <sys/types.h>
2020-08-08 04:44:04 +00:00
# include <sys/wait.h>
2018-05-09 04:22:30 +00:00
# include <errno.h>
2018-12-09 22:15:59 +00:00
# include <pwd.h>
# include <unistd.h>
2018-07-10 17:40:17 +00:00
# include <Poco/Version.h>
2017-03-03 18:59:42 +00:00
# include <Poco/DirectoryIterator.h>
2017-08-09 14:33:07 +00:00
# include <Poco/Net/HTTPServer.h>
2016-06-25 03:06:36 +00:00
# include <Poco/Net/NetException.h>
2019-02-02 14:05:27 +00:00
# include <Poco/Util/HelpFormatter.h>
2021-05-10 05:46:54 +00:00
# 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>
2018-02-08 19:12:37 +00:00
# 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>
2020-06-07 17:29:34 +00:00
# include <common/errnoToString.h>
2019-12-18 10:43:40 +00:00
# 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>
2017-04-01 09:19:00 +00:00
# include <Common/Macros.h>
2021-07-30 09:12:33 +00:00
# include <Common/ShellCommand.h>
2018-01-15 19:07:47 +00:00
# include <Common/StringUtils/StringUtils.h>
2017-08-09 14:33:07 +00:00
# include <Common/ZooKeeper/ZooKeeper.h>
# include <Common/ZooKeeper/ZooKeeperNodeCache.h>
2020-03-18 18:54:27 +00:00
# include <common/getFQDNOrHostName.h>
2017-04-01 09:19:00 +00:00
# include <Common/getMultipleKeysFromConfig.h>
2017-06-22 18:08:14 +00:00
# include <Common/getNumberOfPhysicalCPUCores.h>
2019-08-11 20:37:53 +00:00
# include <Common/getExecutablePath.h>
2020-05-01 18:47:41 +00:00
# include <Common/ThreadProfileEvents.h>
2019-01-15 18:39:54 +00:00
# include <Common/ThreadStatus.h>
2020-09-17 12:53:52 +00:00
# include <Common/getMappedArea.h>
2020-09-10 09:05:57 +00:00
# include <Common/remapExecutable.h>
2020-12-03 21:11:38 +00:00
# include <Common/TLDListsHolder.h>
2017-04-01 09:19:00 +00:00
# include <IO/HTTPCommon.h>
2021-07-30 09:12:33 +00:00
# include <IO/ReadHelpers.h>
2018-09-06 18:05:33 +00:00
# include <IO/UseSSL.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/AsynchronousMetrics.h>
2017-08-09 14:33:07 +00:00
# include <Interpreters/DDLWorker.h>
2019-09-26 10:41:33 +00:00
# include <Interpreters/ExternalDictionariesLoader.h>
2019-10-21 14:59:35 +00:00
# include <Interpreters/ExternalModelsLoader.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/ProcessList.h>
# include <Interpreters/loadMetadata.h>
2020-02-03 12:54:36 +00:00
# include <Interpreters/DatabaseCatalog.h>
2018-05-14 18:36:01 +00:00
# include <Interpreters/DNSCacheUpdater.h>
2019-10-18 15:44:32 +00:00
# include <Interpreters/ExternalLoaderXMLConfigRepository.h>
2020-08-26 08:36:58 +00:00
# include <Interpreters/InterserverCredentials.h>
2021-06-13 12:38:57 +00:00
# include <Interpreters/JIT/CompiledExpressionCache.h>
2020-02-26 22:36:52 +00:00
# include <Access/AccessControlManager.h>
2017-04-01 09:19:00 +00:00
# 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>
2017-12-30 00:36:06 +00:00
# include <Storages/registerStorages.h>
2021-07-14 13:17:30 +00:00
# include <DataStreams/ConnectionCollector.h>
2018-11-28 11:37:12 +00:00
# include <Dictionaries/registerDictionaries.h>
2019-11-27 09:39:44 +00:00
# include <Disks/registerDisks.h>
2018-02-28 20:34:25 +00:00
# include <Common/Config/ConfigReloader.h>
2020-05-27 17:52:52 +00:00
# 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>
2020-05-27 17:52:52 +00:00
# include <Server/TCPHandlerFactory.h>
2019-09-06 17:48:27 +00:00
# include <Common/SensitiveDataMasker.h>
2020-03-01 21:58:50 +00:00
# include <Common/ThreadFuzzer.h>
2021-01-07 02:56:57 +00:00
# include <Common/getHashOfLoadedBinary.h>
# include <Common/Elf.h>
2020-05-27 17:52:52 +00:00
# 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>
2021-02-19 12:51:26 +00:00
# include <Server/HTTP/HTTPServer.h>
2021-05-16 22:06:09 +00:00
# include <filesystem>
2019-07-30 14:04:18 +00:00
2020-04-16 12:31:57 +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
2020-04-16 12:31:57 +00:00
# endif
2019-09-15 10:35:12 +00:00
# if defined(OS_LINUX)
2020-04-16 12:31:57 +00:00
# include <sys / mman.h>
2021-02-26 17:15:11 +00:00
# include <sys / ptrace.h>
2020-04-16 12:31:57 +00:00
# include <Common / hasLinuxCapability.h>
2021-04-25 03:06:38 +00:00
# include <unistd.h>
# include <sys / syscall.h>
2018-11-14 01:11:13 +00:00
# endif
2020-05-08 14:11:19 +00:00
# if USE_SSL
2021-07-30 09:12:33 +00:00
# if USE_INTERNAL_SSL_LIBRARY
# include <Compression / CompressionCodecEncrypted.h>
# endif
2020-04-16 12:31:57 +00:00
# include <Poco / Net / Context.h>
# include <Poco / Net / SecureServerSocket.h>
2017-03-28 20:30:57 +00:00
# endif
2017-03-21 19:08:09 +00:00
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
2021-07-30 09:12:33 +00:00
# if USE_BASE64
# include <turbob64.h>
# endif
2021-04-18 06:43:13 +00:00
# if USE_JEMALLOC
# include <jemalloc / jemalloc.h>
# endif
2017-08-01 14:34:06 +00:00
namespace CurrentMetrics
{
extern const Metric Revision ;
2018-11-22 21:27:52 +00:00
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 ;
2017-08-01 14:34:06 +00:00
}
2021-05-16 22:06:09 +00:00
namespace fs = std : : filesystem ;
2021-04-18 06:43:13 +00:00
# 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 ;
2020-12-23 14:31:10 +00:00
2021-04-18 06:43:13 +00:00
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) " ) ;
}
2020-12-23 14:31:10 +00:00
/// Do not fork separate process from watchdog if we attached to terminal.
/// Otherwise it breaks gdb usage.
2021-01-23 20:58:36 +00:00
/// 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-12-23 14:31:10 +00:00
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 ;
}
}
2020-01-19 14:26:28 +00:00
namespace
{
2020-05-30 21:57:37 +00:00
void setupTmpPath ( Poco : : Logger * log , const std : : string & path )
2020-01-19 14:26:28 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Setting up {} to store temporary data in it " , path ) ;
2020-01-19 14:26:28 +00:00
2021-05-16 22:06:09 +00:00
fs : : create_directories ( path ) ;
2020-01-19 14:26:28 +00:00
/// 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 )
2020-01-19 14:26:28 +00:00
{
2021-05-27 07:35:36 +00:00
if ( it - > is_regular_file ( ) & & startsWith ( it - > path ( ) . filename ( ) , " tmp " ) )
2020-01-19 14:26:28 +00:00
{
2021-05-27 07:35:36 +00:00
LOG_DEBUG ( log , " Removing old temporary file {} " , it - > path ( ) . string ( ) ) ;
fs : : remove ( it - > path ( ) ) ;
2020-01-19 14:26:28 +00:00
}
else
2021-05-27 07:35:36 +00:00
LOG_DEBUG ( log , " Skipped file in temporary path {} " , it - > path ( ) . string ( ) ) ;
2020-01-19 14:26:28 +00:00
}
}
2020-12-16 10:04:46 +00:00
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 ;
2021-04-13 05:20:59 +00:00
for ( ; ; )
2020-12-16 10:04:46 +00:00
{
current_connections = 0 ;
2021-04-13 05:20:59 +00:00
2020-12-16 10:04:46 +00:00
for ( auto & server : servers )
{
server . stop ( ) ;
current_connections + = server . currentConnections ( ) ;
}
2021-04-13 05:20:59 +00:00
2020-12-16 10:04:46 +00:00
if ( ! current_connections )
break ;
2021-04-13 05:20:59 +00:00
2020-12-16 10:04:46 +00:00
sleep_current_ms + = sleep_one_ms ;
2021-04-13 05:20:59 +00:00
if ( sleep_current_ms < sleep_max_ms )
std : : this_thread : : sleep_for ( std : : chrono : : milliseconds ( sleep_one_ms ) ) ;
else
break ;
2020-12-16 10:04:46 +00:00
}
return current_connections ;
}
2020-01-19 14:26:28 +00:00
}
2012-03-09 03:06:09 +00:00
namespace DB
{
2017-08-01 14:34:06 +00:00
2017-08-10 23:25:51 +00:00
namespace ErrorCodes
{
extern const int NO_ELEMENTS_IN_CONFIG ;
extern const int SUPPORT_IS_DISABLED ;
2017-12-20 20:25:22 +00:00
extern const int ARGUMENT_OUT_OF_BOUND ;
2018-07-30 18:32:21 +00:00
extern const int EXCESSIVE_ELEMENT_IN_CONFIG ;
2021-07-30 09:12:33 +00:00
extern const int INCORRECT_DATA ;
2018-11-22 21:19:58 +00:00
extern const int INVALID_CONFIG_PARAMETER ;
2018-11-21 20:56:37 +00:00
extern const int SYSTEM_ERROR ;
2018-12-09 22:15:59 +00:00
extern const int FAILED_TO_GETPWUID ;
extern const int MISMATCHING_USERS_FOR_PROCESS_AND_DATA ;
2019-05-14 19:55:10 +00:00
extern const int NETWORK_ERROR ;
2021-01-07 02:56:57 +00:00
extern const int CORRUPTED_DATA ;
2017-08-10 23:25:51 +00:00
}
2016-12-13 18:51:19 +00:00
static std : : string getCanonicalPath ( std : : string & & path )
2012-03-09 03:06:09 +00:00
{
2017-04-01 07:20:54 +00:00
Poco : : trimInPlace ( path ) ;
if ( path . empty ( ) )
2018-11-22 21:19:58 +00:00
throw Exception ( " path configuration parameter is empty " , ErrorCodes : : INVALID_CONFIG_PARAMETER ) ;
2017-04-01 07:20:54 +00:00
if ( path . back ( ) ! = ' / ' )
path + = ' / ' ;
2018-05-06 06:29:57 +00:00
return std : : move ( path ) ;
2016-12-13 18:51:19 +00:00
}
2018-12-10 17:42:33 +00:00
static std : : string getUserName ( uid_t user_id )
{
2018-12-09 22:15:59 +00:00
/// Try to convert user id into user name.
auto buffer_size = sysconf ( _SC_GETPW_R_SIZE_MAX ) ;
if ( buffer_size < = 0 )
2018-12-10 23:36:20 +00:00
buffer_size = 1024 ;
2018-12-09 22:15:59 +00:00
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
2021-06-21 19:36:32 +00:00
/// 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 } ;
}
}
}
2018-02-08 19:12:37 +00:00
void Server : : uninitialize ( )
{
logger ( ) . information ( " shutting down " ) ;
BaseDaemon : : uninitialize ( ) ;
}
2019-02-02 13:17:55 +00:00
int Server : : run ( )
{
if ( config ( ) . hasOption ( " help " ) )
{
2020-03-23 02:12:31 +00:00
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 ) ;
2020-03-23 02:12:31 +00:00
help_formatter . format ( std : : cout ) ;
2019-02-02 13:17:55 +00:00
return 0 ;
}
2019-02-04 09:21:12 +00:00
if ( config ( ) . hasOption ( " version " ) )
{
2019-04-03 14:06:59 +00:00
std : : cout < < DBMS_NAME < < " server version " < < VERSION_STRING < < VERSION_OFFICIAL < < " . " < < std : : endl ;
2019-02-04 09:21:12 +00:00
return 0 ;
}
2020-03-18 02:02:24 +00:00
return Application : : run ( ) ; // NOLINT
2019-02-02 13:17:55 +00:00
}
2018-02-08 19:12:37 +00:00
void Server : : initialize ( Poco : : Util : : Application & self )
{
BaseDaemon : : initialize ( self ) ;
logger ( ) . information ( " starting up " ) ;
2021-05-10 05:46:54 +00:00
2021-06-22 23:02:57 +00:00
LOG_INFO ( & logger ( ) , " OS name: {}, version: {}, architecture: {} " ,
2021-05-10 05:46:54 +00:00
Poco : : Environment : : osName ( ) ,
Poco : : Environment : : osVersion ( ) ,
Poco : : Environment : : osArchitecture ( ) ) ;
2018-02-08 19:12:37 +00:00
}
2017-01-09 13:42:29 +00:00
std : : string Server : : getDefaultCorePath ( ) const
{
2018-11-27 16:11:46 +00:00
return getCanonicalPath ( config ( ) . getString ( " path " , DBMS_DEFAULT_PATH ) ) + " cores " ;
2017-01-09 13:42:29 +00:00
}
2016-12-13 18:51:19 +00:00
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
2020-06-04 20:33:19 +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 " ) )
2020-06-04 20:34:33 +00:00
{
/// 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.
2020-06-04 20:33:19 +00:00
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 20:34:33 +00:00
}
2020-06-04 20:33:19 +00:00
}
2021-07-30 09:12:33 +00:00
static void loadEncryptionKey ( const std : : string & key_command [[maybe_unused]] , Poco : : Logger * log )
{
# if USE_BASE64 && USE_SSL && USE_INTERNAL_SSL_LIBRARY
auto process = ShellCommand : : execute ( key_command ) ;
std : : string b64_key ;
readStringUntilEOF ( b64_key , process - > out ) ;
process - > wait ( ) ;
// turbob64 doesn't like whitespace characters in input. Strip
// them before decoding.
std : : erase_if ( b64_key , [ ] ( char c )
{
return c = = ' ' | | c = = ' \t ' | | c = = ' \r ' | | c = = ' \n ' ;
} ) ;
std : : vector < char > buf ( b64_key . size ( ) ) ;
const size_t key_size = tb64dec ( reinterpret_cast < const unsigned char * > ( b64_key . data ( ) ) , b64_key . size ( ) ,
reinterpret_cast < unsigned char * > ( buf . data ( ) ) ) ;
if ( ! key_size )
throw Exception ( " Failed to decode encryption key " , ErrorCodes : : INCORRECT_DATA ) ;
else if ( key_size < 16 )
LOG_WARNING ( log , " The encryption key should be at least 16 octets long. " ) ;
const std : : string_view key = std : : string_view ( buf . data ( ) , key_size ) ;
CompressionCodecEncrypted : : setMasterKey ( key ) ;
# else
LOG_WARNING ( log , " Server was built without Base64 or SSL support. Encryption is disabled. " ) ;
# endif
}
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*/ )
2016-12-13 18:51:19 +00:00
{
2020-05-30 21:57:37 +00:00
Poco : : Logger * log = & logger ( ) ;
2020-08-16 11:52:55 +00:00
2018-09-06 18:05:33 +00:00
UseSSL use_ssl ;
Use total_memory_tracker when there is no other MemoryTracker object.
This should significantly reduce the MemoryTracking drift, test shows
that there is 0 drift after query storm (100 queries, via http/tcp/tcp
in one session).
TL;DR;
To track memory, clickhouse creates memory tracker object for each
thread **explicitly**, but until it is not created the memory
allocations are not under account.
There should not be lot of allocations w/o memory tracker, since most of
the time it is created early enough, but even this maybe enough to
trigger some problems.
Plus sometimes it is not possible to create it, for example some 3d
party library does not allow to do this explicitly:
- for example before #15740 allocations from librdkafka threads,
- or even worse, poco threads, they don't have any routines to do this.
This won't be a problem for `MemoryTracking` metric if the deallocation
will be done from the same thread w/o memory tracker (or vise versa),
but this is not always true.
NOTE, that this will slow down per-thread allocations w/o memory
tracker, since before this patch there were no memory tracking for them
while now they will be accounted in total_memory_tracker, and for
total_memory_tracker max_untracked_memory is always reached.
But this should not be significant.
2020-10-18 07:32:49 +00:00
MainThreadStatus : : getInstance ( ) ;
2019-01-15 18:39:54 +00:00
2017-04-21 17:47:27 +00:00
registerFunctions ( ) ;
2017-05-05 20:39:25 +00:00
registerAggregateFunctions ( ) ;
2017-06-10 09:04:31 +00:00
registerTableFunctions ( ) ;
2017-12-30 00:36:06 +00:00
registerStorages ( ) ;
2018-11-28 11:37:12 +00:00
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 ( ) ) ;
2018-11-22 21:27:52 +00:00
CurrentMetrics : : set ( CurrentMetrics : : VersionInteger , ClickHouseRevision : : getVersionInteger ( ) ) ;
2017-08-01 14:34:06 +00:00
2017-04-01 07:20:54 +00:00
/** Context contains all that query execution is dependent:
2020-07-08 10:40:02 +00:00
* settings , available functions , data types , aggregate functions , databases , . . .
2017-04-01 07:20:54 +00:00
*/
2020-04-16 14:37:38 +00:00
auto shared_context = Context : : createShared ( ) ;
2021-04-10 23:33:54 +00:00
global_context = Context : : createGlobal ( shared_context . get ( ) ) ;
2020-04-17 09:47:40 +00:00
2019-07-08 02:14:32 +00:00
global_context - > makeGlobalContext ( ) ;
2017-04-01 07:20:54 +00:00
global_context - > setApplicationType ( Context : : ApplicationType : : SERVER ) ;
2021-07-09 14:40:32 +00:00
# if !defined(NDEBUG) || !defined(__OPTIMIZE__)
global_context - > addWarningMessage ( " Server was built in debug mode. It will work slowly. " ) ;
# endif
2021-07-12 10:57:39 +00:00
if ( ThreadFuzzer : : instance ( ) . isEffective ( ) )
global_context - > addWarningMessage ( " ThreadFuzzer is enabled. Application will run slowly and unstable. " ) ;
# if defined(SANITIZER)
global_context - > addWarningMessage ( " Server was built with sanitizer. It will work slowly. " ) ;
# endif
2020-09-18 10:47:09 +00:00
// 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 ) ) ;
2021-07-14 13:17:30 +00:00
ConnectionCollector : : init ( global_context , config ( ) . getUInt ( " max_threads_for_connection_collector " , 10 ) ) ;
2018-04-03 19:43:33 +00:00
bool has_zookeeper = config ( ) . has ( " zookeeper " ) ;
2017-04-01 07:20:54 +00:00
zkutil : : ZooKeeperNodeCache main_config_zk_node_cache ( [ & ] { return global_context - > getZooKeeper ( ) ; } ) ;
2018-10-17 17:23:10 +00:00
zkutil : : EventPtr main_config_zk_changed_event = std : : make_shared < Poco : : Event > ( ) ;
2017-04-01 07:20:54 +00:00
if ( loaded_config . has_zk_includes )
{
auto old_configuration = loaded_config . configuration ;
2017-11-21 16:54:25 +00:00
ConfigProcessor config_processor ( config_path ) ;
loaded_config = config_processor . loadConfigWithZooKeeperIncludes (
2018-10-17 17:23:10 +00:00
main_config_zk_node_cache , main_config_zk_changed_event , /* fallback_to_preprocessed = */ true ) ;
2018-11-27 16:11:46 +00:00
config_processor . savePreprocessedConfig ( loaded_config , config ( ) . getString ( " path " , DBMS_DEFAULT_PATH ) ) ;
2017-04-01 07:20:54 +00:00
config ( ) . removeConfiguration ( old_configuration . get ( ) ) ;
config ( ) . add ( loaded_config . configuration . duplicate ( ) , PRIO_DEFAULT , false ) ;
}
2020-08-05 19:54:06 +00:00
Settings : : checkNoSettingNamesAtTopLevel ( config ( ) , config_path ) ;
2020-06-04 19:30:30 +00:00
2018-11-14 01:11:13 +00:00
const auto memory_amount = getMemoryAmount ( ) ;
2018-11-14 01:20:46 +00:00
2020-04-16 12:31:57 +00:00
# if defined(OS_LINUX)
2019-08-11 20:37:53 +00:00
std : : string executable_path = getExecutablePath ( ) ;
2021-01-07 02:56:57 +00:00
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
{
2021-02-26 17:15:11 +00:00
/// 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.
2021-07-12 10:57:39 +00:00
global_context - > addWarningMessage (
fmt : : format ( " Server is run under debugger and its binary image is modified (most likely with breakpoints). " ,
calculated_binary_hash )
) ;
2021-02-26 17:15:11 +00:00
}
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 ) ;
}
2021-01-07 02:56:57 +00:00
}
}
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 ) )
2018-11-14 01:11:13 +00:00
{
2020-09-10 09:05:57 +00:00
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-10 09:05:57 +00:00
2020-09-14 18:08:09 +00:00
if ( config ( ) . getBool ( " mlock_executable " , false ) )
{
2018-11-14 01:11:13 +00:00
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 ) ) ;
}
2018-11-14 01:11:13 +00:00
}
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:11:13 +00:00
}
}
2018-11-14 01:20:46 +00:00
# endif
2018-11-14 01:11:13 +00:00
2019-10-10 12:58:06 +00:00
global_context - > setRemoteHostFilter ( config ( ) ) ;
2018-11-27 16:11:46 +00:00
std : : string path = getCanonicalPath ( config ( ) . getString ( " path " , DBMS_DEFAULT_PATH ) ) ;
2017-04-01 07:20:54 +00:00
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.
2018-12-09 22:15:59 +00:00
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 )
2018-12-10 23:32:21 +00:00
{
2018-12-09 22:15:59 +00:00
message + = " Run under 'sudo -u " + data_owner + " '. " ;
2018-12-10 23:32:21 +00:00
throw Exception ( message , ErrorCodes : : MISMATCHING_USERS_FOR_PROCESS_AND_DATA ) ;
}
else
{
2021-07-12 10:57:39 +00:00
global_context - > addWarningMessage ( message ) ;
2018-12-10 23:32:21 +00:00
}
2018-12-09 22:15:59 +00:00
}
2017-04-01 07:20:54 +00:00
global_context - > setPath ( path ) ;
2020-07-04 13:54:24 +00:00
StatusFile status { path + " status " , StatusFile : : write_full_info } ;
2017-04-01 07:20:54 +00:00
/// 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 ) ;
2017-04-01 07:20:54 +00:00
}
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 ) ) ;
2017-04-01 07:20:54 +00:00
else
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Set max number of file descriptors to {} (was {}). " , rlim . rlim_cur , old ) ;
2017-04-01 07:20:54 +00:00
}
}
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. " ) ;
2017-04-01 07:20:54 +00:00
DateLUT : : instance ( ) ;
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Initialized DateLUT with time zone '{}'. " , DateLUT : : instance ( ) . getTimeZone ( ) ) ;
2017-04-01 07:20:54 +00:00
2020-01-19 14:26:28 +00:00
/// Storage with temporary data for processing of heavy queries.
2017-04-01 07:20:54 +00:00
{
std : : string tmp_path = config ( ) . getString ( " tmp_path " , path + " tmp/ " ) ;
2020-01-19 14:26:28 +00:00
std : : string tmp_policy = config ( ) . getString ( " tmp_policy " , " " ) ;
const VolumePtr & volume = global_context - > setTemporaryStorage ( tmp_path , tmp_policy ) ;
2020-05-09 21:24:15 +00:00
for ( const DiskPtr & disk : volume - > getDisks ( ) )
2020-01-19 14:26:28 +00:00
setupTmpPath ( log , disk - > getPath ( ) ) ;
2017-04-01 07:20:54 +00:00
}
/** 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 .
*/
2018-04-19 05:32:09 +00:00
{
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 ) ;
2018-04-19 05:32:09 +00:00
}
/** 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/ " ) ;
2018-04-19 05:32:09 +00:00
global_context - > setUserFilesPath ( user_files_path ) ;
2021-05-16 22:06:09 +00:00
fs : : create_directories ( user_files_path ) ;
2018-04-19 05:32:09 +00:00
}
2017-04-01 07:20:54 +00:00
2019-12-10 17:27:29 +00:00
{
2021-05-16 22:06:09 +00:00
std : : string dictionaries_lib_path = config ( ) . getString ( " dictionaries_lib_path " , fs : : path ( path ) / " dictionaries_lib/ " ) ;
2019-12-10 17:27:29 +00:00
global_context - > setDictionariesLibPath ( dictionaries_lib_path ) ;
2021-05-16 22:06:09 +00:00
fs : : create_directories ( dictionaries_lib_path ) ;
2019-12-10 17:27:29 +00:00
}
2020-12-03 21:11:38 +00:00
/// top_level_domains_lists
2020-12-08 20:54:03 +00:00
{
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-12-08 20:54:03 +00:00
}
2020-12-03 21:11:38 +00:00
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
}
2018-07-30 18:32:21 +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 =
2017-04-01 07:20:54 +00:00
{
2018-07-30 18:32:21 +00:00
std : : make_tuple ( " interserver_http_host " , " interserver_http_port " , " http " ) ,
std : : make_tuple ( " interserver_https_host " , " interserver_https_port " , " https " )
} ;
2017-04-01 07:20:54 +00:00
2018-07-30 18:32:21 +00:00
for ( auto [ host_tag , port_tag , scheme ] : interserver_tags )
{
if ( config ( ) . has ( port_tag ) )
2017-04-01 07:20:54 +00:00
{
2018-07-30 18:32:21 +00:00
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 ) ;
2018-07-30 18:32:21 +00:00
}
2017-04-01 07:20:54 +00:00
2018-07-30 18:32:21 +00:00
String port_str = config ( ) . getString ( port_tag ) ;
int port = parse < int > ( port_str ) ;
2017-04-01 07:20:54 +00:00
2018-07-30 18:32:21 +00:00
if ( port < 0 | | port > 0xFFFF )
throw Exception ( " Out of range ' " + String ( port_tag ) + " ': " + toString ( port ) , ErrorCodes : : ARGUMENT_OUT_OF_BOUND ) ;
2017-04-01 07:20:54 +00:00
2018-07-30 18:32:21 +00:00
global_context - > setInterserverIOAddress ( this_host , port ) ;
global_context - > setInterserverScheme ( scheme ) ;
2017-04-01 07:20:54 +00:00
}
}
2021-04-07 13:52:11 +00:00
LOG_DEBUG ( log , " Initiailizing interserver credentials. " ) ;
2020-08-26 08:36:58 +00:00
global_context - > updateInterserverCredentials ( config ( ) ) ;
2017-04-01 07:20:54 +00:00
if ( config ( ) . has ( " macros " ) )
2020-09-26 19:18:28 +00:00
global_context - > setMacros ( std : : make_unique < Macros > ( config ( ) , " macros " , log ) ) ;
2017-04-01 07:20:54 +00:00
/// 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 " ) )
{
2019-09-06 17:48:27 +00:00
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 ,
2017-04-01 07:20:54 +00:00
include_from_path ,
2018-11-27 16:11:46 +00:00
config ( ) . getString ( " path " , " " ) ,
2017-04-01 07:20:54 +00:00
std : : move ( main_config_zk_node_cache ) ,
2018-10-17 17:23:10 +00:00
main_config_zk_changed_event ,
2021-02-19 16:22:47 +00:00
[ & ] ( ConfigurationPtr config , bool initial_loading )
2018-02-28 20:34:25 +00:00
{
2020-08-05 19:54:06 +00:00
Settings : : checkNoSettingNamesAtTopLevel ( * config , config_path ) ;
2020-06-04 19:30:30 +00:00
2021-01-16 15:33:34 +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 ) ;
2020-01-17 19:19:09 +00:00
// 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());
2018-02-28 20:34:25 +00:00
global_context - > setClustersConfig ( config ) ;
2020-09-26 19:18:28 +00:00
global_context - > setMacros ( std : : make_unique < Macros > ( * config , " macros " , log ) ) ;
2020-07-10 11:59:48 +00:00
global_context - > setExternalAuthenticatorsConfig ( * config ) ;
2021-04-01 10:19:28 +00:00
global_context - > setExternalModelsConfig ( config ) ;
2019-11-14 13:44:15 +00:00
/// 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 " ) ) ;
2020-01-09 14:50:34 +00:00
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 ) ;
2020-09-10 04:00:33 +00:00
2021-02-18 14:13:23 +00:00
global_context - > reloadAuxiliaryZooKeepersConfigIfChanged ( config ) ;
}
2020-11-03 07:52:31 +00:00
2020-01-09 14:50:34 +00:00
global_context - > updateStorageConfiguration ( * config ) ;
2020-08-26 08:36:58 +00:00
global_context - > updateInterserverCredentials ( * config ) ;
2018-02-28 20:34:25 +00:00
} ,
2021-02-18 14:13:23 +00:00
/* already_loaded = */ false ) ; /// Reload it right now (initial loading)
2017-04-01 07:20:54 +00:00
2020-08-05 19:54:06 +00:00
auto & access_control = global_context - > getAccessControlManager ( ) ;
2020-08-03 21:16:28 +00:00
if ( config ( ) . has ( " custom_settings_prefixes " ) )
2020-08-05 19:54:06 +00:00
access_control . setCustomSettingsPrefixes ( config ( ) . getString ( " custom_settings_prefixes " ) ) ;
2020-08-03 21:16:28 +00:00
2020-08-12 11:43:58 +00:00
/// Initialize access storages.
access_control . addStoragesFromMainConfig ( config ( ) , config_path , [ & ] { return global_context - > getZooKeeper ( ) ; } ) ;
2017-04-01 07:20:54 +00:00
2018-03-13 10:41:47 +00:00
/// Reload config in SYSTEM RELOAD CONFIG query.
2018-05-07 02:01:11 +00:00
global_context - > setConfigReloadCallback ( [ & ] ( )
{
2018-03-13 10:41:47 +00:00
main_config_reloader - > reload ( ) ;
2020-08-05 19:54:06 +00:00
access_control . reloadUsersConfigs ( ) ;
2018-03-13 10:41:47 +00:00
} ) ;
2017-08-09 15:34:09 +00:00
/// Limit on total number of concurrently executed queries.
2017-04-01 07:20:54 +00:00
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 ;
2017-04-01 07:20:54 +00:00
/// Size of cache for uncompressed blocks. Zero means disabled.
2017-04-12 16:37:19 +00:00
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-02 13:55:34 +00:00
{
2019-02-09 22:38:26 +00:00
uncompressed_cache_size = max_cache_size ;
2020-06-10 19:34:23 +00:00
LOG_INFO ( log , " Uncompressed cache size was lowered to {} because the system has low amount of memory " ,
formatReadableSizeWithBinarySuffix ( uncompressed_cache_size ) ) ;
2019-02-02 13:55:34 +00:00
}
2019-02-09 22:38:26 +00:00
global_context - > setUncompressedCache ( uncompressed_cache_size ) ;
2017-04-01 07:20:54 +00:00
2018-02-01 13:52:29 +00:00
/// Load global settings from default_profile and system_profile.
global_context - > setDefaultProfiles ( config ( ) ) ;
2020-03-13 14:50:26 +00:00
const Settings & settings = global_context - > getSettingsRef ( ) ;
2017-04-01 07:20:54 +00:00
2019-02-09 22:38:26 +00:00
/// Size of cache for marks (index of MergeTree family of tables). It is mandatory.
2017-12-13 20:32:26 +00:00
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-02 13:55:34 +00:00
{
2019-02-09 22:38:26 +00:00
mark_cache_size = max_cache_size ;
2020-06-10 19:34:23 +00:00
LOG_INFO ( log , " Mark cache size was lowered to {} because the system has low amount of memory " ,
formatReadableSizeWithBinarySuffix ( mark_cache_size ) ) ;
2019-02-02 13:55:34 +00:00
}
2019-02-09 22:38:26 +00:00
global_context - > setMarkCache ( mark_cache_size ) ;
2017-12-13 20:32:26 +00:00
2021-03-28 01:10:30 +00:00
/// 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 ) ;
2021-03-28 01:10:30 +00:00
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
2018-09-05 11:37:41 +00:00
2017-11-10 06:48:28 +00:00
/// 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 ) ;
2017-11-10 06:48:28 +00:00
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
2021-07-30 09:12:33 +00:00
/// Set up encryption.
if ( config ( ) . has ( " encryption.key_command " ) )
loadEncryptionKey ( config ( ) . getString ( " encryption.key_command " ) , log ) ;
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 > (
2021-04-12 12:25:52 +00:00
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
} ) ;
2021-04-12 12:25:52 +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 ( ) )
2020-12-16 10:04:46 +00:00
{
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 )
2020-12-16 10:04:46 +00:00
{
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 ) ) ;
2020-12-16 10:04:46 +00:00
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-12-16 10:04:46 +00:00
}
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
{
2021-04-10 23:33:54 +00:00
loadMetadataSystem ( global_context ) ;
2018-12-10 23:21:03 +00:00
/// After attaching system databases we can initialize system log.
global_context - > initializeSystemLogs ( ) ;
2021-07-29 14:48:49 +00:00
global_context - > setSystemZooKeeperLogAfterInitializationIfNeeded ( ) ;
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 ) ;
2021-02-10 14:12:49 +00:00
/// We load temporary database first, because projections need it.
2021-06-03 14:05:37 +00:00
database_catalog . initializeAndLoadTemporaryDatabase ( ) ;
2018-12-10 23:21:03 +00:00
/// Then, load remaining databases
2021-04-10 23:33:54 +00:00
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. " ) ;
2017-04-01 07:20:54 +00:00
2019-02-03 21:30:45 +00:00
/// Init trace collector only after trace_log system table was created
2019-08-03 04:52:14 +00:00
/// 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
2020-02-16 06:49:24 +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. " ) ;
2020-02-16 06:49:24 +00:00
# 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. " ) ;
2020-02-16 06:49:24 +00:00
# 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. " ) ;
2020-02-16 06:49:24 +00:00
# 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
2020-02-16 06:49:24 +00:00
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). " ) ;
2020-02-16 06:49:24 +00:00
2018-03-29 20:21:01 +00:00
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 ( ) ;
2020-09-02 08:07:46 +00:00
LOG_DEBUG ( log , " DNS caching disabled " ) ;
2018-03-29 20:21:01 +00:00
}
else
{
2019-06-27 16:28:26 +00:00
/// Initialize a watcher periodically updating DNS cache
2021-04-10 23:33:54 +00:00
dns_cache_updater = std : : make_unique < DNSCacheUpdater > ( global_context , config ( ) . getInt ( " dns_cache_update_period " , 15 ) ) ;
2018-03-29 20:21:01 +00:00
}
2018-03-26 14:12:07 +00:00
2020-04-16 12:31:57 +00:00
# if defined(OS_LINUX)
2020-05-01 18:47:41 +00:00
if ( ! TasksStatsCounters : : checkIfAvailable ( ) )
2018-06-14 14:29:42 +00:00
{
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 ) ;
2018-06-14 14:29:42 +00:00
}
2018-08-22 05:56:06 +00:00
# else
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " TaskStats is not implemented for this OS. IO accounting will be disabled. " ) ;
2018-08-22 05:56:06 +00:00
# endif
2018-06-14 14:29:42 +00:00
2020-12-21 23:03:08 +00:00
auto servers = std : : make_shared < std : : vector < ProtocolServerAdapter > > ( ) ;
2017-04-01 07:20:54 +00:00
{
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 ) ;
2020-02-10 13:10:17 +00:00
attachSystemTablesAsync ( * DatabaseCatalog : : instance ( ) . getSystemDatabase ( ) , async_metrics ) ;
2019-11-26 20:27:24 +00:00
2017-04-01 07:20:54 +00:00
for ( const auto & listen_host : listen_hosts )
{
2019-07-30 21:23:42 +00:00
/// HTTP
2020-12-17 13:47:03 +00:00
const char * port_name = " http_port " ;
createServer ( listen_host , port_name , listen_try , [ & ] ( UInt16 port )
2019-07-30 21:23:42 +00:00
{
Poco : : Net : : ServerSocket socket ;
2020-11-11 13:07:06 +00:00
auto address = socketBindListen ( socket , listen_host , port ) ;
2019-07-30 21:23:42 +00:00
socket . setReceiveTimeout ( settings . http_receive_timeout ) ;
socket . setSendTimeout ( settings . http_send_timeout ) ;
2019-11-24 15:43:53 +00:00
2021-02-19 12:51:26 +00:00
servers - > emplace_back (
port_name ,
std : : make_unique < HTTPServer > (
context ( ) , createHandlerFactory ( * this , async_metrics , " HTTPHandler-factory " ) , server_pool , socket , http_params ) ) ;
2019-07-30 21:23:42 +00:00
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Listening for http://{} " , address . toString ( ) ) ;
2019-07-30 21:23:42 +00:00
} ) ;
/// HTTPS
2020-12-17 13:47:03 +00:00
port_name = " https_port " ;
createServer ( listen_host , port_name , listen_try , [ & ] ( UInt16 port )
2019-07-30 21:23:42 +00:00
{
2020-05-08 14:11:19 +00:00
# if USE_SSL
2019-07-30 21:23:42 +00:00
Poco : : Net : : SecureServerSocket socket ;
2020-11-11 13:07:06 +00:00
auto address = socketBindListen ( socket , listen_host , port , /* secure = */ true ) ;
2019-07-30 21:23:42 +00:00
socket . setReceiveTimeout ( settings . http_receive_timeout ) ;
socket . setSendTimeout ( settings . http_send_timeout ) ;
2021-02-19 12:51:26 +00:00
servers - > emplace_back (
port_name ,
std : : make_unique < HTTPServer > (
context ( ) , createHandlerFactory ( * this , async_metrics , " HTTPSHandler-factory " ) , server_pool , socket , http_params ) ) ;
2019-07-30 21:23:42 +00:00
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 ) ;
2019-07-30 21:23:42 +00:00
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
2019-07-30 21:23:42 +00:00
} ) ;
2018-07-30 18:32:21 +00:00
2019-07-30 21:23:42 +00:00
/// TCP
2020-12-17 13:47:03 +00:00
port_name = " tcp_port " ;
createServer ( listen_host , port_name , listen_try , [ & ] ( UInt16 port )
2019-07-30 21:23:42 +00:00
{
Poco : : Net : : ServerSocket socket ;
2020-11-11 13:07:06 +00:00
auto address = socketBindListen ( socket , listen_host , port ) ;
2019-07-30 21:23:42 +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 */ false ) ,
2019-07-30 21:23:42 +00:00
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 ( ) ) ;
2019-07-30 21:23:42 +00:00
} ) ;
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 ( ) ) ;
} ) ;
2019-07-30 21:23:42 +00:00
/// 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 )
2019-07-30 21:23:42 +00:00
{
2020-05-08 14:11:19 +00:00
# if USE_SSL
2019-07-30 21:23:42 +00:00
Poco : : Net : : SecureServerSocket socket ;
2020-11-11 13:07:06 +00:00
auto address = socketBindListen ( socket , listen_host , port , /* secure = */ true ) ;
2019-07-30 21:23:42 +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 */ true , /* proxy protocol */ false ) ,
2019-07-30 21:23:42 +00:00
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 ( ) ) ;
2018-07-30 18:32:21 +00:00
# else
2019-08-06 14:03:41 +00:00
UNUSED ( port ) ;
2019-07-30 21:23:42 +00:00
throw Exception { " SSL support for TCP protocol is disabled because Poco library was built without NetSSL support. " ,
ErrorCodes : : SUPPORT_IS_DISABLED } ;
2018-07-30 18:32:21 +00:00
# endif
2019-07-30 21:23:42 +00:00
} ) ;
2019-03-16 02:08:21 +00:00
2019-07-30 21:23:42 +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 )
2019-07-30 21:23:42 +00:00
{
Poco : : Net : : ServerSocket socket ;
2020-11-11 13:07:06 +00:00
auto address = socketBindListen ( socket , listen_host , port ) ;
2019-07-30 21:23:42 +00:00
socket . setReceiveTimeout ( settings . http_receive_timeout ) ;
socket . setSendTimeout ( settings . http_send_timeout ) ;
2021-02-19 12:51:26 +00:00
servers - > emplace_back (
port_name ,
std : : make_unique < HTTPServer > (
context ( ) ,
createHandlerFactory ( * this , async_metrics , " InterserverIOHTTPHandler-factory " ) ,
server_pool ,
socket ,
http_params ) ) ;
2019-07-30 21:23:42 +00:00
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Listening for replica communication (interserver): http://{} " , address . toString ( ) ) ;
2019-07-30 21:23:42 +00:00
} ) ;
2020-12-17 13:47:03 +00:00
port_name = " interserver_https_port " ;
createServer ( listen_host , port_name , listen_try , [ & ] ( UInt16 port )
2019-07-30 21:23:42 +00:00
{
2020-05-08 14:11:19 +00:00
# if USE_SSL
2019-07-30 21:23:42 +00:00
Poco : : Net : : SecureServerSocket socket ;
2020-11-11 13:07:06 +00:00
auto address = socketBindListen ( socket , listen_host , port , /* secure = */ true ) ;
2019-07-30 21:23:42 +00:00
socket . setReceiveTimeout ( settings . http_receive_timeout ) ;
socket . setSendTimeout ( settings . http_send_timeout ) ;
2021-02-19 12:51:26 +00:00
servers - > emplace_back (
port_name ,
std : : make_unique < HTTPServer > (
context ( ) ,
createHandlerFactory ( * this , async_metrics , " InterserverIOHTTPSHandler-factory " ) ,
server_pool ,
socket ,
http_params ) ) ;
2019-07-30 21:23:42 +00:00
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Listening for secure replica communication (interserver): https://{} " , address . toString ( ) ) ;
2019-07-09 16:41:11 +00:00
# else
2019-08-06 14:03:41 +00:00
UNUSED ( port ) ;
2019-07-30 21:23:42 +00:00
throw Exception { " SSL support for TCP protocol is disabled because Poco library was built without NetSSL support. " ,
ErrorCodes : : SUPPORT_IS_DISABLED } ;
2019-07-09 16:41:11 +00:00
# endif
2019-07-30 21:23:42 +00:00
} ) ;
2020-12-17 13:47:03 +00:00
port_name = " mysql_port " ;
createServer ( listen_host , port_name , listen_try , [ & ] ( UInt16 port )
2017-04-01 07:20:54 +00:00
{
2019-07-30 21:23:42 +00:00
Poco : : Net : : ServerSocket socket ;
2020-11-11 13:07:06 +00:00
auto address = socketBindListen ( socket , listen_host , port , /* secure = */ true ) ;
2019-07-30 21:23:42 +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 > (
2019-07-30 21:23:42 +00:00
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-07-30 21:23:42 +00:00
} ) ;
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 ) ;
2021-02-19 12:51:26 +00:00
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
} ) ;
2017-04-01 07:20:54 +00:00
}
2020-12-21 21:47:10 +00:00
if ( servers - > empty ( ) )
2020-06-15 20:30:36 +00:00
throw Exception ( " No servers started (add valid listen_host and 'tcp_port' or 'http_port' to configuration file.) " ,
ErrorCodes : : NO_ELEMENTS_IN_CONFIG ) ;
2017-04-27 21:51:09 +00:00
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 ( ) ;
2020-03-05 19:23:39 +00:00
global_context - > enableNamedSessions ( ) ;
2020-01-23 20:19:51 +00:00
{
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 ) ;
}
2020-11-25 07:31:16 +00:00
2020-01-17 19:19:09 +00:00
buildLoggers ( config ( ) , logger ( ) ) ;
2018-03-03 14:39:16 +00:00
main_config_reloader - > start ( ) ;
2020-08-05 19:54:06 +00:00
access_control . startPeriodicReloadingUsersConfigs ( ) ;
2019-06-27 16:28:26 +00:00
if ( dns_cache_updater )
dns_cache_updater - > start ( ) ;
2018-03-03 14:39:16 +00:00
2017-06-22 18:08:14 +00:00
{
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 ( ) ) ;
2017-06-22 18:08:14 +00:00
}
2021-02-23 20:20:31 +00:00
/// try to load dictionaries immediately, throw on error and die
try
{
2021-05-12 18:16:01 +00:00
global_context - > loadDictionaries ( config ( ) ) ;
2021-02-23 20:20:31 +00:00
}
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 ) ;
2021-04-10 23:33:54 +00:00
global_context - > setDDLWorker ( std : : make_unique < DDLWorker > ( pool_size , ddl_zookeeper_path , global_context , & config ( ) ,
2021-02-23 20:20:31 +00:00
" distributed_ddl " , " DDLWorker " , & CurrentMetrics : : MaxDDLEntryID ) ) ;
}
2021-03-12 18:44:59 +00:00
for ( auto & server : * servers )
server . start ( ) ;
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Ready for connections. " ) ;
2017-04-01 07:20:54 +00:00
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. " ) ;
2017-04-01 07:20:54 +00:00
is_cancelled = true ;
int current_connections = 0 ;
2020-12-21 21:47:10 +00:00
for ( auto & server : * servers )
2017-04-01 07:20:54 +00:00
{
2020-12-09 08:58:41 +00:00
server . stop ( ) ;
current_connections + = server . currentConnections ( ) ;
2017-04-01 07:20:54 +00:00
}
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. " ) ;
2017-04-01 07:20:54 +00:00
2019-07-30 23:12:04 +00:00
/// Killing remaining queries.
global_context - > getProcessList ( ) . killAllQueries ( ) ;
2017-04-01 07:20:54 +00:00
if ( current_connections )
2020-12-21 21:47:10 +00:00
current_connections = waitServersToFinish ( * servers , config ( ) . getInt ( " shutdown_wait_unfinished " , 5 ) ) ;
2017-04-01 07:20:54 +00:00
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. " ) ;
2017-04-01 07:20:54 +00:00
2019-06-27 16:28:26 +00:00
dns_cache_updater . reset ( ) ;
2017-04-01 07:20:54 +00:00
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).
2019-12-18 10:43:40 +00:00
/// 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
}
2017-04-01 07:20:54 +00:00
} ) ;
std : : vector < std : : unique_ptr < MetricsTransmitter > > metrics_transmitters ;
for ( const auto & graphite_key : DB : : getMultipleKeysFromConfig ( config ( ) , " " , " graphite " ) )
{
2017-08-24 14:51:13 +00:00
metrics_transmitters . emplace_back ( std : : make_unique < MetricsTransmitter > (
2019-07-04 19:08:37 +00:00
global_context - > getConfigRef ( ) , graphite_key , async_metrics ) ) ;
2017-04-01 07:20:54 +00:00
}
waitForTerminationRequest ( ) ;
}
return Application : : EXIT_OK ;
2012-03-09 03:06:09 +00:00
}
}