2021-05-12 10:39:07 +00:00
# include "Keeper.h"
2023-07-11 10:13:30 +00:00
# include <Common/ClickHouseRevision.h>
# include <Common/getMultipleKeysFromConfig.h>
# include <Common/DNSResolver.h>
# include <Interpreters/DNSCacheUpdater.h>
2023-07-11 10:04:43 +00:00
# include <Coordination/Defines.h>
2023-07-11 10:13:30 +00:00
# include <Common/Config/ConfigReloader.h>
# include <filesystem>
2023-07-11 10:04:43 +00:00
# include <IO/UseSSL.h>
2023-07-11 10:13:30 +00:00
# include <Core/ServerUUID.h>
# include <Common/logger_useful.h>
# include <Common/ErrorHandlers.h>
# include <Common/assertProcessUserMatchesDataOwner.h>
# include <Common/makeSocketAddress.h>
2023-05-19 14:23:56 +00:00
# include <Server/waitServersToFinish.h>
2023-07-11 10:04:43 +00:00
# include <base/scope_guard.h>
2023-07-11 10:13:30 +00:00
# include <base/safeExit.h>
2021-10-03 09:54:23 +00:00
# include <Poco/Net/NetException.h>
2023-07-11 10:04:43 +00:00
# include <Poco/Net/TCPServerParams.h>
2023-07-11 10:13:30 +00:00
# include <Poco/Net/TCPServer.h>
2021-05-12 10:39:07 +00:00
# include <Poco/Util/HelpFormatter.h>
2023-07-11 10:13:30 +00:00
# include <Poco/Environment.h>
# include <sys/stat.h>
# include <pwd.h>
2022-11-09 12:37:42 +00:00
2023-05-17 13:32:51 +00:00
# include <Interpreters/Context.h>
2021-10-27 12:26:42 +00:00
# include <Coordination/FourLetterCommand.h>
2022-11-09 12:37:42 +00:00
# include <Coordination/KeeperAsynchronousMetrics.h>
2021-05-12 10:39:07 +00:00
2022-11-09 08:02:04 +00:00
# include <Server/HTTP/HTTPServer.h>
2023-07-11 10:04:43 +00:00
# include <Server/TCPServer.h>
2023-07-11 10:13:30 +00:00
# include <Server/HTTPHandlerFactory.h>
2022-11-09 08:02:04 +00:00
# include "Core/Defines.h"
2022-09-28 13:29:29 +00:00
# include "config.h"
2023-07-11 10:04:43 +00:00
# include "config_version.h"
2023-07-11 10:13:30 +00:00
# include "config_tools.h"
2023-06-14 00:23:39 +00:00
2021-05-12 10:39:07 +00:00
# if USE_SSL
# include <Poco / Net / Context.h>
# include <Poco / Net / SecureServerSocket.h>
2023-07-11 12:23:10 +00:00
# include <Server / CertificateReloader.h>
2021-05-12 10:39:07 +00:00
# endif
2023-07-11 10:04:43 +00:00
# include <Server/ProtocolServerAdapter.h>
2023-07-11 10:13:30 +00:00
# include <Server/KeeperTCPHandlerFactory.h>
2021-05-12 10:39:07 +00:00
2023-05-17 13:32:51 +00:00
# include <Disks/registerDisks.h>
2021-05-17 09:38:44 +00:00
2021-05-12 10:39:07 +00:00
int mainEntryClickHouseKeeper ( int argc , char * * argv )
{
DB : : Keeper app ;
try
{
return app . run ( argc , argv ) ;
}
catch ( . . . )
{
std : : cerr < < DB : : getCurrentExceptionMessage ( true ) < < " \n " ;
auto code = DB : : getCurrentExceptionCode ( ) ;
return code ? code : 1 ;
}
}
2023-05-02 17:41:42 +00:00
# ifdef CLICKHOUSE_PROGRAM_STANDALONE_BUILD
2022-11-10 09:49:00 +00:00
2022-11-30 12:24:08 +00:00
// Weak symbols don't work correctly on Darwin
// so we have a stub implementation to avoid linker errors
2023-07-11 10:13:30 +00:00
void collectCrashLog (
Int32 , UInt64 , const String & , const StackTrace & )
{ }
2022-11-10 09:49:00 +00:00
# endif
2021-05-12 10:39:07 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int NO_ELEMENTS_IN_CONFIG ;
extern const int SUPPORT_IS_DISABLED ;
extern const int NETWORK_ERROR ;
2022-04-15 08:52:34 +00:00
extern const int LOGICAL_ERROR ;
2021-05-12 10:39:07 +00:00
}
2023-07-11 10:13:30 +00:00
Poco : : Net : : SocketAddress Keeper : : socketBindListen ( Poco : : Net : : ServerSocket & socket , const std : : string & host , UInt16 port , [[maybe_unused]] bool secure ) const
2021-05-12 10:39:07 +00:00
{
auto address = makeSocketAddress ( host , port , & logger ( ) ) ;
socket . bind ( address , /* reuseAddress = */ true , /* reusePort = */ config ( ) . getBool ( " listen_reuse_port " , false ) ) ;
socket . listen ( /* backlog = */ config ( ) . getUInt ( " listen_backlog " , 64 ) ) ;
return address ;
}
void Keeper : : createServer ( const std : : string & listen_host , const char * port_name , bool listen_try , CreateServerFunc & & func ) const
{
/// For testing purposes, user may omit tcp_port or http_port or https_port in configuration file.
if ( ! config ( ) . has ( port_name ) )
return ;
auto port = config ( ) . getInt ( port_name ) ;
try
{
func ( port ) ;
}
catch ( const Poco : : Exception & )
{
std : : string message = " Listen [ " + listen_host + " ]: " + std : : to_string ( port ) + " failed: " + getCurrentExceptionMessage ( false ) ;
if ( listen_try )
{
2023-07-11 10:13:30 +00:00
LOG_WARNING ( & logger ( ) , " {}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to "
2021-05-12 10:39:07 +00:00
" specify not disabled IPv4 or IPv6 address to listen in <listen_host> element of configuration "
" file. Example for disabled IPv6: <listen_host>0.0.0.0</listen_host> . "
" Example for disabled IPv4: <listen_host>::</listen_host> " ,
message ) ;
}
else
{
2023-01-23 13:16:14 +00:00
throw Exception : : createDeprecated ( message , ErrorCodes : : NETWORK_ERROR ) ;
2021-05-12 10:39:07 +00:00
}
}
}
void Keeper : : uninitialize ( )
{
logger ( ) . information ( " shutting down " ) ;
BaseDaemon : : uninitialize ( ) ;
}
int Keeper : : run ( )
{
if ( config ( ) . hasOption ( " help " ) )
{
Poco : : Util : : HelpFormatter help_formatter ( Keeper : : options ( ) ) ;
2023-07-11 10:13:30 +00:00
auto header_str = fmt : : format ( " {0} [OPTION] [-- [ARG]...] \n "
2023-06-14 00:23:39 +00:00
# if ENABLE_CLICKHOUSE_KEEPER_CLIENT
2023-07-11 10:13:30 +00:00
" {0} client [OPTION] \n "
2023-06-14 00:23:39 +00:00
# endif
2023-07-11 10:13:30 +00:00
" positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010 " ,
commandName ( ) ) ;
2021-05-12 10:39:07 +00:00
help_formatter . setHeader ( header_str ) ;
help_formatter . format ( std : : cout ) ;
return 0 ;
}
if ( config ( ) . hasOption ( " version " ) )
{
2021-05-12 13:04:34 +00:00
std : : cout < < DBMS_NAME < < " keeper version " < < VERSION_STRING < < VERSION_OFFICIAL < < " . " < < std : : endl ;
2021-05-12 10:39:07 +00:00
return 0 ;
}
return Application : : run ( ) ; // NOLINT
}
void Keeper : : initialize ( Poco : : Util : : Application & self )
{
BaseDaemon : : initialize ( self ) ;
logger ( ) . information ( " starting up " ) ;
2023-07-11 10:13:30 +00:00
LOG_INFO ( & logger ( ) , " OS Name = {}, OS Version = {}, OS Architecture = {} " ,
2021-05-12 10:39:07 +00:00
Poco : : Environment : : osName ( ) ,
Poco : : Environment : : osVersion ( ) ,
Poco : : Environment : : osArchitecture ( ) ) ;
}
2021-05-12 13:04:34 +00:00
std : : string Keeper : : getDefaultConfigFileName ( ) const
2021-05-12 10:39:07 +00:00
{
2021-05-12 13:04:34 +00:00
return " keeper_config.xml " ;
2021-05-12 10:39:07 +00:00
}
2022-04-15 08:52:34 +00:00
void Keeper : : handleCustomArguments ( const std : : string & arg , [[maybe_unused]] const std : : string & value ) // NOLINT
2022-04-08 07:18:18 +00:00
{
if ( arg = = " force-recovery " )
{
assert ( value . empty ( ) ) ;
2022-04-14 12:07:33 +00:00
config ( ) . setBool ( " keeper_server.force_recovery " , true ) ;
2022-04-08 07:18:18 +00:00
return ;
}
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Invalid argument {} provided " , arg ) ;
}
2021-05-12 10:39:07 +00:00
void Keeper : : defineOptions ( Poco : : Util : : OptionSet & options )
{
options . addOption (
2023-07-11 10:13:30 +00:00
Poco : : Util : : Option ( " help " , " h " , " show help and exit " )
2021-05-12 10:39:07 +00:00
. required ( false )
. repeatable ( false )
2023-07-11 10:13:30 +00:00
. binding ( " help " ) ) ;
options . addOption (
Poco : : Util : : Option ( " version " , " V " , " show version and exit " )
. required ( false )
. repeatable ( false )
. binding ( " version " ) ) ;
options . addOption (
Poco : : Util : : Option ( " force-recovery " , " force-recovery " , " Force recovery mode allowing Keeper to overwrite cluster configuration without quorum " )
. required ( false )
. repeatable ( false )
. noArgument ( )
. callback ( Poco : : Util : : OptionCallback < Keeper > ( this , & Keeper : : handleCustomArguments ) ) ) ;
2021-05-12 10:39:07 +00:00
BaseDaemon : : defineOptions ( options ) ;
}
2023-05-22 12:24:16 +00:00
namespace
2022-11-09 08:02:04 +00:00
{
2023-05-22 12:24:16 +00:00
2023-07-11 10:13:30 +00:00
struct KeeperHTTPContext : public IHTTPContext
{
explicit KeeperHTTPContext ( ContextPtr context_ )
: context ( std : : move ( context_ ) )
{ }
2022-11-09 08:02:04 +00:00
2023-07-11 10:13:30 +00:00
uint64_t getMaxHstsAge ( ) const override
{
return context - > getConfigRef ( ) . getUInt64 ( " keeper_server.hsts_max_age " , 0 ) ;
}
2022-11-09 08:02:04 +00:00
2023-07-11 10:13:30 +00:00
uint64_t getMaxUriSize ( ) const override
{
return context - > getConfigRef ( ) . getUInt64 ( " keeper_server.http_max_uri_size " , 1048576 ) ;
}
2022-11-09 08:02:04 +00:00
2023-07-11 10:13:30 +00:00
uint64_t getMaxFields ( ) const override
{
return context - > getConfigRef ( ) . getUInt64 ( " keeper_server.http_max_fields " , 1000000 ) ;
}
2022-11-09 08:02:04 +00:00
2023-07-11 10:13:30 +00:00
uint64_t getMaxFieldNameSize ( ) const override
{
return context - > getConfigRef ( ) . getUInt64 ( " keeper_server.http_max_field_name_size " , 128 * 1024 ) ;
}
2022-11-09 08:02:04 +00:00
2023-07-11 10:13:30 +00:00
uint64_t getMaxFieldValueSize ( ) const override
{
return context - > getConfigRef ( ) . getUInt64 ( " keeper_server.http_max_field_value_size " , 128 * 1024 ) ;
}
2022-11-09 14:51:41 +00:00
2023-07-11 10:13:30 +00:00
uint64_t getMaxChunkSize ( ) const override
{
return context - > getConfigRef ( ) . getUInt64 ( " keeper_server.http_max_chunk_size " , 100 _GiB ) ;
}
2022-11-09 08:02:04 +00:00
2023-07-11 10:13:30 +00:00
Poco : : Timespan getReceiveTimeout ( ) const override
{
return { context - > getConfigRef ( ) . getInt64 ( " keeper_server.http_receive_timeout " , DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC ) , 0 } ;
}
2023-07-11 10:04:43 +00:00
2023-07-11 10:13:30 +00:00
Poco : : Timespan getSendTimeout ( ) const override
2023-07-11 10:04:43 +00:00
{
2023-07-11 10:13:30 +00:00
return { context - > getConfigRef ( ) . getInt64 ( " keeper_server.http_send_timeout " , DBMS_DEFAULT_SEND_TIMEOUT_SEC ) , 0 } ;
2023-07-11 10:04:43 +00:00
}
2023-05-22 12:24:16 +00:00
2023-07-11 10:13:30 +00:00
ContextPtr context ;
} ;
HTTPContextPtr httpContext ( )
{
return std : : make_shared < KeeperHTTPContext > ( Context : : getGlobalContextInstance ( ) ) ;
}
2022-11-09 08:02:04 +00:00
}
2021-05-12 10:39:07 +00:00
int Keeper : : main ( const std : : vector < std : : string > & /*args*/ )
2022-11-18 12:22:55 +00:00
try
2021-05-12 10:39:07 +00:00
{
Poco : : Logger * log = & logger ( ) ;
UseSSL use_ssl ;
MainThreadStatus : : getInstance ( ) ;
# if !defined(NDEBUG) || !defined(__OPTIMIZE__)
2021-05-12 13:04:34 +00:00
LOG_WARNING ( log , " Keeper was built in debug mode. It will work slowly. " ) ;
2021-05-12 10:39:07 +00:00
# endif
# if defined(SANITIZER)
2021-05-12 13:04:34 +00:00
LOG_WARNING ( log , " Keeper was built with sanitizer. It will work slowly. " ) ;
2021-05-12 10:39:07 +00:00
# endif
2021-05-12 13:04:34 +00:00
if ( ! config ( ) . has ( " keeper_server " ) )
throw Exception ( ErrorCodes : : NO_ELEMENTS_IN_CONFIG , " Keeper configuration (<keeper_server> section) not found in config " ) ;
2021-05-12 14:05:44 +00:00
std : : string path ;
if ( config ( ) . has ( " keeper_server.storage_path " ) )
path = config ( ) . getString ( " keeper_server.storage_path " ) ;
else if ( config ( ) . has ( " keeper_server.log_storage_path " ) )
2021-08-25 16:15:56 +00:00
path = std : : filesystem : : path ( config ( ) . getString ( " keeper_server.log_storage_path " ) ) . parent_path ( ) ;
2021-05-12 14:05:44 +00:00
else if ( config ( ) . has ( " keeper_server.snapshot_storage_path " ) )
2021-08-25 16:15:56 +00:00
path = std : : filesystem : : path ( config ( ) . getString ( " keeper_server.snapshot_storage_path " ) ) . parent_path ( ) ;
2021-05-12 14:05:44 +00:00
else
2021-05-18 14:08:56 +00:00
path = std : : filesystem : : path { KEEPER_DEFAULT_PATH } ;
2021-05-12 14:05:44 +00:00
2023-01-30 14:26:57 +00:00
std : : filesystem : : create_directories ( path ) ;
2021-05-12 14:05:44 +00:00
/// Check that the process user id matches the owner of the data.
2023-07-11 10:13:30 +00:00
assertProcessUserMatchesDataOwner ( path , [ & ] ( const std : : string & message ) { LOG_WARNING ( log , fmt : : runtime ( message ) ) ; } ) ;
2021-05-12 14:05:44 +00:00
2021-08-17 13:24:14 +00:00
DB : : ServerUUID : : load ( path + " /uuid " , log ) ;
2021-08-16 18:30:53 +00:00
2021-10-19 14:29:49 +00:00
std : : string include_from_path = config ( ) . getString ( " include_from " , " /etc/metrika.xml " ) ;
2021-11-12 13:24:47 +00:00
GlobalThreadPool : : initialize (
config ( ) . getUInt ( " max_thread_pool_size " , 100 ) ,
config ( ) . getUInt ( " max_thread_pool_free_size " , 1000 ) ,
2023-07-11 10:13:30 +00:00
config ( ) . getUInt ( " thread_pool_queue_size " , 10000 )
) ;
2021-05-12 10:39:07 +00:00
static ServerErrorHandler error_handler ;
Poco : : ErrorHandler : : set ( & error_handler ) ;
/// Initialize DateLUT early, to not interfere with running time of first query.
LOG_DEBUG ( log , " Initializing DateLUT. " ) ;
2023-04-12 10:47:05 +00:00
DateLUT : : serverTimezoneInstance ( ) ;
LOG_TRACE ( log , " Initialized DateLUT with time zone '{}'. " , DateLUT : : serverTimezoneInstance ( ) . getTimeZone ( ) ) ;
2021-05-12 10:39:07 +00:00
2021-05-12 13:04:34 +00:00
/// Don't want to use DNS cache
DNSResolver : : instance ( ) . setDisableCacheFlag ( ) ;
2021-05-12 10:39:07 +00:00
2021-05-12 13:04:34 +00:00
Poco : : ThreadPool server_pool ( 3 , config ( ) . getUInt ( " max_connections " , 1024 ) ) ;
2022-11-09 12:37:42 +00:00
std : : mutex servers_lock ;
auto servers = std : : make_shared < std : : vector < ProtocolServerAdapter > > ( ) ;
2023-05-17 13:32:51 +00:00
auto shared_context = Context : : createShared ( ) ;
auto global_context = Context : : createGlobal ( shared_context . get ( ) ) ;
global_context - > makeGlobalContext ( ) ;
global_context - > setPath ( path ) ;
global_context - > setRemoteHostFilter ( config ( ) ) ;
2023-05-22 12:24:16 +00:00
if ( config ( ) . has ( " macros " ) )
global_context - > setMacros ( std : : make_unique < Macros > ( config ( ) , " macros " , log ) ) ;
2023-05-17 13:32:51 +00:00
registerDisks ( /*global_skip_access_check=*/ false ) ;
2022-11-09 12:37:42 +00:00
/// This object will periodically calculate some metrics.
KeeperAsynchronousMetrics async_metrics (
2023-05-22 12:24:16 +00:00
global_context ,
2022-11-09 12:37:42 +00:00
config ( ) . getUInt ( " asynchronous_metrics_update_period_s " , 1 ) ,
[ & ] ( ) - > std : : vector < ProtocolServerMetrics >
{
std : : vector < ProtocolServerMetrics > metrics ;
std : : lock_guard lock ( servers_lock ) ;
metrics . reserve ( servers - > size ( ) ) ;
for ( const auto & server : * servers )
metrics . emplace_back ( ProtocolServerMetrics { server . getPortName ( ) , server . currentThreads ( ) } ) ;
return metrics ;
2023-07-11 10:13:30 +00:00
}
) ;
2022-11-09 12:37:42 +00:00
2021-05-12 10:39:07 +00:00
std : : vector < std : : string > listen_hosts = DB : : getMultipleValuesFromConfig ( config ( ) , " " , " listen_host " ) ;
bool listen_try = config ( ) . getBool ( " listen_try " , false ) ;
if ( listen_hosts . empty ( ) )
{
listen_hosts . emplace_back ( " ::1 " ) ;
listen_hosts . emplace_back ( " 127.0.0.1 " ) ;
listen_try = true ;
}
2021-10-18 09:13:24 +00:00
/// Initialize keeper RAFT. Do nothing if no keeper_server in config.
2023-05-22 12:24:16 +00:00
global_context - > initializeKeeperDispatcher ( /* start_async = */ true ) ;
FourLetterCommandFactory : : registerCommands ( * global_context - > getKeeperDispatcher ( ) ) ;
2022-03-03 20:27:46 +00:00
2023-07-11 10:13:30 +00:00
auto config_getter = [ & ] ( ) - > const Poco : : Util : : AbstractConfiguration &
{
return global_context - > getConfigRef ( ) ;
} ;
2021-10-27 12:26:42 +00:00
2023-03-28 12:15:28 +00:00
auto tcp_receive_timeout = config ( ) . getInt64 ( " keeper_server.socket_receive_timeout_sec " , DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC ) ;
auto tcp_send_timeout = config ( ) . getInt64 ( " keeper_server.socket_send_timeout_sec " , DBMS_DEFAULT_SEND_TIMEOUT_SEC ) ;
2021-05-12 13:04:34 +00:00
for ( const auto & listen_host : listen_hosts )
{
/// TCP Keeper
const char * port_name = " keeper_server.tcp_port " ;
2023-07-11 10:13:30 +00:00
createServer ( listen_host , port_name , listen_try , [ & ] ( UInt16 port )
{
Poco : : Net : : ServerSocket socket ;
auto address = socketBindListen ( socket , listen_host , port ) ;
socket . setReceiveTimeout ( Poco : : Timespan { tcp_receive_timeout , 0 } ) ;
socket . setSendTimeout ( Poco : : Timespan { tcp_send_timeout , 0 } ) ;
servers - > emplace_back (
listen_host ,
port_name ,
" Keeper (tcp): " + address . toString ( ) ,
std : : make_unique < TCPServer > (
new KeeperTCPHandlerFactory (
config_getter , global_context - > getKeeperDispatcher ( ) ,
tcp_receive_timeout , tcp_send_timeout , false ) , server_pool , socket ) ) ;
} ) ;
2021-05-12 13:04:34 +00:00
const char * secure_port_name = " keeper_server.tcp_port_secure " ;
2023-07-11 10:13:30 +00:00
createServer ( listen_host , secure_port_name , listen_try , [ & ] ( UInt16 port )
{
2021-05-12 10:39:07 +00:00
# if USE_SSL
2023-07-11 10:13:30 +00:00
Poco : : Net : : SecureServerSocket socket ;
auto address = socketBindListen ( socket , listen_host , port , /* secure = */ true ) ;
socket . setReceiveTimeout ( Poco : : Timespan { tcp_receive_timeout , 0 } ) ;
socket . setSendTimeout ( Poco : : Timespan { tcp_send_timeout , 0 } ) ;
servers - > emplace_back (
listen_host ,
secure_port_name ,
" Keeper with secure protocol (tcp_secure): " + address . toString ( ) ,
std : : make_unique < TCPServer > (
new KeeperTCPHandlerFactory (
config_getter , global_context - > getKeeperDispatcher ( ) ,
tcp_receive_timeout , tcp_send_timeout , true ) , server_pool , socket ) ) ;
2021-05-12 10:39:07 +00:00
# else
2023-07-11 10:13:30 +00:00
UNUSED ( port ) ;
throw Exception ( ErrorCodes : : SUPPORT_IS_DISABLED , " SSL support for TCP protocol is disabled because Poco library was built without NetSSL support. " ) ;
2021-05-12 10:39:07 +00:00
# endif
2023-07-11 10:13:30 +00:00
} ) ;
2022-11-09 08:02:04 +00:00
const auto & config = config_getter ( ) ;
2023-03-28 12:15:28 +00:00
auto http_context = httpContext ( ) ;
2022-11-09 08:02:04 +00:00
Poco : : Timespan keep_alive_timeout ( config . getUInt ( " keep_alive_timeout " , 10 ) , 0 ) ;
Poco : : Net : : HTTPServerParams : : Ptr http_params = new Poco : : Net : : HTTPServerParams ;
2023-03-28 12:15:28 +00:00
http_params - > setTimeout ( http_context - > getReceiveTimeout ( ) ) ;
2022-11-09 08:02:04 +00:00
http_params - > setKeepAliveTimeout ( keep_alive_timeout ) ;
/// Prometheus (if defined and not setup yet with http_port)
port_name = " prometheus.port " ;
2023-07-11 10:13:30 +00:00
createServer ( listen_host , port_name , listen_try , [ & , my_http_context = std : : move ( http_context ) ] ( UInt16 port ) mutable
{
Poco : : Net : : ServerSocket socket ;
auto address = socketBindListen ( socket , listen_host , port ) ;
socket . setReceiveTimeout ( my_http_context - > getReceiveTimeout ( ) ) ;
socket . setSendTimeout ( my_http_context - > getSendTimeout ( ) ) ;
servers - > emplace_back (
listen_host ,
port_name ,
" Prometheus: http:// " + address . toString ( ) ,
std : : make_unique < HTTPServer > (
std : : move ( my_http_context ) , createPrometheusMainHandlerFactory ( * this , config_getter ( ) , async_metrics , " PrometheusHandler-factory " ) , server_pool , socket , http_params ) ) ;
} ) ;
2021-05-12 13:04:34 +00:00
}
2021-05-12 10:39:07 +00:00
for ( auto & server : * servers )
2021-10-22 07:15:34 +00:00
{
2021-05-12 10:39:07 +00:00
server . start ( ) ;
2021-10-22 07:15:34 +00:00
LOG_INFO ( log , " Listening for {} " , server . getDescription ( ) ) ;
}
2021-05-12 10:39:07 +00:00
2022-11-09 12:37:42 +00:00
async_metrics . start ( ) ;
2021-10-19 14:29:49 +00:00
zkutil : : EventPtr unused_event = std : : make_shared < Poco : : Event > ( ) ;
zkutil : : ZooKeeperNodeCache unused_cache ( [ ] { return nullptr ; } ) ;
2023-07-11 10:13:30 +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:13:30 +00:00
2021-10-19 14:29:49 +00:00
/// ConfigReloader have to strict parameters which are redundant in our case
auto main_config_reloader = std : : make_unique < ConfigReloader > (
config_path ,
2023-07-11 10:13:30 +00:00
extra_paths ,
2021-10-19 14:29:49 +00:00
config ( ) . getString ( " path " , " " ) ,
std : : move ( unused_cache ) ,
unused_event ,
[ & ] ( ConfigurationPtr config , bool /* initial_loading */ )
{
if ( config - > has ( " keeper_server " ) )
2023-05-22 12:24:16 +00:00
global_context - > updateKeeperConfiguration ( * config ) ;
2023-07-11 10:13:30 +00:00
2023-07-11 10:40:55 +00:00
# if USE_SSL
2023-07-11 10:13:30 +00:00
CertificateReloader : : instance ( ) . tryLoad ( * config ) ;
2023-07-11 10:40:55 +00:00
# endif
2021-10-19 14:29:49 +00:00
} ,
2023-07-11 10:13:30 +00:00
/* already_loaded = */ false ) ; /// Reload it right now (initial loading)
2021-10-19 14:29:49 +00:00
2021-05-12 10:39:07 +00:00
SCOPE_EXIT ( {
LOG_INFO ( log , " Shutting down. " ) ;
2021-10-19 14:29:49 +00:00
main_config_reloader . reset ( ) ;
2021-05-12 10:39:07 +00:00
2022-11-09 12:37:42 +00:00
async_metrics . stop ( ) ;
2021-05-18 14:08:56 +00:00
LOG_DEBUG ( log , " Waiting for current connections to Keeper to finish. " ) ;
2022-10-07 10:46:45 +00:00
size_t current_connections = 0 ;
2021-05-12 10:39:07 +00:00
for ( auto & server : * servers )
{
server . stop ( ) ;
current_connections + = server . currentConnections ( ) ;
}
if ( current_connections )
LOG_INFO ( log , " Closed all listening sockets. Waiting for {} outstanding connections. " , current_connections ) ;
else
LOG_INFO ( log , " Closed all listening sockets. " ) ;
if ( current_connections > 0 )
2023-06-27 18:50:40 +00:00
current_connections = waitServersToFinish ( * servers , servers_lock , config ( ) . getInt ( " shutdown_wait_unfinished " , 5 ) ) ;
2021-05-12 10:39:07 +00:00
if ( current_connections )
2023-07-11 10:13:30 +00:00
LOG_INFO ( log , " Closed connections to Keeper. But {} remain. Probably some users cannot finish their connections after context shutdown. " , current_connections ) ;
2021-05-12 10:39:07 +00:00
else
2021-05-18 14:08:56 +00:00
LOG_INFO ( log , " Closed connections to Keeper. " ) ;
2021-05-12 10:39:07 +00:00
2023-05-22 12:24:16 +00:00
global_context - > shutdownKeeperDispatcher ( ) ;
2021-05-12 10:39:07 +00:00
/// Wait server pool to avoid use-after-free of destroyed context in the handlers
server_pool . joinAll ( ) ;
LOG_DEBUG ( log , " Destroyed global context. " ) ;
if ( current_connections )
{
LOG_INFO ( log , " Will shutdown forcefully. " ) ;
2022-03-02 09:59:10 +00:00
safeExit ( 0 ) ;
2021-05-12 10:39:07 +00:00
}
} ) ;
buildLoggers ( config ( ) , logger ( ) ) ;
2021-10-19 14:29:49 +00:00
main_config_reloader - > start ( ) ;
2021-05-12 10:39:07 +00:00
LOG_INFO ( log , " Ready for connections. " ) ;
waitForTerminationRequest ( ) ;
return Application : : EXIT_OK ;
}
2022-11-18 12:22:55 +00:00
catch ( . . . )
{
/// Poco does not provide stacktrace.
tryLogCurrentException ( " Application " ) ;
throw ;
}
2021-05-12 10:39:07 +00:00
2021-05-12 13:04:34 +00:00
void Keeper : : logRevision ( ) const
{
2023-07-11 10:13:30 +00:00
Poco : : Logger : : root ( ) . information ( " Starting ClickHouse Keeper " + std : : string { VERSION_STRING }
+ " (revision : " + std : : to_string ( ClickHouseRevision : : getVersionRevision ( ) )
+ " , git hash: " + ( git_hash . empty ( ) ? " <unknown> " : git_hash )
+ " , build id: " + ( build_id . empty ( ) ? " <unknown> " : build_id ) + " ) "
+ " , PID " + std : : to_string ( getpid ( ) ) ) ;
2021-05-12 13:04:34 +00:00
}
2021-05-12 10:39:07 +00:00
}