Server.cpp 29.5 KB
Newer Older
1
#include "Server.h"
2

P
proller 已提交
3
#include <memory>
4
#include <sys/resource.h>
5
#include <errno.h>
6
#include <Poco/Version.h>
P
proller 已提交
7
#include <Poco/DirectoryIterator.h>
V
Vadim Skipin 已提交
8
#include <Poco/Net/HTTPServer.h>
9
#include <Poco/Net/NetException.h>
V
Vadim Skipin 已提交
10
#include <ext/scope_guard.h>
11
#include <common/logger_useful.h>
12
#include <common/ErrorHandlers.h>
V
Vadim Skipin 已提交
13 14
#include <common/getMemoryAmount.h>
#include <Common/ClickHouseRevision.h>
15
#include <Common/DNSResolver.h>
V
Vadim Skipin 已提交
16
#include <Common/CurrentMetrics.h>
17
#include <Common/Macros.h>
18
#include <Common/StringUtils/StringUtils.h>
V
Vadim Skipin 已提交
19 20 21
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/ZooKeeperNodeCache.h>
#include <Common/config.h>
22 23
#include <Common/getFQDNOrHostName.h>
#include <Common/getMultipleKeysFromConfig.h>
24
#include <Common/getNumberOfPhysicalCPUCores.h>
25
#include <Common/TaskStatsInfoGetter.h>
26
#include <IO/HTTPCommon.h>
P
proller 已提交
27
#include <IO/UseSSL.h>
28
#include <Interpreters/AsynchronousMetrics.h>
V
Vadim Skipin 已提交
29
#include <Interpreters/DDLWorker.h>
30 31
#include <Interpreters/ProcessList.h>
#include <Interpreters/loadMetadata.h>
32
#include <Interpreters/DNSCacheUpdater.h>
33 34
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/System/attachSystemTables.h>
V
Vadim Skipin 已提交
35 36 37
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <Functions/registerFunctions.h>
#include <TableFunctions/registerTableFunctions.h>
38
#include <Storages/registerStorages.h>
39
#include <Common/Config/ConfigReloader.h>
V
Vadim Skipin 已提交
40
#include "HTTPHandlerFactory.h"
41
#include "MetricsTransmitter.h"
A
Alexey Milovidov 已提交
42
#include <Common/StatusFile.h>
V
Vadim Skipin 已提交
43
#include "TCPHandlerFactory.h"
44

P
proller 已提交
45 46 47 48 49
#if defined(__linux__)
#include <Common/hasLinuxCapability.h>
#include <sys/mman.h>
#endif

50
#if USE_POCO_NETSSL
51 52 53
#include <Poco/Net/Context.h>
#include <Poco/Net/SecureServerSocket.h>
#endif
54

55 56 57 58 59
namespace CurrentMetrics
{
    extern const Metric Revision;
}

60 61
namespace DB
{
62

63 64 65 66
namespace ErrorCodes
{
    extern const int NO_ELEMENTS_IN_CONFIG;
    extern const int SUPPORT_IS_DISABLED;
67
    extern const int ARGUMENT_OUT_OF_BOUND;
68
    extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
69
    extern const int SYSTEM_ERROR;
70 71 72
}


73
static std::string getCanonicalPath(std::string && path)
74
{
75 76 77 78 79
    Poco::trimInPlace(path);
    if (path.empty())
        throw Exception("path configuration parameter is empty");
    if (path.back() != '/')
        path += '/';
80
    return std::move(path);
81 82
}

83 84 85 86 87 88 89 90 91 92 93 94
void Server::uninitialize()
{
    logger().information("shutting down");
    BaseDaemon::uninitialize();
}

void Server::initialize(Poco::Util::Application & self)
{
    BaseDaemon::initialize(self);
    logger().information("starting up");
}

95 96
std::string Server::getDefaultCorePath() const
{
97
    return getCanonicalPath(config().getString("path")) + "cores";
98
}
99

A
Alexey Milovidov 已提交
100
int Server::main(const std::vector<std::string> & /*args*/)
101
{
102 103
    Logger * log = &logger();

P
proller 已提交
104 105
    UseSSL use_ssl;

P
proller 已提交
106
    registerFunctions();
107
    registerAggregateFunctions();
108
    registerTableFunctions();
109
    registerStorages();
P
proller 已提交
110

111 112
    CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::get());

113 114 115
    /** Context contains all that query execution is dependent:
      *  settings, available functions, data types, aggregate functions, databases...
      */
116
    global_context = std::make_unique<Context>(Context::createGlobal());
117 118 119
    global_context->setGlobalContext(*global_context);
    global_context->setApplicationType(Context::ApplicationType::SERVER);

120
    bool has_zookeeper = config().has("zookeeper");
121 122 123 124 125

