ClickHouse/src/Dictionaries/LibraryDictionarySource.cpp

206 lines
6.0 KiB
C++
Raw Normal View History

#include "LibraryDictionarySource.h"
2021-03-05 09:38:00 +00:00
#include <DataStreams/OneBlockInputStream.h>
#include <Interpreters/Context.h>
#include <Poco/File.h>
#include <common/logger_useful.h>
2017-09-08 18:24:15 +00:00
#include <ext/bit_cast.h>
#include <ext/range.h>
#include <ext/scope_guard.h>
2019-12-10 20:39:36 +00:00
#include <Common/StringUtils/StringUtils.h>
#include "DictionarySourceFactory.h"
2021-03-10 13:10:05 +00:00
#include "DictionarySourceHelpers.h"
#include "DictionaryStructure.h"
#include "LibraryDictionarySourceExternal.h"
2019-12-15 06:34:43 +00:00
#include "registerDictionaries.h"
2021-03-05 09:38:00 +00:00
#include <IO/WriteBufferFromString.h>
2021-03-10 18:02:43 +00:00
#include <IO/WriteHelpers.h>
2021-03-05 09:38:00 +00:00
namespace DB
{
2021-03-05 09:38:00 +00:00
namespace ErrorCodes
{
extern const int FILE_DOESNT_EXIST;
extern const int EXTERNAL_LIBRARY_ERROR;
extern const int PATH_ACCESS_DENIED;
}
LibraryDictionarySource::LibraryDictionarySource(
const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config,
2019-08-03 11:02:40 +00:00
const std::string & config_prefix_,
Block & sample_block_,
2021-03-05 09:38:00 +00:00
const Context & context_,
bool check_config)
2020-05-30 21:57:37 +00:00
: log(&Poco::Logger::get("LibraryDictionarySource"))
, dict_struct{dict_struct_}
2019-08-03 11:02:40 +00:00
, config_prefix{config_prefix_}
, path{config.getString(config_prefix + ".path", "")}
2021-03-05 09:38:00 +00:00
, dictionary_id(createDictID())
2019-08-03 11:02:40 +00:00
, sample_block{sample_block_}
2021-03-05 09:38:00 +00:00
, context(context_)
{
if (check_config)
{
const String dictionaries_lib_path = context.getDictionariesLibPath();
if (!startsWith(path, dictionaries_lib_path))
2021-03-05 09:38:00 +00:00
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "LibraryDictionarySource: Library path {} is not inside {}", path, dictionaries_lib_path);
}
if (!Poco::File(path).exists())
2021-03-05 09:38:00 +00:00
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "LibraryDictionarySource: Can't load library {}: file doesn't exist", Poco::File(path).path());
bridge_helper = std::make_shared<LibraryBridgeHelper>(context, dictionary_id);
auto res = bridge_helper->initLibrary(path, getLibrarySettingsString(config, config_prefix + ".settings"));
if (!res)
2021-03-07 19:53:10 +00:00
throw Exception(ErrorCodes::EXTERNAL_LIBRARY_ERROR, "Failed to create shared library from path: {}", path);
description.init(sample_block);
}
2021-03-05 09:38:00 +00:00
LibraryDictionarySource::~LibraryDictionarySource()
{
2021-03-05 15:37:43 +00:00
bridge_helper->removeLibrary();
2021-03-05 09:38:00 +00:00
}
LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource & other)
2020-05-30 21:57:37 +00:00
: log(&Poco::Logger::get("LibraryDictionarySource"))
, dict_struct{other.dict_struct}
, config_prefix{other.config_prefix}
, path{other.path}
2021-03-05 15:37:43 +00:00
, dictionary_id{createDictID()}
, sample_block{other.sample_block}
2021-03-05 09:38:00 +00:00
, context(other.context)
, description{other.description}
{
2021-03-05 09:38:00 +00:00
bridge_helper = std::make_shared<LibraryBridgeHelper>(context, dictionary_id);
2021-03-05 15:37:43 +00:00
bridge_helper->cloneLibrary(other.dictionary_id);
}
2021-03-05 09:38:00 +00:00
2021-03-07 15:23:20 +00:00
bool LibraryDictionarySource::isModified() const
2021-03-05 09:38:00 +00:00
{
2021-03-07 15:23:20 +00:00
return bridge_helper->isModified();
2021-03-05 09:38:00 +00:00
}
2021-03-07 15:23:20 +00:00
bool LibraryDictionarySource::supportsSelectiveLoad() const
{
2021-03-07 15:23:20 +00:00
return bridge_helper->supportsSelectiveLoad();
}
2021-03-05 09:38:00 +00:00
2021-03-07 15:23:20 +00:00
BlockInputStreamPtr LibraryDictionarySource::loadAll()
{
2021-03-07 15:23:20 +00:00
LOG_TRACE(log, "loadAll {}", toString());
return bridge_helper->loadAll(getDictAttributesString(), description.sample_block);
}
2021-03-05 09:38:00 +00:00
2021-03-07 15:23:20 +00:00
BlockInputStreamPtr LibraryDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
2021-03-07 15:23:20 +00:00
LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size());
return bridge_helper->loadIds(getDictAttributesString(), getDictIdsString(ids), description.sample_block);
}
2021-03-05 09:38:00 +00:00
2021-03-10 13:10:05 +00:00
BlockInputStreamPtr LibraryDictionarySource::loadKeys(const Columns & key_columns, const std::vector<std::size_t> & requested_rows)
{
2021-03-07 15:23:20 +00:00
LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size());
2021-03-10 13:10:05 +00:00
auto block = blockForKeys(dict_struct, key_columns, requested_rows);
return bridge_helper->loadKeys(block, description.sample_block);
}
2021-03-05 09:38:00 +00:00
DictionarySourcePtr LibraryDictionarySource::clone() const
{
return std::make_unique<LibraryDictionarySource>(*this);
}
2021-03-05 09:38:00 +00:00
std::string LibraryDictionarySource::toString() const
{
return path;
}
2021-03-05 09:38:00 +00:00
2021-03-05 15:37:43 +00:00
String LibraryDictionarySource::getLibrarySettingsString(const Poco::Util::AbstractConfiguration & config, const std::string & config_root)
{
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(config_root, config_keys);
2021-03-10 18:02:43 +00:00
WriteBufferFromOwnString res;
2021-03-05 15:37:43 +00:00
for (const auto & key : config_keys)
{
std::string key_name = key;
auto bracket_pos = key.find('[');
if (bracket_pos != std::string::npos && bracket_pos > 0)
key_name = key.substr(0, bracket_pos);
2021-03-10 18:02:43 +00:00
if (res.stringRef().size)
writeChar(' ', res);
2021-03-05 15:37:43 +00:00
2021-03-10 18:02:43 +00:00
writeString(key_name, res);
writeChar(' ', res);
writeString(config.getString(config_root + "." + key), res);
2021-03-05 15:37:43 +00:00
}
2021-03-10 18:02:43 +00:00
return res.str();
2021-03-05 15:37:43 +00:00
}
String LibraryDictionarySource::getDictAttributesString()
{
2021-03-10 18:02:43 +00:00
WriteBufferFromOwnString res;
2021-03-05 15:37:43 +00:00
for (const auto & attr : dict_struct.attributes)
{
2021-03-10 18:02:43 +00:00
if (res.stringRef().size)
writeChar(' ', res);
writeString(attr.name, res);
2021-03-05 15:37:43 +00:00
}
2021-03-10 18:02:43 +00:00
return res.str();
2021-03-05 15:37:43 +00:00
}
String LibraryDictionarySource::getDictIdsString(const std::vector<UInt64> & ids)
{
2021-03-10 18:02:43 +00:00
WriteBufferFromOwnString res;
2021-03-05 15:37:43 +00:00
for (const auto & id : ids)
{
2021-03-10 18:02:43 +00:00
if (res.stringRef().size)
writeChar(' ', res);
writeString(std::to_string(id), res);
2021-03-05 15:37:43 +00:00
}
2021-03-10 18:02:43 +00:00
return res.str();
2021-03-05 15:37:43 +00:00
}
void registerDictionarySourceLibrary(DictionarySourceFactory & factory)
{
auto create_table_source = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
const Context & context,
2020-08-15 03:10:57 +00:00
const std::string & /* default_database */,
bool check_config) -> DictionarySourcePtr
{
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, context, check_config);
};
2021-03-05 09:38:00 +00:00
factory.registerSource("library", create_table_source);
}
2021-03-05 09:38:00 +00:00
}