2018-11-28 11:37:12 +00:00
|
|
|
#include "LibraryDictionarySource.h"
|
2021-03-05 09:38:00 +00:00
|
|
|
|
2018-12-10 15:25:45 +00:00
|
|
|
#include <DataStreams/OneBlockInputStream.h>
|
2017-09-05 01:08:26 +00:00
|
|
|
#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>
|
2017-12-09 06:32:22 +00:00
|
|
|
#include <ext/range.h>
|
2018-03-13 23:13:39 +00:00
|
|
|
#include <ext/scope_guard.h>
|
2019-12-10 20:39:36 +00:00
|
|
|
#include <Common/StringUtils/StringUtils.h>
|
2018-11-28 11:37:12 +00:00
|
|
|
#include "DictionarySourceFactory.h"
|
|
|
|
#include "DictionaryStructure.h"
|
2018-12-10 15:25:45 +00:00
|
|
|
#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>
|
|
|
|
|
2017-12-09 06:32:22 +00:00
|
|
|
|
2017-09-05 01:08:26 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
2021-03-05 09:38:00 +00:00
|
|
|
|
2017-09-05 01:08:26 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
|
|
|
|
extern const int FILE_DOESNT_EXIST;
|
2018-02-16 17:12:22 +00:00
|
|
|
extern const int EXTERNAL_LIBRARY_ERROR;
|
2019-12-10 17:27:29 +00:00
|
|
|
extern const int PATH_ACCESS_DENIED;
|
2017-09-05 01:08:26 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-12-10 15:25:45 +00:00
|
|
|
LibraryDictionarySource::LibraryDictionarySource(
|
|
|
|
const DictionaryStructure & dict_struct_,
|
2017-09-05 01:08:26 +00:00
|
|
|
const Poco::Util::AbstractConfiguration & config,
|
2019-08-03 11:02:40 +00:00
|
|
|
const std::string & config_prefix_,
|
2019-12-10 17:27:29 +00:00
|
|
|
Block & sample_block_,
|
2021-03-05 09:38:00 +00:00
|
|
|
const Context & context_,
|
2019-12-10 17:27:29 +00:00
|
|
|
bool check_config)
|
2020-05-30 21:57:37 +00:00
|
|
|
: log(&Poco::Logger::get("LibraryDictionarySource"))
|
2018-02-14 15:03:48 +00:00
|
|
|
, dict_struct{dict_struct_}
|
2019-08-03 11:02:40 +00:00
|
|
|
, config_prefix{config_prefix_}
|
2018-02-14 15:03:48 +00:00
|
|
|
, 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_)
|
2017-09-05 01:08:26 +00:00
|
|
|
{
|
2019-12-10 17:27:29 +00:00
|
|
|
|
|
|
|
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);
|
2019-12-10 17:27:29 +00:00
|
|
|
}
|
|
|
|
|
2017-09-05 01:08:26 +00:00
|
|
|
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)
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to create shared library from path: {}", path);
|
2019-12-10 17:27:29 +00:00
|
|
|
|
2017-09-05 01:08:26 +00:00
|
|
|
description.init(sample_block);
|
|
|
|
}
|
|
|
|
|
2021-03-05 09:38:00 +00:00
|
|
|
|
|
|
|
LibraryDictionarySource::~LibraryDictionarySource()
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-09-05 01:08:26 +00:00
|
|
|
LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource & other)
|
2020-05-30 21:57:37 +00:00
|
|
|
: log(&Poco::Logger::get("LibraryDictionarySource"))
|
2018-02-14 15:03:48 +00:00
|
|
|
, dict_struct{other.dict_struct}
|
|
|
|
, config_prefix{other.config_prefix}
|
|
|
|
, path{other.path}
|
2021-03-05 09:38:00 +00:00
|
|
|
, dictionary_id{other.dictionary_id}
|
2018-02-14 15:03:48 +00:00
|
|
|
, sample_block{other.sample_block}
|
2021-03-05 09:38:00 +00:00
|
|
|
, context(other.context)
|
2018-02-14 15:03:48 +00:00
|
|
|
, description{other.description}
|
2018-02-26 16:57:14 +00:00
|
|
|
{
|
2021-03-05 09:38:00 +00:00
|
|
|
bridge_helper = std::make_shared<LibraryBridgeHelper>(context, dictionary_id);
|
2018-02-26 16:57:14 +00:00
|
|
|
}
|
|
|
|
|
2021-03-05 09:38:00 +00:00
|
|
|
|
|
|
|
String LibraryDictionarySource::getLibrarySettingsString(const Poco::Util::AbstractConfiguration & config, const std::string & config_root)
|
2017-09-05 01:08:26 +00:00
|
|
|
{
|
2021-03-05 09:38:00 +00:00
|
|
|
Poco::Util::AbstractConfiguration::Keys config_keys;
|
|
|
|
config.keys(config_root, config_keys);
|
|
|
|
std::string res;
|
|
|
|
|
|
|
|
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);
|
|
|
|
|
|
|
|
if (!res.empty())
|
|
|
|
res += ' ';
|
|
|
|
|
|
|
|
res += key_name + ' ' + config.getString(config_root + "." + key);
|
|
|
|
}
|
|
|
|
|
|
|
|
return res;
|
2017-09-05 01:08:26 +00:00
|
|
|
}
|
|
|
|
|
2021-03-05 09:38:00 +00:00
|
|
|
|
|
|
|
String LibraryDictionarySource::getDictAttributesString()
|
2017-09-05 01:08:26 +00:00
|
|
|
{
|
2021-03-05 09:38:00 +00:00
|
|
|
std::string res;
|
|
|
|
for (const auto & attr : dict_struct.attributes)
|
|
|
|
{
|
|
|
|
if (!res.empty())
|
|
|
|
res += ',';
|
|
|
|
res += attr.name;
|
|
|
|
}
|
2017-09-05 01:08:26 +00:00
|
|
|
|
2021-03-05 09:38:00 +00:00
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
String LibraryDictionarySource::getDictIdsString(const std::vector<UInt64> & ids)
|
|
|
|
{
|
|
|
|
std::string res;
|
|
|
|
for (const auto & id : ids)
|
2017-09-05 01:08:26 +00:00
|
|
|
{
|
2021-03-05 09:38:00 +00:00
|
|
|
if (!res.empty())
|
|
|
|
res += ',';
|
|
|
|
res += std::to_string(id);
|
2017-09-05 01:08:26 +00:00
|
|
|
}
|
2021-03-05 09:38:00 +00:00
|
|
|
|
|
|
|
return res;
|
2017-09-05 01:08:26 +00:00
|
|
|
}
|
|
|
|
|
2021-03-05 09:38:00 +00:00
|
|
|
|
|
|
|
BlockInputStreamPtr LibraryDictionarySource::loadAll()
|
|
|
|
{
|
|
|
|
LOG_TRACE(log, "loadAll {}", toString());
|
|
|
|
return bridge_helper->loadAll(getDictAttributesString(), description.sample_block);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-09-05 01:08:26 +00:00
|
|
|
BlockInputStreamPtr LibraryDictionarySource::loadIds(const std::vector<UInt64> & ids)
|
|
|
|
{
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size());
|
2021-03-05 09:38:00 +00:00
|
|
|
return bridge_helper->loadIds(getDictAttributesString(), getDictIdsString(ids), description.sample_block);
|
2017-09-05 01:08:26 +00:00
|
|
|
}
|
|
|
|
|
2021-03-05 09:38:00 +00:00
|
|
|
|
|
|
|
BlockInputStreamPtr LibraryDictionarySource::loadKeys(const Columns &/* key_columns */, const std::vector<std::size_t> & requested_rows)
|
2017-09-05 01:08:26 +00:00
|
|
|
{
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size());
|
2021-03-05 09:38:00 +00:00
|
|
|
return {};
|
|
|
|
//return bridge_helper->loadIds(getDictAttributesStrig(), getDictIdsString(), escription.sample_block);
|
2017-09-05 01:08:26 +00:00
|
|
|
|
2021-03-05 09:38:00 +00:00
|
|
|
//auto holder = std::make_unique<ClickHouseLibrary::Row[]>(key_columns.size());
|
|
|
|
//std::vector<std::unique_ptr<ClickHouseLibrary::Field[]>> column_data_holders;
|
|
|
|
|
|
|
|
//for (size_t i = 0; i < key_columns.size(); ++i)
|
|
|
|
//{
|
|
|
|
// auto cell_holder = std::make_unique<ClickHouseLibrary::Field[]>(requested_rows.size());
|
|
|
|
|
|
|
|
// for (size_t j = 0; j < requested_rows.size(); ++j)
|
|
|
|
// {
|
|
|
|
// auto data_ref = key_columns[i]->getDataAt(requested_rows[j]);
|
|
|
|
// cell_holder[j] = ClickHouseLibrary::Field{.data = static_cast<const void *>(data_ref.data), .size = data_ref.size};
|
|
|
|
// }
|
|
|
|
|
|
|
|
// holder[i] = ClickHouseLibrary::Row{.data = static_cast<ClickHouseLibrary::Field *>(cell_holder.get()), .size = requested_rows.size()};
|
|
|
|
// column_data_holders.push_back(std::move(cell_holder));
|
|
|
|
//}
|
|
|
|
|
|
|
|
//ClickHouseLibrary::Table request_cols{.data = static_cast<ClickHouseLibrary::Row *>(holder.get()), .size = key_columns.size()};
|
|
|
|
|
|
|
|
//void * data_ptr = nullptr;
|
|
|
|
|
|
|
|
///// Get function pointer before dataNew call because library->get may throw.
|
|
|
|
//auto func_load_keys = library->get<void * (*)(decltype(data_ptr), decltype(&settings->strings), decltype(&request_cols))>("ClickHouseDictionary_v3_loadKeys");
|
|
|
|
|
|
|
|
//data_ptr = library->get<decltype(data_ptr) (*)(decltype(lib_data))>("ClickHouseDictionary_v3_dataNew")(lib_data);
|
|
|
|
//auto * data = func_load_keys(data_ptr, &settings->strings, &request_cols);
|
|
|
|
//auto block = dataToBlock(description.sample_block, data);
|
2017-09-05 01:08:26 +00:00
|
|
|
|
2021-03-05 09:38:00 +00:00
|
|
|
//SCOPE_EXIT(library->get<void (*)(decltype(lib_data), decltype(data_ptr))>("ClickHouseDictionary_v3_dataDelete")(lib_data, data_ptr));
|
2017-09-05 01:08:26 +00:00
|
|
|
}
|
|
|
|
|
2021-03-05 09:38:00 +00:00
|
|
|
|
2017-09-05 01:08:26 +00:00
|
|
|
bool LibraryDictionarySource::isModified() const
|
|
|
|
{
|
2021-03-05 10:43:47 +00:00
|
|
|
return bridge_helper->isModified();
|
2017-09-05 01:08:26 +00:00
|
|
|
}
|
|
|
|
|
2021-03-05 09:38:00 +00:00
|
|
|
|
2017-09-05 01:08:26 +00:00
|
|
|
bool LibraryDictionarySource::supportsSelectiveLoad() const
|
|
|
|
{
|
2021-03-05 10:43:47 +00:00
|
|
|
return bridge_helper->supportsSelectiveLoad();
|
2017-09-05 01:08:26 +00:00
|
|
|
}
|
|
|
|
|
2021-03-05 09:38:00 +00:00
|
|
|
|
2017-09-05 01:08:26 +00:00
|
|
|
DictionarySourcePtr LibraryDictionarySource::clone() const
|
|
|
|
{
|
|
|
|
return std::make_unique<LibraryDictionarySource>(*this);
|
|
|
|
}
|
|
|
|
|
2021-03-05 09:38:00 +00:00
|
|
|
|
2017-09-05 01:08:26 +00:00
|
|
|
std::string LibraryDictionarySource::toString() const
|
|
|
|
{
|
|
|
|
return path;
|
|
|
|
}
|
2018-11-28 11:37:12 +00:00
|
|
|
|
2021-03-05 09:38:00 +00:00
|
|
|
|
2018-11-28 11:37:12 +00:00
|
|
|
void registerDictionarySourceLibrary(DictionarySourceFactory & factory)
|
|
|
|
{
|
2020-03-23 02:12:31 +00:00
|
|
|
auto create_table_source = [=](const DictionaryStructure & dict_struct,
|
2018-11-28 11:37:12 +00:00
|
|
|
const Poco::Util::AbstractConfiguration & config,
|
|
|
|
const std::string & config_prefix,
|
|
|
|
Block & sample_block,
|
2019-12-10 17:27:29 +00:00
|
|
|
const Context & context,
|
2020-08-15 03:10:57 +00:00
|
|
|
const std::string & /* default_database */,
|
2019-12-10 17:27:29 +00:00
|
|
|
bool check_config) -> DictionarySourcePtr
|
2019-01-29 19:05:32 +00:00
|
|
|
{
|
2019-12-10 17:27:29 +00:00
|
|
|
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, context, check_config);
|
2018-11-28 11:37:12 +00:00
|
|
|
};
|
2021-03-05 09:38:00 +00:00
|
|
|
|
2020-03-23 02:12:31 +00:00
|
|
|
factory.registerSource("library", create_table_source);
|
2018-11-28 11:37:12 +00:00
|
|
|
}
|
|
|
|
|
2021-03-05 09:38:00 +00:00
|
|
|
|
2017-09-05 01:08:26 +00:00
|
|
|
}
|