    zkutil::ZooKeeperNodeCache main_config_zk_node_cache([&] { return global_context->getZooKeeper(); });
    if (loaded_config.has_zk_includes)
    {
        auto old_configuration = loaded_config.configuration;
126 127 128 129
        ConfigProcessor config_processor(config_path);
        loaded_config = config_processor.loadConfigWithZooKeeperIncludes(
            main_config_zk_node_cache, /* fallback_to_preprocessed = */ true);
        config_processor.savePreprocessedConfig(loaded_config);
130 131 132 133
        config().removeConfiguration(old_configuration.get());
        config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false);
    }

P
proller 已提交
134
    const auto memory_amount = getMemoryAmount();
A
Alexey Milovidov 已提交
135

P
proller 已提交
136
#if defined(__linux__)
A
Alexey Milovidov 已提交
137 138 139
    /// After full config loaded
    {
        if (config().getBool("mlock_executable", false))
P
proller 已提交
140 141 142
        {
            if (hasLinuxCapability(CAP_IPC_LOCK))
            {
A
Alexey Milovidov 已提交
143
                LOG_TRACE(log, "Will mlockall to prevent executable memory from being paged out. It may take a few seconds.");
P
proller 已提交
144
                if (0 != mlockall(MCL_CURRENT))
145
                    LOG_WARNING(log, "Failed mlockall: " + errnoToString(ErrorCodes::SYSTEM_ERROR));
P
proller 已提交
146
                else
A
Alexey Milovidov 已提交
147
                    LOG_TRACE(log, "The memory map of clickhouse executable has been mlock'ed");
P
proller 已提交
148 149 150
            }
            else
            {
A
Alexey Milovidov 已提交
151 152 153 154
                LOG_INFO(log, "It looks like the process has no CAP_IPC_LOCK capability, binary mlock will be disabled."
                    " It could happen due to incorrect ClickHouse package installation."
                    " You could resolve the problem manually with 'sudo setcap cap_ipc_lock=+ep /usr/bin/clickhouse'."
                    " Note that it will not work on 'nosuid' mounted filesystems.");
P
proller 已提交
155 156 157
            }
        }
    }
A
Alexey Milovidov 已提交
158
#endif
P
proller 已提交
159

160 161 162 163 164 165 166 167 168 169 170
    std::string path = getCanonicalPath(config().getString("path"));
    std::string default_database = config().getString("default_database", "default");

    global_context->setPath(path);

    /// Create directories for 'path' and for default database, if not exist.
    Poco::File(path + "data/" + default_database).createDirectories();
    Poco::File(path + "metadata/" + default_database).createDirectories();

    StatusFile status{path + "status"};

