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>
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>
2017-08-09 14:33:07 +00:00
# include <ext/scope_guard.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>
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>
2017-04-01 09:19:00 +00:00
# include <IO/HTTPCommon.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-08-01 13:05:59 +00:00
# include <Interpreters/SystemLog.cpp>
2019-10-18 15:44:32 +00:00
# include <Interpreters/ExternalLoaderXMLConfigRepository.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>
2017-12-30 00:36:06 +00:00
# include <Storages/registerStorages.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>
2020-05-27 17:52:52 +00:00
# include <Server/MySQLHandlerFactory.h>
2020-05-30 20:02:11 +00:00
# include <Server/PostgreSQLHandlerFactory.h>
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>
# include <Common / hasLinuxCapability.h>
2018-11-14 01:11:13 +00:00
# endif
2020-05-08 14:11:19 +00:00
# if USE_SSL
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
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 ;
2017-08-01 14:34:06 +00:00
}
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
Poco : : File ( path ) . createDirectories ( ) ;
/// Clearing old temporary files.
Poco : : DirectoryIterator dir_end ;
for ( Poco : : DirectoryIterator it ( path ) ; it ! = dir_end ; + + it )
{
if ( it - > isFile ( ) & & startsWith ( it . name ( ) , " tmp " ) )
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Removing old temporary file {} " , it - > path ( ) ) ;
2020-01-19 14:26:28 +00:00
it - > remove ( ) ;
}
else
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Skipped file in temporary path {} " , it - > path ( ) ) ;
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 ;
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 ;
2020-06-04 19:30:30 +00:00
extern const int UNKNOWN_ELEMENT_IN_CONFIG ;
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 ) ;
}
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 ( ) ) ;
2019-02-02 15:37:54 +00:00
std : : stringstream header ;
header < < commandName ( ) < < " [OPTION] [-- [ARG]...] \n " ;
header < < " positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010 " ;
2020-03-23 02:12:31 +00:00
help_formatter . setHeader ( header . str ( ) ) ;
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 " ) ;
}
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
/// Check that there is no user-level settings at the top level in config.
/// This is a common source of mistake (user don't know where to write user-level setting).
2020-06-04 20:33:19 +00:00
void checkForUserSettingsAtTopLevel ( const Poco : : Util : : AbstractConfiguration & config , const std : : string & path )
2020-06-04 19:30:30 +00:00
{
if ( config . getBool ( " skip_check_for_incorrect_settings " , false ) )
return ;
Settings settings ;
for ( const auto & setting : settings )
{
std : : string name = setting . getName ( ) . toString ( ) ;
if ( config . has ( name ) )
{
throw Exception ( fmt : : format ( " A setting '{}' appeared at top level in config {}. "
" But it is user-level setting that should be located in users.xml inside <profiles> section for specific profile. "
" You can add it to <profiles><default> if you want to change default value of this setting. "
" You can also disable the check - specify <skip_check_for_incorrect_settings>1</skip_check_for_incorrect_settings> "
" in the main configuration file. " ,
name , path ) ,
ErrorCodes : : UNKNOWN_ELEMENT_IN_CONFIG ) ;
}
}
}
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
}
2020-06-04 19:30:30 +00:00
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 ( ) ;
2018-09-06 18:05:33 +00:00
UseSSL use_ssl ;
2019-01-15 18:39:54 +00:00
ThreadStatus thread_status ;
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 ( ) ;
2017-04-21 17:47:27 +00:00
2020-05-18 10:26:23 +00:00
# if !defined(ARCADIA_BUILD)
2020-05-15 08:31:10 +00:00
# if USE_OPENCL
2020-05-18 21:41:23 +00:00
BitonicSort : : getInstance ( ) . configure ( ) ;
2020-05-18 10:26:23 +00:00
# endif
2020-05-14 23:57:22 +00:00
# endif
2020-05-15 00:01:14 +00:00
2017-08-01 14:34:06 +00:00
CurrentMetrics : : set ( CurrentMetrics : : Revision , ClickHouseRevision : : get ( ) ) ;
2018-11-22 21:27:52 +00:00
CurrentMetrics : : set ( CurrentMetrics : : VersionInteger , ClickHouseRevision : : getVersionInteger ( ) ) ;
2017-08-01 14:34:06 +00:00
2020-03-01 21:58:50 +00:00
if ( ThreadFuzzer : : instance ( ) . isEffective ( ) )
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " ThreadFuzzer is enabled. Application will run slowly and unstable. " ) ;
2020-03-01 21:58:50 +00:00
2020-05-29 21:41:33 +00:00
# if !defined(NDEBUG) || !defined(__OPTIMIZE__)
LOG_WARNING ( log , " Server was built in debug mode. It will work slowly. " ) ;
# endif
# if defined(ADDRESS_SANITIZER) || defined(THREAD_SANITIZER) || defined(MEMORY_SANITIZER)
LOG_WARNING ( log , " Server was built with sanitizer. It will work slowly. " ) ;
# endif
2017-04-01 07:20:54 +00:00
/** Context contains all that query execution is dependent:
* settings , available functions , data types , aggregate functions , databases . . .
*/
2020-04-16 14:37:38 +00:00
auto shared_context = Context : : createShared ( ) ;
2020-04-17 12:58:52 +00:00
auto global_context = std : : make_unique < Context > ( Context : : createGlobal ( shared_context . get ( ) ) ) ;
global_context_ptr = global_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 ) ;
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-06-04 20:33:19 +00:00
checkForUserSettingsAtTopLevel ( 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 ( ) ;
if ( executable_path . empty ( ) )
executable_path = " /usr/bin/clickhouse " ; /// It is used for information messages.
2018-11-14 01:20:46 +00:00
/// After full config loaded
{
if ( config ( ) . getBool ( " mlock_executable " , false ) )
2018-11-14 01:11:13 +00:00
{
if ( hasLinuxCapability ( CAP_IPC_LOCK ) )
{
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " Will mlockall to prevent executable memory from being paged out. It may take a few seconds. " ) ;
2018-11-14 01:11:13 +00:00
if ( 0 ! = mlockall ( MCL_CURRENT ) )
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , " Failed mlockall: {} " , errnoToString ( ErrorCodes : : SYSTEM_ERROR ) ) ;
2018-11-14 01:11:13 +00:00
else
2020-05-23 22:24:01 +00:00
LOG_TRACE ( log , " The memory map of clickhouse executable has been mlock'ed " ) ;
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 ) ;
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 " ) ;
2018-12-09 22:15:59 +00:00
/// Check that the process' user id matches the owner of the data.
const auto effective_user_id = geteuid ( ) ;
struct stat statbuf ;
if ( stat ( path . c_str ( ) , & statbuf ) = = 0 & & effective_user_id ! = statbuf . st_uid )
{
const auto effective_user = getUserName ( effective_user_id ) ;
const auto data_owner = getUserName ( statbuf . st_uid ) ;
std : : string message = " Effective user of the process ( " + effective_user +
" ) does not match the owner of the data ( " + data_owner + " ). " ;
if ( effective_user_id = = 0 )
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
{
2020-05-23 22:24:01 +00:00
LOG_WARNING ( log , 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 ) ;
StatusFile status { path + " status " } ;
2017-08-30 14:47:35 +00:00
SCOPE_EXIT ( {
2019-03-12 19:30:01 +00:00
/** 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 .
*/
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Shutting down storages. " ) ;
2019-06-20 07:17:21 +00:00
2019-03-12 19:30:01 +00:00
global_context - > shutdown ( ) ;
2019-06-20 07:17:21 +00:00
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Shut down storages. " ) ;
2019-03-12 19:30:01 +00:00
2017-08-30 14:47:35 +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 .
*/
2020-04-17 12:58:52 +00:00
global_context_ptr = nullptr ;
2017-08-30 14:47:35 +00:00
global_context . reset ( ) ;
2020-04-17 12:58:52 +00:00
shared_context . reset ( ) ;
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Destroyed global context. " ) ;
2017-08-30 14:47:35 +00:00
} ) ;
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-06-22 19:04:12 +00:00
/// Initialize global thread pool
GlobalThreadPool : : initialize ( config ( ) . getUInt ( " max_thread_pool_size " , 10000 ) ) ;
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
{
Poco : : File ( path + " flags/ " ) . createDirectories ( ) ;
global_context - > setFlagsPath ( path + " flags/ " ) ;
}
/** Directory with user provided files that are usable by 'file' table function.
*/
{
std : : string user_files_path = config ( ) . getString ( " user_files_path " , path + " user_files/ " ) ;
global_context - > setUserFilesPath ( user_files_path ) ;
Poco : : File ( user_files_path ) . createDirectories ( ) ;
}
2017-04-01 07:20:54 +00:00
2019-12-10 17:27:29 +00:00
{
std : : string dictionaries_lib_path = config ( ) . getString ( " dictionaries_lib_path " , path + " dictionaries_lib/ " ) ;
global_context - > setDictionariesLibPath ( dictionaries_lib_path ) ;
Poco : : File ( dictionaries_lib_path ) . createDirectories ( ) ;
}
2020-03-19 21:14:52 +00:00
{
/// Directory with metadata of tables, which was marked as dropped by Atomic database
Poco : : File ( path + " metadata_dropped/ " ) . createDirectories ( ) ;
}
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
}
}
2018-07-26 16:01:43 +00:00
if ( config ( ) . has ( " interserver_http_credentials " ) )
2018-07-26 15:10:57 +00:00
{
String user = config ( ) . getString ( " interserver_http_credentials.user " , " " ) ;
String password = config ( ) . getString ( " interserver_http_credentials.password " , " " ) ;
2017-04-01 07:20:54 +00:00
2018-07-26 15:10:57 +00:00
if ( user . empty ( ) )
2018-07-26 16:10:21 +00:00
throw Exception ( " Configuration parameter interserver_http_credentials user can't be empty " , ErrorCodes : : NO_ELEMENTS_IN_CONFIG ) ;
2017-04-01 07:20:54 +00:00
2018-07-30 18:32:21 +00:00
global_context - > setInterserverCredentials ( user , password ) ;
2017-04-01 07:20:54 +00:00
}
if ( config ( ) . has ( " macros " ) )
2018-03-13 23:44:23 +00:00
global_context - > setMacros ( std : : make_unique < Macros > ( config ( ) , " macros " ) ) ;
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 ,
2018-02-28 20:34:25 +00:00
[ & ] ( ConfigurationPtr config )
{
2020-06-04 20:33:19 +00:00
checkForUserSettingsAtTopLevel ( * config , config_path ) ;
2020-06-04 19:30:30 +00:00
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 ) ;
2018-03-13 23:44:23 +00:00
global_context - > setMacros ( std : : make_unique < Macros > ( * config , " macros " ) ) ;
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
global_context - > updateStorageConfiguration ( * config ) ;
2018-02-28 20:34:25 +00:00
} ,
2017-04-01 07:20:54 +00:00
/* already_loaded = */ true ) ;
/// Initialize users config reloader.
std : : string users_config_path = config ( ) . getString ( " users_config " , config_path ) ;
/// If path to users' config isn't absolute, try guess its root (current) dir.
/// At first, try to find it in dir of main config, after will use current dir.
if ( users_config_path . empty ( ) | | users_config_path [ 0 ] ! = ' / ' )
{
std : : string config_dir = Poco : : Path ( config_path ) . parent ( ) . toString ( ) ;
if ( Poco : : File ( config_dir + users_config_path ) . exists ( ) )
users_config_path = config_dir + users_config_path ;
}
2020-06-04 20:33:19 +00:00
if ( users_config_path ! = config_path )
checkForUsersNotInMainConfig ( config ( ) , config_path , users_config_path , log ) ;
2020-06-04 19:30:30 +00:00
auto users_config_reloader = std : : make_unique < ConfigReloader > (
users_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
zkutil : : ZooKeeperNodeCache ( [ & ] { return global_context - > getZooKeeper ( ) ; } ) ,
2018-10-17 17:23:10 +00:00
std : : make_shared < Poco : : Event > ( ) ,
2020-06-04 19:30:30 +00:00
[ & ] ( ConfigurationPtr config )
{
global_context - > setUsersConfig ( config ) ;
2020-06-04 20:33:19 +00:00
checkForUserSettingsAtTopLevel ( * config , users_config_path ) ;
2020-06-04 19:30:30 +00:00
} ,
2017-04-01 07:20:54 +00:00
/* already_loaded = */ false ) ;
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 ( ) ;
users_config_reloader - > reload ( ) ;
} ) ;
2020-02-26 22:36:52 +00:00
/// Sets a local directory storing information about access control.
std : : string access_control_local_path = config ( ) . getString ( " access_control_path " , " " ) ;
if ( ! access_control_local_path . empty ( ) )
global_context - > getAccessControlManager ( ) . setLocalDirectory ( access_control_local_path ) ;
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
2018-09-06 19:13:49 +00:00
# if USE_EMBEDDED_COMPILER
2019-01-11 19:03:23 +00:00
size_t compiled_expression_cache_size = config ( ) . getUInt64 ( " compiled_expression_cache_size " , 500 ) ;
2018-09-05 11:37:41 +00:00
if ( compiled_expression_cache_size )
2018-09-05 12:42:37 +00:00
global_context - > setCompiledExpressionCache ( compiled_expression_cache_size ) ;
2018-09-06 19:13:49 +00:00
# endif
2018-09-05 11:37:41 +00:00
2017-11-10 06:48:28 +00:00
/// Set path for format schema files
auto format_schema_path = Poco : : File ( config ( ) . getString ( " format_schema_path " , path + " format_schemas/ " ) ) ;
2019-01-27 09:15:32 +00:00
global_context - > setFormatSchemaPath ( format_schema_path . path ( ) ) ;
2017-11-10 06:48:28 +00:00
format_schema_path . createDirectories ( ) ;
2020-04-19 21:43:06 +00:00
/// Limit on total memory usage
2020-04-21 22:40:18 +00:00
size_t max_server_memory_usage = config ( ) . getUInt64 ( " max_server_memory_usage " , 0 ) ;
2020-04-19 22:08:09 +00:00
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 ;
2020-06-23 12:30:45 +00:00
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 ) ;
2020-04-19 22:08:09 +00:00
}
else if ( max_server_memory_usage > default_max_server_memory_usage )
{
max_server_memory_usage = default_max_server_memory_usage ;
2020-06-23 12:30:45 +00:00
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 ) ;
2020-04-19 22:08:09 +00:00
}
total_memory_tracker . setOrRaiseHardLimit ( max_server_memory_usage ) ;
2020-04-19 21:43:06 +00:00
total_memory_tracker . setDescription ( " (total) " ) ;
total_memory_tracker . setMetric ( CurrentMetrics : : MemoryTracking ) ;
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Loading metadata from {} " , path ) ;
2019-07-30 14:04:18 +00:00
2018-12-10 23:21:03 +00:00
try
{
loadMetadataSystem ( * global_context ) ;
/// After attaching system databases we can initialize system log.
global_context - > initializeSystemLogs ( ) ;
/// After the system database is created, attach virtual system tables (in addition to query_log and part_log)
2020-02-10 13:10:17 +00:00
attachSystemTablesServer ( * DatabaseCatalog : : instance ( ) . getSystemDatabase ( ) , has_zookeeper ) ;
2018-12-10 23:21:03 +00:00
/// Then, load remaining databases
2020-03-17 23:51:35 +00:00
loadMetadata ( * global_context , default_database ) ;
2020-02-10 13:10:17 +00:00
DatabaseCatalog : : instance ( ) . loadDatabases ( ) ;
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
///
# if USE_UNWIND && !WITH_COVERAGE && !defined(SANITIZER)
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
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
2017-04-01 07:20:54 +00:00
global_context - > setCurrentDatabase ( default_database ) ;
2017-04-13 13:42:29 +00:00
if ( has_zookeeper & & config ( ) . has ( " distributed_ddl " ) )
{
2017-04-18 15:44:31 +00:00
/// DDL worker should be started after all tables were loaded
2017-04-13 16:12:56 +00:00
String ddl_zookeeper_path = config ( ) . getString ( " distributed_ddl.path " , " /clickhouse/task_queue/ddl/ " ) ;
2019-02-21 16:41:27 +00:00
global_context - > setDDLWorker ( std : : make_unique < DDLWorker > ( ddl_zookeeper_path , * global_context , & config ( ) , " distributed_ddl " ) ) ;
2017-04-13 13:42:29 +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 ( ) ;
2018-03-29 20:21:01 +00:00
}
else
{
2019-06-27 16:28:26 +00:00
/// Initialize a watcher periodically updating DNS cache
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-05-23 22:24:01 +00:00
LOG_INFO ( log , " It looks like the process has no CAP_SYS_NICE capability, the setting 'os_thread_nice' 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
2017-04-01 07:20:54 +00:00
{
2017-09-08 16:41:35 +00:00
Poco : : Timespan keep_alive_timeout ( config ( ) . getUInt ( " keep_alive_timeout " , 10 ) , 0 ) ;
2017-04-01 07:20:54 +00:00
2017-09-08 16:41:35 +00:00
Poco : : ThreadPool server_pool ( 3 , config ( ) . getUInt ( " max_connections " , 1024 ) ) ;
2017-04-01 07:20:54 +00:00
Poco : : Net : : HTTPServerParams : : Ptr http_params = new Poco : : Net : : HTTPServerParams ;
2018-07-10 18:39:32 +00:00
http_params - > setTimeout ( settings . http_receive_timeout ) ;
2017-04-01 07:20:54 +00:00
http_params - > setKeepAliveTimeout ( keep_alive_timeout ) ;
std : : vector < std : : unique_ptr < Poco : : Net : : TCPServer > > servers ;
2017-06-22 18:56:40 +00:00
std : : vector < std : : string > listen_hosts = DB : : getMultipleValuesFromConfig ( config ( ) , " " , " listen_host " ) ;
2017-04-01 07:20:54 +00:00
2018-03-29 17:24:46 +00:00
bool listen_try = config ( ) . getBool ( " listen_try " , false ) ;
2017-04-01 07:20:54 +00:00
if ( listen_hosts . empty ( ) )
{
listen_hosts . emplace_back ( " ::1 " ) ;
listen_hosts . emplace_back ( " 127.0.0.1 " ) ;
2018-02-12 19:36:18 +00:00
listen_try = true ;
2017-04-01 07:20:54 +00:00
}
2017-09-06 02:42:44 +00:00
auto make_socket_address = [ & ] ( const std : : string & host , UInt16 port )
{
2017-04-01 07:20:54 +00:00
Poco : : Net : : SocketAddress socket_address ;
try
{
socket_address = Poco : : Net : : SocketAddress ( host , port ) ;
}
catch ( const Poco : : Net : : DNSException & e )
{
2018-03-12 15:48:55 +00:00
const auto code = e . code ( ) ;
if ( code = = EAI_FAMILY
2017-03-07 19:01:37 +00:00
# if defined(EAI_ADDRFAMILY)
2018-03-12 15:48:55 +00:00
| | code = = EAI_ADDRFAMILY
2017-03-07 19:01:37 +00:00
# endif
2017-04-01 07:20:54 +00:00
)
{
2020-05-23 22:24:01 +00:00
LOG_ERROR ( log , " Cannot resolve listen_host ({}), error {}: {}. "
2020-05-23 22:21:29 +00:00
" If it is an IPv6 address and your host has disabled IPv6, then consider to "
" specify IPv4 address to listen in <listen_host> element of configuration "
" file. Example: <listen_host>0.0.0.0</listen_host> " ,
host , e . code ( ) , e . message ( ) ) ;
2017-04-01 07:20:54 +00:00
}
throw ;
}
return socket_address ;
} ;
2020-03-09 03:41:03 +00:00
auto socket_bind_listen = [ & ] ( auto & socket , const std : : string & host , UInt16 port , [[maybe_unused]] bool secure = false )
2018-03-29 17:24:46 +00:00
{
auto address = make_socket_address ( host , port ) ;
2019-05-14 19:55:10 +00:00
# if !defined(POCO_CLICKHOUSE_PATCH) || POCO_VERSION < 0x01090100
2018-03-29 17:24:46 +00:00
if ( secure )
2019-05-14 19:55:10 +00:00
/// Bug in old (<1.9.1) poco, listen() after bind() with reusePort param will fail because have no implementation in SecureServerSocketImpl
2018-03-29 17:24:46 +00:00
/// https://github.com/pocoproject/poco/pull/2257
2018-03-30 12:42:06 +00:00
socket . bind ( address , /* reuseAddress = */ true ) ;
2018-03-29 17:24:46 +00:00
else
# endif
2018-03-30 12:42:06 +00:00
# if POCO_VERSION < 0x01080000
socket . bind ( address , /* reuseAddress = */ true ) ;
# else
2018-03-29 17:24:46 +00:00
socket . bind ( address , /* reuseAddress = */ true , /* reusePort = */ config ( ) . getBool ( " listen_reuse_port " , false ) ) ;
2018-03-30 12:42:06 +00:00
# endif
2018-03-29 17:24:46 +00:00
socket . listen ( /* backlog = */ config ( ) . getUInt ( " listen_backlog " , 64 ) ) ;
return address ;
} ;
2019-11-26 20:27:24 +00:00
/// This object will periodically calculate some metrics.
AsynchronousMetrics async_metrics ( * global_context ) ;
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
auto create_server = [ & ] ( const char * port_name , auto & & func )
2017-04-01 07:20:54 +00:00
{
2019-07-30 21:23:42 +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 ;
2017-04-01 07:20:54 +00:00
2019-07-30 22:06:13 +00:00
auto port = config ( ) . getInt ( port_name ) ;
2019-07-30 21:23:42 +00:00
try
2017-04-27 21:51:09 +00:00
{
2019-07-30 21:23:42 +00:00
func ( port ) ;
2017-04-27 21:51:09 +00:00
}
2019-07-30 22:08:57 +00:00
catch ( const Poco : : Exception & )
2017-04-27 21:51:09 +00:00
{
2019-07-30 21:23:42 +00:00
std : : string message = " Listen [ " + listen_host + " ]: " + std : : to_string ( port ) + " failed: " + getCurrentExceptionMessage ( false ) ;
if ( listen_try )
{
2020-06-15 20:30:36 +00:00
LOG_WARNING ( log , " {}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to "
2020-05-23 22:21:29 +00:00
" 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 ) ;
2019-07-30 21:23:42 +00:00
}
else
{
throw Exception { message , ErrorCodes : : NETWORK_ERROR } ;
}
2017-04-27 21:51:09 +00:00
}
2019-07-30 21:23:42 +00:00
} ;
2017-04-01 07:20:54 +00:00
2019-07-30 21:23:42 +00:00
/// HTTP
create_server ( " http_port " , [ & ] ( UInt16 port )
{
Poco : : Net : : ServerSocket socket ;
auto address = socket_bind_listen ( socket , listen_host , port ) ;
socket . setReceiveTimeout ( settings . http_receive_timeout ) ;
socket . setSendTimeout ( settings . http_send_timeout ) ;
2019-11-24 15:43:53 +00:00
2019-07-30 21:23:42 +00:00
servers . emplace_back ( std : : make_unique < Poco : : Net : : HTTPServer > (
2020-04-21 11:30:45 +00:00
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
2019-08-06 14:03:41 +00:00
create_server ( " https_port " , [ & ] ( 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 ;
auto address = socket_bind_listen ( socket , listen_host , port , /* secure = */ true ) ;
socket . setReceiveTimeout ( settings . http_receive_timeout ) ;
socket . setSendTimeout ( settings . http_send_timeout ) ;
servers . emplace_back ( std : : make_unique < Poco : : Net : : HTTPServer > (
2020-04-21 11:30:45 +00:00
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
create_server ( " tcp_port " , [ & ] ( UInt16 port )
{
Poco : : Net : : ServerSocket socket ;
auto address = socket_bind_listen ( socket , listen_host , port ) ;
socket . setReceiveTimeout ( settings . receive_timeout ) ;
socket . setSendTimeout ( settings . send_timeout ) ;
servers . emplace_back ( std : : make_unique < Poco : : Net : : TCPServer > (
new TCPHandlerFactory ( * this ) ,
server_pool ,
socket ,
new Poco : : Net : : TCPServerParams ) ) ;
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
} ) ;
/// TCP with SSL
2019-08-06 14:04:51 +00:00
create_server ( " tcp_port_secure " , [ & ] ( 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 ;
auto address = socket_bind_listen ( socket , listen_host , port , /* secure = */ true ) ;
socket . setReceiveTimeout ( settings . receive_timeout ) ;
socket . setSendTimeout ( settings . send_timeout ) ;
servers . emplace_back ( std : : make_unique < Poco : : Net : : TCPServer > (
new TCPHandlerFactory ( * this , /* secure= */ true ) ,
server_pool ,
socket ,
new Poco : : Net : : TCPServerParams ) ) ;
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
create_server ( " interserver_http_port " , [ & ] ( UInt16 port )
{
Poco : : Net : : ServerSocket socket ;
auto address = socket_bind_listen ( socket , listen_host , port ) ;
socket . setReceiveTimeout ( settings . http_receive_timeout ) ;
socket . setSendTimeout ( settings . http_send_timeout ) ;
servers . emplace_back ( std : : make_unique < Poco : : Net : : HTTPServer > (
2020-04-21 11:30:45 +00:00
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
} ) ;
2019-08-06 14:03:41 +00:00
create_server ( " interserver_https_port " , [ & ] ( 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 ;
auto address = socket_bind_listen ( socket , listen_host , port , /* secure = */ true ) ;
socket . setReceiveTimeout ( settings . http_receive_timeout ) ;
socket . setSendTimeout ( settings . http_send_timeout ) ;
servers . emplace_back ( std : : make_unique < Poco : : Net : : HTTPServer > (
2020-04-21 11:30:45 +00:00
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
} ) ;
2019-08-06 14:03:41 +00:00
create_server ( " mysql_port " , [ & ] ( UInt16 port )
2017-04-01 07:20:54 +00:00
{
2019-07-30 21:23:42 +00:00
Poco : : Net : : ServerSocket socket ;
auto address = socket_bind_listen ( socket , listen_host , port , /* secure = */ true ) ;
socket . setReceiveTimeout ( Poco : : Timespan ( ) ) ;
socket . setSendTimeout ( settings . send_timeout ) ;
servers . emplace_back ( std : : make_unique < Poco : : Net : : TCPServer > (
new MySQLHandlerFactory ( * this ) ,
server_pool ,
socket ,
new Poco : : Net : : TCPServerParams ) ) ;
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-05-30 20:02:11 +00:00
create_server ( " postgresql_port " , [ & ] ( UInt16 port )
{
Poco : : Net : : ServerSocket socket ;
auto address = socket_bind_listen ( socket , listen_host , port , /* secure = */ true ) ;
socket . setReceiveTimeout ( Poco : : Timespan ( ) ) ;
socket . setSendTimeout ( settings . send_timeout ) ;
servers . emplace_back ( std : : make_unique < Poco : : Net : : TCPServer > (
new PostgreSQLHandlerFactory ( * this ) ,
server_pool ,
socket ,
new Poco : : Net : : TCPServerParams ) ) ;
LOG_INFO ( log , " Listening for PostgreSQL compatibility protocol: " + address . toString ( ) ) ;
} ) ;
2019-11-24 16:27:00 +00:00
/// Prometheus (if defined and not setup yet with http_port)
create_server ( " prometheus.port " , [ & ] ( UInt16 port )
{
Poco : : Net : : ServerSocket socket ;
auto address = socket_bind_listen ( socket , listen_host , port ) ;
socket . setReceiveTimeout ( settings . http_receive_timeout ) ;
socket . setSendTimeout ( settings . http_send_timeout ) ;
servers . emplace_back ( std : : make_unique < Poco : : Net : : HTTPServer > (
2020-04-21 11:30:45 +00:00
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
}
2017-04-27 21:51:09 +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-03-05 19:23:39 +00:00
global_context - > enableNamedSessions ( ) ;
2017-04-01 07:20:54 +00:00
for ( auto & server : servers )
server - > start ( ) ;
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-01-17 19:19:09 +00:00
buildLoggers ( config ( ) , logger ( ) ) ;
2018-03-03 14:39:16 +00:00
main_config_reloader - > start ( ) ;
users_config_reloader - > start ( ) ;
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
}
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 ;
for ( auto & server : servers )
{
server - > stop ( ) ;
current_connections + = server - > currentConnections ( ) ;
}
2020-05-23 22:21:29 +00:00
if ( current_connections )
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Closed all listening sockets. Waiting for {} outstanding connections. " , current_connections ) ;
2020-05-23 22:21:29 +00:00
else
2020-05-23 22:24:01 +00:00
LOG_INFO ( log , " Closed all listening sockets. " ) ;
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 )
{
const int sleep_max_ms = 1000 * config ( ) . getInt ( " shutdown_wait_unfinished " , 5 ) ;
const int sleep_one_ms = 100 ;
int sleep_current_ms = 0 ;
while ( sleep_current_ms < sleep_max_ms )
{
current_connections = 0 ;
for ( auto & server : servers )
current_connections + = server - > currentConnections ( ) ;
if ( ! current_connections )
break ;
sleep_current_ms + = sleep_one_ms ;
std : : this_thread : : sleep_for ( std : : chrono : : milliseconds ( sleep_one_ms ) ) ;
}
}
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 ( ) ;
users_config_reloader . reset ( ) ;
2019-07-30 23:12:04 +00:00
if ( current_connections )
{
/// There is no better way to force connections to close in Poco.
/// Otherwise connection handlers will continue to live
/// (they are effectively dangling objects, but they use global thread pool
/// and global thread pool destructor will wait for threads, preventing server shutdown).
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. " ) ;
2019-07-30 23:12:04 +00:00
_exit ( Application : : EXIT_OK ) ;
}
2017-04-01 07:20:54 +00:00
} ) ;
/// try to load dictionaries immediately, throw on error and die
2019-12-30 23:30:06 +00:00
ext : : scope_guard dictionaries_xmls , models_xmls ;
2017-04-01 07:20:54 +00:00
try
{
if ( ! config ( ) . getBool ( " dictionaries_lazy_load " , true ) )
{
global_context - > tryCreateEmbeddedDictionaries ( ) ;
2019-09-26 10:41:33 +00:00
global_context - > getExternalDictionariesLoader ( ) . enableAlwaysLoadEverything ( true ) ;
2017-04-01 07:20:54 +00:00
}
2019-12-30 23:30:06 +00:00
dictionaries_xmls = global_context - > getExternalDictionariesLoader ( ) . addConfigRepository (
std : : make_unique < ExternalLoaderXMLConfigRepository > ( config ( ) , " dictionaries_config " ) ) ;
models_xmls = global_context - > getExternalModelsLoader ( ) . addConfigRepository (
std : : make_unique < ExternalLoaderXMLConfigRepository > ( config ( ) , " models_config " ) ) ;
2017-04-01 07:20:54 +00:00
}
catch ( . . . )
{
2020-05-23 22:24:01 +00:00
LOG_ERROR ( log , " Caught exception while loading dictionaries. " ) ;
2017-04-01 07:20:54 +00:00
throw ;
}
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
}
}
2016-10-31 19:54:49 +00:00
2019-12-15 06:34:43 +00:00
# pragma GCC diagnostic ignored "-Wunused-function"
# pragma GCC diagnostic ignored "-Wmissing-declarations"
2017-12-02 02:47:12 +00:00
int mainEntryClickHouseServer ( int argc , char * * argv )
{
DB : : Server app ;
try
{
return app . run ( argc , argv ) ;
}
catch ( . . . )
{
std : : cerr < < DB : : getCurrentExceptionMessage ( true ) < < " \n " ;
auto code = DB : : getCurrentExceptionCode ( ) ;
return code ? code : 1 ;
}
}