Added functions dictGetChildren, dictGetDescendants

This commit is contained in:
Maksim Kita 2021-03-24 22:55:06 +03:00
parent 9f2f0d1095
commit bbd180caf3
5 changed files with 235 additions and 7 deletions

View File

@ -221,6 +221,49 @@ ColumnUInt8::Ptr FlatDictionary::isInHierarchy(
return result;
}
ColumnPtr FlatDictionary::getDescendands(
ColumnPtr key_column,
const DataTypePtr &,
size_t level) const
{
PaddedPODArray<UInt64> keys_backup;
const auto & keys = getColumnVectorData(this, key_column, keys_backup);
size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index;
const auto & hierarchical_attribute = attributes[hierarchical_attribute_index];
const UInt64 null_value = std::get<UInt64>(hierarchical_attribute.null_values);
const ContainerType<UInt64> & parent_keys = std::get<ContainerType<UInt64>>(hierarchical_attribute.arrays);
HashMap<UInt64, UInt64> parent_to_child;
for (size_t i = 0; i < parent_keys.size(); ++i)
{
auto parent_key = parent_keys[i];
parent_to_child[parent_key] = static_cast<UInt64>(i);
}
auto is_key_valid_func = [&](auto & key)
{
return parent_to_child.find(key) != nullptr;
};
auto get_child_key_func = [&](auto & key)
{
std::optional<UInt64> result;
auto it = parent_to_child.find(key);
if (it)
result = it->getMapped();
return result;
};
auto result = getDescendandsArray(keys, null_value, level, is_key_valid_func, get_child_key_func);
return result;
}
void FlatDictionary::createAttributes()
{
const auto size = dict_struct.attributes.size();

View File

@ -82,6 +82,11 @@ public:
ColumnPtr in_key_column,
const DataTypePtr & key_type) const override;
ColumnPtr getDescendands(
ColumnPtr key_column,
const DataTypePtr & key_type,
size_t level) const override;
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
private:

View File

@ -32,17 +32,17 @@ struct IsKeyValidFuncInterface
};
template <typename T>
struct GetParentKeyFuncInterface
struct GetKeyFuncInterface
{
std::optional<T> operator()(T key [[maybe_unused]]) { return {}; }
};
template <typename KeyType, typename IsKeyValidFunc, typename GetParentKeyFunc>
template <typename KeyType, typename IsKeyValidFunc, typename GetKeyFunc>
ElementsAndOffsets<KeyType> getKeysHierarchy(
const PaddedPODArray<KeyType> & hierarchy_keys,
const KeyType & hierarchy_null_value,
IsKeyValidFunc && is_key_valid_func,
GetParentKeyFunc && get_parent_func)
GetKeyFunc && get_key_func)
{
size_t hierarchy_keys_size = hierarchy_keys.size();
@ -114,7 +114,7 @@ ElementsAndOffsets<KeyType> getKeysHierarchy(
elements.emplace_back(hierarchy_key);
++current_hierarchy_depth;
std::optional<KeyType> parent_key = std::forward<GetParentKeyFunc>(get_parent_func)(hierarchy_key);
std::optional<KeyType> parent_key = std::forward<GetKeyFunc>(get_key_func)(hierarchy_key);
if (!parent_key.has_value())
break;
@ -162,7 +162,11 @@ PaddedPODArray<UInt8> isInKeysHierarchy(
PaddedPODArray<UInt8> result;
result.resize_fill(hierarchy_keys.size());
ElementsAndOffsets<KeyType> hierarchy = getKeysHierarchy(hierarchy_keys, hierarchy_null_value, std::forward<IsKeyValidFunc>(is_key_valid_func), std::forward<GetParentKeyFunc>(get_parent_func));
ElementsAndOffsets<KeyType> hierarchy = getKeysHierarchy(
hierarchy_keys,
hierarchy_null_value,
std::forward<IsKeyValidFunc>(is_key_valid_func),
std::forward<GetParentKeyFunc>(get_parent_func));
auto & offsets = hierarchy.offsets;
auto & elements = hierarchy.elements;
@ -186,6 +190,53 @@ PaddedPODArray<UInt8> isInKeysHierarchy(
return result;
}
template <typename KeyType, typename IsKeyValidFunc, typename GetDescendantKeyFunc>
ColumnPtr getDescendandsArray(
const PaddedPODArray<KeyType> & hierarchy_keys,
const KeyType & hierarchy_null_value,
size_t level,
IsKeyValidFunc && is_key_valid_func,
GetDescendantKeyFunc && get_descendant_func)
{
auto elements_and_offsets = getKeysHierarchy(
hierarchy_keys,
hierarchy_null_value,
std::forward<IsKeyValidFunc>(is_key_valid_func),
std::forward<GetDescendantKeyFunc>(get_descendant_func));
auto & elements = elements_and_offsets.elements;
auto & offsets = elements_and_offsets.offsets;
PaddedPODArray<KeyType> descendants;
descendants.reserve(elements.size());
PaddedPODArray<size_t> descendants_offsets;
descendants_offsets.reserve(elements.size());
for (size_t i = 0; i < offsets.size(); ++i)
{
size_t offset_start_index = i > 0 ? offsets[i - 1] : 0;
size_t offset_end_index = offsets[i];
size_t size = offset_end_index - offset_start_index;
if (level == 0)
descendants.insert(elements.begin() + offset_start_index + 1, elements.begin() + offset_end_index);
else if (level < size)
descendants.emplace_back(elements[offset_start_index + level]);
descendants_offsets.emplace_back(descendants.size());
}
auto elements_column = ColumnVector<KeyType>::create();
elements_column->getData() = std::move(elements_and_offsets.elements);
auto offsets_column = ColumnVector<IColumn::Offset>::create();
offsets_column->getData() = std::move(offsets);
auto column_array = ColumnArray::create(std::move(elements_column), std::move(offsets_column));
return column_array;
}
ColumnPtr getHierarchyDefaultImplementation(const IDictionary * dictionary, ColumnPtr key_column, const DataTypePtr & key_type);
ColumnUInt8::Ptr isInHierarchyDefaultImplementation(

View File

@ -165,7 +165,7 @@ struct IDictionary : public IExternalLoadable
const DataTypePtr & key_type [[maybe_unused]]) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Hierarchy is not supported for {} dictionary.",
"Method getHierarchy is not supported for {} dictionary.",
getDictionaryID().getNameForLogs());
}
@ -175,7 +175,17 @@ struct IDictionary : public IExternalLoadable
const DataTypePtr & key_type [[maybe_unused]]) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Hierarchy is not supported for {} dictionary.",
"Method isInHierarchy is not supported for {} dictionary.",
getDictionaryID().getNameForLogs());
}
virtual ColumnPtr getDescendands(
ColumnPtr key_column [[maybe_unused]],
const DataTypePtr & key_type [[maybe_unused]],
size_t level [[maybe_unused]]) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Method getDescendands is not supported for {} dictionary.",
getDictionaryID().getNameForLogs());
}