171 172 173 174 175 176 177 178 179
    SCOPE_EXIT({
        /** 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();

        LOG_DEBUG(log, "Destroyed global context.");
    });

180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196
    /// Try to increase limit on number of open files.
    {
        rlimit rlim;
        if (getrlimit(RLIMIT_NOFILE, &rlim))
            throw Poco::Exception("Cannot getrlimit");

        if (rlim.rlim_cur == rlim.rlim_max)
        {
            LOG_DEBUG(log, "rlimit on number of file descriptors is " << rlim.rlim_cur);
        }
        else
        {
            rlim_t old = rlim.rlim_cur;
            rlim.rlim_cur = config().getUInt("max_open_files", rlim.rlim_max);
            int rc = setrlimit(RLIMIT_NOFILE, &rlim);
            if (rc != 0)
                LOG_WARNING(log,
197 198 199
                    "Cannot set max number of file descriptors to " << rlim.rlim_cur
                        << ". Try to specify max_open_files according to your system limits. error: "
                        << strerror(errno));
200 201 202 203 204 205 206 207 208 209 210 211 212
            else
                LOG_DEBUG(log, "Set max number of file descriptors to " << rlim.rlim_cur << " (was " << old << ").");
        }
    }

    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.");
    DateLUT::instance();
    LOG_TRACE(log, "Initialized DateLUT with time zone `" << DateLUT::instance().getTimeZone() << "'.");

213
    /// Directory with temporary data for processing of heavy queries.
214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234
    {
        std::string tmp_path = config().getString("tmp_path", path + "tmp/");
        global_context->setTemporaryPath(tmp_path);
        Poco::File(tmp_path).createDirectories();

        /// Clearing old temporary files.
        Poco::DirectoryIterator dir_end;
        for (Poco::DirectoryIterator it(tmp_path); it != dir_end; ++it)
        {
            if (it->isFile() && startsWith(it.name(), "tmp"))
            {
                LOG_DEBUG(log, "Removing old temporary file " << it->path());
                it->remove();
            }
        }
    }

    /** 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.
      */
235 236 237 238 239 240 241 242 243 244 245 246 247
    {
        Poco::File(path + "flags/").createDirectories();
        global_context->setFlagsPath(path + "flags/");
    }

    /** Directory with user provided files that are usable by 'file' table function.
      */
    {

        std::string user_files_path = config().getString("user_files_path", path + "user_files/");
        global_context->setUserFilesPath(user_files_path);
        Poco::File(user_files_path).createDirectories();
    }
248

249 250 251 252
    if (config().has("interserver_http_port") && config().has("interserver_https_port"))
        throw Exception("Both http and https interserver ports are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);

    static const auto interserver_tags =
253
    {
254 255 256
        std::make_tuple("interserver_http_host", "interserver_http_port", "http"),
        std::make_tuple("interserver_https_host", "interserver_https_port", "https")
    };
257

258 259 260
    for (auto [host_tag, port_tag, scheme] : interserver_tags)
    {
        if (config().has(port_tag))
261
        {
262 263 264 265 266 267 268 269 270
            String this_host = config().getString(host_tag, "");

            if (this_host.empty())
            {
                this_host = getFQDNOrHostName();
                LOG_DEBUG(log,
                    "Configuration parameter '" + String(host_tag) + "' doesn't exist or exists and empty. Will use '" + this_host
                        + "' as replica host.");
            }
271

272 273
            String port_str = config().getString(port_tag);
            int port = parse<int>(port_str);
274

275 276
            if (port < 0 || port > 0xFFFF)
                throw Exception("Out of range '" + String(port_tag) + "': " + toString(port), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
277

278 279
            global_context->setInterserverIOAddress(this_host, port);
            global_context->setInterserverScheme(scheme);
280 281 282
        }
    }

283
    if (config().has("interserver_http_credentials"))
284 285 286
    {
        String user = config().getString("interserver_http_credentials.user", "");
        String password = config().getString("interserver_http_credentials.password", "");
287

288
        if (user.empty())
A
alesapin 已提交
289
            throw Exception("Configuration parameter interserver_http_credentials user can't be empty", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
290

291
        global_context->setInterserverCredentials(user, password);
292 293 294
    }

    if (config().has("macros"))
A
Alexey Milovidov 已提交
295
        global_context->setMacros(std::make_unique<Macros>(config(), "macros"));
296 297 298 299 300 301

    /// Initialize main config reloader.
    std::string include_from_path = config().getString("include_from", "/etc/metrika.xml");
    auto main_config_reloader = std::make_unique<ConfigReloader>(config_path,
        include_from_path,
        std::move(main_config_zk_node_cache),
302 303
        [&](ConfigurationPtr config)
        {
304
            buildLoggers(*config);
305
            global_context->setClustersConfig(config);
A
Alexey Milovidov 已提交
306
            global_context->setMacros(std::make_unique<Macros>(*config, "macros"));
307
        },
308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325
        /* already_loaded = */ true);

    /// Initialize users config reloader.
    std::string users_config_path = config().getString("users_config", config_path);
    /// If path to users' config isn't absolute, try guess its root (current) dir.
    /// At first, try to find it in dir of main config, after will use current dir.
    if (users_config_path.empty() || users_config_path[0] != '/')
    {
        std::string config_dir = Poco::Path(config_path).parent().toString();
        if (Poco::File(config_dir + users_config_path).exists())
            users_config_path = config_dir + users_config_path;
    }
    auto users_config_reloader = std::make_unique<ConfigReloader>(users_config_path,
        include_from_path,
        zkutil::ZooKeeperNodeCache([&] { return global_context->getZooKeeper(); }),
        [&](ConfigurationPtr config) { global_context->setUsersConfig(config); },
        /* already_loaded = */ false);

326
    /// Reload config in SYSTEM RELOAD CONFIG query.
327 328
    global_context->setConfigReloadCallback([&]()
    {
329 330 331 332
        main_config_reloader->reload();
        users_config_reloader->reload();
    });

A
Alexey Milovidov 已提交
333
    /// Limit on total number of concurrently executed queries.
334 335 336 337 338 339
    global_context->getProcessList().setMaxSize(config().getInt("max_concurrent_queries", 0));

    /// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default)
    if (config().has("max_table_size_to_drop"))
        global_context->setMaxTableSizeToDrop(config().getUInt64("max_table_size_to_drop"));

340
    if (config().has("max_partition_size_to_drop"))
341
        global_context->setMaxPartitionSizeToDrop(config().getUInt64("max_partition_size_to_drop"));
342

343
    /// Size of cache for uncompressed blocks. Zero means disabled.
344
    size_t uncompressed_cache_size = config().getUInt64("uncompressed_cache_size", 0);
345 346 347
    if (uncompressed_cache_size)
        global_context->setUncompressedCache(uncompressed_cache_size);

348 349
    /// Load global settings from default_profile and system_profile.
    global_context->setDefaultProfiles(config());
350 351
    Settings & settings = global_context->getSettingsRef();

352 353 354 355 356
    /// Size of cache for marks (index of MergeTree family of tables). It is necessary.
    size_t mark_cache_size = config().getUInt64("mark_cache_size");
    if (mark_cache_size)
        global_context->setMarkCache(mark_cache_size);

A
alesapin 已提交
357
#if USE_EMBEDDED_COMPILER
358 359
    size_t compiled_expression_cache_size = config().getUInt64("compiled_expression_cache_size", std::numeric_limits<UInt64>::max());
    if (compiled_expression_cache_size)
A
alesapin 已提交
360
        global_context->setCompiledExpressionCache(compiled_expression_cache_size);
A
alesapin 已提交
361
#endif
362

363 364 365 366 367
    /// Set path for format schema files
    auto format_schema_path = Poco::File(config().getString("format_schema_path", path + "format_schemas/"));
    global_context->setFormatSchemaPath(format_schema_path.path() + "/");
    format_schema_path.createDirectories();

368
    LOG_INFO(log, "Loading metadata.");
369
    loadMetadataSystem(*global_context);
370 371
    /// After attaching system databases we can initialize system log.
    global_context->initializeSystemLogs();
372 373 374
    /// After the system database is created, attach virtual system tables (in addition to query_log and part_log)
    attachSystemTablesServer(*global_context->getDatabase("system"), has_zookeeper);
    /// Then, load remaining databases
375 376 377 378 379
    loadMetadata(*global_context);
    LOG_DEBUG(log, "Loaded metadata.");

    global_context->setCurrentDatabase(default_database);

380 381 382 383 384 385 386 387 388 389 390
    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, "Shutted down storages.");
    });

391 392
    if (has_zookeeper && config().has("distributed_ddl"))
    {
393
        /// DDL worker should be started after all tables were loaded
394
        String ddl_zookeeper_path = config().getString("distributed_ddl.path", "/clickhouse/task_queue/ddl/");
395
        global_context->setDDLWorker(std::make_shared<DDLWorker>(ddl_zookeeper_path, *global_context, &config(), "distributed_ddl"));
396 397
    }

398 399 400 401
    std::unique_ptr<DNSCacheUpdater> dns_cache_updater;
    if (config().has("disable_internal_dns_cache") && config().getInt("disable_internal_dns_cache"))
    {
        /// Disable DNS caching at all
402
        DNSResolver::instance().setDisableCacheFlag();
403 404 405 406 407 408
    }
    else
    {
        /// Initialize a watcher updating DNS cache in case of network errors
        dns_cache_updater = std::make_unique<DNSCacheUpdater>(*global_context);
    }
409

410
#if defined(__linux__)
A
Alexey Milovidov 已提交
411
    if (!TaskStatsInfoGetter::checkPermissions())
412
    {
A
Alexey Milovidov 已提交
413
        LOG_INFO(log, "It looks like the process has no CAP_NET_ADMIN capability, 'taskstats' performance statistics will be disabled."
A
Alexey Milovidov 已提交
414
                      " It could happen due to incorrect ClickHouse package installation."
A
Alexey Milovidov 已提交
415 416 417
                      " You could resolve the problem manually with 'sudo setcap cap_net_admin=+ep /usr/bin/clickhouse'."
                      " 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.");
418
    }
419 420 421
#else
    LOG_INFO(log, "TaskStats is not implemented for this OS. IO accounting will be disabled.");
#endif
422

423
    {
424
        Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0);
425

426
        Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024));
