Make libraries storage as singleton

This commit is contained in:
kssenii 2021-03-06 18:21:40 +00:00
parent 2c080da51b
commit 94af06588e
8 changed files with 176 additions and 86 deletions

View File

@ -5,6 +5,7 @@ set (CLICKHOUSE_LIBRARY_BRIDGE_SOURCES
Handlers.cpp
HandlerFactory.cpp
SharedLibraryHandler.cpp
SharedLibraryHandlerFactory.cpp
)
if (OS_LINUX)

View File

@ -20,54 +20,7 @@ namespace DB
{
/// Remove '/' in the beginning.
auto dictionary_id = uri.getPath().substr(1);
/// Keep a map: dictionary_id -> SharedLibraryHandler.
auto library_handler = library_handlers.find(dictionary_id);
HTMLForm params(request);
params.read(request.getStream());
std::string method = params.get("method");
if (!params.has("method"))
return std::make_unique<LibraryErrorResponseHandler>("No 'method' in request URL");
/// If such handler exists, then current method can be: loadAll, loadIds, loadKeys, isModified, supportsSelectiveLoad.
if (library_handler != library_handlers.end())
return std::make_unique<LibraryRequestHandler>(keep_alive_timeout, context, library_handler->second);
/// If there is no such dictionary_id in map, then current method is either libNew or libClone or libDelete.
if (method == "libNew")
{
auto library_handler_ptr = std::make_shared<SharedLibraryHandler>();
library_handlers[dictionary_id] = library_handler_ptr;
return std::make_unique<LibraryRequestHandler>(keep_alive_timeout, context, library_handler_ptr);
}
if (method == "libClone")
{
if (!params.has("other_dictionary_id"))
return std::make_unique<LibraryErrorResponseHandler>("No 'other_dictionary_id' in request URL");
std::string other_dictionary_id = params.get("other_dictionary_id");
LOG_INFO(log, "libClone from dictionaryID {} to {}", other_dictionary_id, dictionary_id);
auto other_library_handler = library_handlers.find(other_dictionary_id);
if (other_library_handler != library_handlers.end())
{
/// libClone method for lib_data will be called in copy constructor.
auto other_library_handler_ptr = other_library_handler->second;
auto library_handler_ptr = std::make_shared<SharedLibraryHandler>(*other_library_handler_ptr);
library_handlers[dictionary_id] = library_handler_ptr;
return std::make_unique<LibraryRequestHandler>(keep_alive_timeout, context, library_handler_ptr);
}
/// cloneLibrary is called in copy constructor for LibraryDictionarySource.
/// SharedLibraryHandler is removed from map only in LibraryDictionarySource desctructor.
/// Therefore other_library_handler is supposed to always exist at this moment.
return std::make_unique<LibraryErrorResponseHandler>("SharedLibraryHandler for dictionary to clone from does not exist");
}
return std::make_unique<LibraryErrorResponseHandler>("Unknown 'method' in request URL");
return std::make_unique<LibraryRequestHandler>(keep_alive_timeout, context, dictionary_id);
}
return std::make_unique<LibraryErrorResponseHandler>("Unknown request");

View File

