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>
|
2014-02-13 07:17:22 +00:00
|
|
|
|
#include <Poco/Util/XMLConfiguration.h>
|
2012-03-09 03:06:09 +00:00
|
|
|
|
|
|
|
|
|
#include <Yandex/ApplicationServerExt.h>
|
2014-02-21 13:02:12 +00:00
|
|
|
|
#include <statdaemons/ConfigProcessor.h>
|
2012-03-09 03:06:09 +00:00
|
|
|
|
|
|
|
|
|
#include <DB/Interpreters/loadMetadata.h>
|
|
|
|
|
#include <DB/Storages/StorageSystemNumbers.h>
|
2012-05-08 11:19:00 +00:00
|
|
|
|
#include <DB/Storages/StorageSystemTables.h>
|
2012-06-18 07:49:19 +00:00
|
|
|
|
#include <DB/Storages/StorageSystemDatabases.h>
|
2013-09-03 20:21:28 +00:00
|
|
|
|
#include <DB/Storages/StorageSystemProcesses.h>
|
2014-01-03 08:20:13 +00:00
|
|
|
|
#include <DB/Storages/StorageSystemEvents.h>
|
2012-03-09 03:06:09 +00:00
|
|
|
|
#include <DB/Storages/StorageSystemOne.h>
|
|
|
|
|
|
|
|
|
|
#include "Server.h"
|
2012-03-09 15:46:52 +00:00
|
|
|
|
#include "HTTPHandler.h"
|
2014-03-21 13:42:14 +00:00
|
|
|
|
#include "InterserverIOHTTPHandler.h"
|
2012-12-14 11:21:07 +00:00
|
|
|
|
#include "OLAPHTTPHandler.h"
|
2012-03-09 15:46:52 +00:00
|
|
|
|
#include "TCPHandler.h"
|
2012-03-09 03:06:09 +00:00
|
|
|
|
|
2014-07-21 11:21:09 +00:00
|
|
|
|
#include <thread>
|
|
|
|
|
#include <atomic>
|
|
|
|
|
|
|
|
|
|
namespace
|
|
|
|
|
{
|
|
|
|
|
|
|
|
|
|
/** Automatically sends ProfileEvents to Graphite every minute
|
|
|
|
|
*/
|
|
|
|
|
class ProfileEventsTransmitter
|
|
|
|
|
{
|
|
|
|
|
public:
|
|
|
|
|
~ProfileEventsTransmitter()
|
|
|
|
|
{
|
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
quit.store(true, std::memory_order_relaxed);
|
|
|
|
|
thread.join();
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
DB::tryLogCurrentException(__FUNCTION__);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private:
|
|
|
|
|
void run()
|
|
|
|
|
{
|
|
|
|
|
while (!quit.load(std::memory_order_relaxed))
|
|
|
|
|
{
|
|
|
|
|
std::this_thread::sleep_until(std::chrono::system_clock::now() + std::chrono::minutes(1));
|
|
|
|
|
transmitCounters();
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void transmitCounters()
|
|
|
|
|
{
|
|
|
|
|
decltype(prev_counters) counters;
|
|
|
|
|
std::copy(std::begin(prev_counters), std::end(prev_counters), counters);
|
|
|
|
|
std::transform(
|
|
|
|
|
std::begin(counters), std::end(counters), ProfileEvents::counters,
|
|
|
|
|
prev_counters, [] (size_t& prev, size_t& current)
|
|
|
|
|
{
|
|
|
|
|
prev = current - prev;
|
|
|
|
|
return prev;
|
|
|
|
|
}
|
|
|
|
|
);
|
|
|
|
|
|
|
|
|
|
auto key_vals = GraphiteWriter::KeyValueVector<size_t>{};
|
|
|
|
|
key_vals.reserve(ProfileEvents::END);
|
|
|
|
|
|
|
|
|
|
for (auto i = 0; i < ProfileEvents::END; ++i)
|
|
|
|
|
{
|
|
|
|
|
key_vals.push_back({
|
|
|
|
|
descriptionToKey(ProfileEvents::getDescription(static_cast<ProfileEvents::Event>(i))),
|
|
|
|
|
counters[i]
|
|
|
|
|
});
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
Daemon::instance().writeToGraphite(key_vals);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
std::string descriptionToKey(std::string desc)
|
|
|
|
|
{
|
|
|
|
|
std::transform(std::begin(desc), std::end(desc), std::begin(desc),
|
|
|
|
|
[] (const char c) { return c == ' ' ? '_' : std::tolower(c); }
|
|
|
|
|
);
|
|
|
|
|
|
|
|
|
|
return desc;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
decltype(ProfileEvents::counters) prev_counters{0};
|
|
|
|
|
std::atomic<bool> quit;
|
|
|
|
|
std::thread thread{&ProfileEventsTransmitter::run, this};
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
}
|
2012-03-09 03:06:09 +00:00
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
2014-07-21 11:21:09 +00:00
|
|
|
|
/** Каждые две секунды проверяет, не изменился ли конфиг.
|
|
|
|
|
* Когда изменился, запускает на нем ConfigProcessor и вызывает setUsersConfig у контекста.
|
|
|
|
|
* NOTE: Не перезагружает конфиг, если изменились другие файлы, влияющие на обработку конфига: metrika.xml
|
|
|
|
|
* и содержимое conf.d и users.d. Это можно исправить, переместив проверку времени изменения файлов в ConfigProcessor.
|
|
|
|
|
*/
|
|
|
|
|
class UsersConfigReloader
|
|
|
|
|
{
|
|
|
|
|
public:
|
|
|
|
|
UsersConfigReloader(const std::string & path, Context * context);
|
|
|
|
|
~UsersConfigReloader();
|
|
|
|
|
private:
|
|
|
|
|
std::string path;
|
|
|
|
|
Context * context;
|
|
|
|
|
|
|
|
|
|
time_t file_modification_time;
|
|
|
|
|
std::atomic<bool> quit;
|
|
|
|
|
std::thread thread;
|
|
|
|
|
|
|
|
|
|
Logger * log;
|
|
|
|
|
|
|
|
|
|
void reloadIfNewer(bool force);
|
|
|
|
|
void run();
|
|
|
|
|
};
|
|
|
|
|
|
2012-03-09 03:06:09 +00:00
|
|
|
|
/// Отвечает "Ok.\n", если получен любой GET запрос. Используется для проверки живости.
|
|
|
|
|
class PingRequestHandler : public Poco::Net::HTTPRequestHandler
|
|
|
|
|
{
|
|
|
|
|
public:
|
|
|
|
|
PingRequestHandler()
|
|
|
|
|
{
|
|
|
|
|
LOG_TRACE((&Logger::get("PingRequestHandler")), "Ping request.");
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response)
|
|
|
|
|
{
|
2014-07-16 00:53:56 +00:00
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
const char * data = "Ok.\n";
|
|
|
|
|
response.sendBuffer(data, strlen(data));
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException("PingRequestHandler");
|
|
|
|
|
}
|
2012-03-09 03:06:09 +00:00
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
2012-12-14 11:21:07 +00:00
|
|
|
|
template<typename HandlerType>
|
|
|
|
|
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"));
|
|
|
|
|
|
|
|
|
|
if (request.getURI().find('?') != std::string::npos || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST)
|
|
|
|
|
return new HandlerType(server);
|
|
|
|
|
else if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET)
|
|
|
|
|
return new PingRequestHandler();
|
|
|
|
|
else
|
|
|
|
|
return 0;
|
|
|
|
|
}
|
|
|
|
|
};
|
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
|
|
|
|
|
|
|
|
|
|
2014-07-21 11:21:09 +00:00
|
|
|
|
UsersConfigReloader::UsersConfigReloader(const std::string & path_, Context * context_)
|
2014-02-13 07:17:22 +00:00
|
|
|
|
: path(path_), context(context_), file_modification_time(0), quit(false), log(&Logger::get("UsersConfigReloader"))
|
|
|
|
|
{
|
2014-02-24 12:08:53 +00:00
|
|
|
|
/// Если путь к конфигу не абсолютный, угадаем, относительно чего он задан.
|
|
|
|
|
/// Сначала поищем его рядом с основным конфигом, потом - в текущей директории.
|
|
|
|
|
if (path.empty() || path[0] != '/')
|
|
|
|
|
{
|
|
|
|
|
std::string main_config_path = Application::instance().config().getString("config-file", "config.xml");
|
|
|
|
|
std::string config_dir = Poco::Path(main_config_path).parent().toString();
|
|
|
|
|
if (Poco::File(config_dir + path).exists())
|
|
|
|
|
path = config_dir + path;
|
|
|
|
|
}
|
|
|
|
|
|
2014-02-13 07:17:22 +00:00
|
|
|
|
reloadIfNewer(true);
|
|
|
|
|
thread = std::thread(&UsersConfigReloader::run, this);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
UsersConfigReloader::~UsersConfigReloader()
|
|
|
|
|
{
|
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
quit = true;
|
|
|
|
|
thread.join();
|
|
|
|
|
}
|
|
|
|
|
catch(...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException("~UsersConfigReloader");
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void UsersConfigReloader::run()
|
|
|
|
|
{
|
|
|
|
|
while (!quit)
|
|
|
|
|
{
|
|
|
|
|
std::this_thread::sleep_for(std::chrono::seconds(2));
|
|
|
|
|
reloadIfNewer(false);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void UsersConfigReloader::reloadIfNewer(bool force)
|
|
|
|
|
{
|
|
|
|
|
Poco::File f(path);
|
2014-02-25 09:54:25 +00:00
|
|
|
|
if (!f.exists())
|
|
|
|
|
{
|
|
|
|
|
if (force)
|
|
|
|
|
throw Exception("Users config not found at: " + path, ErrorCodes::FILE_DOESNT_EXIST);
|
|
|
|
|
if (file_modification_time)
|
|
|
|
|
{
|
|
|
|
|
LOG_ERROR(log, "Users config not found at: " << path);
|
|
|
|
|
file_modification_time = 0;
|
|
|
|
|
}
|
|
|
|
|
return;
|
|
|
|
|
}
|
2014-02-13 07:17:22 +00:00
|
|
|
|
time_t new_modification_time = f.getLastModified().epochTime();
|
|
|
|
|
if (!force && new_modification_time == file_modification_time)
|
|
|
|
|
return;
|
|
|
|
|
file_modification_time = new_modification_time;
|
|
|
|
|
|
|
|
|
|
LOG_DEBUG(log, "Loading users config");
|
|
|
|
|
|
|
|
|
|
ConfigurationPtr config;
|
|
|
|
|
|
2014-02-21 13:02:12 +00:00
|
|
|
|
try
|
|
|
|
|
{
|
2014-02-24 11:19:23 +00:00
|
|
|
|
config = ConfigProcessor(!force).loadConfig(path);
|
2014-02-21 13:02:12 +00:00
|
|
|
|
}
|
|
|
|
|
catch (Poco::Exception & e)
|
|
|
|
|
{
|
|
|
|
|
if (force)
|
|
|
|
|
throw;
|
|
|
|
|
|
2014-02-24 11:19:23 +00:00
|
|
|
|
LOG_ERROR(log, "Error loading users config: " << e.what() << ": " << e.displayText());
|
2014-02-21 13:02:12 +00:00
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
if (force)
|
|
|
|
|
throw;
|
|
|
|
|
|
2014-02-24 11:19:23 +00:00
|
|
|
|
LOG_ERROR(log, "Error loading users config.");
|
2014-02-13 07:17:22 +00:00
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
context->setUsersConfig(config);
|
|
|
|
|
}
|
|
|
|
|
catch (Exception & e)
|
|
|
|
|
{
|
|
|
|
|
if (force)
|
|
|
|
|
throw;
|
|
|
|
|
|
|
|
|
|
LOG_ERROR(log, "Error updating users config: " << e.what() << ": " << e.displayText() << "\n" << e.getStackTrace().toString());
|
|
|
|
|
}
|
|
|
|
|
catch (Poco::Exception & e)
|
|
|
|
|
{
|
|
|
|
|
if (force)
|
|
|
|
|
throw;
|
|
|
|
|
|
|
|
|
|
LOG_ERROR(log, "Error updating users config: " << e.what() << ": " << e.displayText());
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
if (force)
|
|
|
|
|
throw;
|
|
|
|
|
|
|
|
|
|
LOG_ERROR(log, "Error updating users config.");
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
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
|
|
|
|
|
|
|
|
|
/// Попробуем повысить ограничение на число открытых файлов.
|
|
|
|
|
{
|
|
|
|
|
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 << ")");
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
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
|
|
|
|
|
2014-07-21 11:21:09 +00:00
|
|
|
|
global_context.reset(new 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-05-12 00:49:24 +00:00
|
|
|
|
global_context->setPath(config().getString("path"));
|
2012-03-09 03:06:09 +00:00
|
|
|
|
|
2014-05-12 00:49:24 +00:00
|
|
|
|
if (config().has("zookeeper"))
|
|
|
|
|
global_context->setZooKeeper(new zkutil::ZooKeeper(config(), "zookeeper"));
|
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
|
|
|
|
{
|
|
|
|
|
String this_host;
|
2014-05-12 00:49:24 +00:00
|
|
|
|
if (config().has("interserver_http_host"))
|
|
|
|
|
this_host = config().getString("interserver_http_host");
|
2014-04-03 08:47:59 +00:00
|
|
|
|
else
|
|
|
|
|
this_host = Poco::Net::DNS::hostName();
|
|
|
|
|
|
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);
|
|
|
|
|
|
|
|
|
|
global_context->setInterserverIOHost(this_host, port);
|
|
|
|
|
}
|
|
|
|
|
|
2014-05-12 00:49:24 +00:00
|
|
|
|
if (config().has("replica_name"))
|
|
|
|
|
global_context->setDefaultReplicaName(config().getString("replica_name"));
|
2014-04-14 10:18:23 +00:00
|
|
|
|
|
2014-05-14 14:19:58 +00:00
|
|
|
|
std::string users_config_path = config().getString("users_config", config().getString("config-file", "config.xml"));
|
2014-07-21 11:21:09 +00:00
|
|
|
|
auto users_config_reloader = std::unique_ptr<UsersConfigReloader>{
|
|
|
|
|
new 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.");
|
|
|
|
|
|
|
|
|
|
/// Создаём системные таблицы.
|
2013-09-14 05:14:22 +00:00
|
|
|
|
global_context->addDatabase("system");
|
2014-07-16 00:53:56 +00:00
|
|
|
|
|
2013-09-14 05:14:22 +00:00
|
|
|
|
global_context->addTable("system", "one", StorageSystemOne::create("one"));
|
|
|
|
|
global_context->addTable("system", "numbers", StorageSystemNumbers::create("numbers"));
|
|
|
|
|
global_context->addTable("system", "tables", StorageSystemTables::create("tables", *global_context));
|
2014-01-03 08:20:13 +00:00
|
|
|
|
global_context->addTable("system", "databases", StorageSystemDatabases::create("databases", *global_context));
|
|
|
|
|
global_context->addTable("system", "processes", StorageSystemProcesses::create("processes", *global_context));
|
|
|
|
|
global_context->addTable("system", "events", StorageSystemEvents::create("events"));
|
2014-07-16 00:53: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
|
|
|
|
|
2013-09-14 05:14:22 +00:00
|
|
|
|
{
|
2014-07-21 11:21:09 +00:00
|
|
|
|
const auto profile_events_transmitter = config().getBool("use_graphite", true)
|
|
|
|
|
? std::unique_ptr<ProfileEventsTransmitter>{new ProfileEventsTransmitter{}}
|
|
|
|
|
: nullptr;
|
|
|
|
|
|
2014-05-12 00:49:24 +00:00
|
|
|
|
bool use_olap_server = config().getBool("use_olap_http_server", false);
|
|
|
|
|
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);
|
|
|
|
|
|
|
|
|
|
/// HTTP
|
2014-05-12 00:49:24 +00:00
|
|
|
|
Poco::Net::ServerSocket http_socket(Poco::Net::SocketAddress("[::]:" + config().getString("http_port")));
|
2013-09-14 05:14:22 +00:00
|
|
|
|
http_socket.setReceiveTimeout(settings.receive_timeout);
|
|
|
|
|
http_socket.setSendTimeout(settings.send_timeout);
|
|
|
|
|
Poco::Net::HTTPServer http_server(
|
|
|
|
|
new HTTPRequestHandlerFactory<HTTPHandler>(*this, "HTTPHandler-factory"),
|
|
|
|
|
server_pool,
|
|
|
|
|
http_socket,
|
|
|
|
|
http_params);
|
|
|
|
|
|
|
|
|
|
/// TCP
|
2014-05-12 00:49:24 +00:00
|
|
|
|
Poco::Net::ServerSocket tcp_socket(Poco::Net::SocketAddress("[::]:" + config().getString("tcp_port")));
|
2013-09-14 05:14:22 +00:00
|
|
|
|
tcp_socket.setReceiveTimeout(settings.receive_timeout);
|
|
|
|
|
tcp_socket.setSendTimeout(settings.send_timeout);
|
|
|
|
|
Poco::Net::TCPServer tcp_server(
|
|
|
|
|
new TCPConnectionFactory(*this),
|
|
|
|
|
server_pool,
|
|
|
|
|
tcp_socket,
|
|
|
|
|
new Poco::Net::TCPServerParams);
|
|
|
|
|
|
2014-03-21 13:42:14 +00:00
|
|
|
|
/// Interserver IO HTTP
|
|
|
|
|
Poco::SharedPtr<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-05-12 00:49:24 +00:00
|
|
|
|
String port_str = config().getString("interserver_http_port");
|
2014-03-21 19:49:27 +00:00
|
|
|
|
|
2014-03-21 13:42:14 +00:00
|
|
|
|
Poco::Net::ServerSocket interserver_io_http_socket(Poco::Net::SocketAddress("[::]:"
|
2014-03-21 19:49:27 +00:00
|
|
|
|
+ port_str));
|
2014-03-21 13:42:14 +00:00
|
|
|
|
interserver_io_http_socket.setReceiveTimeout(settings.receive_timeout);
|
|
|
|
|
interserver_io_http_socket.setSendTimeout(settings.send_timeout);
|
|
|
|
|
interserver_io_http_server = new Poco::Net::HTTPServer(
|
|
|
|
|
new HTTPRequestHandlerFactory<InterserverIOHTTPHandler>(*this, "InterserverIOHTTPHandler-factory"),
|
|
|
|
|
server_pool,
|
|
|
|
|
interserver_io_http_socket,
|
|
|
|
|
http_params);
|
|
|
|
|
}
|
|
|
|
|
|
2013-09-14 05:14:22 +00:00
|
|
|
|
/// OLAP HTTP
|
|
|
|
|
Poco::SharedPtr<Poco::Net::HTTPServer> olap_http_server;
|
|
|
|
|
if (use_olap_server)
|
|
|
|
|
{
|
2014-07-21 11:21:09 +00:00
|
|
|
|
olap_parser.reset(new OLAP::QueryParser());
|
|
|
|
|
olap_converter.reset(new OLAP::QueryConverter(config()));
|
2013-09-14 05:14:22 +00:00
|
|
|
|
|
2014-05-12 00:49:24 +00:00
|
|
|
|
Poco::Net::ServerSocket olap_http_socket(Poco::Net::SocketAddress("[::]:" + config().getString("olap_http_port")));
|
2013-09-14 05:14:22 +00:00
|
|
|
|
olap_http_socket.setReceiveTimeout(settings.receive_timeout);
|
|
|
|
|
olap_http_socket.setSendTimeout(settings.send_timeout);
|
|
|
|
|
olap_http_server = new Poco::Net::HTTPServer(
|
|
|
|
|
new HTTPRequestHandlerFactory<OLAPHTTPHandler>(*this, "OLAPHTTPHandler-factory"),
|
|
|
|
|
server_pool,
|
|
|
|
|
olap_http_socket,
|
2014-03-21 13:42:14 +00:00
|
|
|
|
http_params);
|
2013-09-14 05:14:22 +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();
|
|
|
|
|
if (olap_http_server)
|
2013-09-14 05:14:22 +00:00
|
|
|
|
olap_http_server->start();
|
|
|
|
|
|
|
|
|
|
LOG_INFO(log, "Ready for connections.");
|
2014-07-16 00:53:56 +00:00
|
|
|
|
|
2013-09-14 05:14:22 +00:00
|
|
|
|
waitForTerminationRequest();
|
2014-07-16 00:53:56 +00:00
|
|
|
|
|
2013-09-14 05:14:22 +00:00
|
|
|
|
LOG_DEBUG(log, "Received termination signal. Waiting for current connections to close.");
|
2014-02-13 07:17:22 +00:00
|
|
|
|
|
2014-07-21 11:21:09 +00:00
|
|
|
|
users_config_reloader.reset();
|
2014-02-13 07:17:22 +00:00
|
|
|
|
|
2013-09-14 05:14:22 +00:00
|
|
|
|
is_cancelled = true;
|
2012-12-27 17:54:16 +00:00
|
|
|
|
|
2013-09-14 05:14:22 +00:00
|
|
|
|
http_server.stop();
|
|
|
|
|
tcp_server.stop();
|
|
|
|
|
if (use_olap_server)
|
|
|
|
|
olap_http_server->stop();
|
2012-12-14 11:21:07 +00:00
|
|
|
|
}
|
2012-03-09 15:46:52 +00:00
|
|
|
|
|
2013-09-14 05:14:22 +00:00
|
|
|
|
LOG_DEBUG(log, "Closed all connections.");
|
2012-03-09 15:46:52 +00:00
|
|
|
|
|
2014-03-03 20:11:09 +00:00
|
|
|
|
/** Попросим завершить фоновую работу у всех движков таблиц.
|
|
|
|
|
* Это важно делать заранее, не в деструкторе Context-а, так как
|
|
|
|
|
* движки таблиц могут при уничтожении всё ещё пользоваться Context-ом.
|
|
|
|
|
*/
|
|
|
|
|
LOG_INFO(log, "Shutting down storages.");
|
|
|
|
|
global_context->shutdown();
|
|
|
|
|
LOG_DEBUG(log, "Shutted down storages.");
|
|
|
|
|
|
2013-09-14 05:14:22 +00:00
|
|
|
|
/** Явно уничтожаем контекст - это удобнее, чем в деструкторе Server-а, так как ещё доступен логгер.
|
|
|
|
|
* В этот момент никто больше не должен владеть shared-частью контекста.
|
|
|
|
|
*/
|
2014-07-21 11:21:09 +00:00
|
|
|
|
global_context.reset();
|
2012-03-09 15:46:52 +00:00
|
|
|
|
|
2013-09-14 05:14:22 +00:00
|
|
|
|
LOG_DEBUG(log, "Destroyed global context.");
|
2014-07-16 00:53:56 +00:00
|
|
|
|
|
2012-03-09 03:06:09 +00:00
|
|
|
|
return Application::EXIT_OK;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
YANDEX_APP_SERVER_MAIN(DB::Server);
|