427
        Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
428
        http_params->setTimeout(settings.http_receive_timeout);
429 430 431 432
        http_params->setKeepAliveTimeout(keep_alive_timeout);

        std::vector<std::unique_ptr<Poco::Net::TCPServer>> servers;

433
        std::vector<std::string> listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host");
434

435
        bool listen_try = config().getBool("listen_try", false);
436 437 438 439
        if (listen_hosts.empty())
        {
            listen_hosts.emplace_back("::1");
            listen_hosts.emplace_back("127.0.0.1");
P
proller 已提交
440
            listen_try = true;
441 442
        }

443 444
        auto make_socket_address = [&](const std::string & host, UInt16 port)
        {
445 446 447 448 449 450 451
            Poco::Net::SocketAddress socket_address;
            try
            {
                socket_address = Poco::Net::SocketAddress(host, port);
            }
            catch (const Poco::Net::DNSException & e)
            {
P
proller 已提交
452 453
                const auto code = e.code();
                if (code == EAI_FAMILY
P
proller 已提交
454
#if defined(EAI_ADDRFAMILY)
P
proller 已提交
455
                    || code == EAI_ADDRFAMILY
P
proller 已提交
456
#endif
457 458 459
                    )
                {
                    LOG_ERROR(log,
P
proller 已提交
460
                        "Cannot resolve listen_host (" << host << "), error " << e.code() << ": " << e.message() << ". "
A
alexey-milovidov 已提交
461 462 463
                        "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>");
464 465 466 467 468 469 470
                }

                throw;
            }
            return socket_address;
        };

