2014-04-15 16:39:56 +00:00
# include <sys/resource.h>
2012-03-09 03:06:09 +00:00
# include <Poco/Net/HTTPServerRequest.h>
2015-04-16 06:12:35 +00:00
# include <Poco/Net/DNS.h>
2016-06-25 03:06:36 +00:00
# include <Poco/Net/NetException.h>
2014-02-13 07:17:22 +00:00
# include <Poco/Util/XMLConfiguration.h>
2016-01-17 13:34:36 +00:00
# include <Poco/DirectoryIterator.h>
2012-03-09 03:06:09 +00:00
2015-09-29 19:19:54 +00:00
# include <common/ApplicationServerExt.h>
# include <common/ErrorHandlers.h>
2015-04-16 07:36:38 +00:00
2015-10-05 00:33:43 +00:00
# include <ext/scope_guard.hpp>
2015-04-16 07:36:38 +00:00
2015-02-10 21:10:58 +00:00
# include <memory>
2016-05-28 14:14:18 +00:00
# include <experimental/optional>
2012-03-09 03:06:09 +00:00
2015-04-16 06:12:35 +00:00
# include <DB/Common/Macros.h>
2015-05-29 00:33:56 +00:00
# include <DB/Common/getFQDNOrHostName.h>
2016-07-14 05:22:09 +00:00
# include <DB/Common/StringUtils.h>
2016-01-17 13:34:36 +00:00
2012-03-09 03:06:09 +00:00
# include <DB/Interpreters/loadMetadata.h>
2015-06-21 06:06:04 +00:00
# include <DB/Interpreters/ProcessList.h>
2016-01-17 13:34:36 +00:00
2015-09-24 03:50:09 +00:00
# include <DB/Storages/System/StorageSystemNumbers.h>
# include <DB/Storages/System/StorageSystemTables.h>
# include <DB/Storages/System/StorageSystemParts.h>
# include <DB/Storages/System/StorageSystemDatabases.h>
# include <DB/Storages/System/StorageSystemProcesses.h>
# include <DB/Storages/System/StorageSystemEvents.h>
# include <DB/Storages/System/StorageSystemOne.h>
# include <DB/Storages/System/StorageSystemMerges.h>
# include <DB/Storages/System/StorageSystemSettings.h>
# include <DB/Storages/System/StorageSystemZooKeeper.h>
# include <DB/Storages/System/StorageSystemReplicas.h>
# include <DB/Storages/System/StorageSystemReplicationQueue.h>
# include <DB/Storages/System/StorageSystemDictionaries.h>
# include <DB/Storages/System/StorageSystemColumns.h>
# include <DB/Storages/System/StorageSystemFunctions.h>
# include <DB/Storages/System/StorageSystemClusters.h>
2016-01-21 01:47:28 +00:00
# include <DB/Storages/System/StorageSystemMetrics.h>
2016-01-28 01:00:27 +00:00
# include <DB/Storages/StorageReplicatedMergeTree.h>
# include <DB/Storages/MergeTree/ReshardingWorker.h>
2016-03-19 01:18:49 +00:00
# include <DB/Databases/DatabaseOrdinary.h>
2014-12-21 03:18:40 +00:00
2015-04-16 07:36:38 +00:00
# include <zkutil/ZooKeeper.h>
2012-03-09 03:06:09 +00:00
# include "Server.h"
2012-03-09 15:46:52 +00:00
# include "HTTPHandler.h"
2016-01-17 13:34:36 +00:00
# include "ReplicasStatusHandler.h"
2014-03-21 13:42:14 +00:00
# include "InterserverIOHTTPHandler.h"
2012-03-09 15:46:52 +00:00
# include "TCPHandler.h"
2016-01-17 13:34:36 +00:00
# include "MetricsTransmitter.h"
# include "UsersConfigReloader.h"
# include "StatusFile.h"
2012-03-09 03:06:09 +00:00
2014-07-21 11:21:09 +00:00
2012-03-09 03:06:09 +00:00
namespace DB
{
2016-09-21 15:22:38 +00:00
namespace ErrorCodes
{
extern const int NO_ELEMENTS_IN_CONFIG ;
}
2015-11-05 20:08:18 +00:00
/// Отвечает "Ok.\n". Используется для проверки живости.
2012-03-09 03:06:09 +00:00
class PingRequestHandler : public Poco : : Net : : HTTPRequestHandler
{
public :
2015-11-05 20:08:18 +00:00
void handleRequest ( Poco : : Net : : HTTPServerRequest & request , Poco : : Net : : HTTPServerResponse & response )
2012-03-09 03:06:09 +00:00
{
2015-11-05 20:08:18 +00:00
try
{
const char * data = " Ok. \n " ;
response . sendBuffer ( data , strlen ( data ) ) ;
}
catch ( . . . )
{
tryLogCurrentException ( " PingRequestHandler " ) ;
}
}
} ;
/// Отвечает 404 с подробным объяснением.
class NotFoundHandler : public Poco : : Net : : HTTPRequestHandler
{
public :
void handleRequest ( Poco : : Net : : HTTPServerRequest & request , Poco : : Net : : HTTPServerResponse & response )
{
try
{
response . setStatusAndReason ( Poco : : Net : : HTTPResponse : : HTTP_NOT_FOUND ) ;
response . send ( ) < < " There is no handle " < < request . getURI ( ) < < " \n \n "
< < " Use / or /ping for health checks. \n "
< < " Or /replicas_status for more sophisticated health checks. \n \n "
< < " Send queries from your program with POST method or GET /?query=... \n \n "
< < " Use clickhouse-client: \n \n "
< < " For interactive data analysis: \n "
< < " clickhouse-client \n \n "
< < " For batch query processing: \n "
< < " clickhouse-client --query='SELECT 1' > result \n "
< < " clickhouse-client < query > result \n " ;
}
catch ( . . . )
{
tryLogCurrentException ( " NotFoundHandler " ) ;
2014-07-16 00:53:56 +00:00
}
2012-03-09 03:06:09 +00:00
}
} ;
2015-09-25 16:28:17 +00:00
template < typename HandlerType >
2012-12-14 11:21:07 +00:00
class HTTPRequestHandlerFactory : public Poco : : Net : : HTTPRequestHandlerFactory
2012-03-09 03:06:09 +00:00
{
2012-12-14 11:21:07 +00:00
private :
Server & server ;
Logger * log ;
std : : string name ;
2014-07-16 00:53:56 +00:00
2012-12-14 11:21:07 +00:00
public :
HTTPRequestHandlerFactory ( Server & server_ , const std : : string & name_ )
: server ( server_ ) , log ( & Logger : : get ( name_ ) ) , name ( name_ ) { }
2014-07-16 00:53:56 +00:00
2012-12-14 11:21:07 +00:00
Poco : : Net : : HTTPRequestHandler * createRequestHandler ( const Poco : : Net : : HTTPServerRequest & request )
{
LOG_TRACE ( log , " HTTP Request for " < < name < < " . "
< < " Method: " < < request . getMethod ( )
< < " , Address: " < < request . clientAddress ( ) . toString ( )
< < " , User-Agent: " < < ( request . has ( " User-Agent " ) ? request . get ( " User-Agent " ) : " none " ) ) ;
2015-11-05 20:08:18 +00:00
const auto & uri = request . getURI ( ) ;
2016-01-17 13:41:36 +00:00
if ( request . getMethod ( ) = = Poco : : Net : : HTTPRequest : : HTTP_GET
2015-09-25 15:54:54 +00:00
| | request . getMethod ( ) = = Poco : : Net : : HTTPRequest : : HTTP_HEAD )
2015-09-25 16:28:17 +00:00
{
2015-11-05 20:08:18 +00:00
if ( uri = = " / " | | uri = = " /ping " )
return new PingRequestHandler ;
2016-07-14 05:22:09 +00:00
else if ( startsWith ( uri , " /replicas_status " ) )
2015-11-05 20:24:27 +00:00
return new ReplicasStatusHandler ( * server . global_context ) ;
2015-09-25 16:28:17 +00:00
}
2016-01-17 13:41:36 +00:00
if ( uri . find ( ' ? ' ) ! = std : : string : : npos
| | request . getMethod ( ) = = Poco : : Net : : HTTPRequest : : HTTP_POST )
{
return new HandlerType ( server ) ;
}
if ( request . getMethod ( ) = = Poco : : Net : : HTTPRequest : : HTTP_GET
| | request . getMethod ( ) = = Poco : : Net : : HTTPRequest : : HTTP_HEAD
| | request . getMethod ( ) = = Poco : : Net : : HTTPRequest : : HTTP_POST )
{
return new NotFoundHandler ;
}
return nullptr ;
2012-12-14 11:21:07 +00:00
}
} ;
2012-03-09 03:06:09 +00:00
2012-12-14 11:21:07 +00:00
class TCPConnectionFactory : public Poco : : Net : : TCPServerConnectionFactory
2012-03-09 15:46:52 +00:00
{
2012-12-14 11:21:07 +00:00
private :
Server & server ;
Logger * log ;
2014-07-16 00:53:56 +00:00
2012-12-14 11:21:07 +00:00
public :
TCPConnectionFactory ( Server & server_ ) : server ( server_ ) , log ( & Logger : : get ( " TCPConnectionFactory " ) ) { }
2012-03-09 15:46:52 +00:00
2012-12-14 11:21:07 +00:00
Poco : : Net : : TCPServerConnection * createConnection ( const Poco : : Net : : StreamSocket & socket )
{
2013-01-28 18:20:20 +00:00
LOG_TRACE ( log , " TCP Request. " < < " Address: " < < socket . peerAddress ( ) . toString ( ) ) ;
2014-07-16 00:53:56 +00:00
2012-12-14 11:21:07 +00:00
return new TCPHandler ( server , socket ) ;
}
} ;
2012-03-09 15:46:52 +00:00
2012-03-09 03:06:09 +00:00
int Server : : main ( const std : : vector < std : : string > & args )
{
2012-06-24 23:35:13 +00:00
Logger * log = & logger ( ) ;
2014-04-15 16:39:56 +00:00
2014-12-21 03:18:40 +00:00
std : : string path = config ( ) . getString ( " path " ) ;
Poco : : trimInPlace ( path ) ;
if ( path . empty ( ) )
throw Exception ( " path configuration parameter is empty " ) ;
if ( path . back ( ) ! = ' / ' )
path + = ' / ' ;
StatusFile status { path + " status " } ;
2014-04-15 16:39:56 +00:00
/// Попробуем повысить ограничение на число открытых файлов.
{
rlimit rlim ;
if ( getrlimit ( RLIMIT_NOFILE , & rlim ) )
throw Poco : : Exception ( " Cannot getrlimit " ) ;
if ( rlim . rlim_cur = = rlim . rlim_max )
{
LOG_DEBUG ( log , " rlimit on number of file descriptors is " < < rlim . rlim_cur ) ;
}
else
{
rlim_t old = rlim . rlim_cur ;
rlim . rlim_cur = rlim . rlim_max ;
if ( setrlimit ( RLIMIT_NOFILE , & rlim ) )
throw Poco : : Exception ( " Cannot setrlimit " ) ;
LOG_DEBUG ( log , " Set rlimit on number of file descriptors to " < < rlim . rlim_cur < < " (was " < < old < < " ) " ) ;
}
}
2014-12-14 22:51:21 +00:00
static ServerErrorHandler error_handler ;
Poco : : ErrorHandler : : set ( & error_handler ) ;
2012-03-09 03:06:09 +00:00
/// Заранее инициализируем DateLUT, чтобы первая инициализация потом не влияла на измеряемую скорость выполнения.
2012-06-24 23:25:07 +00:00
LOG_DEBUG ( log , " Initializing DateLUT. " ) ;
2014-07-08 23:52:53 +00:00
DateLUT : : instance ( ) ;
2012-06-24 23:25:07 +00:00
LOG_TRACE ( log , " Initialized DateLUT. " ) ;
2012-03-09 03:56:12 +00:00
2016-09-23 20:41:14 +00:00
global_context = std : : make_unique < Context > ( ) ;
2013-09-14 05:14:22 +00:00
2012-03-09 03:56:12 +00:00
/** Контекст содержит всё, что влияет на обработку запроса:
* н а с т р о й к и , н а б о р ф у н к ц и й , т и п о в д а н н ы х , а г р е г а т н ы х ф у н к ц и й , б а з д а н н ы х . . .
*/
2013-09-14 05:14:22 +00:00
global_context - > setGlobalContext ( * global_context ) ;
2014-12-21 03:18:40 +00:00
global_context - > setPath ( path ) ;
2012-03-09 03:06:09 +00:00
2016-08-10 00:47:00 +00:00
/// Directory with temporary data for processing of hard queries.
2015-12-08 01:43:39 +00:00
{
std : : string tmp_path = config ( ) . getString ( " tmp_path " , path + " tmp/ " ) ;
global_context - > setTemporaryPath ( tmp_path ) ;
Poco : : File ( tmp_path ) . createDirectories ( ) ;
2016-08-10 00:47:00 +00:00
/// Clearing old temporary files.
2015-12-08 01:43:39 +00:00
Poco : : DirectoryIterator dir_end ;
for ( Poco : : DirectoryIterator it ( tmp_path ) ; it ! = dir_end ; + + it )
{
2016-07-14 05:22:09 +00:00
if ( it - > isFile ( ) & & startsWith ( it . name ( ) , " tmp " ) )
2015-12-08 01:43:39 +00:00
{
LOG_DEBUG ( log , " Removing old temporary file " < < it - > path ( ) ) ;
it - > remove ( ) ;
}
}
}
2015-01-07 17:19:23 +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 .
*/
Poco : : File ( path + " flags/ " ) . createDirectories ( ) ;
2014-10-06 03:49:56 +00:00
bool has_zookeeper = false ;
2014-05-12 00:49:24 +00:00
if ( config ( ) . has ( " zookeeper " ) )
2014-10-06 03:49:56 +00:00
{
2015-04-16 07:32:40 +00:00
global_context - > setZooKeeper ( std : : make_shared < zkutil : : ZooKeeper > ( config ( ) , " zookeeper " ) ) ;
2014-10-06 03:49:56 +00:00
has_zookeeper = true ;
}
2014-03-21 19:17:59 +00:00
2014-05-12 00:49:24 +00:00
if ( config ( ) . has ( " interserver_http_port " ) )
2014-04-03 08:47:59 +00:00
{
2015-08-11 21:11:54 +00:00
String this_host = config ( ) . getString ( " interserver_http_host " , " " ) ;
if ( this_host . empty ( ) )
2015-05-29 00:33:56 +00:00
{
this_host = getFQDNOrHostName ( ) ;
2015-08-11 21:11:54 +00:00
LOG_DEBUG ( log , " Configuration parameter 'interserver_http_host' doesn't exist or exists and empty. Will use ' " + this_host + " ' as replica host. " ) ;
2015-05-29 00:33:56 +00:00
}
2014-04-03 08:47:59 +00:00
2014-05-12 00:49:24 +00:00
String port_str = config ( ) . getString ( " interserver_http_port " ) ;
2014-04-03 08:47:59 +00:00
int port = parse < int > ( port_str ) ;
2014-11-19 20:40:51 +00:00
if ( port < 0 | | port > 0xFFFF )
throw Exception ( " Out of range 'interserver_http_port': " + toString ( port ) , ErrorCodes : : ARGUMENT_OUT_OF_BOUND ) ;
global_context - > setInterserverIOAddress ( this_host , port ) ;
2014-04-03 08:47:59 +00:00
}
2014-08-11 15:59:01 +00:00
if ( config ( ) . has ( " macros " ) )
global_context - > setMacros ( Macros ( config ( ) , " macros " ) ) ;
2014-05-14 14:19:58 +00:00
std : : string users_config_path = config ( ) . getString ( " users_config " , config ( ) . getString ( " config-file " , " config.xml " ) ) ;
2015-02-10 21:10:58 +00:00
auto users_config_reloader = std : : make_unique < UsersConfigReloader > ( users_config_path , global_context . get ( ) ) ;
2013-08-12 00:36:18 +00:00
2013-09-03 20:21:28 +00:00
/// Максимальное количество одновременно выполняющихся запросов.
2014-05-12 00:49:24 +00:00
global_context - > getProcessList ( ) . setMaxSize ( config ( ) . getInt ( " max_concurrent_queries " , 0 ) ) ;
2013-09-03 20:21:28 +00:00
2013-09-08 05:53:10 +00:00
/// Размер кэша разжатых блоков. Если нулевой - кэш отключён.
2014-05-12 00:49:24 +00:00
size_t uncompressed_cache_size = parse < size_t > ( config ( ) . getString ( " uncompressed_cache_size " , " 0 " ) ) ;
2013-09-08 05:53:10 +00:00
if ( uncompressed_cache_size )
2013-09-14 05:14:22 +00:00
global_context - > setUncompressedCache ( uncompressed_cache_size ) ;
2013-09-08 05:53:10 +00:00
2014-06-04 13:39:56 +00:00
/// Размер кэша засечек. Обязательный параметр.
size_t mark_cache_size = parse < size_t > ( config ( ) . getString ( " mark_cache_size " ) ) ;
2014-02-11 13:30:42 +00:00
if ( mark_cache_size )
global_context - > setMarkCache ( mark_cache_size ) ;
2012-10-22 20:08:28 +00:00
/// Загружаем настройки.
2013-09-14 05:14:22 +00:00
Settings & settings = global_context - > getSettingsRef ( ) ;
2014-05-12 00:49:24 +00:00
global_context - > setSetting ( " profile " , config ( ) . getString ( " default_profile " , " default " ) ) ;
2012-10-22 20:08:28 +00:00
LOG_INFO ( log , " Loading metadata. " ) ;
2013-09-14 05:14:22 +00:00
loadMetadata ( * global_context ) ;
2012-10-22 20:08:28 +00:00
LOG_DEBUG ( log , " Loaded metadata. " ) ;
2016-08-09 21:48:05 +00:00
/// Create system tables.
2015-06-26 22:37:55 +00:00
if ( ! global_context - > isDatabaseExist ( " system " ) )
{
Poco : : File ( path + " data/system " ) . createDirectories ( ) ;
Poco : : File ( path + " metadata/system " ) . createDirectories ( ) ;
2016-03-19 01:18:49 +00:00
2016-06-10 20:46:58 +00:00
auto system_database = std : : make_shared < DatabaseOrdinary > ( " system " , path + " metadata/system/ " ) ;
global_context - > addDatabase ( " system " , system_database ) ;
2016-08-09 21:48:05 +00:00
/// 'has_force_restore_data_flag' is true, to not fail on loading query_log table, if it is corrupted.
system_database - > loadTables ( * global_context , nullptr , true ) ;
2015-06-26 22:37:55 +00:00
}
2014-07-16 00:53:56 +00:00
2016-03-19 01:18:49 +00:00
DatabasePtr system_database = global_context - > getDatabase ( " system " ) ;
system_database - > attachTable ( " one " , StorageSystemOne : : create ( " one " ) ) ;
system_database - > attachTable ( " numbers " , StorageSystemNumbers : : create ( " numbers " ) ) ;
system_database - > attachTable ( " numbers_mt " , StorageSystemNumbers : : create ( " numbers_mt " , true ) ) ;
system_database - > attachTable ( " tables " , StorageSystemTables : : create ( " tables " ) ) ;
system_database - > attachTable ( " parts " , StorageSystemParts : : create ( " parts " ) ) ;
system_database - > attachTable ( " databases " , StorageSystemDatabases : : create ( " databases " ) ) ;
system_database - > attachTable ( " processes " , StorageSystemProcesses : : create ( " processes " ) ) ;
system_database - > attachTable ( " settings " , StorageSystemSettings : : create ( " settings " ) ) ;
system_database - > attachTable ( " events " , StorageSystemEvents : : create ( " events " ) ) ;
system_database - > attachTable ( " metrics " , StorageSystemMetrics : : create ( " metrics " ) ) ;
system_database - > attachTable ( " merges " , StorageSystemMerges : : create ( " merges " ) ) ;
system_database - > attachTable ( " replicas " , StorageSystemReplicas : : create ( " replicas " ) ) ;
system_database - > attachTable ( " replication_queue " , StorageSystemReplicationQueue : : create ( " replication_queue " ) ) ;
system_database - > attachTable ( " dictionaries " , StorageSystemDictionaries : : create ( " dictionaries " ) ) ;
system_database - > attachTable ( " columns " , StorageSystemColumns : : create ( " columns " ) ) ;
system_database - > attachTable ( " functions " , StorageSystemFunctions : : create ( " functions " ) ) ;
system_database - > attachTable ( " clusters " , StorageSystemClusters : : create ( " clusters " , * global_context ) ) ;
2014-07-16 00:53:56 +00:00
2014-10-06 03:49:56 +00:00
if ( has_zookeeper )
2016-03-19 01:18:49 +00:00
system_database - > attachTable ( " zookeeper " , StorageSystemZooKeeper : : create ( " zookeeper " ) ) ;
2014-10-06 03:49:56 +00:00
2014-05-12 00:49:24 +00:00
global_context - > setCurrentDatabase ( config ( ) . getString ( " default_database " , " default " ) ) ;
2012-10-22 20:08:28 +00:00
2016-03-01 17:47:53 +00:00
bool has_resharding_worker = false ;
2016-01-28 01:00:42 +00:00
if ( has_zookeeper & & config ( ) . has ( " resharding " ) )
2016-01-28 01:00:27 +00:00
{
2016-01-28 01:00:42 +00:00
auto resharding_worker = std : : make_shared < ReshardingWorker > ( config ( ) , " resharding " , * global_context ) ;
global_context - > setReshardingWorker ( resharding_worker ) ;
resharding_worker - > start ( ) ;
2016-03-01 17:47:53 +00:00
has_resharding_worker = true ;
2016-01-28 01:00:27 +00:00
}
2015-04-02 16:30:18 +00:00
SCOPE_EXIT (
LOG_DEBUG ( log , " Closed all connections. " ) ;
2016-03-22 19:35:23 +00:00
/** Попросим завершить фоновую работу у всех движков таблиц,
* а т а к ж е у query_log - а .
2015-04-02 16:30:18 +00:00
* Э т о в а ж н о д е л а т ь з а р а н е е , н е в д е с т р у к т о р е Context - а , т а к к а к
* д в и ж к и т а б л и ц м о г у т п р и у н и ч т о ж е н и и в с ё е щ ё п о л ь з о в а т ь с я Context - о м .
*/
LOG_INFO ( log , " Shutting down storages. " ) ;
global_context - > shutdown ( ) ;
LOG_DEBUG ( log , " Shutted down storages. " ) ;
/** Явно уничтожаем контекст - это удобнее, чем в деструкторе Server-а , так как ещё доступен логгер.
* В э т о т м о м е н т н и к т о б о л ь ш е н е д о л ж е н в л а д е т ь shared - ч а с т ь ю к о н т е к с т а .
*/
global_context . reset ( ) ;
LOG_DEBUG ( log , " Destroyed global context. " ) ;
) ;
2013-09-14 05:14:22 +00:00
{
2016-01-17 13:34:36 +00:00
const auto metrics_transmitter = config ( ) . getBool ( " use_graphite " , true )
? std : : make_unique < MetricsTransmitter > ( )
2014-07-21 11:21:09 +00:00
: nullptr ;
2014-11-19 20:52:42 +00:00
const std : : string listen_host = config ( ) . getString ( " listen_host " , " :: " ) ;
2014-05-12 00:49:24 +00:00
Poco : : Timespan keep_alive_timeout ( config ( ) . getInt ( " keep_alive_timeout " , 10 ) , 0 ) ;
2013-09-14 05:14:22 +00:00
2014-05-12 00:49:24 +00:00
Poco : : ThreadPool server_pool ( 3 , config ( ) . getInt ( " max_connections " , 1024 ) ) ;
2014-03-21 13:42:14 +00:00
Poco : : Net : : HTTPServerParams : : Ptr http_params = new Poco : : Net : : HTTPServerParams ;
2013-09-14 05:14:22 +00:00
http_params - > setTimeout ( settings . receive_timeout ) ;
http_params - > setKeepAliveTimeout ( keep_alive_timeout ) ;
2016-09-21 15:22:38 +00:00
/// For testing purposes, user may omit tcp_port or http_port in configuration file.
2016-06-25 03:06:36 +00:00
2016-09-21 15:22:38 +00:00
/// HTTP
std : : experimental : : optional < Poco : : Net : : HTTPServer > http_server ;
if ( config ( ) . has ( " http_port " ) )
2016-06-25 03:06:36 +00:00
{
2016-09-21 15:22:38 +00:00
Poco : : Net : : SocketAddress http_socket_address ;
try
2016-06-25 03:06:36 +00:00
{
2016-09-21 15:22:38 +00:00
http_socket_address = Poco : : Net : : SocketAddress ( listen_host , config ( ) . getInt ( " http_port " ) ) ;
}
catch ( const Poco : : Net : : DNSException & e )
{
/// Better message when IPv6 is disabled on host.
if ( e . code ( ) = = EAI_ADDRFAMILY )
{
LOG_ERROR ( log , " Cannot resolve listen_host ( " < < listen_host + " ), error: " < < e . message ( ) < < " . "
" If it is an IPv6 address and your host has disabled IPv6, then consider to specify IPv4 address to listen in <listen_host> element of configuration file. Example: <listen_host>0.0.0.0</listen_host> " ) ;
}
throw ;
2016-06-25 03:06:36 +00:00
}
2016-09-21 15:22:38 +00:00
Poco : : Net : : ServerSocket http_socket ( http_socket_address ) ;
http_socket . setReceiveTimeout ( settings . receive_timeout ) ;
http_socket . setSendTimeout ( settings . send_timeout ) ;
http_server . emplace (
new HTTPRequestHandlerFactory < HTTPHandler > ( * this , " HTTPHandler-factory " ) ,
server_pool ,
http_socket ,
http_params ) ;
2016-06-25 03:06:36 +00:00
}
2013-09-14 05:14:22 +00:00
/// TCP
2016-09-21 15:22:38 +00:00
std : : experimental : : optional < Poco : : Net : : TCPServer > tcp_server ;
if ( config ( ) . has ( " tcp_port " ) )
{
Poco : : Net : : ServerSocket tcp_socket ( Poco : : Net : : SocketAddress ( listen_host , config ( ) . getInt ( " tcp_port " ) ) ) ;
tcp_socket . setReceiveTimeout ( settings . receive_timeout ) ;
tcp_socket . setSendTimeout ( settings . send_timeout ) ;
tcp_server . emplace (
new TCPConnectionFactory ( * this ) ,
server_pool ,
tcp_socket ,
new Poco : : Net : : TCPServerParams ) ;
}
/// At least one of TCP and HTTP servers must be created.
if ( ! http_server & & ! tcp_server )
throw Exception ( " No 'tcp_port' and 'http_port' is specified in configuration file. " , ErrorCodes : : NO_ELEMENTS_IN_CONFIG ) ;
2013-09-14 05:14:22 +00:00
2014-03-21 13:42:14 +00:00
/// Interserver IO HTTP
2016-05-28 14:14:18 +00:00
std : : experimental : : optional < Poco : : Net : : HTTPServer > interserver_io_http_server ;
2014-05-12 00:49:24 +00:00
if ( config ( ) . has ( " interserver_http_port " ) )
2014-03-21 13:42:14 +00:00
{
2014-11-19 20:52:42 +00:00
Poco : : Net : : ServerSocket interserver_io_http_socket ( Poco : : Net : : SocketAddress ( listen_host , config ( ) . getInt ( " interserver_http_port " ) ) ) ;
2014-03-21 13:42:14 +00:00
interserver_io_http_socket . setReceiveTimeout ( settings . receive_timeout ) ;
interserver_io_http_socket . setSendTimeout ( settings . send_timeout ) ;
2016-05-28 14:37:51 +00:00
interserver_io_http_server . emplace (
2014-03-21 13:42:14 +00:00
new HTTPRequestHandlerFactory < InterserverIOHTTPHandler > ( * this , " InterserverIOHTTPHandler-factory " ) ,
server_pool ,
interserver_io_http_socket ,
http_params ) ;
}
2016-09-21 15:22:38 +00:00
http_server - > start ( ) ;
tcp_server - > start ( ) ;
2014-03-21 13:42:14 +00:00
if ( interserver_io_http_server )
interserver_io_http_server - > start ( ) ;
2013-09-14 05:14:22 +00:00
2015-04-02 16:41:07 +00:00
LOG_INFO ( log , " Ready for connections. " ) ;
2015-04-02 16:30:18 +00:00
SCOPE_EXIT (
2016-03-02 20:08:38 +00:00
LOG_DEBUG ( log , " Received termination signal. " ) ;
if ( has_resharding_worker )
{
LOG_INFO ( log , " Shutting down resharding thread " ) ;
auto & resharding_worker = global_context - > getReshardingWorker ( ) ;
if ( resharding_worker . isStarted ( ) )
resharding_worker . shutdown ( ) ;
LOG_DEBUG ( log , " Shut down resharding thread " ) ;
}
LOG_DEBUG ( log , " Waiting for current connections to close. " ) ;
2015-04-02 16:30:18 +00:00
users_config_reloader . reset ( ) ;
is_cancelled = true ;
2016-09-21 15:22:38 +00:00
http_server - > stop ( ) ;
tcp_server - > stop ( ) ;
2015-04-02 16:30:18 +00:00
) ;
2015-03-27 13:11:22 +00:00
/// try to load dictionaries immediately, throw on error and die
try
{
if ( ! config ( ) . getBool ( " dictionaries_lazy_load " , true ) )
{
2015-04-02 16:30:18 +00:00
global_context - > tryCreateDictionaries ( ) ;
global_context - > tryCreateExternalDictionaries ( ) ;
2015-03-27 13:11:22 +00:00
}
waitForTerminationRequest ( ) ;
}
catch ( . . . )
{
LOG_ERROR ( log , " Caught exception while loading dictionaries. " ) ;
2015-04-02 16:30:18 +00:00
throw ;
2015-03-27 13:11:22 +00:00
}
2012-12-14 11:21:07 +00:00
}
2012-03-09 15:46:52 +00:00
2012-03-09 03:06:09 +00:00
return Application : : EXIT_OK ;
}
}
YANDEX_APP_SERVER_MAIN ( DB : : Server ) ;