mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 10:02:01 +00:00
dbms: dummy types and functions for external dictionaries. [#METR-13298]
This commit is contained in:
parent
c52d6b0ce8
commit
300b7342e7
21
dbms/include/DB/Dictionaries/DictionaryFactory.h
Normal file
21
dbms/include/DB/Dictionaries/DictionaryFactory.h
Normal file
@ -0,0 +1,21 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Dictionaries/FlatDictionary.h>
|
||||
#include <DB/Dictionaries/IDictionary.h>
|
||||
#include <Yandex/singleton.h>
|
||||
#include <statdaemons/ext/memory.hpp>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class DictionaryFactory : public Singleton<DictionaryFactory>
|
||||
{
|
||||
public:
|
||||
DictionaryPtr create() const
|
||||
{
|
||||
return ext::make_unique<FlatDictionary>();
|
||||
}
|
||||
};
|
||||
|
||||
}
|
32
dbms/include/DB/Dictionaries/FlatDictionary.h
Normal file
32
dbms/include/DB/Dictionaries/FlatDictionary.h
Normal file
@ -0,0 +1,32 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Dictionaries/IDictionary.h>
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
#include <map>
|
||||
#include <vector>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class FlatDictionary : public IDictionary
|
||||
{
|
||||
public:
|
||||
FlatDictionary() = default;
|
||||
|
||||
StringRef getString(const id_t id, const std::string & attribute_name) const override {
|
||||
return { "", 0 };
|
||||
}
|
||||
|
||||
UInt64 getUInt64(const id_t id, const std::string & attribute_name) const override {
|
||||
return 0;
|
||||
}
|
||||
|
||||
private:
|
||||
using value_t = std::pair<id_t, Field>;
|
||||
using attribute_t = std::vector<value_t>;
|
||||
using attributes_t = std::map<std::string, attribute_t>;
|
||||
|
||||
attribute_t attributes;
|
||||
};
|
||||
|
||||
}
|
22
dbms/include/DB/Dictionaries/IDictionary.h
Normal file
22
dbms/include/DB/Dictionaries/IDictionary.h
Normal file
@ -0,0 +1,22 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Core/Field.h>
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IDictionary
|
||||
{
|
||||
public:
|
||||
using id_t = std::uint64_t;
|
||||
|
||||
virtual StringRef getString(const id_t id, const std::string & attribute_name) const = 0;
|
||||
virtual UInt64 getUInt64(const id_t id, const std::string & attribute_name) const = 0;
|
||||
|
||||
virtual ~IDictionary() = default;
|
||||
};
|
||||
|
||||
using DictionaryPtr = std::unique_ptr<IDictionary>;
|
||||
|
||||
}
|
@ -778,4 +778,134 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionDictGetString : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "dictGetString";
|
||||
|
||||
static IFunction * create(const Context & context)
|
||||
{
|
||||
return new FunctionDictGetString{context.getDictionaries()};
|
||||
};
|
||||
|
||||
FunctionDictGetString(const Dictionaries & dictionaries) : dictionaries(dictionaries) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
private:
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 3)
|
||||
throw Exception{
|
||||
"Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 3.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
|
||||
};
|
||||
|
||||
if (!typeid_cast<const DataTypeString *>(arguments[0].get()))
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
}
|
||||
|
||||
if (!typeid_cast<const DataTypeString *>(arguments[1].get()))
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
}
|
||||
|
||||
const auto id_arg = arguments[2].get();
|
||||
if (!typeid_cast<const DataTypeUInt8 *>(id_arg) &&
|
||||
!typeid_cast<const DataTypeUInt16 *>(id_arg) &&
|
||||
!typeid_cast<const DataTypeUInt32 *>(id_arg) &&
|
||||
!typeid_cast<const DataTypeUInt64 *>(id_arg) &&
|
||||
!typeid_cast<const DataTypeInt8 *>(id_arg) &&
|
||||
!typeid_cast<const DataTypeInt16 *>(id_arg) &&
|
||||
!typeid_cast<const DataTypeInt32 *>(id_arg) &&
|
||||
!typeid_cast<const DataTypeInt64 *>(id_arg))
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal type " + arguments[2]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
}
|
||||
|
||||
return new DataTypeString;
|
||||
}
|
||||
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
{
|
||||
const auto dict_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[0]).column.get());
|
||||
if (!dict_name_col)
|
||||
throw Exception{
|
||||
"First argument of function " + getName() + " must be a constant string",
|
||||
ErrorCodes::ILLEGAL_COLUMN
|
||||
};
|
||||
|
||||
auto dict = dictionaries.getExternalDictionary(dict_name_col->getData());
|
||||
|
||||
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
|
||||
if (!attr_name_col)
|
||||
throw Exception{
|
||||
"Second argument of function " + getName() + " must be a constant string",
|
||||
ErrorCodes::ILLEGAL_COLUMN
|
||||
};
|
||||
|
||||
const auto & attr_name = attr_name_col->getData();
|
||||
|
||||
const auto id_col = block.getByPosition(arguments[2]).column.get();
|
||||
if (!execute<UInt8>(block, result, dict, attr_name, id_col) &&
|
||||
!execute<UInt16>(block, result, dict, attr_name, id_col) &&
|
||||
!execute<UInt32>(block, result, dict, attr_name, id_col) &&
|
||||
!execute<UInt64>(block, result, dict, attr_name, id_col) &&
|
||||
!execute<Int8>(block, result, dict, attr_name, id_col) &&
|
||||
!execute<Int16>(block, result, dict, attr_name, id_col) &&
|
||||
!execute<Int32>(block, result, dict, attr_name, id_col) &&
|
||||
!execute<Int64>(block, result, dict, attr_name, id_col))
|
||||
{
|
||||
throw Exception{
|
||||
"Third argument of function " + getName() + " must be integral",
|
||||
ErrorCodes::ILLEGAL_COLUMN
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool execute(Block & block, const size_t result, const MultiVersion<IDictionary>::Version & dictionary,
|
||||
const std::string & attr_name, const IColumn * const id_col_untyped)
|
||||
{
|
||||
if (const auto id_col = typeid_cast<const ColumnVector<T> *>(id_col_untyped))
|
||||
{
|
||||
const auto out = new ColumnString;
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
for (const auto & id : id_col->getData())
|
||||
{
|
||||
const auto string_ref = dictionary->getString(id, attr_name);
|
||||
out->insertData(string_ref.data, string_ref.size);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (const auto id_col = typeid_cast<const ColumnConst<T> *>(id_col_untyped))
|
||||
{
|
||||
block.getByPosition(result).column = new ColumnConst<String>{
|
||||
id_col->size(),
|
||||
dictionary->getString(id_col->getData(), attr_name).toString()
|
||||
};
|
||||
|
||||
return true;
|
||||
};
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
const Dictionaries & dictionaries;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <thread>
|
||||
#include <unordered_map>
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
@ -11,6 +12,9 @@
|
||||
#include <statdaemons/CategoriesHierarchy.h>
|
||||
#include <statdaemons/RegionsNames.h>
|
||||
|
||||
#include <DB/Dictionaries/IDictionary.h>
|
||||
#include <DB/Dictionaries/FlatDictionary.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -26,6 +30,7 @@ private:
|
||||
MultiVersion<TechDataHierarchy> tech_data_hierarchy;
|
||||
MultiVersion<CategoriesHierarchy> categories_hierarchy;
|
||||
MultiVersion<RegionsNames> regions_names;
|
||||
std::unordered_map<std::string, std::shared_ptr<MultiVersion<IDictionary>>> external_dictionaries;
|
||||
|
||||
/// Периодичность обновления справочников, в секундах.
|
||||
int reload_period;
|
||||
@ -116,10 +121,24 @@ private:
|
||||
was_exception = true;
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
reloadExternalDictionaries();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
handleException();
|
||||
was_exception = true;
|
||||
}
|
||||
|
||||
|
||||
if (!was_exception)
|
||||
LOG_INFO(log, "Loaded dictionaries.");
|
||||
}
|
||||
|
||||
|
||||
void reloadExternalDictionaries();
|
||||
|
||||
/// Обновляет каждые reload_period секунд.
|
||||
void reloadPeriodically()
|
||||
{
|
||||
@ -167,6 +186,18 @@ public:
|
||||
{
|
||||
return regions_names.get();
|
||||
}
|
||||
|
||||
MultiVersion<IDictionary>::Version getExternalDictionary(const std::string & name) const
|
||||
{
|
||||
const auto it = external_dictionaries.find(name);
|
||||
if (it == std::end(external_dictionaries))
|
||||
throw Exception{
|
||||
"No such dictionary: " + name,
|
||||
ErrorCodes::BAD_ARGUMENTS
|
||||
};
|
||||
|
||||
return it->second->get();
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -25,6 +25,7 @@ void registerFunctionsDictionaries(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionSEHierarchy>();
|
||||
factory.registerFunction<FunctionCategoryHierarchy>();
|
||||
factory.registerFunction<FunctionRegionToName>();
|
||||
factory.registerFunction<FunctionDictGetString>();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
58
dbms/src/Interpreters/Dictionaries.cpp
Normal file
58
dbms/src/Interpreters/Dictionaries.cpp
Normal file
@ -0,0 +1,58 @@
|
||||
#include <DB/Interpreters/Dictionaries.h>
|
||||
#include <DB/Dictionaries/DictionaryFactory.h>
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
template <typename T> struct release
|
||||
{
|
||||
void operator()(const T * const ptr) { ptr->release(); }
|
||||
};
|
||||
template <typename T> using config_ptr_t = std::unique_ptr<T, release<T>>;
|
||||
};
|
||||
|
||||
void Dictionaries::reloadExternalDictionaries()
|
||||
{
|
||||
const auto config_path = Poco::Util::Application::instance().config().getString("dictionaries_config");
|
||||
const config_ptr_t<Poco::Util::XMLConfiguration> config{new Poco::Util::XMLConfiguration{config_path}};
|
||||
|
||||
/// get all dictionaries' definitions
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config->keys(keys);
|
||||
|
||||
/// for each dictionary defined in xml config
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
if (0 != strncmp(key.data(), "dictionary", strlen("dictionary")))
|
||||
{
|
||||
/// @todo maybe output a warning
|
||||
continue;
|
||||
}
|
||||
|
||||
std::cout << key << std::endl;
|
||||
const auto & prefix = key + '.';
|
||||
|
||||
const auto & name = config->getString(prefix + "name");
|
||||
if (name.empty())
|
||||
{
|
||||
/// @todo handle error, dictionary name cannot be empty
|
||||
}
|
||||
|
||||
auto dict_ptr = DictionaryFactory::instance().create();
|
||||
const auto it = external_dictionaries.find(name);
|
||||
if (it == std::end(external_dictionaries))
|
||||
{
|
||||
external_dictionaries.emplace(name, std::make_shared<MultiVersion<IDictionary>>(dict_ptr.release()));
|
||||
}
|
||||
else
|
||||
{
|
||||
it->second->set(dict_ptr.release());
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
}
|
Loading…
Reference in New Issue
Block a user