471 472 473 474 475 476 477
        auto socket_bind_listen = [&](auto & socket, const std::string & host, UInt16 port, bool secure = 0)
        {
               auto address = make_socket_address(host, port);
#if !POCO_CLICKHOUSE_PATCH || POCO_VERSION <= 0x02000000 // TODO: fill correct version
               if (secure)
                   /// Bug in old poco, listen() after bind() with reusePort param will fail because have no implementation in SecureServerSocketImpl
                   /// https://github.com/pocoproject/poco/pull/2257
P
proller 已提交
478
                   socket.bind(address, /* reuseAddress = */ true);
479 480
               else
#endif
P
proller 已提交
481 482 483
#if POCO_VERSION < 0x01080000
                   socket.bind(address, /* reuseAddress = */ true);
#else
484
                   socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ config().getBool("listen_reuse_port", false));
P
proller 已提交
485
#endif
486 487 488 489 490 491

               socket.listen(/* backlog = */ config().getUInt("listen_backlog", 64));

               return address;
        };

492 493 494
        for (const auto & listen_host : listen_hosts)
        {
            /// For testing purposes, user may omit tcp_port or http_port or https_port in configuration file.
495
            try
496
            {
497 498 499
                /// HTTP
                if (config().has("http_port"))
                {
500 501 502 503
                    Poco::Net::ServerSocket socket;
                    auto address = socket_bind_listen(socket, listen_host, config().getInt("http_port"));
                    socket.setReceiveTimeout(settings.http_receive_timeout);
                    socket.setSendTimeout(settings.http_send_timeout);
504
                    servers.emplace_back(new Poco::Net::HTTPServer(
V
Vadim Skipin 已提交
505 506
                        new HTTPHandlerFactory(*this, "HTTPHandler-factory"),
                        server_pool,
507
                        socket,
V
Vadim Skipin 已提交
508
                        http_params));
509

510
                    LOG_INFO(log, "Listening http://" + address.toString());
511
                }
512

513 514 515
                /// HTTPS
                if (config().has("https_port"))
                {
516
#if USE_POCO_NETSSL
517 518 519 520
                    Poco::Net::SecureServerSocket socket;
                    auto address = socket_bind_listen(socket, listen_host, config().getInt("https_port"), /* secure = */ true);
                    socket.setReceiveTimeout(settings.http_receive_timeout);
                    socket.setSendTimeout(settings.http_send_timeout);
521
                    servers.emplace_back(new Poco::Net::HTTPServer(
P
proller 已提交
522
                        new HTTPHandlerFactory(*this, "HTTPSHandler-factory"),
V
Vadim Skipin 已提交
523
                        server_pool,
524
                        socket,
V
Vadim Skipin 已提交
525
                        http_params));
526

527
                    LOG_INFO(log, "Listening https://" + address.toString());
V
Vadim Skipin 已提交
528
#else
A
Alexey Milovidov 已提交
529
                    throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.",
530
                        ErrorCodes::SUPPORT_IS_DISABLED};
V
Vadim Skipin 已提交
531
#endif
532
                }
533

534 535 536
                /// TCP
                if (config().has("tcp_port"))
                {
537 538 539 540
                    Poco::Net::ServerSocket socket;
                    auto address = socket_bind_listen(socket, listen_host, config().getInt("tcp_port"));
                    socket.setReceiveTimeout(settings.receive_timeout);
                    socket.setSendTimeout(settings.send_timeout);
V
Vadim Skipin 已提交
541 542 543
                    servers.emplace_back(new Poco::Net::TCPServer(
                        new TCPHandlerFactory(*this),
                        server_pool,
544
                        socket,
V
Vadim Skipin 已提交
545
                        new Poco::Net::TCPServerParams));
546

547
                    LOG_INFO(log, "Listening tcp: " + address.toString());
548
                }
549

A
alexey-milovidov 已提交
550
                /// TCP with SSL
551
                if (config().has("tcp_port_secure"))
P
proller 已提交
552
                {
553
#if USE_POCO_NETSSL
554 555 556 557
                    Poco::Net::SecureServerSocket socket;
                    auto address = socket_bind_listen(socket, listen_host, config().getInt("tcp_port_secure"), /* secure = */ true);
                    socket.setReceiveTimeout(settings.receive_timeout);
                    socket.setSendTimeout(settings.send_timeout);
P
proller 已提交
558
                    servers.emplace_back(new Poco::Net::TCPServer(
P
proller 已提交
559
                        new TCPHandlerFactory(*this, /* secure= */ true ),
P
proller 已提交
560
                                                                  server_pool,
561
                                                                  socket,
P
proller 已提交
562
                                                                  new Poco::Net::TCPServerParams));
563
                    LOG_INFO(log, "Listening tcp_secure: " + address.toString());
P
proller 已提交
564
#else
A
Alexey Milovidov 已提交
565
                    throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
P
proller 已提交
566 567 568 569
                        ErrorCodes::SUPPORT_IS_DISABLED};
#endif
                }

