mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
dbms: add dictGetHierarchy and dictIsIn. [#METR-13298]
This commit is contained in:
parent
6b406df4d9
commit
f4e7d2d4f5
@ -50,6 +50,29 @@ attribute_type getAttributeTypeByName(const std::string & type)
|
||||
};
|
||||
}
|
||||
|
||||
std::string toString(const attribute_type type)
|
||||
{
|
||||
switch (type)
|
||||
{
|
||||
case attribute_type::uint8: return "UInt8";
|
||||
case attribute_type::uint16: return "UInt16";
|
||||
case attribute_type::uint32: return "UInt32";
|
||||
case attribute_type::uint64: return "UInt64";
|
||||
case attribute_type::int8: return "Int8";
|
||||
case attribute_type::int16: return "Int16";
|
||||
case attribute_type::int32: return "Int32";
|
||||
case attribute_type::int64: return "Int64";
|
||||
case attribute_type::float32: return "Float32";
|
||||
case attribute_type::float64: return "Float64";
|
||||
case attribute_type::string: return "String";
|
||||
}
|
||||
|
||||
throw Exception{
|
||||
"Unknown attribute_type " + toString(type),
|
||||
ErrorCodes::ARGUMENT_OUT_OF_BOUND
|
||||
};
|
||||
}
|
||||
|
||||
struct DictionaryAttribute
|
||||
{
|
||||
std::string name;
|
||||
|
@ -20,13 +20,16 @@ public:
|
||||
const std::string & config_prefix, DictionarySourcePtr source_ptr)
|
||||
: source_ptr{std::move(source_ptr)}
|
||||
{
|
||||
attributes.reserve(dict_struct.attributes.size());
|
||||
for (const auto & attribute : dict_struct.attributes)
|
||||
{
|
||||
attributes.emplace(attribute.name,
|
||||
createAttributeWithType(getAttributeTypeByName(attribute.type), attribute.null_value));
|
||||
attribute_index_by_name.emplace(attribute.name, attributes.size());
|
||||
attributes.emplace_back(
|
||||
createAttributeWithType(getAttributeTypeByName(attribute.type),
|
||||
attribute.null_value));
|
||||
|
||||
if (attribute.hierarchical)
|
||||
hierarchical_attribute = &attributes[attribute.name];
|
||||
hierarchical_attribute = &attributes.back();
|
||||
}
|
||||
|
||||
auto stream = this->source_ptr->loadAll();
|
||||
@ -38,7 +41,7 @@ public:
|
||||
for (const auto attribute_idx : ext::range(0, attributes.size()))
|
||||
{
|
||||
const auto & attribute_column = *block.getByPosition(attribute_idx + 1).column;
|
||||
auto & attribute = attributes[dict_struct.attributes[attribute_idx].name];
|
||||
auto & attribute = attributes[attribute_idx];
|
||||
|
||||
for (const auto row_idx : ext::range(0, id_column.size()))
|
||||
setAttributeValue(attribute, id_column[row_idx].get<UInt64>(), attribute_column[row_idx]);
|
||||
@ -49,14 +52,41 @@ public:
|
||||
this->source_ptr->reset();
|
||||
}
|
||||
|
||||
private:
|
||||
UInt64 getUInt64(const id_t id, const std::string & attribute_name) const override
|
||||
id_t toParent(const id_t id) const override
|
||||
{
|
||||
const auto & attribute = findAttribute(attribute_name);
|
||||
const auto exists = id < max_array_size;
|
||||
const auto attr = hierarchical_attribute;
|
||||
|
||||
switch (hierarchical_attribute->type)
|
||||
{
|
||||
case attribute_type::uint8: return exists ? attr->uint8_array[id] : attr->uint8_null_value;
|
||||
case attribute_type::uint16: return exists ? attr->uint16_array[id] : attr->uint16_null_value;
|
||||
case attribute_type::uint32: return exists ? attr->uint32_array[id] : attr->uint32_null_value;
|
||||
case attribute_type::uint64: return exists ? attr->uint64_array[id] : attr->uint64_null_value;
|
||||
case attribute_type::int8: return exists ? attr->int8_array[id] : attr->int8_null_value;
|
||||
case attribute_type::int16: return exists ? attr->int16_array[id] : attr->int16_null_value;
|
||||
case attribute_type::int32: return exists ? attr->int32_array[id] : attr->int32_null_value;
|
||||
case attribute_type::int64: return exists ? attr->int64_array[id] : attr->int64_null_value;
|
||||
case attribute_type::float32:
|
||||
case attribute_type::float64:
|
||||
case attribute_type::string:
|
||||
break;
|
||||
}
|
||||
|
||||
throw Exception{
|
||||
"Hierarchical attribute has non-integer type " + toString(hierarchical_attribute->type),
|
||||
ErrorCodes::TYPE_MISMATCH
|
||||
};
|
||||
}
|
||||
|
||||
UInt64 getUInt64(const std::string & attribute_name, const id_t id) const override
|
||||
{
|
||||
const auto idx = getAttributeIndex(attribute_name);
|
||||
const auto & attribute = attributes[idx];
|
||||
|
||||
if (attribute.type != attribute_type::uint64)
|
||||
throw Exception{
|
||||
"Type mismatch: attribute " + attribute_name + " has a type different from UInt64",
|
||||
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
|
||||
ErrorCodes::TYPE_MISMATCH
|
||||
};
|
||||
|
||||
@ -66,13 +96,14 @@ private:
|
||||
return attribute.uint64_null_value;
|
||||
}
|
||||
|
||||
StringRef getString(const id_t id, const std::string & attribute_name) const override
|
||||
StringRef getString(const std::string & attribute_name, const id_t id) const override
|
||||
{
|
||||
const auto & attribute = findAttribute(attribute_name);
|
||||
const auto idx = getAttributeIndex(attribute_name);
|
||||
const auto & attribute = attributes[idx];
|
||||
|
||||
if (attribute.type != attribute_type::string)
|
||||
throw Exception{
|
||||
"Type mismatch: attribute " + attribute_name + " has a type different from String",
|
||||
"Type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
|
||||
ErrorCodes::TYPE_MISMATCH
|
||||
};
|
||||
|
||||
@ -82,8 +113,44 @@ private:
|
||||
return { attribute.string_null_value.data(), attribute.string_null_value.size() };
|
||||
}
|
||||
|
||||
std::size_t getAttributeIndex(const std::string & attribute_name) const override
|
||||
{
|
||||
const auto it = attribute_index_by_name.find(attribute_name);
|
||||
if (it == std::end(attribute_index_by_name))
|
||||
throw Exception{
|
||||
"No such attribute '" + attribute_name + "'",
|
||||
ErrorCodes::BAD_ARGUMENTS
|
||||
};
|
||||
|
||||
return it->second;
|
||||
}
|
||||
|
||||
bool isUInt64(const std::size_t attribute_idx) const override
|
||||
{
|
||||
return attributes[attribute_idx].type == attribute_type::uint64;
|
||||
}
|
||||
|
||||
bool isString(const std::size_t attribute_idx) const override
|
||||
{
|
||||
return attributes[attribute_idx].type == attribute_type::string;
|
||||
}
|
||||
|
||||
UInt64 getUInt64Unsafe(const std::size_t attribute_idx, const id_t id) const override
|
||||
{
|
||||
const auto & attribute = attributes[attribute_idx];
|
||||
return id < max_array_size ? attribute.uint64_array[id] : attribute.uint64_null_value;
|
||||
}
|
||||
|
||||
StringRef getStringUnsafe(const std::size_t attribute_idx, const id_t id) const override
|
||||
{
|
||||
const auto & attribute = attributes[attribute_idx];
|
||||
return id < max_array_size ? attribute.string_array[id] : attribute.string_null_value;
|
||||
}
|
||||
|
||||
bool isComplete() const override { return true; }
|
||||
|
||||
bool hasHierarchy() const override { return hierarchical_attribute; }
|
||||
|
||||
struct attribute_t
|
||||
{
|
||||
attribute_type type;
|
||||
@ -112,8 +179,6 @@ private:
|
||||
std::vector<StringRef> string_array;
|
||||
};
|
||||
|
||||
using attributes_t = std::map<std::string, attribute_t>;
|
||||
|
||||
attribute_t createAttributeWithType(const attribute_type type, const std::string & null_value)
|
||||
{
|
||||
attribute_t attr{type};
|
||||
@ -221,19 +286,8 @@ private:
|
||||
}
|
||||
}
|
||||
|
||||
const attribute_t & findAttribute(const std::string & attribute_name) const
|
||||
{
|
||||
const auto it = attributes.find(attribute_name);
|
||||
if (it == std::end(attributes))
|
||||
throw Exception{
|
||||
"No such attribute '" + attribute_name + "'",
|
||||
ErrorCodes::BAD_ARGUMENTS
|
||||
};
|
||||
|
||||
return it->second;
|
||||
}
|
||||
|
||||
attributes_t attributes;
|
||||
std::map<std::string, std::size_t> attribute_index_by_name;
|
||||
std::vector<attribute_t> attributes;
|
||||
const attribute_t * hierarchical_attribute = nullptr;
|
||||
|
||||
DictionarySourcePtr source_ptr;
|
||||
|
@ -12,9 +12,37 @@ class IDictionary
|
||||
public:
|
||||
using id_t = std::uint64_t;
|
||||
|
||||
virtual UInt64 getUInt64(const id_t id, const std::string & attribute_name) const = 0;
|
||||
virtual StringRef getString(const id_t id, const std::string & attribute_name) const = 0;
|
||||
virtual bool hasHierarchy() const = 0;
|
||||
|
||||
/// do not call unless you ensure that hasHierarchy() returns true
|
||||
virtual id_t toParent(id_t id) const = 0;
|
||||
|
||||
bool in(id_t child_id, const id_t ancestor_id) const
|
||||
{
|
||||
while (child_id != 0 && child_id != ancestor_id)
|
||||
child_id = toParent(child_id);
|
||||
|
||||
return child_id != 0;
|
||||
}
|
||||
|
||||
/// safe and slow functions, perform map lookup and type checks
|
||||
virtual UInt64 getUInt64(const std::string & attribute_name, id_t id) const = 0;
|
||||
virtual StringRef getString(const std::string & attribute_name, id_t id) const = 0;
|
||||
|
||||
/// unsafe functions for maximum performance, you are on your own ensuring type-safety
|
||||
|
||||
/// returns persistent attribute index for usage with following functions
|
||||
virtual std::size_t getAttributeIndex(const std::string & attribute_name) const = 0;
|
||||
|
||||
/// type-checking functions
|
||||
virtual bool isUInt64(std::size_t attribute_idx) const = 0;
|
||||
virtual bool isString(std::size_t attribute_idx) const = 0;
|
||||
|
||||
/// plain load from target container without any checks
|
||||
virtual UInt64 getUInt64Unsafe(std::size_t attribute_idx, id_t id) const = 0;
|
||||
virtual StringRef getStringUnsafe(std::size_t attribute_idx, id_t id) const = 0;
|
||||
|
||||
/// entirely-loaded dictionaries should be immutable
|
||||
virtual bool isComplete() const = 0;
|
||||
virtual void reload() {}
|
||||
|
||||
|
@ -12,6 +12,7 @@
|
||||
|
||||
#include <DB/Functions/IFunction.h>
|
||||
#include <statdaemons/CategoriesHierarchy.h>
|
||||
#include <statdaemons/ext/range.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -884,9 +885,16 @@ private:
|
||||
const auto out = new ColumnString;
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
const auto attribute_idx = dictionary->getAttributeIndex(attr_name);
|
||||
if (!dictionary->isString(attribute_idx))
|
||||
throw Exception{
|
||||
"Type mismatch: attribute " + attr_name + " has type different from String",
|
||||
ErrorCodes::TYPE_MISMATCH
|
||||
};
|
||||
|
||||
for (const auto & id : id_col->getData())
|
||||
{
|
||||
const auto string_ref = dictionary->getString(id, attr_name);
|
||||
const auto string_ref = dictionary->getStringUnsafe(attribute_idx, id);
|
||||
out->insertData(string_ref.data, string_ref.size);
|
||||
}
|
||||
|
||||
@ -896,7 +904,7 @@ private:
|
||||
{
|
||||
block.getByPosition(result).column = new ColumnConst<String>{
|
||||
id_col->size(),
|
||||
dictionary->getString(id_col->getData(), attr_name).toString()
|
||||
dictionary->getString(attr_name, id_col->getData()).toString()
|
||||
};
|
||||
|
||||
return true;
|
||||
@ -910,7 +918,7 @@ private:
|
||||
|
||||
|
||||
template <typename IntegralType>
|
||||
class FunctionDictGetInteger: public IFunction
|
||||
class FunctionDictGetInteger final : public IFunction
|
||||
{
|
||||
public:
|
||||
static const std::string name;
|
||||
@ -961,7 +969,7 @@ private:
|
||||
!typeid_cast<const DataTypeInt64 *>(id_arg))
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal type " + arguments[2]->getName() + " of argument of function " + getName(),
|
||||
"Illegal type " + id_arg->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
}
|
||||
@ -969,7 +977,7 @@ private:
|
||||
return new typename DataTypeFromFieldType<IntegralType>::Type;
|
||||
}
|
||||
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
const auto dict_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[0]).column.get());
|
||||
if (!dict_name_col)
|
||||
@ -1015,8 +1023,15 @@ private:
|
||||
const auto out = new ColumnVector<IntegralType>;
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
const auto attribute_idx = dictionary->getAttributeIndex(attr_name);
|
||||
if (!dictionary->isUInt64(attribute_idx))
|
||||
throw Exception{
|
||||
"Type mismatch: attribute " + attr_name + " has type different from UInt64",
|
||||
ErrorCodes::TYPE_MISMATCH
|
||||
};
|
||||
|
||||
for (const auto & id : id_col->getData())
|
||||
out->insert(dictionary->getUInt64(id, attr_name));
|
||||
out->insert(dictionary->getUInt64Unsafe(attribute_idx, id));
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -1024,7 +1039,7 @@ private:
|
||||
{
|
||||
block.getByPosition(result).column = new ColumnConst<IntegralType>{
|
||||
id_col->size(),
|
||||
static_cast<IntegralType>(dictionary->getUInt64(id_col->getData(), attr_name))
|
||||
static_cast<IntegralType>(dictionary->getUInt64(attr_name, id_col->getData()))
|
||||
};
|
||||
|
||||
return true;
|
||||
@ -1050,4 +1065,364 @@ using FunctionDictGetInt32 = FunctionDictGetInteger<Int32>;
|
||||
using FunctionDictGetInt64 = FunctionDictGetInteger<Int64>;
|
||||
|
||||
|
||||
|
||||
class FunctionDictGetHierarchy final : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "dictGetHierarchy";
|
||||
|
||||
static IFunction * create(const Context & context)
|
||||
{
|
||||
return new FunctionDictGetHierarchy{context.getDictionaries()};
|
||||
};
|
||||
|
||||
FunctionDictGetHierarchy(const Dictionaries & dictionaries) : dictionaries(dictionaries) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
private:
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
throw Exception{
|
||||
"Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 2.",
|
||||
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
|
||||
};
|
||||
}
|
||||
|
||||
const auto id_arg = arguments[1].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 " + id_arg->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
}
|
||||
|
||||
return new DataTypeArray{new DataTypeUInt64};
|
||||
};
|
||||
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
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());
|
||||
|
||||
if (!dict->hasHierarchy())
|
||||
throw Exception{
|
||||
"Dictionary does not have a hierarchy",
|
||||
ErrorCodes::UNSUPPORTED_METHOD
|
||||
};
|
||||
|
||||
const auto id_col = block.getByPosition(arguments[1]).column.get();
|
||||
if (!execute<UInt8>(block, result, dict, id_col) &&
|
||||
!execute<UInt16>(block, result, dict, id_col) &&
|
||||
!execute<UInt32>(block, result, dict, id_col) &&
|
||||
!execute<UInt64>(block, result, dict, id_col) &&
|
||||
!execute<Int8>(block, result, dict, id_col) &&
|
||||
!execute<Int16>(block, result, dict, id_col) &&
|
||||
!execute<Int32>(block, result, dict, id_col) &&
|
||||
!execute<Int64>(block, result, dict, id_col))
|
||||
{
|
||||
throw Exception{
|
||||
"Second 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 IColumn * const id_col_untyped)
|
||||
{
|
||||
if (const auto id_col = typeid_cast<const ColumnVector<T> *>(id_col_untyped))
|
||||
{
|
||||
const auto backend = new ColumnVector<UInt64>;
|
||||
const auto array = new ColumnArray{backend};
|
||||
block.getByPosition(result).column = array;
|
||||
|
||||
const auto & in = id_col->getData();
|
||||
const auto size = in.size();
|
||||
auto & out = backend->getData();
|
||||
auto & offsets = array->getOffsets();
|
||||
offsets.resize(size);
|
||||
out.reserve(size * 4);
|
||||
|
||||
for (const auto idx : ext::range(0, size))
|
||||
{
|
||||
IDictionary::id_t cur = in[idx];
|
||||
while (cur)
|
||||
{
|
||||
out.push_back(cur);
|
||||
cur = dictionary->toParent(cur);
|
||||
}
|
||||
offsets[idx] = out.size();
|
||||
};
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (const auto id_col = typeid_cast<const ColumnConst<T> *>(id_col_untyped))
|
||||
{
|
||||
Array res;
|
||||
|
||||
IDictionary::id_t cur = id_col->getData();
|
||||
while (cur)
|
||||
{
|
||||
res.push_back(static_cast<typename NearestFieldType<T>::Type>(cur));
|
||||
cur = dictionary->toParent(cur);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = new ColumnConstArray{
|
||||
id_col->size(),
|
||||
res,
|
||||
new DataTypeArray{new DataTypeUInt64}
|
||||
};
|
||||
|
||||
return true;
|
||||
};
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
const Dictionaries & dictionaries;
|
||||
};
|
||||
|
||||
|
||||
class FunctionDictIsIn final : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "dictIsIn";
|
||||
|
||||
static IFunction * create(const Context & context)
|
||||
{
|
||||
return new FunctionDictIsIn{context.getDictionaries()};
|
||||
};
|
||||
|
||||
FunctionDictIsIn(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
|
||||
};
|
||||
}
|
||||
|
||||
const auto child_id_arg = arguments[1].get();
|
||||
if (!typeid_cast<const DataTypeUInt8 *>(child_id_arg) &&
|
||||
!typeid_cast<const DataTypeUInt16 *>(child_id_arg) &&
|
||||
!typeid_cast<const DataTypeUInt32 *>(child_id_arg) &&
|
||||
!typeid_cast<const DataTypeUInt64 *>(child_id_arg) &&
|
||||
!typeid_cast<const DataTypeInt8 *>(child_id_arg) &&
|
||||
!typeid_cast<const DataTypeInt16 *>(child_id_arg) &&
|
||||
!typeid_cast<const DataTypeInt32 *>(child_id_arg) &&
|
||||
!typeid_cast<const DataTypeInt64 *>(child_id_arg))
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal type " + child_id_arg->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
}
|
||||
|
||||
const auto ancestor_id_arg = arguments[2].get();
|
||||
if (!typeid_cast<const DataTypeUInt8 *>(ancestor_id_arg) &&
|
||||
!typeid_cast<const DataTypeUInt16 *>(ancestor_id_arg) &&
|
||||
!typeid_cast<const DataTypeUInt32 *>(ancestor_id_arg) &&
|
||||
!typeid_cast<const DataTypeUInt64 *>(ancestor_id_arg) &&
|
||||
!typeid_cast<const DataTypeInt8 *>(ancestor_id_arg) &&
|
||||
!typeid_cast<const DataTypeInt16 *>(ancestor_id_arg) &&
|
||||
!typeid_cast<const DataTypeInt32 *>(ancestor_id_arg) &&
|
||||
!typeid_cast<const DataTypeInt64 *>(ancestor_id_arg))
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal type " + ancestor_id_arg->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
|
||||
};
|
||||
}
|
||||
|
||||
return new DataTypeUInt8;
|
||||
}
|
||||
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
|
||||
{
|
||||
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());
|
||||
|
||||
if (!dict->hasHierarchy())
|
||||
throw Exception{
|
||||
"Dictionary does not have a hierarchy",
|
||||
ErrorCodes::UNSUPPORTED_METHOD
|
||||
};
|
||||
|
||||
const auto child_id_col = block.getByPosition(arguments[1]).column.get();
|
||||
const auto ancestor_id_col = block.getByPosition(arguments[2]).column.get();
|
||||
if (!execute<UInt8>(block, result, dict, child_id_col, ancestor_id_col) &&
|
||||
!execute<UInt16>(block, result, dict, child_id_col, ancestor_id_col) &&
|
||||
!execute<UInt32>(block, result, dict, child_id_col, ancestor_id_col) &&
|
||||
!execute<UInt64>(block, result, dict, child_id_col, ancestor_id_col) &&
|
||||
!execute<Int8>(block, result, dict, child_id_col, ancestor_id_col) &&
|
||||
!execute<Int16>(block, result, dict, child_id_col, ancestor_id_col) &&
|
||||
!execute<Int32>(block, result, dict, child_id_col, ancestor_id_col) &&
|
||||
!execute<Int64>(block, result, dict, child_id_col, ancestor_id_col))
|
||||
{
|
||||
throw Exception{
|
||||
"Illegal column " + child_id_col->getName()
|
||||
+ " of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool execute(Block & block, const size_t result, const MultiVersion<IDictionary>::Version & dictionary,
|
||||
const IColumn * const child_id_col_untyped, const IColumn * const ancestor_id_col_untyped)
|
||||
{
|
||||
if (execute<T, ColumnVector<T>>(block, result, dictionary, child_id_col_untyped, ancestor_id_col_untyped) ||
|
||||
execute<T, ColumnConst<T>>(block, result, dictionary, child_id_col_untyped, ancestor_id_col_untyped))
|
||||
return true;
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T, typename ColumnType>
|
||||
bool execute(Block & block, const size_t result, const MultiVersion<IDictionary>::Version & dictionary,
|
||||
const IColumn * const child_id_col_untyped, const IColumn * const ancestor_id_col_untyped)
|
||||
{
|
||||
if (const auto child_id_col = typeid_cast<const ColumnType *>(child_id_col_untyped))
|
||||
{
|
||||
if (execute<T, UInt8>(block, result, dictionary, child_id_col, ancestor_id_col_untyped) ||
|
||||
execute<T, UInt16>(block, result, dictionary, child_id_col, ancestor_id_col_untyped) ||
|
||||
execute<T, UInt32>(block, result, dictionary, child_id_col, ancestor_id_col_untyped) ||
|
||||
execute<T, UInt64>(block, result, dictionary, child_id_col, ancestor_id_col_untyped) ||
|
||||
execute<T, Int8>(block, result, dictionary, child_id_col, ancestor_id_col_untyped) ||
|
||||
execute<T, Int16>(block, result, dictionary, child_id_col, ancestor_id_col_untyped) ||
|
||||
execute<T, Int32>(block, result, dictionary, child_id_col, ancestor_id_col_untyped) ||
|
||||
execute<T, Int64>(block, result, dictionary, child_id_col, ancestor_id_col_untyped))
|
||||
return true;
|
||||
else
|
||||
throw Exception{
|
||||
"Illegal column " + ancestor_id_col_untyped->getName()
|
||||
+ " of third argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN
|
||||
};
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T, typename U>
|
||||
bool execute(Block & block, const size_t result, const MultiVersion<IDictionary>::Version & dictionary,
|
||||
const ColumnVector<T> * const child_id_col, const IColumn * const ancestor_id_col_untyped)
|
||||
{
|
||||
if (const auto ancestor_id_col = typeid_cast<const ColumnVector<T> *>(ancestor_id_col_untyped))
|
||||
{
|
||||
const auto out = new ColumnVector<UInt8>;
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
const auto & child_ids = child_id_col->getData();
|
||||
const auto & ancestor_ids = ancestor_id_col->getData();
|
||||
auto & data = out->getData();
|
||||
const auto size = child_id_col->size();
|
||||
data.resize(size);
|
||||
|
||||
for (const auto idx : ext::range(0, size))
|
||||
data[idx] = dictionary->in(child_ids[idx], ancestor_ids[idx]);
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (const auto ancestor_id_col = typeid_cast<const ColumnConst<T> *>(ancestor_id_col_untyped))
|
||||
{
|
||||
const auto out = new ColumnVector<UInt8>;
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
const auto & child_ids = child_id_col->getData();
|
||||
const auto ancestor_id = ancestor_id_col->getData();
|
||||
auto & data = out->getData();
|
||||
const auto size = child_id_col->size();
|
||||
data.resize(size);
|
||||
|
||||
for (const auto idx : ext::range(0, size))
|
||||
data[idx] = dictionary->in(child_ids[idx], ancestor_id);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T, typename U>
|
||||
bool execute(Block & block, const size_t result, const MultiVersion<IDictionary>::Version & dictionary,
|
||||
const ColumnConst<T> * const child_id_col, const IColumn * const ancestor_id_col_untyped)
|
||||
{
|
||||
if (const auto ancestor_id_col = typeid_cast<const ColumnVector<T> *>(ancestor_id_col_untyped))
|
||||
{
|
||||
const auto out = new ColumnVector<UInt8>;
|
||||
block.getByPosition(result).column = out;
|
||||
|
||||
const auto child_id = child_id_col->getData();
|
||||
const auto & ancestor_ids = ancestor_id_col->getData();
|
||||
auto & data = out->getData();
|
||||
const auto size = child_id_col->size();
|
||||
data.resize(size);
|
||||
|
||||
for (const auto idx : ext::range(0, size))
|
||||
data[idx] = dictionary->in(child_id, ancestor_ids[idx]);
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (const auto ancestor_id_col = typeid_cast<const ColumnConst<T> *>(ancestor_id_col_untyped))
|
||||
{
|
||||
block.getByPosition(result).column = new ColumnConst<UInt8>{
|
||||
child_id_col->size(),
|
||||
dictionary->in(child_id_col->getData(), ancestor_id_col->getData())
|
||||
};
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
const Dictionaries & dictionaries;
|
||||
};
|
||||
|
||||
|
||||
}
|
||||
|
@ -34,6 +34,8 @@ void registerFunctionsDictionaries(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionDictGetInt32>();
|
||||
factory.registerFunction<FunctionDictGetInt64>();
|
||||
factory.registerFunction<FunctionDictGetString>();
|
||||
factory.registerFunction<FunctionDictGetHierarchy>();
|
||||
factory.registerFunction<FunctionDictIsIn>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -496,7 +496,7 @@ const Dictionaries & Context::getDictionaries() const
|
||||
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
|
||||
|
||||
if (!shared->dictionaries)
|
||||
shared->dictionaries = new Dictionaries{*this};
|
||||
shared->dictionaries = new Dictionaries{*this->global_context};
|
||||
|
||||
return *shared->dictionaries;
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user