@ -1,4 +1,5 @@
#include "Handlers.h"
#include "SharedLibraryHandlerFactory.h"
#include <DataStreams/copyData.h>
#include <DataTypes/DataTypeFactory.h>
@ -77,22 +78,37 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe
LOG_TRACE(log, "Library path: '{}', library_settings: '{}'", library_path, library_settings);
library_handler->libNew(library_path, library_settings);
writeStringBinary("1", out);
bool res = SharedLibraryHandlerFactory::instance().create(dictionary_id, library_path, library_settings);
writeStringBinary(std::to_string(res), out);
}
else if (method == "libClone")
{
/// libClone was already called, only need to send the responce.
writeStringBinary("1", out);
if (!params.has("from_dictionary_id"))
{
processError(response, "No 'from_dictionary_id' in request URL");
return;
}
std::string from_dictionary_id = params.get("from_dictionary_id");
LOG_TRACE(log, "Calling libClone from {} to {}", from_dictionary_id, dictionary_id);
bool res = SharedLibraryHandlerFactory::instance().clone(from_dictionary_id, dictionary_id);
writeStringBinary(std::to_string(res), out);
}
else if (method == "libDelete")
{
bool res = SharedLibraryHandlerFactory::instance().remove(dictionary_id);
writeStringBinary(std::to_string(res), out);
}
else if (method == "isModified")
{
auto res = library_handler->isModified();
auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id);
bool res = library_handler->isModified();
writeStringBinary(std::to_string(res), out);
}
else if (method == "supportsSelectiveLoad")
{
auto res = library_handler->supportsSelectiveLoad();
auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id);
bool res = library_handler->supportsSelectiveLoad();
writeStringBinary(std::to_string(res), out);
}
else if (method == "loadAll")
@ -118,6 +134,7 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe
return;
}
auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id);
auto input = library_handler->loadAll(attributes, *sample_block);
BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream("RowBinary", out, *sample_block, context);
@ -154,11 +171,21 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe
return;
}
auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id);
auto input = library_handler->loadIds(attributes, ids, *sample_block);
BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream("RowBinary", out, *sample_block, context);
copyData(*input, *output);
}
else if (method == "loadKeys")
{
if (!params.has("query"))
{
processError(response, "No 'query' in request URL");
return;
}
processError(response, "KSSENII SUCCESS!");
}
}
catch (...)
{

View File

@ -18,11 +18,11 @@ public:
LibraryRequestHandler(
size_t keep_alive_timeout_,
Context & context_,
SharedLibraryHandlerPtr library_handler_)
const std::string & dictionary_id_)
: log(&Poco::Logger::get("LibraryRequestHandler"))
, keep_alive_timeout(keep_alive_timeout_)
, context(context_)
, library_handler(library_handler_)
, dictionary_id(dictionary_id_)
{
}
@ -32,12 +32,9 @@ private:
void processError(HTTPServerResponse & response, const std::string & message);
Poco::Logger * log;
size_t keep_alive_timeout;
Context & context;
SharedLibraryHandlerPtr library_handler;
const std::string dictionary_id;
};
@ -54,7 +51,6 @@ public:
private:
Poco::Logger * log;
const std::string message;
};

View File