570 571 572
                /// At least one of TCP and HTTP servers must be created.
                if (servers.empty())
                    throw Exception("No 'tcp_port' and 'http_port' is specified in configuration file.", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
573

574 575 576
                /// Interserver IO HTTP
                if (config().has("interserver_http_port"))
                {
577 578 579 580
                    Poco::Net::ServerSocket socket;
                    auto address = socket_bind_listen(socket, listen_host, config().getInt("interserver_http_port"));
                    socket.setReceiveTimeout(settings.http_receive_timeout);
                    socket.setSendTimeout(settings.http_send_timeout);
581
                    servers.emplace_back(new Poco::Net::HTTPServer(
V
Vadim Skipin 已提交
582
                        new InterserverIOHTTPHandlerFactory(*this, "InterserverIOHTTPHandler-factory"),
583
                        server_pool,
584
                        socket,
585 586
                        http_params));

587
                    LOG_INFO(log, "Listening interserver http: " + address.toString());
588
                }
589 590 591 592 593 594 595 596 597 598 599 600 601 602 603 604 605 606 607 608

                if (config().has("interserver_https_port"))
                {
#if USE_POCO_NETSSL
                    Poco::Net::SecureServerSocket socket;
                    auto address = socket_bind_listen(socket, listen_host, config().getInt("interserver_https_port"), /* secure = */ true);
                    socket.setReceiveTimeout(settings.http_receive_timeout);
                    socket.setSendTimeout(settings.http_send_timeout);
                    servers.emplace_back(new Poco::Net::HTTPServer(
                        new InterserverIOHTTPHandlerFactory(*this, "InterserverIOHTTPHandler-factory"),
                        server_pool,
                        socket,
                        http_params));

                    LOG_INFO(log, "Listening interserver https: " + address.toString());
#else
                    throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
                            ErrorCodes::SUPPORT_IS_DISABLED};
#endif
                }