View File

@ -806,4 +806,123 @@ private:
mutable FunctionDictHelper helper;
};
class FunctionDictGetChildren final : public IFunction
{
public:
static constexpr auto name = "dictGetChildren";
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionDictGetChildren>(context);
}
explicit FunctionDictGetChildren(const Context & context_)
: helper(context_) {}
String getName() const override { return name; }
private:
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!WhichDataType(arguments[1]).isUInt64())
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
bool isDeterministic() const override { return false; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
if (input_rows_count == 0)
return result_type->createColumn();
auto dict = helper.getDictionary(arguments[0]);
if (!dict->hasHierarchy())
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary ({}) does not support hierarchy", dict->getFullName());
ColumnPtr res = dict->getDescendands(arguments[1].column, std::make_shared<DataTypeUInt64>(), 0);
return res;
}
mutable FunctionDictHelper helper;
};
class FunctionDictGetDescendands final : public IFunction
{
public:
static constexpr auto name = "dictGetDescendands";
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionDictGetDescendands>(context);
}
explicit FunctionDictGetDescendands(const Context & context_)
: helper(context_) {}
String getName() const override { return name; }
private:
size_t getNumberOfArguments() const override { return 3; }
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 2}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!WhichDataType(arguments[1]).isUInt64())
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!WhichDataType(arguments[2]).isUInt64())
throw Exception{"Illegal type " + arguments[1]->getName() + " of third argument of function " + getName()
+ ", must be const UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataTypeUInt8>();
}
bool isDeterministic() const override { return false; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
if (input_rows_count == 0)
return result_type->createColumn();
const auto * level_const_column = checkAndGetColumnConst<ColumnVector<UInt64>>(arguments[2].column.get());
if (!level_const_column)
throw Exception{"Illegal type " + arguments[1].type->getName() + " of third argument of function " + getName()
+ ", must be const UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
auto dict = helper.getDictionary(arguments[0]);
if (!dict->hasHierarchy())
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Dictionary ({}) does not support hierarchy", dict->getFullName());
size_t level = static_cast<size_t>(level_const_column->getValue<UInt64>());
ColumnPtr res = dict->getDescendands(arguments[1].column, std::make_shared<DataTypeUInt64>(), level);
return res;
}
mutable FunctionDictHelper helper;
};
}