@ -8,6 +8,29 @@
namespace DB
{
SharedLibraryHandler::SharedLibraryHandler(
const std::string & library_path_,
const std::string & library_settings)
: library_path(library_path_)
{
library = std::make_shared<SharedLibrary>(library_path, RTLD_LAZY
#if defined(RTLD_DEEPBIND) && !defined(ADDRESS_SANITIZER) // Does not exists in FreeBSD. Cannot work with Address Sanitizer.
| RTLD_DEEPBIND
#endif
);
std::vector<std::string> lib_settings;
boost::split(lib_settings, library_settings, [](char c) { return c == ' '; });
settings_holder = std::make_shared<CStringsHolder>(CStringsHolder(lib_settings));
auto lib_new = library->tryGet<decltype(lib_data) (*)(
decltype(&settings_holder->strings), decltype(&ClickHouseLibrary::log))>("ClickHouseDictionary_v3_libNew");
if (lib_new)
lib_data = lib_new(&settings_holder->strings, ClickHouseLibrary::log);
}
SharedLibraryHandler::SharedLibraryHandler(const SharedLibraryHandler & other)
: library_path{other.library_path}
, library{other.library}
@ -29,28 +52,6 @@ SharedLibraryHandler::~SharedLibraryHandler()
}
void SharedLibraryHandler::libNew(const std::string & path, const std::string & settings)
{
library_path = path;
library = std::make_shared<SharedLibrary>(library_path, RTLD_LAZY
#if defined(RTLD_DEEPBIND) && !defined(ADDRESS_SANITIZER) // Does not exists in FreeBSD. Cannot work with Address Sanitizer.
| RTLD_DEEPBIND
#endif
);
std::vector<std::string> lib_settings;
boost::split(lib_settings, settings, [](char c) { return c == ' '; });
settings_holder = std::make_shared<CStringsHolder>(CStringsHolder(lib_settings));
auto lib_new = library->tryGet<decltype(lib_data) (*)(
decltype(&settings_holder->strings), decltype(&ClickHouseLibrary::log))>("ClickHouseDictionary_v3_libNew");
if (lib_new)
lib_data = lib_new(&settings_holder->strings, ClickHouseLibrary::log);
}
bool SharedLibraryHandler::isModified()
{
if (auto func_is_modified = library->tryGet<bool (*)(

View File

@ -14,14 +14,12 @@ class SharedLibraryHandler
{
public:
SharedLibraryHandler() {}
SharedLibraryHandler(const std::string & library_path_, const std::string & library_settings);
SharedLibraryHandler(const SharedLibraryHandler & other);
~SharedLibraryHandler();
void libNew(const std::string & path, const std::string & settings);
BlockInputStreamPtr loadAll(const std::string & attributes_string, const Block & sample_block);
BlockInputStreamPtr loadIds(const std::string & ids_string, const std::string & attributes_string, const Block & sample_block);

View File

@ -0,0 +1,84 @@
#include "SharedLibraryHandlerFactory.h"
namespace DB
{
SharedLibraryHandlerPtr SharedLibraryHandlerFactory::get(const std::string & dictionary_id)
{
std::lock_guard lock(mutex);
auto library_handler = library_handlers.find(dictionary_id);
if (library_handler != library_handlers.end())
return library_handler->second;
return nullptr;
}
bool SharedLibraryHandlerFactory::create(const std::string & dictionary_id, const std::string & library_path, const std::string & library_settings)
{
std::lock_guard lock(mutex);
try
{
library_handlers[dictionary_id] = std::make_shared<SharedLibraryHandler>(library_path, library_settings);
}
catch (...)
{
return false;
}
return true;
}
bool SharedLibraryHandlerFactory::clone(const std::string & from_dictionary_id, const std::string & to_dictionary_id)
{
std::lock_guard lock(mutex);
auto from_library_handler = library_handlers.find(from_dictionary_id);
/// This is impossible to happen as libClone is called from copy constructor of LibraryDictionarySource
/// object, and shared library handler of from_dictionary is removed only in its destructor.
/// And if for from_dictionary there was no shared library handler, it would have received and exception in
/// its constructor, so no libClone would ever be made for it.
if (from_library_handler == library_handlers.end())
return false;
try
{
/// libClone method will be called in copy constructor
library_handlers[to_dictionary_id] = std::make_shared<SharedLibraryHandler>(*from_library_handler->second);
}
catch (...)
{
return false;
}
return true;
}
bool SharedLibraryHandlerFactory::remove(const std::string & dictionary_id)
{
std::lock_guard lock(mutex);
try
{
/// libDelete is called in destructor.
library_handlers.erase(dictionary_id);
}
catch (...)
{
return false;
}
return true;
}
SharedLibraryHandlerFactory & SharedLibraryHandlerFactory::instance()
{
static SharedLibraryHandlerFactory ret;
return ret;
}
}

View File

@ -0,0 +1,30 @@
#pragma once
#include "SharedLibraryHandler.h"
#include <unordered_map>
#include <mutex>
namespace DB
{
/// Creates map object: dictionary_id -> SharedLibararyHandler.
class SharedLibraryHandlerFactory final : private boost::noncopyable
{
public:
static SharedLibraryHandlerFactory & instance();
SharedLibraryHandlerPtr get(const std::string & dictionary_id);
bool create(const std::string & dictionary_id, const std::string & library_path, const std::string & library_settings);
bool clone(const std::string & from_dictionary_id, const std::string & to_dictionary_id);
bool remove(const std::string & dictionary_id);
private:
std::unordered_map<std::string, SharedLibraryHandlerPtr> library_handlers;
std::mutex mutex;
};
}