609 610
            }
            catch (const Poco::Net::NetException & e)
611
            {
612
                if (listen_try)
P
proller 已提交
613
                    LOG_ERROR(log, "Listen [" << listen_host << "]: " << e.code() << ": " << e.what() << ": " << e.message()
A
alexey-milovidov 已提交
614 615 616 617
                        << "  If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to "
                        "specify not disabled IPv4 or IPv6 address to listen in <listen_host> element of configuration "
                        "file. Example for disabled IPv6: <listen_host>0.0.0.0</listen_host> ."
                        " Example for disabled IPv4: <listen_host>::</listen_host>");
618 619
                else
                    throw;
620 621 622
            }
        }

623 624 625
        if (servers.empty())
             throw Exception("No servers started (add valid listen_host and 'tcp_port' or 'http_port' to configuration file.)", ErrorCodes::NO_ELEMENTS_IN_CONFIG);

626 627 628
        for (auto & server : servers)
            server->start();

629 630 631
        main_config_reloader->start();
        users_config_reloader->start();

632 633
        {
            std::stringstream message;
P
proller 已提交
634
            message << "Available RAM = " << formatReadableSizeWithBinarySuffix(memory_amount) << ";"
635 636
                << " physical cores = " << getNumberOfPhysicalCPUCores() << ";"
                // on ARM processors it can show only enabled at current moment cores
637
                << " threads = " <<  std::thread::hardware_concurrency() << ".";
638 639 640
            LOG_INFO(log, message.str());
        }

