ClickHouse/dbms/src/Server/Server.cpp

120 lines
4.1 KiB
C++
Raw Normal View History

2012-03-09 03:06:09 +00:00
#include <Poco/Net/HTTPServerRequest.h>
#include <Yandex/ApplicationServerExt.h>
#include <DB/Functions/FunctionsLibrary.h>
#include <DB/Interpreters/loadMetadata.h>
#include <DB/Storages/StorageSystemNumbers.h>
2012-05-08 11:19:00 +00:00
#include <DB/Storages/StorageSystemTables.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"
#include "TCPHandler.h"
2012-03-09 03:06:09 +00:00
namespace DB
{
/// Отвечает "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)
{
response.send() << "Ok." << std::endl;
}
};
Poco::Net::HTTPRequestHandler * HTTPRequestHandlerFactory::createRequestHandler(
const Poco::Net::HTTPServerRequest & request)
{
LOG_TRACE(log, "HTTP Request. "
<< "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)
2012-03-09 15:46:52 +00:00
return new HTTPHandler(server);
2012-03-09 03:06:09 +00:00
else if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET)
return new PingRequestHandler();
else
return 0;
}
2012-03-09 15:46:52 +00:00
Poco::Net::TCPServerConnection * TCPConnectionFactory::createConnection(const Poco::Net::StreamSocket & socket)
{
LOG_TRACE(log, "TCP Request. " << "Address: " << socket.address().toString());
return new TCPHandler(server, socket);
}
2012-03-09 03:06:09 +00:00
int Server::main(const std::vector<std::string> & args)
{
/// Заранее инициализируем DateLUT, чтобы первая инициализация потом не влияла на измеряемую скорость выполнения.
Yandex::DateLUTSingleton::instance();
2012-03-09 03:56:12 +00:00
/** Контекст содержит всё, что влияет на обработку запроса:
* настройки, набор функций, типов данных, агрегатных функций, баз данных...
*/
2012-03-09 03:06:09 +00:00
global_context.path = config.getString("path");
global_context.functions = FunctionsLibrary::get();
global_context.aggregate_function_factory = new AggregateFunctionFactory;
global_context.data_type_factory = new DataTypeFactory;
global_context.storage_factory = new StorageFactory;
2012-03-19 12:57:56 +00:00
global_context.format_factory = new FormatFactory;
2012-03-09 03:06:09 +00:00
loadMetadata(global_context);
(*global_context.databases)["system"]["one"] = new StorageSystemOne("one");
(*global_context.databases)["system"]["numbers"] = new StorageSystemNumbers("numbers");
2012-05-08 11:19:00 +00:00
(*global_context.databases)["system"]["tables"] = new StorageSystemTables("tables", global_context);
2012-03-09 03:06:09 +00:00
global_context.current_database = config.getString("default_database", "default");
2012-03-09 03:56:12 +00:00
global_context.settings.asynchronous = config.getBool("asynchronous", global_context.settings.asynchronous);
global_context.settings.max_block_size = config.getInt("max_block_size", global_context.settings.max_block_size);
global_context.settings.max_query_size = config.getInt("max_query_size", global_context.settings.max_query_size);
global_context.settings.max_threads = config.getInt("max_threads", global_context.settings.max_threads);
2012-03-09 03:06:09 +00:00
2012-03-09 15:46:52 +00:00
Poco::Net::ServerSocket http_socket(Poco::Net::SocketAddress("[::]:" + config.getString("http_port")));
Poco::Net::ServerSocket tcp_socket(Poco::Net::SocketAddress("[::]:" + config.getString("tcp_port")));
2012-03-09 03:06:09 +00:00
Poco::ThreadPool server_pool(2, config.getInt("max_threads", 128));
2012-03-09 15:46:52 +00:00
Poco::Net::HTTPServer http_server(
2012-03-09 03:06:09 +00:00
new HTTPRequestHandlerFactory(*this),
server_pool,
2012-03-09 15:46:52 +00:00
http_socket,
2012-03-09 03:06:09 +00:00
new Poco::Net::HTTPServerParams);
2012-03-09 15:46:52 +00:00
Poco::Net::TCPServer tcp_server(
new TCPConnectionFactory(*this),
server_pool,
tcp_socket,
new Poco::Net::TCPServerParams);
http_server.start();
tcp_server.start();
2012-03-09 03:06:09 +00:00
waitForTerminationRequest();
2012-03-09 15:46:52 +00:00
http_server.stop();
tcp_server.stop();
2012-03-09 03:06:09 +00:00
return Application::EXIT_OK;
}
}
YANDEX_APP_SERVER_MAIN(DB::Server);