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>
# include <pwd.h>
# include <unistd.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>
2022-04-27 15:05:45 +00:00
# include <Common/scope_guard_safe.h>
# include <Common/logger_useful.h>
2021-10-02 07:13:14 +00:00
# include <base/phdr_cache.h>
2022-04-27 15:05:45 +00:00
# include <Common/ErrorHandlers.h>
2021-10-02 07:13:14 +00:00
# include <base/getMemoryAmount.h>
2022-02-15 10:59:00 +00:00
# include <base/getAvailableMemoryAmount.h>
2021-10-02 07:13:14 +00:00
# include <base/errnoToString.h>
# include <base/coverage.h>
2022-03-02 09:59:10 +00:00
# include <base/getFQDNOrHostName.h>
# include <base/safeExit.h>
2023-11-06 13:08:27 +00:00
# include <Common/PoolId.h>
2022-01-18 12:21:59 +00:00
# include <Common/MemoryTracker.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>
2024-02-01 21:09:39 +00:00
# include <Common/CgroupsMemoryUsageObserver.h>
2017-08-09 14:33:07 +00:00
# include <Common/CurrentMetrics.h>
2022-05-31 16:43:15 +00:00
# include <Common/ConcurrencyControl.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>
2017-08-09 14:33:07 +00:00
# include <Common/ZooKeeper/ZooKeeper.h>
# include <Common/ZooKeeper/ZooKeeperNodeCache.h>
2023-06-29 18:33:48 +00:00
# include <Common/formatReadable.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>
2021-10-22 07:15:34 +00:00
# include <Common/ProfileEvents.h>
2024-03-19 16:04:29 +00:00
# include <Common/Scheduler/IResourceManager.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>
2022-09-09 20:00:27 +00:00
# include <Common/Config/AbstractConfigurationComparison.h>
2023-05-19 14:23:56 +00:00
# include <Common/assertProcessUserMatchesDataOwner.h>
2023-08-31 11:33:52 +00:00
# include <Common/FailPoint.h>
2024-02-16 12:06:57 +00:00
# include <Common/CPUID.h>
2024-03-03 13:22:40 +00:00
# include <Common/HTTPConnectionPool.h>
2023-12-07 17:30:43 +00:00
# include <Interpreters/Cache/FileCacheFactory.h>
2021-08-18 12:15:31 +00:00
# include <Core/ServerUUID.h>
2021-07-30 09:12:33 +00:00
# include <IO/ReadHelpers.h>
2022-03-29 16:20:20 +00:00
# include <IO/ReadBufferFromFile.h>
2023-05-03 16:56:45 +00:00
# include <IO/SharedThreadPools.h>
2018-09-06 18:05:33 +00:00
# include <IO/UseSSL.h>
2022-11-09 12:37:42 +00:00
# include <Interpreters/ServerAsynchronousMetrics.h>
2017-08-09 14:33:07 +00:00
# include <Interpreters/DDLWorker.h>
2021-03-05 14:57:16 +00:00
# include <Interpreters/DNSCacheUpdater.h>
# include <Interpreters/DatabaseCatalog.h>
2019-09-26 10:41:33 +00:00
# include <Interpreters/ExternalDictionariesLoader.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/ProcessList.h>
# include <Interpreters/loadMetadata.h>
2024-01-09 06:33:48 +00:00
# include <Interpreters/registerInterpreters.h>
2021-06-13 12:38:57 +00:00
# include <Interpreters/JIT/CompiledExpressionCache.h>
2021-11-02 11:06:20 +00:00
# include <Access/AccessControl.h>
2017-04-01 09:19:00 +00:00
# include <Storages/StorageReplicatedMergeTree.h>
# include <Storages/System/attachSystemTables.h>
2021-09-09 09:37:51 +00:00
# include <Storages/System/attachInformationSchemaTables.h>
2021-12-23 03:50:26 +00:00
# include <Storages/Cache/ExternalDataSourceCache.h>
2022-01-04 07:06:19 +00:00
# include <Storages/Cache/registerRemoteFileMetadatas.h>
2022-12-16 22:57:09 +00:00
# include <Common/NamedCollections/NamedCollectionUtils.h>
2017-08-09 14:33:07 +00:00
# include <AggregateFunctions/registerAggregateFunctions.h>
2023-12-12 14:03:49 +00:00
# include <Functions/UserDefined/IUserDefinedSQLObjectsStorage.h>
2017-08-09 14:33:07 +00:00
# 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>
2023-12-29 21:52:20 +00:00
# include <Databases/registerDatabases.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>
2024-01-20 12:06:26 +00:00
# include <Common/Scheduler/Nodes/registerSchedulerNodes.h>
# include <Common/Scheduler/Nodes/registerResourceManagers.h>
2018-02-28 20:34:25 +00:00
# include <Common/Config/ConfigReloader.h>
2016-01-17 13:34:36 +00:00
# include "MetricsTransmitter.h"
2018-06-05 20:09:51 +00:00
# include <Common/StatusFile.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>
2022-02-13 16:58:38 +00:00
# include <Common/filesystemHelpers.h>
2022-06-01 20:07:26 +00:00
# include <Compression/CompressionCodecEncrypted.h>
2024-02-13 09:40:20 +00:00
# include <Server/CloudPlacementInfo.h>
2024-05-23 23:14:26 +00:00
# include <Server/ServersManager/InterServersManager.h>
# include <Server/ServersManager/ProtocolServersManager.h>
2021-09-01 23:42:50 +00:00
# include <Interpreters/AsynchronousInsertQueue.h>
2023-02-18 04:46:09 +00:00
# include <Core/ServerSettings.h>
2021-09-14 20:02:30 +00:00
# include <filesystem>
2019-07-30 14:04:18 +00:00
2022-09-28 13:29:29 +00:00
# include "config.h"
2023-11-13 09:09:23 +00:00
# include <Common/config_version.h>
2020-04-16 12:31:57 +00:00
2019-09-15 10:35:12 +00:00
# if defined(OS_LINUX)
2022-11-19 18:18:03 +00:00
# include <cstdlib>
# include <sys / un.h>
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>
2018-11-14 01:11:13 +00:00
# endif
2020-05-08 14:11:19 +00:00
# if USE_SSL
2023-07-11 12:23:10 +00:00
# include <Server / CertificateReloader.h>
2017-03-28 20:30:57 +00:00
# endif
2017-03-21 19:08:09 +00:00
2021-04-18 06:43:13 +00:00
# if USE_JEMALLOC
# include <jemalloc / jemalloc.h>
# endif
2023-01-30 19:00:48 +00:00
# if USE_AZURE_BLOB_STORAGE
# include <azure / storage / common / internal / xml_wrapper.hpp>
2024-04-12 15:38:50 +00:00
# include <azure / core / diagnostics / logger.hpp>
2023-01-30 19:00:48 +00:00
# endif
2023-07-23 02:56:47 +00:00
# include <incbin.h>
/// A minimal file used when the server is run without installation
2023-07-23 22:03:40 +00:00
INCBIN ( resource_embedded_xml , SOURCE_DIR " /programs/server/embedded.xml " ) ;
2023-07-23 02:56:47 +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 ;
2023-04-14 14:34:19 +00:00
extern const Metric MergesMutationsMemoryTracking ;
2021-02-15 10:26:34 +00:00
extern const Metric MaxDDLEntryID ;
2021-08-11 03:40:06 +00:00
extern const Metric MaxPushedDDLEntryID ;
2017-08-01 14:34:06 +00:00
}
2021-10-22 07:15:34 +00:00
namespace ProfileEvents
{
extern const Event MainConfigLoads ;
2023-01-13 13:38:54 +00:00
extern const Event ServerStartupMilliseconds ;
2021-10-22 07:15:34 +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
2024-02-28 22:47:34 +00:00
static bool jemallocOptionEnabled ( const char * ) { return false ; }
2021-04-18 06:43:13 +00:00
# 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 )
{
2022-08-21 18:24:17 +00:00
const char * env_watchdog = getenv ( " CLICKHOUSE_WATCHDOG_ENABLE " ) ; // NOLINT(concurrency-mt-unsafe)
2021-01-23 20:58:36 +00:00
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 ;
}
}
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 ;
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 ;
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
{
2014-12-21 03:18:40 +00:00
Poco : : trimInPlace ( path ) ;
if ( path . empty ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INVALID_CONFIG_PARAMETER , " path configuration parameter is empty " ) ;
2014-12-21 03:18:40 +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
}
2023-01-01 22:22:26 +00:00
2023-01-02 02:27:21 +00:00
# if defined(OS_LINUX)
2023-01-01 22:22:26 +00:00
namespace
{
2024-01-23 17:04:50 +00:00
void setOOMScore ( int value , LoggerRawPtr log )
2023-01-01 22:22:26 +00:00
{
try
{
std : : string value_string = std : : to_string ( value ) ;
DB : : WriteBufferFromFile buf ( " /proc/self/oom_score_adj " ) ;
buf . write ( value_string . c_str ( ) , value_string . size ( ) ) ;
buf . next ( ) ;
buf . close ( ) ;
}
catch ( const Poco : : Exception & e )
{
LOG_WARNING ( log , " Failed to adjust OOM score: '{}'. " , e . displayText ( ) ) ;
return ;
}
LOG_INFO ( log , " Set OOM score adjustment to {} " , value ) ;
}
}
2023-01-02 02:27:21 +00:00
# endif
2023-01-01 22:22:26 +00:00
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 " ,
2023-04-20 22:54:34 +00:00
commandName ( ) ) ;
2020-11-10 18:22:26 +00:00
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 " ) )
{
2023-08-09 03:02:50 +00:00
std : : cout < < VERSION_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 )
{
2023-07-23 02:56:47 +00:00
ConfigProcessor : : registerEmbeddedConfig ( " config.xml " , std : : string_view ( reinterpret_cast < const char * > ( gresource_embedded_xmlData ) , gresource_embedded_xmlSize ) ) ;
2018-02-08 19:12:37 +00:00
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 ,
2024-01-23 17:04:50 +00:00
LoggerPtr log )
2020-06-04 20:33:19 +00:00
{
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
}
2022-03-29 12:20:46 +00:00
/// Unused in other builds
# if defined(OS_LINUX)
2023-08-23 09:42:29 +00:00
static String readLine ( const String & path )
2022-02-13 16:58:38 +00:00
{
ReadBufferFromFile in ( path ) ;
String contents ;
2023-08-23 09:42:29 +00:00
readStringUntilNewlineInto ( contents , in ) ;
2022-02-13 16:58:38 +00:00
return contents ;
}
2022-02-14 14:16:05 +00:00
static int readNumber ( const String & path )
2022-02-13 16:58:38 +00:00
{
ReadBufferFromFile in ( path ) ;
int result ;
readText ( result , in ) ;
return result ;
}
2022-03-29 12:20:46 +00:00
# endif
2022-04-30 11:19:25 +00:00
static void sanityChecks ( Server & server )
2022-02-13 16:58:38 +00:00
{
2022-04-30 11:19:25 +00:00
std : : string data_path = getCanonicalPath ( server . config ( ) . getString ( " path " , DBMS_DEFAULT_PATH ) ) ;
std : : string logs_path = server . config ( ) . getString ( " logger.log " , " " ) ;
if ( server . logger ( ) . is ( Poco : : Message : : PRIO_TEST ) )
server . context ( ) - > addWarningMessage ( " Server logging level is set to 'test' and performance is degraded. This cannot be used in production. " ) ;
2022-02-13 16:58:38 +00:00
# if defined(OS_LINUX)
2022-02-15 09:56:21 +00:00
try
{
2023-08-23 09:42:29 +00:00
const std : : unordered_set < std : : string > fastClockSources = {
// ARM clock
" arch_sys_counter " ,
// KVM guest clock
" kvm-clock " ,
// X86 clock
" tsc " ,
} ;
2022-05-31 11:50:09 +00:00
const char * filename = " /sys/devices/system/clocksource/clocksource0/current_clocksource " ;
2023-08-23 12:35:30 +00:00
if ( ! fastClockSources . contains ( readLine ( filename ) ) )
2022-09-13 10:44:42 +00:00
server . context ( ) - > addWarningMessage ( " Linux is not using a fast clock source. Performance can be degraded. Check " + String ( filename ) ) ;
2022-02-15 09:56:21 +00:00
}
2023-09-25 20:19:09 +00:00
catch ( . . . ) // NOLINT(bugprone-empty-catch)
2022-02-15 09:56:21 +00:00
{
}
2022-02-13 16:58:38 +00:00
2022-02-15 09:56:21 +00:00
try
{
2022-05-31 11:50:09 +00:00
const char * filename = " /proc/sys/vm/overcommit_memory " ;
if ( readNumber ( filename ) = = 2 )
server . context ( ) - > addWarningMessage ( " Linux memory overcommit is disabled. Check " + String ( filename ) ) ;
2022-02-15 09:56:21 +00:00
}
2023-09-25 20:19:09 +00:00
catch ( . . . ) // NOLINT(bugprone-empty-catch)
2022-02-15 09:56:21 +00:00
{
}
2022-02-13 16:58:38 +00:00
2022-02-15 09:56:21 +00:00
try
{
2022-05-31 11:50:09 +00:00
const char * filename = " /sys/kernel/mm/transparent_hugepage/enabled " ;
2023-08-23 09:42:29 +00:00
if ( readLine ( filename ) . find ( " [always] " ) ! = std : : string : : npos )
2022-05-31 11:50:09 +00:00
server . context ( ) - > addWarningMessage ( " Linux transparent hugepages are set to \" always \" . Check " + String ( filename ) ) ;
2022-02-15 09:56:21 +00:00
}
2023-09-25 20:19:09 +00:00
catch ( . . . ) // NOLINT(bugprone-empty-catch)
2022-02-15 09:56:21 +00:00
{
}
2022-02-13 16:58:38 +00:00
2022-02-15 09:56:21 +00:00
try
{
2022-05-31 11:50:09 +00:00
const char * filename = " /proc/sys/kernel/pid_max " ;
if ( readNumber ( filename ) < 30000 )
server . context ( ) - > addWarningMessage ( " Linux max PID is too low. Check " + String ( filename ) ) ;
2022-02-15 09:56:21 +00:00
}
2023-09-25 20:19:09 +00:00
catch ( . . . ) // NOLINT(bugprone-empty-catch)
2022-02-15 09:56:21 +00:00
{
}
2022-02-13 16:58:38 +00:00
2022-02-15 09:56:21 +00:00
try
{
2022-05-31 11:50:09 +00:00
const char * filename = " /proc/sys/kernel/threads-max " ;
if ( readNumber ( filename ) < 30000 )
server . context ( ) - > addWarningMessage ( " Linux threads max count is too low. Check " + String ( filename ) ) ;
2023-11-02 12:21:27 +00:00
}
catch ( . . . ) // NOLINT(bugprone-empty-catch)
{
}
try
{
const char * filename = " /proc/sys/kernel/task_delayacct " ;
if ( readNumber ( filename ) = = 0 )
2024-02-10 13:49:43 +00:00
server . context ( ) - > addWarningMessage ( " Delay accounting is not enabled, OSIOWaitMicroseconds will not be gathered. You can enable it using `echo 1 > " + String ( filename ) + " ` or by using sysctl. " ) ;
2022-02-15 09:56:21 +00:00
}
2023-09-25 20:19:09 +00:00
catch ( . . . ) // NOLINT(bugprone-empty-catch)
2022-02-15 09:56:21 +00:00
{
}
2022-02-13 16:58:38 +00:00
2022-02-15 10:39:45 +00:00
std : : string dev_id = getBlockDeviceId ( data_path ) ;
if ( getBlockDeviceType ( dev_id ) = = BlockDeviceType : : ROT & & getBlockDeviceReadAheadBytes ( dev_id ) = = 0 )
2022-05-31 07:26:26 +00:00
server . context ( ) - > addWarningMessage ( " Rotational disk with disabled readahead is in use. Performance can be degraded. Used for data: " + String ( data_path ) ) ;
2022-02-13 16:58:38 +00:00
# endif
2022-03-30 22:22:36 +00:00
2022-02-15 10:59:00 +00:00
try
{
if ( getAvailableMemoryAmount ( ) < ( 2l < < 30 ) )
2022-04-30 11:19:25 +00:00
server . context ( ) - > addWarningMessage ( " Available memory at server startup is too low (2GiB). " ) ;
2022-08-13 23:49:00 +00:00
}
2023-09-25 20:19:09 +00:00
catch ( . . . ) // NOLINT(bugprone-empty-catch)
2022-08-13 23:49:00 +00:00
{
}
2022-02-13 16:58:38 +00:00
2022-08-13 23:49:00 +00:00
try
{
2022-03-30 22:22:36 +00:00
if ( ! enoughSpaceInDirectory ( data_path , 1ull < < 30 ) )
2022-04-30 11:19:25 +00:00
server . context ( ) - > addWarningMessage ( " Available disk space for data at server startup is too low (1GiB): " + String ( data_path ) ) ;
2022-08-13 23:49:00 +00:00
}
2023-09-25 20:19:09 +00:00
catch ( . . . ) // NOLINT(bugprone-empty-catch)
2022-08-13 23:49:00 +00:00
{
}
2022-02-13 16:58:38 +00:00
2022-08-13 23:49:00 +00:00
try
{
2022-03-30 22:22:36 +00:00
if ( ! logs_path . empty ( ) )
{
2022-04-21 05:45:08 +00:00
auto logs_parent = fs : : path ( logs_path ) . parent_path ( ) ;
if ( ! enoughSpaceInDirectory ( logs_parent , 1ull < < 30 ) )
2022-04-30 11:19:25 +00:00
server . context ( ) - > addWarningMessage ( " Available disk space for logs at server startup is too low (1GiB): " + String ( logs_parent ) ) ;
2022-03-30 22:22:36 +00:00
}
}
2023-09-25 20:19:09 +00:00
catch ( . . . ) // NOLINT(bugprone-empty-catch)
2022-02-15 10:39:45 +00:00
{
2022-02-13 16:58:38 +00:00
}
2022-08-13 23:49:00 +00:00
if ( server . context ( ) - > getMergeTreeSettings ( ) . allow_remote_fs_zero_copy_replication )
{
server . context ( ) - > addWarningMessage ( " The setting 'allow_remote_fs_zero_copy_replication' is enabled for MergeTree tables. "
" But the feature of 'zero-copy replication' is under development and is not ready for production. "
" The usage of this feature can lead to data corruption and loss. The setting should be disabled in production. " ) ;
}
2022-02-13 16:58:38 +00:00
}
2021-04-25 03:06:38 +00:00
2024-04-12 18:41:40 +00:00
static void initializeAzureSDKLogger (
[[ maybe_unused ]] const ServerSettings & server_settings ,
[[ maybe_unused ]] int server_logs_level )
2024-04-12 15:38:50 +00:00
{
# if USE_AZURE_BLOB_STORAGE
if ( ! server_settings . enable_azure_sdk_logging )
return ;
using AzureLogsLevel = Azure : : Core : : Diagnostics : : Logger : : Level ;
static const std : : unordered_map < AzureLogsLevel , std : : pair < Poco : : Message : : Priority , DB : : LogsLevel > > azure_to_server_mapping =
{
{ AzureLogsLevel : : Error , { Poco : : Message : : PRIO_DEBUG , LogsLevel : : debug } } ,
{ AzureLogsLevel : : Warning , { Poco : : Message : : PRIO_DEBUG , LogsLevel : : debug } } ,
{ AzureLogsLevel : : Informational , { Poco : : Message : : PRIO_TRACE , LogsLevel : : trace } } ,
{ AzureLogsLevel : : Verbose , { Poco : : Message : : PRIO_TEST , LogsLevel : : test } } ,
} ;
static const std : : map < Poco : : Message : : Priority , AzureLogsLevel > server_to_azure_mapping =
{
{ Poco : : Message : : PRIO_DEBUG , AzureLogsLevel : : Warning } ,
{ Poco : : Message : : PRIO_TRACE , AzureLogsLevel : : Informational } ,
{ Poco : : Message : : PRIO_TEST , AzureLogsLevel : : Verbose } ,
} ;
static const LoggerPtr azure_sdk_logger = getLogger ( " AzureSDK " ) ;
auto it = server_to_azure_mapping . lower_bound ( static_cast < Poco : : Message : : Priority > ( server_logs_level ) ) ;
chassert ( it ! = server_to_azure_mapping . end ( ) ) ;
Azure : : Core : : Diagnostics : : Logger : : SetLevel ( it - > second ) ;
Azure : : Core : : Diagnostics : : Logger : : SetListener ( [ ] ( AzureLogsLevel level , const std : : string & message )
{
auto [ poco_level , db_level ] = azure_to_server_mapping . at ( level ) ;
LOG_IMPL ( azure_sdk_logger , db_level , poco_level , fmt : : runtime ( message ) ) ;
} ) ;
# endif
}
2017-12-02 02:47:12 +00:00
int Server : : main ( const std : : vector < std : : string > & /*args*/ )
2022-11-18 12:22:55 +00:00
try
2016-12-13 18:51:19 +00:00
{
2023-01-13 13:38:54 +00:00
Stopwatch startup_watch ;
2023-01-13 12:47:54 +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
2023-02-18 04:46:09 +00:00
ServerSettings server_settings ;
server_settings . loadSettingsFromConfig ( config ( ) ) ;
2024-05-23 23:14:26 +00:00
Poco : : ThreadPool server_pool ( 3 , server_settings . max_connections ) ;
2023-02-18 04:46:09 +00:00
2024-02-22 18:17:15 +00:00
ASTAlterCommand : : setFormatAlterCommandsWithParentheses ( server_settings . format_alter_operations_with_parentheses ) ;
2023-02-18 04:46:09 +00:00
StackTrace : : setShowAddresses ( server_settings . show_addresses_in_stack_traces ) ;
2022-07-28 22:13:02 +00:00
2022-08-20 15:09:20 +00:00
# if USE_HDFS
/// This will point libhdfs3 to the right location for its config.
/// Note: this has to be done once at server initialization, because 'setenv' is not thread-safe.
String libhdfs3_conf = config ( ) . getString ( " hdfs.libhdfs3_conf " , " " ) ;
if ( ! libhdfs3_conf . empty ( ) )
{
if ( std : : filesystem : : path { libhdfs3_conf } . is_relative ( ) & & ! std : : filesystem : : exists ( libhdfs3_conf ) )
{
const String config_path = config ( ) . getString ( " config-file " , " config.xml " ) ;
const auto config_dir = std : : filesystem : : path { config_path } . remove_filename ( ) ;
if ( std : : filesystem : : exists ( config_dir / libhdfs3_conf ) )
libhdfs3_conf = std : : filesystem : : absolute ( config_dir / libhdfs3_conf ) ;
}
setenv ( " LIBHDFS3_CONF " , libhdfs3_conf . c_str ( ) , true /* overwrite */ ) ; // NOLINT
}
# endif
2022-12-06 22:18:35 +00:00
/// When building openssl into clickhouse, clickhouse owns the configuration
2022-12-06 22:44:10 +00:00
/// Therefore, the clickhouse openssl configuration should be kept separate from
/// the OS. Default to the one in the standard config directory, unless overridden
2022-12-06 22:18:35 +00:00
/// by a key in the config.
2024-02-10 14:35:04 +00:00
/// Note: this has to be done once at server initialization, because 'setenv' is not thread-safe.
2022-12-06 22:18:35 +00:00
if ( config ( ) . has ( " opensslconf " ) )
{
std : : string opensslconf_path = config ( ) . getString ( " opensslconf " ) ;
2024-02-10 14:35:04 +00:00
setenv ( " OPENSSL_CONF " , opensslconf_path . c_str ( ) , true ) ; /// NOLINT
2022-12-06 22:18:35 +00:00
}
else
{
const String config_path = config ( ) . getString ( " config-file " , " config.xml " ) ;
2022-12-16 20:31:17 +00:00
const auto config_dir = std : : filesystem : : path { config_path } . replace_filename ( " openssl.conf " ) ;
2024-02-10 14:35:04 +00:00
setenv ( " OPENSSL_CONF " , config_dir . c_str ( ) , true ) ; /// NOLINT
2022-12-06 22:18:35 +00:00
}
2024-01-09 06:33:48 +00:00
registerInterpreters ( ) ;
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 ( ) ;
2023-12-29 21:52:20 +00:00
registerDatabases ( ) ;
2017-12-30 00:36:06 +00:00
registerStorages ( ) ;
2018-11-28 11:37:12 +00:00
registerDictionaries ( ) ;
2022-11-19 08:09:24 +00:00
registerDisks ( /* global_skip_access_check= */ false ) ;
2020-10-29 03:39:43 +00:00
registerFormats ( ) ;
2022-01-04 07:06:19 +00:00
registerRemoteFileMetadatas ( ) ;
2022-09-27 13:26:41 +00:00
registerSchedulerNodes ( ) ;
registerResourceManagers ( ) ;
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
2016-10-13 20:01:18 +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 , . . .
2016-10-13 20:01:18 +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 ( ) ;
2016-11-11 17:01:02 +00:00
global_context - > setApplicationType ( Context : : ApplicationType : : SERVER ) ;
2017-04-01 07:20:54 +00:00
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
2022-02-13 16:58:38 +00:00
if ( ThreadFuzzer : : instance ( ) . isEffective ( ) )
global_context - > addWarningMessage ( " ThreadFuzzer is enabled. Application will run slowly and unstable. " ) ;
2021-07-12 10:57:39 +00:00
# if defined(SANITIZER)
global_context - > addWarningMessage ( " Server was built with sanitizer. It will work slowly. " ) ;
# endif
2023-10-29 15:55:47 +00:00
# if defined(SANITIZE_COVERAGE) || WITH_COVERAGE
global_context - > addWarningMessage ( " Server was built with code coverage. It will work slowly. " ) ;
2021-07-12 10:57:39 +00:00
# endif
2023-08-15 21:04:40 +00:00
const size_t physical_server_memory = getMemoryAmount ( ) ;
2022-12-26 12:30:13 +00:00
LOG_INFO ( log , " Available RAM: {}; physical cores: {}; logical cores: {}. " ,
2023-08-15 21:04:40 +00:00
formatReadableSizeWithBinarySuffix ( physical_server_memory ) ,
2022-12-26 12:30:13 +00:00
getNumberOfPhysicalCPUCores ( ) , // on ARM processors it can show only enabled at current moment cores
std : : thread : : hardware_concurrency ( ) ) ;
2024-02-17 00:54:54 +00:00
# if defined(__x86_64__)
2024-02-16 12:06:57 +00:00
String cpu_info ;
# define COLLECT_FLAG(X) \
if ( CPU : : have # # X ( ) ) \
{ \
if ( ! cpu_info . empty ( ) ) \
cpu_info + = " , " ; \
cpu_info + = # X ; \
}
CPU_ID_ENUMERATE ( COLLECT_FLAG )
# undef COLLECT_FLAG
2024-02-16 12:09:07 +00:00
LOG_INFO ( log , " Available CPU instruction sets: {} " , cpu_info ) ;
2024-02-17 00:54:54 +00:00
# endif
2024-02-16 12:06:57 +00:00
2024-04-02 16:04:53 +00:00
bool will_have_trace_collector = hasPHDRCache ( ) & & config ( ) . has ( " trace_log " ) ;
2024-05-23 23:14:26 +00:00
std : : mutex servers_lock ;
ProtocolServersManager servers ( context ( ) , & logger ( ) ) ;
InterServersManager servers_to_start_before_tables ( context ( ) , & logger ( ) ) ;
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.
2021-11-12 13:24:47 +00:00
GlobalThreadPool : : initialize (
2023-02-18 04:46:09 +00:00
server_settings . max_thread_pool_size ,
server_settings . max_thread_pool_free_size ,
2024-04-02 16:04:53 +00:00
server_settings . thread_pool_queue_size ,
will_have_trace_collector ? server_settings . global_profiler_real_time_period_ns : 0 ,
will_have_trace_collector ? server_settings . global_profiler_cpu_time_period_ns : 0 ) ;
2023-07-28 07:47:09 +00:00
/// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed).
SCOPE_EXIT ( {
Stopwatch watch ;
LOG_INFO ( log , " Waiting for background threads " ) ;
GlobalThreadPool : : instance ( ) . shutdown ( ) ;
LOG_INFO ( log , " Background threads finished in {} ms " , watch . elapsedMilliseconds ( ) ) ;
} ) ;
2023-12-10 06:57:38 +00:00
/// NOTE: global context should be destroyed *before* GlobalThreadPool::shutdown()
2023-12-10 14:37:10 +00:00
/// Otherwise GlobalThreadPool::shutdown() will hang, since Context holds some threads.
2023-12-10 06:57:38 +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. " ) ;
2024-05-23 23:14:26 +00:00
servers_to_start_before_tables . stopServers ( server_settings , servers_lock ) ;
2023-12-10 06:57:38 +00:00
global_context - > shutdownKeeperDispatcher ( ) ;
/// Wait server pool to avoid use-after-free of destroyed context in the handlers
server_pool . joinAll ( ) ;
/** 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-18 10:47:09 +00:00
2023-01-30 19:00:48 +00:00
# if USE_AZURE_BLOB_STORAGE
/// It makes sense to deinitialize libxml after joining of all threads
/// in global pool because libxml uses thread-local memory allocations via
/// 'pthread_key_create' and 'pthread_setspecific' which should be deallocated
/// at 'pthread_exit'. Deinitialization of libxml leads to call of 'pthread_key_delete'
/// and if it is done before joining of threads, allocated memory will not be freed
/// and there may be memory leaks in threads that used libxml.
GlobalThreadPool : : instance ( ) . addOnDestroyCallback ( [ ]
{
Azure : : Storage : : _internal : : XmlGlobalDeinitialize ( ) ;
} ) ;
# endif
2023-06-06 12:42:56 +00:00
getIOThreadPool ( ) . initialize (
2023-02-18 04:46:09 +00:00
server_settings . max_io_thread_pool_size ,
server_settings . max_io_thread_pool_free_size ,
server_settings . io_thread_pool_queue_size ) ;
2021-11-29 08:22:43 +00:00
2023-06-06 12:42:56 +00:00
getBackupsIOThreadPool ( ) . initialize (
2023-03-03 19:05:42 +00:00
server_settings . max_backups_io_thread_pool_size ,
server_settings . max_backups_io_thread_pool_free_size ,
server_settings . backups_io_thread_pool_queue_size ) ;
2023-06-06 12:42:56 +00:00
getActivePartsLoadingThreadPool ( ) . initialize (
server_settings . max_active_parts_loading_thread_pool_size ,
0 , // We don't need any threads once all the parts will be loaded
server_settings . max_active_parts_loading_thread_pool_size ) ;
getOutdatedPartsLoadingThreadPool ( ) . initialize (
2023-05-03 16:56:45 +00:00
server_settings . max_outdated_parts_loading_thread_pool_size ,
2023-06-06 12:42:56 +00:00
0 , // We don't need any threads once all the parts will be loaded
2023-06-02 12:10:26 +00:00
server_settings . max_outdated_parts_loading_thread_pool_size ) ;
2023-05-03 16:56:45 +00:00
2023-06-06 12:42:56 +00:00
/// It could grow if we need to synchronously wait until all the data parts will be loaded.
getOutdatedPartsLoadingThreadPool ( ) . setMaxTurboThreads (
server_settings . max_active_parts_loading_thread_pool_size
) ;
2024-05-08 13:04:16 +00:00
getUnexpectedPartsLoadingThreadPool ( ) . initialize (
server_settings . max_unexpected_parts_loading_thread_pool_size ,
0 , // We don't need any threads once all the parts will be loaded
server_settings . max_unexpected_parts_loading_thread_pool_size ) ;
/// It could grow if we need to synchronously wait until all the data parts will be loaded.
getUnexpectedPartsLoadingThreadPool ( ) . setMaxTurboThreads (
server_settings . max_active_parts_loading_thread_pool_size
) ;
2023-06-06 12:42:56 +00:00
getPartsCleaningThreadPool ( ) . initialize (
server_settings . max_parts_cleaning_thread_pool_size ,
0 , // We don't need any threads one all the parts will be deleted
server_settings . max_parts_cleaning_thread_pool_size ) ;
2023-05-03 16:56:45 +00:00
2024-02-06 18:09:04 +00:00
auto max_database_replicated_create_table_thread_pool_size = server_settings . max_database_replicated_create_table_thread_pool_size ?
server_settings . max_database_replicated_create_table_thread_pool_size : getNumberOfPhysicalCPUCores ( ) ;
2024-01-31 18:58:28 +00:00
getDatabaseReplicatedCreateTablesThreadPool ( ) . initialize (
2024-02-06 18:09:04 +00:00
max_database_replicated_create_table_thread_pool_size ,
2024-01-31 18:58:28 +00:00
0 , // We don't need any threads once all the tables will be created
2024-02-06 18:09:04 +00:00
max_database_replicated_create_table_thread_pool_size ) ;
2024-01-31 18:58:28 +00:00
2021-11-29 08:22:43 +00:00
/// Initialize global local cache for remote filesystem.
if ( config ( ) . has ( " local_cache_for_remote_fs " ) )
{
bool enable = config ( ) . getBool ( " local_cache_for_remote_fs.enable " , false ) ;
if ( enable )
{
String root_dir = config ( ) . getString ( " local_cache_for_remote_fs.root_dir " ) ;
UInt64 limit_size = config ( ) . getUInt64 ( " local_cache_for_remote_fs.limit_size " ) ;
UInt64 bytes_read_before_flush
= config ( ) . getUInt64 ( " local_cache_for_remote_fs.bytes_read_before_flush " , DBMS_DEFAULT_BUFFER_SIZE ) ;
2021-12-23 03:50:26 +00:00
ExternalDataSourceCache : : instance ( ) . initOnce ( global_context , root_dir , limit_size , bytes_read_before_flush ) ;
2021-11-29 08:22:43 +00:00
}
}
2021-11-15 08:47:12 +00:00
2021-10-22 07:15:34 +00:00
/// This object will periodically calculate some metrics.
2022-11-09 12:37:42 +00:00
ServerAsynchronousMetrics async_metrics (
2022-08-29 20:00:26 +00:00
global_context ,
2023-02-18 04:46:09 +00:00
server_settings . asynchronous_metrics_update_period_s ,
server_settings . asynchronous_heavy_metrics_update_period_s ,
2021-10-22 07:15:34 +00:00
[ & ] ( ) - > std : : vector < ProtocolServerMetrics >
{
2023-07-18 14:06:03 +00:00
std : : lock_guard lock ( servers_lock ) ;
2024-05-23 23:14:26 +00:00
std : : vector < ProtocolServerMetrics > metrics1 = servers_to_start_before_tables . getMetrics ( ) ;
std : : vector < ProtocolServerMetrics > metrics2 = servers . getMetrics ( ) ;
metrics1 . reserve ( metrics1 . size ( ) + metrics2 . size ( ) ) ;
metrics1 . insert ( metrics1 . end ( ) , std : : make_move_iterator ( metrics2 . begin ( ) ) , std : : make_move_iterator ( metrics2 . end ( ) ) ) ;
return metrics1 ;
} ) ;
2021-10-22 07:15:34 +00:00
2023-03-23 08:58:56 +00:00
zkutil : : validateZooKeeperConfig ( config ( ) ) ;
bool has_zookeeper = zkutil : : hasZooKeeperConfig ( config ( ) ) ;
2017-04-01 07:20:54 +00:00
2017-03-17 00:44:00 +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-03-17 00:44:00 +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-03-17 00:44:00 +00:00
config ( ) . removeConfiguration ( old_configuration . get ( ) ) ;
config ( ) . add ( loaded_config . configuration . duplicate ( ) , PRIO_DEFAULT , false ) ;
2024-03-13 12:12:48 +00:00
global_context - > setConfig ( loaded_config . configuration ) ;
2017-03-17 00:44:00 +00:00
}
2017-04-01 07:20:54 +00:00
2020-08-05 19:54:06 +00:00
Settings : : checkNoSettingNamesAtTopLevel ( config ( ) , config_path ) ;
2020-06-04 19:30:30 +00:00
2023-02-18 04:46:09 +00:00
/// We need to reload server settings because config could be updated via zookeeper.
server_settings . loadSettingsFromConfig ( config ( ) ) ;
2024-03-13 15:13:17 +00:00
/// NOTE: Do sanity checks after we loaded all possible substitutions (for the configuration) from ZK
2024-03-13 12:12:48 +00:00
sanityChecks ( * this ) ;
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.
2022-06-15 22:07:59 +00:00
/// Please keep the below log messages in-sync with the ones in daemon/BaseDaemon.cpp
2021-01-07 02:56:57 +00:00
if ( stored_binary_hash . empty ( ) )
{
2022-11-14 11:17:21 +00:00
LOG_WARNING ( log , " Integrity check of the executable skipped because the reference checksum could not be read. " ) ;
2021-01-07 02:56:57 +00:00
}
else
{
2022-11-14 11:17:21 +00:00
String calculated_binary_hash = getHashOfLoadedBinaryHex ( ) ;
if ( calculated_binary_hash = = stored_binary_hash )
2021-02-26 17:15:11 +00:00
{
2022-11-14 11:17:21 +00:00
LOG_INFO ( log , " Integrity check of the executable successfully passed (checksum: {}) " , calculated_binary_hash ) ;
2021-02-26 17:15:11 +00:00
}
else
{
2022-11-14 11:17:21 +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.
global_context - > addWarningMessage ( fmt : : format (
" Server is run under debugger and its binary image is modified (most likely with breakpoints). " ,
calculated_binary_hash ) ) ;
}
else
{
throw Exception (
ErrorCodes : : CORRUPTED_DATA ,
" Calculated checksum of the executable ({0}) does not correspond "
" to the reference checksum stored in the executable ({1}). "
" This may indicate one of the following: "
" - the executable {2} was changed just after startup; "
" - the executable {2} was corrupted on disk due to faulty hardware; "
" - the loaded executable was corrupted in memory due to faulty hardware; "
" - the file {2} was intentionally modified; "
" - a logical error in the code. " ,
calculated_binary_hash ,
stored_binary_hash ,
executable_path ) ;
}
2021-02-26 17:15:11 +00:00
}
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. " ) ;
2021-11-03 05:11:36 +00:00
size_t size = remapExecutable ( ) ;
LOG_DEBUG ( log , " The code ({}) in memory has been successfully remapped. " , ReadableSize ( size ) ) ;
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 ) )
2022-08-20 15:09:20 +00:00
LOG_WARNING ( log , " Failed mlock: {} " , errnoToString ( ) ) ;
2020-09-25 02:03:58 +00:00
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
}
}
2023-01-01 22:22:26 +00:00
2023-08-31 11:33:52 +00:00
FailPointInjection : : enableFromGlobalConfig ( config ( ) ) ;
2023-01-01 22:22:26 +00:00
int default_oom_score = 0 ;
# if !defined(NDEBUG)
/// In debug version on Linux, increase oom score so that clickhouse is killed
/// first, instead of some service. Use a carefully chosen random score of 555:
/// the maximum is 1000, and chromium uses 300 for its tab processes. Ignore
/// whatever errors that occur, because it's just a debugging aid and we don't
/// care if it breaks.
default_oom_score = 555 ;
# endif
int oom_score = config ( ) . getInt ( " oom_score " , default_oom_score ) ;
if ( oom_score )
setOOMScore ( oom_score , log ) ;
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 ( ) ) ;
2023-06-15 13:49:49 +00:00
global_context - > setHTTPHeaderFilter ( config ( ) ) ;
2019-10-10 12:58:06 +00:00
2021-08-16 18:30:53 +00:00
std : : string path_str = getCanonicalPath ( config ( ) . getString ( " path " , DBMS_DEFAULT_PATH ) ) ;
fs : : path path = path_str ;
2023-02-18 04:46:09 +00:00
std : : string default_database = server_settings . default_database . toString ( ) ;
2017-04-01 07:20:54 +00:00
2020-08-08 01:52:28 +00:00
/// Check that the process user id matches the owner of the data.
2023-05-19 14:23:56 +00:00
assertProcessUserMatchesDataOwner ( path_str , [ & ] ( const std : : string & message ) { global_context - > addWarningMessage ( message ) ; } ) ;
2018-12-09 22:15:59 +00:00
2021-08-16 18:30:53 +00:00
global_context - > setPath ( path_str ) ;
2017-04-01 07:20:54 +00:00
2021-08-16 18:30:53 +00:00
StatusFile status { path / " status " , StatusFile : : write_full_info } ;
2017-04-01 07:20:54 +00:00
2023-04-09 01:04:26 +00:00
ServerUUID : : load ( path / " uuid " , log ) ;
2017-04-01 07:20:54 +00:00
2016-11-03 19:59:53 +00:00
/// Try to increase limit on number of open files.
2014-04-15 16:39:56 +00:00
{
rlimit rlim ;
if ( getrlimit ( RLIMIT_NOFILE , & rlim ) )
throw Poco : : Exception ( " Cannot getrlimit " ) ;
2017-04-01 07:20:54 +00:00
2014-04-15 16:39:56 +00:00
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 ) ;
2014-04-15 16:39:56 +00:00
}
else
{
rlim_t old = rlim . rlim_cur ;
2022-10-07 10:46:45 +00:00
rlim . rlim_cur = config ( ) . getUInt ( " max_open_files " , static_cast < unsigned > ( rlim . rlim_max ) ) ;
2016-11-03 19:59:53 +00:00
int rc = setrlimit ( RLIMIT_NOFILE , & rlim ) ;
if ( rc ! = 0 )
2022-08-20 15:09:20 +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 , errnoToString ( ) ) ;
2017-01-26 21:46:17 +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 ) ;
2014-04-15 16:39:56 +00:00
}
}
2017-04-01 07:20:54 +00:00
2022-02-13 16:58:38 +00:00
/// Try to increase limit on number of threads.
{
rlimit rlim ;
if ( getrlimit ( RLIMIT_NPROC , & rlim ) )
throw Poco : : Exception ( " Cannot getrlimit " ) ;
if ( rlim . rlim_cur = = rlim . rlim_max )
{
LOG_DEBUG ( log , " rlimit on number of threads is {} " , rlim . rlim_cur ) ;
}
else
{
rlim_t old = rlim . rlim_cur ;
rlim . rlim_cur = rlim . rlim_max ;
int rc = setrlimit ( RLIMIT_NPROC , & rlim ) ;
2022-02-15 10:39:45 +00:00
if ( rc ! = 0 )
{
2022-08-20 15:09:20 +00:00
LOG_WARNING ( log , " Cannot set max number of threads to {}. error: {} " , rlim . rlim_cur , errnoToString ( ) ) ;
2022-02-13 16:58:38 +00:00
rlim . rlim_cur = old ;
}
else
{
LOG_DEBUG ( log , " Set max number of threads to {} (was {}). " , rlim . rlim_cur , old ) ;
}
}
2022-02-15 10:39:45 +00:00
if ( rlim . rlim_cur < 30000 )
{
2022-02-13 16:58:38 +00:00
global_context - > addWarningMessage ( " Maximum number of threads is lower than 30000. There could be problems with handling a lot of simultaneous queries. " ) ;
}
}
2014-12-14 22:51:21 +00:00
static ServerErrorHandler error_handler ;
Poco : : ErrorHandler : : set ( & error_handler ) ;
2017-04-01 07:20:54 +00:00
2016-10-13 20:01:18 +00:00
/// 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. " ) ;
2023-04-12 10:47:05 +00:00
DateLUT : : serverTimezoneInstance ( ) ;
LOG_TRACE ( log , " Initialized DateLUT with time zone '{}'. " , DateLUT : : serverTimezoneInstance ( ) . 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.
2023-02-18 04:46:09 +00:00
if ( ! server_settings . tmp_policy . value . empty ( ) )
2022-12-06 10:04:15 +00:00
{
2023-02-18 04:46:09 +00:00
global_context - > setTemporaryStoragePolicy ( server_settings . tmp_policy , server_settings . max_temporary_data_on_disk_size ) ;
2022-12-06 10:04:15 +00:00
}
2023-02-18 04:46:09 +00:00
else if ( ! server_settings . temporary_data_in_cache . value . empty ( ) )
2022-12-06 10:04:15 +00:00
{
2023-02-18 04:46:09 +00:00
global_context - > setTemporaryStorageInCache ( server_settings . temporary_data_in_cache , server_settings . max_temporary_data_on_disk_size ) ;
2022-12-06 10:04:15 +00:00
}
else
2015-12-08 01:43:39 +00:00
{
2022-10-05 16:35:10 +00:00
std : : string temporary_path = config ( ) . getString ( " tmp_path " , path / " tmp/ " ) ;
2023-02-18 04:46:09 +00:00
global_context - > setTemporaryStoragePath ( temporary_path , server_settings . max_temporary_data_on_disk_size ) ;
2015-12-08 01:43:39 +00:00
}
2017-04-01 07:20:54 +00:00
2016-08-10 00:47:00 +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-08-16 18:30:53 +00:00
auto flags_path = path / " flags/ " ;
2021-05-16 22:06:09 +00:00
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-08-16 18:30:53 +00:00
std : : string user_files_path = config ( ) . getString ( " user_files_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-08-16 18:30:53 +00:00
std : : string dictionaries_lib_path = config ( ) . getString ( " dictionaries_lib_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
}
2021-09-01 23:46:23 +00:00
{
std : : string user_scripts_path = config ( ) . getString ( " user_scripts_path " , path / " user_scripts/ " ) ;
global_context - > setUserScriptsPath ( user_scripts_path ) ;
fs : : create_directories ( user_scripts_path ) ;
}
2020-12-03 21:11:38 +00:00
/// top_level_domains_lists
2020-12-08 20:54:03 +00:00
{
2021-08-16 18:30:53 +00:00
const std : : string & top_level_domains_path = config ( ) . getString ( " top_level_domains_path " , path / " top_level_domains/ " ) ;
2021-05-16 22:06:09 +00:00
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
{
2024-04-27 16:47:03 +00:00
fs : : create_directories ( path / " data " ) ;
fs : : create_directories ( 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
2024-04-27 16:47:03 +00:00
fs : : create_directories ( 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 " ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : EXCESSIVE_ELEMENT_IN_CONFIG , " Both http and https interserver ports are specified " ) ;
2018-07-30 18:32:21 +00:00
static const auto interserver_tags =
2014-04-03 08:47:59 +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 ) )
2015-05-29 00:33:56 +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 )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : ARGUMENT_OUT_OF_BOUND , " Out of range '{}': {} " , String ( port_tag ) , port ) ;
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 ) ;
2015-05-29 00:33:56 +00:00
}
2014-04-03 08:47:59 +00:00
}
2017-04-01 07:20:54 +00:00
2022-09-05 01:50:24 +00:00
LOG_DEBUG ( log , " Initializing interserver credentials. " ) ;
2020-08-26 08:36:58 +00:00
global_context - > updateInterserverCredentials ( config ( ) ) ;
2017-04-01 07:20:54 +00:00
2014-08-11 15:59:01 +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
2023-06-29 13:02:11 +00:00
/// Set up caches.
const size_t max_cache_size = static_cast < size_t > ( physical_server_memory * server_settings . cache_size_to_ram_max_ratio ) ;
String uncompressed_cache_policy = server_settings . uncompressed_cache_policy ;
size_t uncompressed_cache_size = server_settings . uncompressed_cache_size ;
2023-08-21 19:08:34 +00:00
double uncompressed_cache_size_ratio = server_settings . uncompressed_cache_size_ratio ;
2023-06-29 13:02:11 +00:00
if ( uncompressed_cache_size > max_cache_size )
{
uncompressed_cache_size = max_cache_size ;
LOG_INFO ( log , " Lowered uncompressed cache size to {} because the system has limited RAM " , formatReadableSizeWithBinarySuffix ( uncompressed_cache_size ) ) ;
}
2023-08-21 19:08:34 +00:00
global_context - > setUncompressedCache ( uncompressed_cache_policy , uncompressed_cache_size , uncompressed_cache_size_ratio ) ;
2023-06-29 13:02:11 +00:00
String mark_cache_policy = server_settings . mark_cache_policy ;
size_t mark_cache_size = server_settings . mark_cache_size ;
2023-08-21 19:08:34 +00:00
double mark_cache_size_ratio = server_settings . mark_cache_size_ratio ;
2023-06-29 13:02:11 +00:00
if ( mark_cache_size > max_cache_size )
{
mark_cache_size = max_cache_size ;
LOG_INFO ( log , " Lowered mark cache size to {} because the system has limited RAM " , formatReadableSizeWithBinarySuffix ( mark_cache_size ) ) ;
}
2023-08-21 19:08:34 +00:00
global_context - > setMarkCache ( mark_cache_policy , mark_cache_size , mark_cache_size_ratio ) ;
2023-06-29 13:02:11 +00:00
2024-02-29 11:27:32 +00:00
size_t page_cache_size = server_settings . page_cache_size ;
if ( page_cache_size ! = 0 )
global_context - > setPageCache (
server_settings . page_cache_chunk_size , server_settings . page_cache_mmap_size ,
page_cache_size , server_settings . page_cache_use_madv_free ,
server_settings . page_cache_use_transparent_huge_pages ) ;
2023-08-21 19:08:34 +00:00
String index_uncompressed_cache_policy = server_settings . index_uncompressed_cache_policy ;
2023-06-29 13:02:11 +00:00
size_t index_uncompressed_cache_size = server_settings . index_uncompressed_cache_size ;
2023-08-21 19:08:34 +00:00
double index_uncompressed_cache_size_ratio = server_settings . index_uncompressed_cache_size_ratio ;
2023-06-29 13:02:11 +00:00
if ( index_uncompressed_cache_size > max_cache_size )
{
index_uncompressed_cache_size = max_cache_size ;
LOG_INFO ( log , " Lowered index uncompressed cache size to {} because the system has limited RAM " , formatReadableSizeWithBinarySuffix ( uncompressed_cache_size ) ) ;
}
2023-08-21 19:08:34 +00:00
global_context - > setIndexUncompressedCache ( index_uncompressed_cache_policy , index_uncompressed_cache_size , index_uncompressed_cache_size_ratio ) ;
2023-06-29 13:02:11 +00:00
2023-08-21 19:08:34 +00:00
String index_mark_cache_policy = server_settings . index_mark_cache_policy ;
2023-06-29 13:02:11 +00:00
size_t index_mark_cache_size = server_settings . index_mark_cache_size ;
2023-08-21 19:08:34 +00:00
double index_mark_cache_size_ratio = server_settings . index_mark_cache_size_ratio ;
2023-06-29 13:02:11 +00:00
if ( index_mark_cache_size > max_cache_size )
{
index_mark_cache_size = max_cache_size ;
LOG_INFO ( log , " Lowered index mark cache size to {} because the system has limited RAM " , formatReadableSizeWithBinarySuffix ( uncompressed_cache_size ) ) ;
}
2023-08-21 19:08:34 +00:00
global_context - > setIndexMarkCache ( index_mark_cache_policy , index_mark_cache_size , index_mark_cache_size_ratio ) ;
2023-06-29 13:02:11 +00:00
size_t mmap_cache_size = server_settings . mmap_cache_size ;
if ( mmap_cache_size > max_cache_size )
{
mmap_cache_size = max_cache_size ;
LOG_INFO ( log , " Lowered mmap file cache size to {} because the system has limited RAM " , formatReadableSizeWithBinarySuffix ( uncompressed_cache_size ) ) ;
}
global_context - > setMMappedFileCache ( mmap_cache_size ) ;
size_t query_cache_max_size_in_bytes = config ( ) . getUInt64 ( " query_cache.max_size_in_bytes " , DEFAULT_QUERY_CACHE_MAX_SIZE ) ;
size_t query_cache_max_entries = config ( ) . getUInt64 ( " query_cache.max_entries " , DEFAULT_QUERY_CACHE_MAX_ENTRIES ) ;
size_t query_cache_query_cache_max_entry_size_in_bytes = config ( ) . getUInt64 ( " query_cache.max_entry_size_in_bytes " , DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_BYTES ) ;
size_t query_cache_max_entry_size_in_rows = config ( ) . getUInt64 ( " query_cache.max_entry_rows_in_rows " , DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_ROWS ) ;
if ( query_cache_max_size_in_bytes > max_cache_size )
{
query_cache_max_size_in_bytes = max_cache_size ;
LOG_INFO ( log , " Lowered query cache size to {} because the system has limited RAM " , formatReadableSizeWithBinarySuffix ( uncompressed_cache_size ) ) ;
}
global_context - > setQueryCache ( query_cache_max_size_in_bytes , query_cache_max_entries , query_cache_query_cache_max_entry_size_in_bytes , query_cache_max_entry_size_in_rows ) ;
# if USE_EMBEDDED_COMPILER
size_t compiled_expression_cache_max_size_in_bytes = config ( ) . getUInt64 ( " compiled_expression_cache_size " , DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE ) ;
size_t compiled_expression_cache_max_elements = config ( ) . getUInt64 ( " compiled_expression_cache_elements_size " , DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_ENTRIES ) ;
CompiledExpressionCacheFactory : : instance ( ) . init ( compiled_expression_cache_max_size_in_bytes , compiled_expression_cache_max_elements ) ;
# endif
2023-11-23 12:22:06 +00:00
NamedCollectionUtils : : loadIfNot ( ) ;
2017-03-17 00:44:00 +00:00
/// Initialize main config reloader.
2016-10-25 12:27:09 +00:00
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
}
2024-02-01 21:09:39 +00:00
std : : optional < CgroupsMemoryUsageObserver > cgroups_memory_usage_observer ;
try
{
2024-03-13 16:54:20 +00:00
auto wait_time = server_settings . cgroups_memory_usage_observer_wait_time ;
2024-02-01 21:09:39 +00:00
if ( wait_time ! = 0 )
cgroups_memory_usage_observer . emplace ( std : : chrono : : seconds ( wait_time ) ) ;
}
catch ( Exception & )
{
tryLogCurrentException ( log , " Disabling cgroup memory observer because of an error during initialization " ) ;
}
2023-07-11 10:04:43 +00:00
const std : : string cert_path = config ( ) . getString ( " openSSL.server.certificateFile " , " " ) ;
const std : : string key_path = config ( ) . getString ( " openSSL.server.privateKeyFile " , " " ) ;
std : : vector < std : : string > extra_paths = { include_from_path } ;
2023-07-23 02:56:47 +00:00
if ( ! cert_path . empty ( ) )
extra_paths . emplace_back ( cert_path ) ;
if ( ! key_path . empty ( ) )
extra_paths . emplace_back ( key_path ) ;
2023-07-11 10:04:43 +00:00
2020-06-04 19:30:30 +00:00
auto main_config_reloader = std : : make_unique < ConfigReloader > (
config_path ,
2023-07-11 10:04:43 +00:00
extra_paths ,
2024-02-21 19:33:39 +00:00
config ( ) . getString ( " path " , DBMS_DEFAULT_PATH ) ,
2017-03-21 19:08:09 +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
2024-01-04 16:21:46 +00:00
ServerSettings new_server_settings ;
new_server_settings . loadSettingsFromConfig ( * config ) ;
2023-02-18 04:46:09 +00:00
2024-01-04 16:21:46 +00:00
size_t max_server_memory_usage = new_server_settings . max_server_memory_usage ;
double max_server_memory_usage_to_ram_ratio = new_server_settings . max_server_memory_usage_to_ram_ratio ;
2023-08-15 21:04:40 +00:00
size_t current_physical_server_memory = getMemoryAmount ( ) ; /// With cgroups, the amount of memory available to the server can be changed dynamically.
size_t default_max_server_memory_usage = static_cast < size_t > ( current_physical_server_memory * max_server_memory_usage_to_ram_ratio ) ;
2021-01-16 15:33:34 +00:00
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 ) ,
2023-08-15 21:04:40 +00:00
formatReadableSizeWithBinarySuffix ( current_physical_server_memory ) ,
2021-01-16 15:33:34 +00:00
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 ) ,
2023-08-15 21:04:40 +00:00
formatReadableSizeWithBinarySuffix ( current_physical_server_memory ) ,
2021-01-16 15:33:34 +00:00
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 ) ;
2021-10-22 15:15:33 +00:00
2024-02-01 21:09:39 +00:00
if ( cgroups_memory_usage_observer )
{
double hard_limit_ratio = new_server_settings . cgroup_memory_watcher_hard_limit_ratio ;
double soft_limit_ratio = new_server_settings . cgroup_memory_watcher_soft_limit_ratio ;
2024-03-13 16:54:20 +00:00
cgroups_memory_usage_observer - > setMemoryUsageLimits (
2024-02-01 21:09:39 +00:00
static_cast < uint64_t > ( max_server_memory_usage * hard_limit_ratio ) ,
static_cast < uint64_t > ( max_server_memory_usage * soft_limit_ratio ) ) ;
}
2024-01-04 16:21:46 +00:00
size_t merges_mutations_memory_usage_soft_limit = new_server_settings . merges_mutations_memory_usage_soft_limit ;
2023-04-14 14:34:19 +00:00
2024-01-04 16:21:46 +00:00
size_t default_merges_mutations_server_memory_usage = static_cast < size_t > ( current_physical_server_memory * new_server_settings . merges_mutations_memory_usage_to_ram_ratio ) ;
2023-07-06 01:04:58 +00:00
if ( merges_mutations_memory_usage_soft_limit = = 0 )
{
merges_mutations_memory_usage_soft_limit = default_merges_mutations_server_memory_usage ;
LOG_INFO ( log , " Setting merges_mutations_memory_usage_soft_limit was set to {} "
" ({} available * {:.2f} merges_mutations_memory_usage_to_ram_ratio) " ,
formatReadableSizeWithBinarySuffix ( merges_mutations_memory_usage_soft_limit ) ,
2023-08-15 21:04:40 +00:00
formatReadableSizeWithBinarySuffix ( current_physical_server_memory ) ,
2024-01-04 16:21:46 +00:00
new_server_settings . merges_mutations_memory_usage_to_ram_ratio ) ;
2023-07-06 01:04:58 +00:00
}
else if ( merges_mutations_memory_usage_soft_limit > default_merges_mutations_server_memory_usage )
2023-04-14 14:34:19 +00:00
{
merges_mutations_memory_usage_soft_limit = default_merges_mutations_server_memory_usage ;
LOG_WARNING ( log , " Setting merges_mutations_memory_usage_soft_limit was set to {} "
" ({} available * {:.2f} merges_mutations_memory_usage_to_ram_ratio) " ,
formatReadableSizeWithBinarySuffix ( merges_mutations_memory_usage_soft_limit ) ,
2023-08-15 21:04:40 +00:00
formatReadableSizeWithBinarySuffix ( current_physical_server_memory ) ,
2024-01-04 16:21:46 +00:00
new_server_settings . merges_mutations_memory_usage_to_ram_ratio ) ;
2023-04-14 14:34:19 +00:00
}
LOG_INFO ( log , " Merges and mutations memory limit is set to {} " ,
formatReadableSizeWithBinarySuffix ( merges_mutations_memory_usage_soft_limit ) ) ;
background_memory_tracker . setSoftLimit ( merges_mutations_memory_usage_soft_limit ) ;
background_memory_tracker . setDescription ( " (background) " ) ;
background_memory_tracker . setMetric ( CurrentMetrics : : MergesMutationsMemoryTracking ) ;
2024-01-04 16:21:46 +00:00
total_memory_tracker . setAllowUseJemallocMemory ( new_server_settings . allow_use_jemalloc_memory ) ;
2022-11-21 16:10:47 +00:00
2021-10-22 15:15:33 +00:00
auto * global_overcommit_tracker = global_context - > getGlobalOvercommitTracker ( ) ;
total_memory_tracker . setOvercommitTracker ( global_overcommit_tracker ) ;
2021-01-16 15:33:34 +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());
2021-09-30 19:46:12 +00:00
updateLevels ( * config , logger ( ) ) ;
2021-11-16 13:01:57 +00:00
global_context - > setClustersConfig ( config , has_zookeeper ) ;
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-09-29 12:52:58 +00:00
2023-05-05 13:25:18 +00:00
if ( global_context - > isServerCompletelyStarted ( ) )
{
/// It does not make sense to reload anything before server has started.
/// Moreover, it may break initialization order.
global_context - > loadOrReloadDictionaries ( * config ) ;
global_context - > loadOrReloadUserDefinedExecutableFunctions ( * config ) ;
}
2019-11-14 13:44:15 +00:00
2022-03-15 13:00:31 +00:00
global_context - > setRemoteHostFilter ( * config ) ;
2023-06-15 13:49:49 +00:00
global_context - > setHTTPHeaderFilter ( * config ) ;
2022-03-15 13:00:31 +00:00
2024-01-04 16:21:46 +00:00
global_context - > setMaxTableSizeToDrop ( new_server_settings . max_table_size_to_drop ) ;
global_context - > setMaxPartitionSizeToDrop ( new_server_settings . max_partition_size_to_drop ) ;
global_context - > setMaxTableNumToWarn ( new_server_settings . max_table_num_to_warn ) ;
2024-05-21 09:29:07 +00:00
global_context - > setMaxViewNumToWarn ( new_server_settings . max_view_num_to_warn ) ;
global_context - > setMaxDictionaryNumToWarn ( new_server_settings . max_dictionary_num_to_warn ) ;
2024-01-04 16:21:46 +00:00
global_context - > setMaxDatabaseNumToWarn ( new_server_settings . max_database_num_to_warn ) ;
global_context - > setMaxPartNumToWarn ( new_server_settings . max_part_num_to_warn ) ;
2020-01-09 14:50:34 +00:00
2024-01-28 20:26:55 +00:00
SlotCount concurrent_threads_soft_limit = UnlimitedSlots ;
2024-01-04 16:21:46 +00:00
if ( new_server_settings . concurrent_threads_soft_limit_num > 0 & & new_server_settings . concurrent_threads_soft_limit_num < concurrent_threads_soft_limit )
concurrent_threads_soft_limit = new_server_settings . concurrent_threads_soft_limit_num ;
if ( new_server_settings . concurrent_threads_soft_limit_ratio_to_cores > 0 )
2022-05-17 16:35:40 +00:00
{
2024-01-04 16:21:46 +00:00
auto value = new_server_settings . concurrent_threads_soft_limit_ratio_to_cores * std : : thread : : hardware_concurrency ( ) ;
2022-08-18 17:30:33 +00:00
if ( value > 0 & & value < concurrent_threads_soft_limit )
concurrent_threads_soft_limit = value ;
2022-08-18 12:51:08 +00:00
}
ConcurrencyControl : : instance ( ) . setMaxConcurrency ( concurrent_threads_soft_limit ) ;
2022-04-22 14:17:29 +00:00
2024-01-04 16:21:46 +00:00
global_context - > getProcessList ( ) . setMaxSize ( new_server_settings . max_concurrent_queries ) ;
global_context - > getProcessList ( ) . setMaxInsertQueriesAmount ( new_server_settings . max_concurrent_insert_queries ) ;
global_context - > getProcessList ( ) . setMaxSelectQueriesAmount ( new_server_settings . max_concurrent_select_queries ) ;
2024-03-07 17:05:54 +00:00
global_context - > getProcessList ( ) . setMaxWaitingQueriesAmount ( new_server_settings . max_waiting_queries ) ;
2021-12-12 08:07:06 +00:00
2021-10-18 15:27:51 +00:00
if ( config - > has ( " keeper_server " ) )
global_context - > updateKeeperConfiguration ( * config ) ;
2022-04-19 15:01:41 +00:00
/// Reload the number of threads for global pools.
/// Note: If you specified it in the top level config (not it config of default profile)
/// then ClickHouse will use it exactly.
/// This is done for backward compatibility.
2023-02-22 20:15:09 +00:00
if ( global_context - > areBackgroundExecutorsInitialized ( ) )
2022-04-19 15:01:41 +00:00
{
2024-01-04 16:21:46 +00:00
auto new_pool_size = new_server_settings . background_pool_size ;
auto new_ratio = new_server_settings . background_merges_mutations_concurrency_ratio ;
Refactor reading the pool setting & from server config. (#48055)
After #36425 there was a lot of confusions/problems with configuring pools - when the message was confusing, and settings need to be ajusted in several places.
See some examples in #44251, #43351, #47900, #46515.
The commit includes the following changes:
1) Introduced a unified mechanism for reading pool sizes from the configuration file(s). Previously, pool sizes were read from the Context.cpp with fallbacks to profiles, whereas main_config_reloader in Server.cpp read them directly without fallbacks.
2) Corrected the data type for background_merges_mutations_concurrency_ratio. It should be float instead of int.
3) Refactored the default values for settings. Previously, they were defined in multiple places throughout the codebase, but they are now defined in one place (or two, to be exact: Settings.h and ServerSettings.h).
4) Improved documentation, including the correct message in system.settings.
Additionally make the code more conform with #46550.
2023-03-30 14:44:11 +00:00
global_context - > getMergeMutateExecutor ( ) - > increaseThreadsAndMaxTasksCount ( new_pool_size , static_cast < size_t > ( new_pool_size * new_ratio ) ) ;
2024-01-04 16:21:46 +00:00
global_context - > getMergeMutateExecutor ( ) - > updateSchedulingPolicy ( new_server_settings . background_merges_mutations_scheduling_policy . toString ( ) ) ;
2022-04-19 15:01:41 +00:00
}
2023-02-22 20:15:09 +00:00
if ( global_context - > areBackgroundExecutorsInitialized ( ) )
2022-04-19 15:01:41 +00:00
{
2024-01-04 16:21:46 +00:00
auto new_pool_size = new_server_settings . background_move_pool_size ;
2022-04-19 15:01:41 +00:00
global_context - > getMovesExecutor ( ) - > increaseThreadsAndMaxTasksCount ( new_pool_size , new_pool_size ) ;
}
2023-02-22 20:15:09 +00:00
if ( global_context - > areBackgroundExecutorsInitialized ( ) )
2022-04-19 15:01:41 +00:00
{
2024-01-04 16:21:46 +00:00
auto new_pool_size = new_server_settings . background_fetches_pool_size ;
2022-04-19 15:01:41 +00:00
global_context - > getFetchesExecutor ( ) - > increaseThreadsAndMaxTasksCount ( new_pool_size , new_pool_size ) ;
}
2023-02-22 20:15:09 +00:00
if ( global_context - > areBackgroundExecutorsInitialized ( ) )
2022-04-19 15:01:41 +00:00
{
2024-01-04 16:21:46 +00:00
auto new_pool_size = new_server_settings . background_common_pool_size ;
2022-04-19 15:01:41 +00:00
global_context - > getCommonExecutor ( ) - > increaseThreadsAndMaxTasksCount ( new_pool_size , new_pool_size ) ;
}
2024-01-04 16:21:46 +00:00
global_context - > getBufferFlushSchedulePool ( ) . increaseThreadsCount ( new_server_settings . background_buffer_flush_schedule_pool_size ) ;
global_context - > getSchedulePool ( ) . increaseThreadsCount ( new_server_settings . background_schedule_pool_size ) ;
global_context - > getMessageBrokerSchedulePool ( ) . increaseThreadsCount ( new_server_settings . background_message_broker_schedule_pool_size ) ;
global_context - > getDistributedSchedulePool ( ) . increaseThreadsCount ( new_server_settings . background_distributed_schedule_pool_size ) ;
2022-04-20 13:35:13 +00:00
2024-01-04 16:21:46 +00:00
global_context - > getAsyncLoader ( ) . setMaxThreads ( TablesLoaderForegroundPoolId , new_server_settings . tables_loader_foreground_pool_size ) ;
global_context - > getAsyncLoader ( ) . setMaxThreads ( TablesLoaderBackgroundLoadPoolId , new_server_settings . tables_loader_background_pool_size ) ;
global_context - > getAsyncLoader ( ) . setMaxThreads ( TablesLoaderBackgroundStartupPoolId , new_server_settings . tables_loader_background_pool_size ) ;
2023-04-26 18:27:23 +00:00
2023-06-06 12:42:56 +00:00
getIOThreadPool ( ) . reloadConfiguration (
2024-01-04 16:21:46 +00:00
new_server_settings . max_io_thread_pool_size ,
new_server_settings . max_io_thread_pool_free_size ,
new_server_settings . io_thread_pool_queue_size ) ;
2023-06-06 12:42:56 +00:00
getBackupsIOThreadPool ( ) . reloadConfiguration (
2024-01-04 16:21:46 +00:00
new_server_settings . max_backups_io_thread_pool_size ,
new_server_settings . max_backups_io_thread_pool_free_size ,
new_server_settings . backups_io_thread_pool_queue_size ) ;
2023-06-06 12:42:56 +00:00
getActivePartsLoadingThreadPool ( ) . reloadConfiguration (
2024-01-04 16:21:46 +00:00
new_server_settings . max_active_parts_loading_thread_pool_size ,
2023-06-06 12:42:56 +00:00
0 , // We don't need any threads once all the parts will be loaded
2024-01-04 16:21:46 +00:00
new_server_settings . max_active_parts_loading_thread_pool_size ) ;
2023-06-06 12:42:56 +00:00
getOutdatedPartsLoadingThreadPool ( ) . reloadConfiguration (
2024-01-04 16:21:46 +00:00
new_server_settings . max_outdated_parts_loading_thread_pool_size ,
2023-06-06 12:42:56 +00:00
0 , // We don't need any threads once all the parts will be loaded
2024-01-04 16:21:46 +00:00
new_server_settings . max_outdated_parts_loading_thread_pool_size ) ;
2023-06-06 12:42:56 +00:00
/// It could grow if we need to synchronously wait until all the data parts will be loaded.
getOutdatedPartsLoadingThreadPool ( ) . setMaxTurboThreads (
2024-01-04 16:21:46 +00:00
new_server_settings . max_active_parts_loading_thread_pool_size
2023-06-06 12:42:56 +00:00
) ;
getPartsCleaningThreadPool ( ) . reloadConfiguration (
2024-01-04 16:21:46 +00:00
new_server_settings . max_parts_cleaning_thread_pool_size ,
2023-06-06 12:42:56 +00:00
0 , // We don't need any threads one all the parts will be deleted
2024-01-04 16:21:46 +00:00
new_server_settings . max_parts_cleaning_thread_pool_size ) ;
2023-06-06 12:42:56 +00:00
2022-09-27 13:26:41 +00:00
if ( config - > has ( " resources " ) )
{
global_context - > getResourceManager ( ) - > updateConfiguration ( * config ) ;
}
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.
2023-03-23 12:58:39 +00:00
if ( zkutil : : hasZooKeeperConfig ( * config ) )
2021-02-18 14:13:23 +00:00
global_context - > reloadZooKeeperIfChanged ( config ) ;
2020-09-10 04:00:33 +00:00
2021-02-18 14:13:23 +00:00
global_context - > reloadAuxiliaryZooKeepersConfigIfChanged ( config ) ;
2021-10-22 07:15:34 +00:00
2023-12-18 10:24:50 +00:00
global_context - > reloadQueryMaskingRulesIfChanged ( config ) ;
2024-04-30 12:32:03 +00:00
if ( global_context - > isServerCompletelyStarted ( ) )
{
std : : lock_guard lock ( servers_lock ) ;
2024-05-23 23:14:26 +00:00
servers . updateServers ( * config , * this , servers_lock , server_pool , async_metrics , latest_config ) ;
servers_to_start_before_tables . updateServers ( * config , * this , servers_lock , server_pool , async_metrics , latest_config ) ;
2024-04-30 12:32:03 +00:00
}
2021-02-18 14:13:23 +00:00
}
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 ) ;
2023-06-29 13:02:11 +00:00
global_context - > updateUncompressedCacheConfiguration ( * config ) ;
global_context - > updateMarkCacheConfiguration ( * config ) ;
global_context - > updateIndexUncompressedCacheConfiguration ( * config ) ;
global_context - > updateIndexMarkCacheConfiguration ( * config ) ;
global_context - > updateMMappedFileCacheConfiguration ( * config ) ;
2023-01-29 20:43:52 +00:00
global_context - > updateQueryCacheConfiguration ( * config ) ;
2023-06-29 13:02:11 +00:00
2021-09-13 08:25:36 +00:00
CompressionCodecEncrypted : : Configuration : : instance ( ) . tryLoad ( * config , " encryption_codecs " ) ;
2021-07-10 02:57:44 +00:00
# if USE_SSL
2021-11-10 16:56:07 +00:00
CertificateReloader : : instance ( ) . tryLoad ( * config ) ;
2021-07-10 02:57:44 +00:00
# endif
2022-11-15 14:49:25 +00:00
NamedCollectionUtils : : reloadFromConfig ( * config ) ;
2023-12-07 17:30:43 +00:00
FileCacheFactory : : instance ( ) . updateSettingsFromConfig ( * config ) ;
2024-03-03 13:22:40 +00:00
HTTPConnectionPools : : instance ( ) . setLimits (
HTTPConnectionPools : : Limits {
new_server_settings . disk_connections_soft_limit ,
new_server_settings . disk_connections_warn_limit ,
new_server_settings . disk_connections_store_limit ,
} ,
HTTPConnectionPools : : Limits {
new_server_settings . storage_connections_soft_limit ,
new_server_settings . storage_connections_warn_limit ,
new_server_settings . storage_connections_store_limit ,
} ,
HTTPConnectionPools : : Limits {
new_server_settings . http_connections_soft_limit ,
new_server_settings . http_connections_warn_limit ,
new_server_settings . http_connections_store_limit ,
} ) ;
2024-04-03 18:57:12 +00:00
if ( global_context - > isServerCompletelyStarted ( ) )
CannotAllocateThreadFaultInjector : : setFaultProbability ( new_server_settings . cannot_allocate_thread_fault_injection_probability ) ;
2024-04-03 17:49:23 +00:00
2021-10-22 07:15:34 +00:00
ProfileEvents : : increment ( ProfileEvents : : MainConfigLoads ) ;
2022-09-09 20:00:27 +00:00
/// Must be the last.
latest_config = config ;
2018-02-28 20:34:25 +00:00
} ,
2024-05-23 23:14:26 +00:00
/* already_loaded = */ false ) ; /// Reload it right now (initial loading)
servers_to_start_before_tables . createServers (
config ( ) ,
* this ,
servers_lock ,
server_pool ,
async_metrics ,
/* start_servers= */ false ,
ServerType ( ServerType : : Type : : QUERIES_ALL )
) ;
2022-01-25 13:40:12 +00:00
2020-08-12 11:43:58 +00:00
/// Initialize access storages.
2022-05-06 23:37:23 +00:00
auto & access_control = global_context - > getAccessControl ( ) ;
2021-05-21 08:25:43 +00:00
try
{
2022-05-06 23:37:23 +00:00
access_control . setUpFromMainConfig ( config ( ) , config_path , [ & ] { return global_context - > getZooKeeper ( ) ; } ) ;
2021-05-21 08:25:43 +00:00
}
catch ( . . . )
{
2022-12-06 10:04:15 +00:00
tryLogCurrentException ( log , " Caught exception while setting up access control. " ) ;
2021-05-21 08:25:43 +00:00
throw ;
}
2017-04-01 07:20:54 +00:00
2024-03-08 16:30:58 +00:00
if ( cgroups_memory_usage_observer )
2024-03-09 01:28:39 +00:00
{
2024-03-18 10:44:56 +00:00
cgroups_memory_usage_observer - > setOnMemoryAmountAvailableChangedFn ( [ & ] ( ) { main_config_reloader - > reload ( ) ; } ) ;
2024-03-12 22:31:33 +00:00
cgroups_memory_usage_observer - > startThread ( ) ;
2024-03-09 01:28:39 +00:00
}
2024-03-08 16:30:58 +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 ( ) ;
2022-09-16 11:19:39 +00:00
access_control . reload ( AccessControl : : ReloadMode : : USERS_CONFIG_ONLY ) ;
2018-03-13 10:41:47 +00:00
} ) ;
2023-07-26 17:58:41 +00:00
global_context - > setStopServersCallback ( [ & ] ( const ServerType & server_type )
{
2023-09-22 20:20:21 +00:00
std : : lock_guard lock ( servers_lock ) ;
2024-05-23 23:14:26 +00:00
servers . stopServers ( server_type ) ;
2023-07-26 17:58:41 +00:00
} ) ;
global_context - > setStartServersCallback ( [ & ] ( const ServerType & server_type )
{
2023-09-22 20:20:21 +00:00
std : : lock_guard lock ( servers_lock ) ;
2024-05-23 23:14:26 +00:00
servers . createServers (
2023-07-26 17:58:41 +00:00
config ( ) ,
2024-05-23 23:14:26 +00:00
* this ,
servers_lock ,
2023-07-26 17:58:41 +00:00
server_pool ,
async_metrics ,
/* start_servers= */ true ,
server_type ) ;
} ) ;
2017-08-09 15:34:09 +00:00
/// Limit on total number of concurrently executed queries.
2023-02-18 04:46:09 +00:00
global_context - > getProcessList ( ) . setMaxSize ( server_settings . max_concurrent_queries ) ;
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 ( ) ) ;
2017-04-01 07:20:54 +00:00
2021-09-30 21:26:24 +00:00
/// Initialize background executors after we load default_profile config.
/// This is needed to load proper values of background_pool_size etc.
2021-10-19 08:19:43 +00:00
global_context - > initializeBackgroundExecutorsIfNeeded ( ) ;
2021-09-30 21:26:24 +00:00
2023-06-07 11:49:41 +00:00
if ( server_settings . async_insert_threads )
2023-04-25 21:30:03 +00:00
{
2021-04-21 13:16:16 +00:00
global_context - > setAsynchronousInsertQueue ( std : : make_shared < AsynchronousInsertQueue > (
2021-08-27 03:00:12 +00:00
global_context ,
2023-06-07 11:49:41 +00:00
server_settings . async_insert_threads ,
server_settings . async_insert_queue_flush_on_shutdown ) ) ;
2023-04-25 21:30:03 +00:00
}
2021-04-19 19:16:34 +00:00
2017-11-10 06:48:28 +00:00
/// Set path for format schema files
2021-08-16 18:30:53 +00:00
fs : : path format_schema_path ( config ( ) . getString ( " format_schema_path " , path / " format_schemas/ " ) ) ;
2021-05-16 22:06:09 +00:00
global_context - > setFormatSchemaPath ( format_schema_path ) ;
fs : : create_directories ( format_schema_path ) ;
2017-11-10 06:48:28 +00:00
2023-11-15 13:55:43 +00:00
/// Set the path for google proto files
if ( config ( ) . has ( " google_protos_path " ) )
global_context - > setGoogleProtosPath ( fs : : weakly_canonical ( config ( ) . getString ( " google_protos_path " ) ) ) ;
2023-08-07 15:38:30 +00:00
/// Set path for filesystem caches
fs : : path filesystem_caches_path ( config ( ) . getString ( " filesystem_caches_path " , " " ) ) ;
if ( ! filesystem_caches_path . empty ( ) )
global_context - > setFilesystemCachesPath ( filesystem_caches_path ) ;
2020-07-30 19:08:13 +00:00
/// Check sanity of MergeTreeSettings on server startup
2022-05-26 12:14:58 +00:00
{
size_t background_pool_tasks = global_context - > getMergeMutateExecutor ( ) - > getMaxTasksCount ( ) ;
global_context - > getMergeTreeSettings ( ) . sanityCheck ( background_pool_tasks ) ;
global_context - > getReplicatedMergeTreeSettings ( ) . sanityCheck ( background_pool_tasks ) ;
}
2021-09-13 08:25:36 +00:00
/// try set up encryption. There are some errors in config, error will be printed and server wouldn't start.
CompressionCodecEncrypted : : Configuration : : instance ( ) . load ( config ( ) , " encryption_codecs " ) ;
2021-07-30 09:12:33 +00:00
2022-09-22 11:02:45 +00:00
/// DNSCacheUpdater uses BackgroundSchedulePool which lives in shared context
/// and thus this object must be created after the SCOPE_EXIT object where shared
/// context is destroyed.
/// In addition this object has to be created before the loading of the tables.
std : : unique_ptr < DNSCacheUpdater > dns_cache_updater ;
2023-02-18 04:46:09 +00:00
if ( server_settings . disable_internal_dns_cache )
2022-09-22 11:02:45 +00:00
{
/// Disable DNS caching at all
DNSResolver : : instance ( ) . setDisableCacheFlag ( ) ;
LOG_DEBUG ( log , " DNS caching disabled " ) ;
}
else
{
2024-02-28 23:32:13 +00:00
DNSResolver : : instance ( ) . setCacheMaxEntries ( server_settings . dns_cache_max_entries ) ;
2024-02-26 20:48:02 +00:00
2022-09-22 11:02:45 +00:00
/// Initialize a watcher periodically updating DNS cache
dns_cache_updater = std : : make_unique < DNSCacheUpdater > (
2023-02-18 04:46:09 +00:00
global_context , server_settings . dns_cache_update_period , server_settings . dns_max_consecutive_failures ) ;
2022-09-22 11:02:45 +00:00
}
if ( dns_cache_updater )
dns_cache_updater - > start ( ) ;
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 ) ;
2021-08-16 18:30:53 +00:00
LOG_INFO ( log , " Loading metadata from {} " , path_str ) ;
2019-07-30 14:04:18 +00:00
2023-10-24 21:49:15 +00:00
LoadTaskPtrs load_metadata_tasks ;
2024-01-22 19:38:26 +00:00
// Make sure that if exception is thrown during startup async, new async loading jobs are not going to be called.
// This is important for the case when exception is thrown from loading of metadata with `async_load_databases = false`
// to avoid simultaneously running table startups and destructing databases.
SCOPE_EXIT_SAFE (
LOG_INFO ( log , " Stopping AsyncLoader. " ) ;
2024-01-22 23:50:53 +00:00
// Waits for all currently running jobs to finish and do not run any other pending jobs.
global_context - > getAsyncLoader ( ) . stop ( ) ;
2024-01-22 19:38:26 +00:00
) ;
2018-12-10 23:21:03 +00:00
try
{
2021-09-03 19:21:01 +00:00
auto & database_catalog = DatabaseCatalog : : instance ( ) ;
/// We load temporary database first, because projections need it.
database_catalog . initializeAndLoadTemporaryDatabase ( ) ;
2023-05-12 19:49:47 +00:00
auto system_startup_tasks = loadMetadataSystem ( global_context ) ;
maybeConvertSystemDatabase ( global_context , system_startup_tasks ) ;
2023-08-07 16:57:30 +00:00
/// This has to be done before the initialization of system logs,
/// otherwise there is a race condition between the system database initialization
/// and creation of new tables in the database.
2023-11-06 13:08:27 +00:00
waitLoad ( TablesLoaderForegroundPoolId , system_startup_tasks ) ;
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 ( ) ;
2023-06-24 11:13:49 +00:00
/// Build loggers before tables startup to make log messages from tables
/// attach available in system.text_log
2023-07-12 15:48:09 +00:00
buildLoggers ( config ( ) , logger ( ) ) ;
2024-04-12 15:38:50 +00:00
initializeAzureSDKLogger ( server_settings , logger ( ) . getLevel ( ) ) ;
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)
2021-11-10 13:13:27 +00:00
attachSystemTablesServer ( global_context , * database_catalog . getSystemDatabase ( ) , has_zookeeper ) ;
2021-09-09 09:37:51 +00:00
attachInformationSchema ( global_context , * database_catalog . getDatabase ( DatabaseCatalog : : INFORMATION_SCHEMA ) ) ;
attachInformationSchema ( global_context , * database_catalog . getDatabase ( DatabaseCatalog : : INFORMATION_SCHEMA_UPPERCASE ) ) ;
2021-09-02 18:05:13 +00:00
/// Firstly remove partially dropped databases, to avoid race with MaterializedMySQLSyncThread,
/// that may execute DROP before loadMarkedAsDroppedTables() in background,
/// and so loadMarkedAsDroppedTables() will find it and try to add, and UUID will overlap.
database_catalog . loadMarkedAsDroppedTables ( ) ;
2023-02-17 16:57:49 +00:00
database_catalog . createBackgroundTasks ( ) ;
2023-05-01 11:56:00 +00:00
/// Then, load remaining databases (some of them maybe be loaded asynchronously)
2023-10-24 10:09:10 +00:00
load_metadata_tasks = loadMetadata ( global_context , default_database , server_settings . async_load_databases ) ;
2023-05-01 11:56:00 +00:00
/// If we need to convert database engines, disable async tables loading
2023-10-24 10:09:10 +00:00
convertDatabasesEnginesIfNeed ( load_metadata_tasks , global_context ) ;
2023-09-12 06:00:40 +00:00
database_catalog . startupBackgroundTasks ( ) ;
2020-09-25 11:27:00 +00:00
/// After loading validate that default database exists
database_catalog . assertDatabaseExists ( default_database ) ;
2022-09-24 21:24:39 +00:00
/// Load user-defined SQL functions.
2023-12-12 14:03:49 +00:00
global_context - > getUserDefinedSQLObjectsStorage ( ) . loadObjects ( ) ;
2018-12-10 23:21:03 +00:00
}
catch ( . . . )
{
tryLogCurrentException ( log , " Caught exception while loading metadata " ) ;
throw ;
}
2023-05-12 17:54:51 +00:00
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.
2023-05-03 08:09:11 +00:00
# if !WITH_COVERAGE
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).
2023-07-30 12:01:34 +00:00
if ( server_settings . total_memory_profiler_step )
2020-04-22 17:52:21 +00:00
{
2023-07-30 12:01:34 +00:00
total_memory_tracker . setProfilerStep ( server_settings . total_memory_profiler_step ) ;
2020-04-22 17:52:21 +00:00
}
2020-04-30 13:25:17 +00:00
2023-07-30 12:01:34 +00:00
if ( server_settings . total_memory_tracker_sample_probability > 0.0 )
2020-04-30 13:25:17 +00:00
{
2023-07-30 12:01:34 +00:00
total_memory_tracker . setSampleProbability ( server_settings . total_memory_tracker_sample_probability ) ;
2020-04-30 13:25:17 +00:00
}
2023-07-30 12:01:34 +00:00
if ( server_settings . total_memory_profiler_sample_min_allocation_size )
2020-04-22 17:52:21 +00:00
{
2023-07-30 12:01:34 +00:00
total_memory_tracker . setSampleMinAllocationSize ( server_settings . total_memory_profiler_sample_min_allocation_size ) ;
2020-04-22 17:52:21 +00:00
}
2020-04-30 13:25:17 +00:00
2023-07-30 12:01:34 +00:00
if ( server_settings . total_memory_profiler_sample_max_allocation_size )
2020-04-30 13:25:17 +00:00
{
2023-07-30 12:01:34 +00:00
total_memory_tracker . setSampleMaxAllocationSize ( server_settings . total_memory_profiler_sample_max_allocation_size ) ;
2020-04-30 13:25:17 +00:00
}
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 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)
2021-10-11 07:03:35 +00:00
LOG_INFO ( log , " Query Profiler 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
2020-04-16 12:31:57 +00:00
# if defined(OS_LINUX)
2022-04-17 12:01:56 +00:00
auto tasks_stats_provider = TasksStatsCounters : : findBestAvailableProvider ( ) ;
if ( tasks_stats_provider = = TasksStatsCounters : : MetricsProvider : : None )
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
}
2022-04-17 12:01:56 +00:00
else
{
LOG_INFO ( log , " Tasks stats provider: {} " , TasksStatsCounters : : metricsProviderString ( tasks_stats_provider ) ) ;
}
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
2013-09-14 05:14:22 +00:00
{
2021-11-10 13:13:27 +00:00
attachSystemTablesAsync ( global_context , * DatabaseCatalog : : instance ( ) . getSystemDatabase ( ) , async_metrics ) ;
2019-11-26 20:27:24 +00:00
2017-03-03 00:24:56 +00:00
{
2021-10-22 07:15:34 +00:00
std : : lock_guard lock ( servers_lock ) ;
2024-05-23 23:14:26 +00:00
servers . createServers (
config ( ) ,
* this ,
servers_lock ,
server_pool ,
async_metrics ,
false ,
ServerType ( ServerType : : Type : : QUERIES_ALL ) ) ;
2021-10-22 07:15:34 +00:00
if ( servers . empty ( ) )
2024-05-23 23:14:26 +00:00
throw Exception (
ErrorCodes : : NO_ELEMENTS_IN_CONFIG ,
" No servers started (add valid listen_host and 'tcp_port' "
" or 'http_port' to configuration file.) " ) ;
2014-03-21 13:42:14 +00:00
}
2017-04-01 07:20:54 +00:00
2021-07-10 02:59:11 +00:00
# if USE_SSL
2021-11-10 16:56:07 +00:00
CertificateReloader : : instance ( ) . tryLoad ( config ( ) ) ;
2021-07-10 02:59:11 +00:00
# endif
2020-10-08 16:07:58 +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 ( ) ;
2023-08-22 21:29:05 +00:00
global_context - > setAsynchronousMetrics ( & async_metrics ) ;
2020-03-05 19:23:39 +00:00
2018-03-03 14:39:16 +00:00
main_config_reloader - > start ( ) ;
2022-05-16 18:43:55 +00:00
access_control . startPeriodicReloading ( ) ;
2018-03-03 14:39:16 +00:00
2021-02-23 20:20:31 +00:00
/// try to load dictionaries immediately, throw on error and die
try
{
2021-09-29 12:52:58 +00:00
global_context - > loadOrReloadDictionaries ( config ( ) ) ;
2023-11-15 09:58:51 +00:00
2023-11-22 23:42:04 +00:00
if ( ! config ( ) . getBool ( " dictionaries_lazy_load " , true ) & & config ( ) . getBool ( " wait_dictionaries_load_at_startup " , true ) )
2023-11-15 09:58:51 +00:00
global_context - > waitForDictionariesLoad ( ) ;
2021-09-29 12:52:58 +00:00
}
catch ( . . . )
{
2021-10-01 13:44:09 +00:00
tryLogCurrentException ( log , " Caught exception while loading dictionaries. " ) ;
2021-09-29 12:52:58 +00:00
throw ;
}
/// try to load embedded dictionaries immediately, throw on error and die
try
{
global_context - > tryCreateEmbeddedDictionaries ( config ( ) ) ;
}
catch ( . . . )
{
2021-10-01 13:44:09 +00:00
tryLogCurrentException ( log , " Caught exception while loading embedded dictionaries. " ) ;
2021-09-29 12:52:58 +00:00
throw ;
}
2021-09-15 20:52:18 +00:00
/// try to load user defined executable functions, throw on error and die
2021-09-07 23:55:17 +00:00
try
{
2021-09-29 12:52:58 +00:00
global_context - > loadOrReloadUserDefinedExecutableFunctions ( config ( ) ) ;
2021-09-07 23:55:17 +00:00
}
catch ( . . . )
{
2021-10-01 13:44:09 +00:00
tryLogCurrentException ( log , " Caught exception while loading user defined executable functions. " ) ;
2021-09-07 23:55:17 +00:00
throw ;
}
2021-02-23 20:20:31 +00:00
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 )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : ARGUMENT_OUT_OF_BOUND , " distributed_ddl.pool_size should be greater then 0 " ) ;
2021-04-10 23:33:54 +00:00
global_context - > setDDLWorker ( std : : make_unique < DDLWorker > ( pool_size , ddl_zookeeper_path , global_context , & config ( ) ,
2021-08-11 03:40:06 +00:00
" distributed_ddl " , " DDLWorker " ,
2023-10-24 10:09:10 +00:00
& CurrentMetrics : : MaxDDLEntryID , & CurrentMetrics : : MaxPushedDDLEntryID ) ,
load_metadata_tasks ) ;
2021-02-23 20:20:31 +00:00
}
2024-02-13 09:40:20 +00:00
if ( config ( ) . has ( DB : : PlacementInfo : : PLACEMENT_CONFIG_PREFIX ) )
{
PlacementInfo : : PlacementInfo : : instance ( ) . initialize ( config ( ) ) ;
}
2023-10-24 10:09:10 +00:00
/// Do not keep tasks in server, they should be kept inside databases. Used here to make dependent tasks only.
load_metadata_tasks . clear ( ) ;
load_metadata_tasks . shrink_to_fit ( ) ;
2021-10-22 07:15:34 +00:00
{
std : : lock_guard lock ( servers_lock ) ;
2024-05-23 23:14:26 +00:00
servers . startServers ( ) ;
2022-03-14 20:43:34 +00:00
global_context - > setServerCompletelyStarted ( ) ;
2021-10-22 07:15:34 +00:00
LOG_INFO ( log , " Ready for connections. " ) ;
}
2017-04-01 07:20:54 +00:00
2023-01-16 07:32:25 +00:00
startup_watch . stop ( ) ;
ProfileEvents : : increment ( ProfileEvents : : ServerStartupMilliseconds , startup_watch . elapsedMilliseconds ( ) ) ;
2024-04-03 17:49:23 +00:00
CannotAllocateThreadFaultInjector : : setFaultProbability ( server_settings . cannot_allocate_thread_fault_injection_probability ) ;
2021-11-19 09:42:00 +00:00
try
{
global_context - > startClusterDiscovery ( ) ;
}
catch ( . . . )
{
tryLogCurrentException ( log , " Caught exception while starting cluster discovery " ) ;
}
2017-04-01 07:20:54 +00:00
2022-11-19 18:18:03 +00:00
# if defined(OS_LINUX)
2022-11-23 22:37:02 +00:00
/// Tell the service manager that service startup is finished.
2022-12-05 21:20:04 +00:00
/// NOTE: the parent clickhouse-watchdog process must do systemdNotify("MAINPID={}\n", child_pid); before
2022-12-06 14:17:43 +00:00
/// the child process notifies 'READY=1'.
2022-11-21 14:59:50 +00:00
systemdNotify ( " READY=1 \n " ) ;
2022-11-19 18:18:03 +00:00
# endif
Fix uncaught exception during server termination
Example of a stacktrace:
<details>
```
[ 47463 ] {} <Trace> BaseDaemon: Received signal 15
[ 47463 ] {} <Information> Application: Received termination signal (Terminated)
[ 47462 ] {} <Debug> Application: Received termination signal.
[ 47462 ] {} <Debug> Application: Waiting for current connections to close.
[ 47463 ] {} <Trace> BaseDaemon: Received signal 15
[ 47463 ] {} <Information> Application: Received termination signal (Terminated)
...
[ 47463 ] {} <Trace> BaseDaemon: Received signal -1
[ 47463 ] {} <Fatal> BaseDaemon: (version 21.9.1.1, build id: 63945F58FC2C28ED) (from thread 47462) Terminate called for uncaught exception:
[ 47463 ] {} <Fatal> BaseDaemon: Code: 210. DB::NetException: Connection reset by peer, while writing to socket (10.7.141.42:9000). (NETWORK_ERROR), Stack trace (when copying this message, always include the lines below):
[ 47463 ] {} <Fatal> BaseDaemon:
[ 47463 ] {} <Fatal> BaseDaemon: 0. DB::Exception::Exception(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, int, bool) @ 0x94ca99a in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 47463 ] {} <Fatal> BaseDaemon: 1. DB::WriteBufferFromPocoSocket::nextImpl() @ 0x10676a3b in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 47463 ] {} <Fatal> BaseDaemon: 2. DB::Connection::sendCancel() @ 0x11554701 in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 47463 ] {} <Fatal> BaseDaemon: 3. DB::MultiplexedConnections::sendCancel() @ 0x1157e766 in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 47463 ] {} <Fatal> BaseDaemon: 4. DB::RemoteQueryExecutor::tryCancel(char const*, std::__1::unique_ptr<DB::RemoteQueryExecutorReadContext, std::__1::default_delete<DB::RemoteQueryExecutorReadContext> >*) @ 0x10392000 in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 47463 ] {} <Fatal> BaseDaemon: 5. DB::PipelineExecutor::cancel() @ 0x11697ffe in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 47463 ] {} <Fatal> BaseDaemon: 6. DB::QueryStatus::cancelQuery(bool) @ 0x10c19fc8 in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 47463 ] {} <Fatal> BaseDaemon: 7. DB::ProcessList::killAllQueries() @ 0x10c1a6ae in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 47463 ] {} <Fatal> BaseDaemon: 8. 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&)::$_18>::~basic_scope_guard() @ 0x95587ad in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 47463 ] {} <Fatal> BaseDaemon: 9. 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&) @ 0x95528a2 in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 47463 ] {} <Fatal> BaseDaemon: 10. Poco::Util::Application::run() @ 0x141e85a3 in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 47463 ] {} <Fatal> BaseDaemon: 11. DB::Server::run() @ 0x9541dac in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 47463 ] {} <Fatal> BaseDaemon: 12. mainEntryClickHouseServer(int, char**) @ 0x9540153 in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 47463 ] {} <Fatal> BaseDaemon: 13. main @ 0x94c569e in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 47463 ] {} <Fatal> BaseDaemon: 14. __libc_start_main @ 0x26d0a in /usr/lib/x86_64-linux-gnu/libc-2.31.so
[ 47463 ] {} <Fatal> BaseDaemon: 15. _start @ 0x9490a2a in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 47463 ] {} <Fatal> BaseDaemon: (version 21.9.1.1)
[ 47463 ] {} <Trace> BaseDaemon: Received signal 6
[ 11858 ] {} <Fatal> BaseDaemon: ########################################
[ 11858 ] {} <Fatal> BaseDaemon: (version 21.9.1.1, build id: 63945F58FC2C28ED) (from thread 47462) (no query) Received signal Aborted (6)
[ 11858 ] {} <Fatal> BaseDaemon:
[ 11858 ] {} <Fatal> BaseDaemon: Stack trace: 0x7ff04c196ce1 0x7ff04c180537 0xff91f28 0x163304e3 0x1633044c 0x94c60cb 0x10c1a135 0x10c1a6ae 0x95587ad 0x95528a2 0x141e85a3 0x9541dac 0x9540153 0x94c569e 0x7ff04c181d0a 0x9490a2a
[ 11858 ] {} <Fatal> BaseDaemon: 1. raise @ 0x3bce1 in /usr/lib/x86_64-linux-gnu/libc-2.31.so
[ 11858 ] {} <Fatal> BaseDaemon: 2. abort @ 0x25537 in /usr/lib/x86_64-linux-gnu/libc-2.31.so
[ 11858 ] {} <Fatal> BaseDaemon: 3. terminate_handler() @ 0xff91f28 in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 11858 ] {} <Fatal> BaseDaemon: 4. std::__terminate(void (*)()) @ 0x163304e3 in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 11858 ] {} <Fatal> BaseDaemon: 5. std::terminate() @ 0x1633044c in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 11858 ] {} <Fatal> BaseDaemon: 6. ? @ 0x94c60cb in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 11858 ] {} <Fatal> BaseDaemon: 7. ? @ 0x10c1a135 in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 11858 ] {} <Fatal> BaseDaemon: 8. DB::ProcessList::killAllQueries() @ 0x10c1a6ae in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 11858 ] {} <Fatal> BaseDaemon: 9. 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&)::$_18>::~basic_scope_guard() @ 0x95587ad in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 11858 ] {} <Fatal> BaseDaemon: 10. 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&) @ 0x95528a2 in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 11858 ] {} <Fatal> BaseDaemon: 11. Poco::Util::Application::run() @ 0x141e85a3 in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 11858 ] {} <Fatal> BaseDaemon: 12. DB::Server::run() @ 0x9541dac in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 11858 ] {} <Fatal> BaseDaemon: 13. mainEntryClickHouseServer(int, char**) @ 0x9540153 in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 11858 ] {} <Fatal> BaseDaemon: 14. main @ 0x94c569e in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
[ 11858 ] {} <Fatal> BaseDaemon: 15. __libc_start_main @ 0x26d0a in /usr/lib/x86_64-linux-gnu/libc-2.31.so
[ 11858 ] {} <Fatal> BaseDaemon: 16. _start @ 0x9490a2a in /usr/lib/debug/.build-id/63/945f58fc2c28ed.debug
```
</details>
2021-09-08 18:10:34 +00:00
SCOPE_EXIT_SAFE ( {
2020-05-23 22:24:01 +00:00
LOG_DEBUG ( log , " Received termination signal. " ) ;
2022-11-13 19:17:41 +00:00
/// Stop reloading of the main config. This must be done before everything else because it
/// can try to access/modify already deleted objects.
/// E.g. it can recreate new servers or it may pass a changed config to some destroyed parts of ContextSharedPart.
main_config_reloader . reset ( ) ;
access_control . stopPeriodicReloading ( ) ;
2017-04-01 07:20:54 +00:00
2017-03-03 21:10:41 +00:00
is_cancelled = true ;
2024-05-23 23:14:26 +00:00
const auto remaining_connections = servers . stopServers ( server_settings , servers_lock ) ;
2019-06-27 16:28:26 +00:00
dns_cache_updater . reset ( ) ;
2019-07-30 23:12:04 +00:00
2024-05-23 23:14:26 +00:00
if ( remaining_connections )
2019-07-30 23:12:04 +00:00
{
/// 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 ( ) ;
2023-05-15 12:17:32 +00:00
LOG_WARNING ( log , " Will shutdown forcefully. " ) ;
2022-03-02 09:59:10 +00:00
safeExit ( 0 ) ;
2019-07-30 23:12:04 +00:00
}
2017-03-03 21:10:41 +00:00
} ) ;
2017-04-01 07:20:54 +00:00
2017-03-21 19:08:09 +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-03-21 19:08:09 +00:00
}
2017-04-01 07:20:54 +00:00
2016-10-23 06:12:50 +00:00
waitForTerminationRequest ( ) ;
2012-12-14 11:21:07 +00:00
}
2017-04-01 07:20:54 +00:00
2012-03-09 03:06:09 +00:00
return Application : : EXIT_OK ;
}
2022-11-18 12:22:55 +00:00
catch ( . . . )
{
/// Poco does not provide stacktrace.
tryLogCurrentException ( " Application " ) ;
2023-11-23 17:13:12 +00:00
auto code = getCurrentExceptionCode ( ) ;
return code ? code : - 1 ;
2022-11-18 12:22:55 +00:00
}
2021-10-22 07:15:34 +00:00
2012-03-09 03:06:09 +00:00
}