641 642 643 644 645 646 647 648 649 650 651 652 653 654 655 656 657
        LOG_INFO(log, "Ready for connections.");

        SCOPE_EXIT({
            LOG_DEBUG(log, "Received termination signal.");
            LOG_DEBUG(log, "Waiting for current connections to close.");

            is_cancelled = true;

            int current_connections = 0;
            for (auto & server : servers)
            {
                server->stop();
                current_connections += server->currentConnections();
            }

            LOG_DEBUG(log,
                "Closed all listening sockets."
658
                    << (current_connections ? " Waiting for " + toString(current_connections) + " outstanding connections." : ""));
659 660 661 662 663 664 665 666 667 668 669 670 671 672 673 674 675 676 677

            if (current_connections)
            {
                const int sleep_max_ms = 1000 * config().getInt("shutdown_wait_unfinished", 5);
                const int sleep_one_ms = 100;
                int sleep_current_ms = 0;
                while (sleep_current_ms < sleep_max_ms)
                {
                    current_connections = 0;
                    for (auto & server : servers)
                        current_connections += server->currentConnections();
                    if (!current_connections)
                        break;
                    sleep_current_ms += sleep_one_ms;
                    std::this_thread::sleep_for(std::chrono::milliseconds(sleep_one_ms));
                }
            }

            LOG_DEBUG(
678
                log, "Closed connections." << (current_connections ? " But " + toString(current_connections) + " remains."
679
                    " Tip: To increase wait time add to config: <shutdown_wait_unfinished>60</shutdown_wait_unfinished>" : ""));
680 681 682 683 684 685 686 687 688 689 690 691 692 693 694 695 696 697 698 699 700 701

            main_config_reloader.reset();
            users_config_reloader.reset();
        });

        /// try to load dictionaries immediately, throw on error and die
        try
        {
            if (!config().getBool("dictionaries_lazy_load", true))
            {
                global_context->tryCreateEmbeddedDictionaries();
                global_context->tryCreateExternalDictionaries();
            }
        }
        catch (...)
        {
            LOG_ERROR(log, "Caught exception while loading dictionaries.");
            throw;
        }

        /// This object will periodically calculate some metrics.
        AsynchronousMetrics async_metrics(*global_context);
702
        attachSystemTablesAsync(*global_context->getDatabase("system"), async_metrics);
703 704 705 706

        std::vector<std::unique_ptr<MetricsTransmitter>> metrics_transmitters;
        for (const auto & graphite_key : DB::getMultipleKeysFromConfig(config(), "", "graphite"))
        {
707 708
            metrics_transmitters.emplace_back(std::make_unique<MetricsTransmitter>(
                *global_context, async_metrics, graphite_key));
709 710
        }

711 712
        SessionCleaner session_cleaner(*global_context);

713 714 715 716
        waitForTerminationRequest();
    }

    return Application::EXIT_OK;
717 718
}
}
719

A
Alexey Milovidov 已提交
720 721 722 723 724 725 726 727 728 729 730 731 732 733
int mainEntryClickHouseServer(int argc, char ** argv)
{
    DB::Server app;
    try
    {
        return app.run(argc, argv);
    }
    catch (...)
    {
        std::cerr << DB::getCurrentExceptionMessage(true) << "\n";
        auto code = DB::getCurrentExceptionCode();
        return code ? code : 1;
    }
}