Dictionaries support array type

This commit is contained in:
Maksim Kita 2021-06-09 13:43:40 +03:00
parent 1c7c999b25
commit 45b8dc772b
15 changed files with 178 additions and 52 deletions

View File

@ -561,7 +561,6 @@ private:
attributes.emplace_back();
auto & last_attribute = attributes.back();
last_attribute.type = attribute_type;
last_attribute.is_complex_type = dictionary_attribute.is_nullable || dictionary_attribute.is_array;
if (dictionary_attribute.is_nullable)
last_attribute.attribute_container = std::vector<Field>();
@ -609,6 +608,8 @@ private:
PaddedPODArray<Float64>,
PaddedPODArray<UUID>,
PaddedPODArray<StringRef>,
/// TODO: FIX
PaddedPODArray<Array>,
std::vector<Field>> attribute_container;
};

View File

@ -6,8 +6,10 @@
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnArray.h>
#include <DataStreams/IBlockInputStream.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeArray.h>
#include <Core/Block.h>
#include <Dictionaries/IDictionary.h>
#include <Dictionaries/DictionaryStructure.h>
@ -231,14 +233,27 @@ class DictionaryAttributeColumnProvider
{
public:
using ColumnType =
std::conditional_t<std::is_same_v<DictionaryAttributeType, String>, ColumnString,
std::conditional_t<IsDecimalNumber<DictionaryAttributeType>, ColumnDecimal<DictionaryAttributeType>,
ColumnVector<DictionaryAttributeType>>>;
std::conditional_t<std::is_same_v<DictionaryAttributeType, Array>, ColumnArray,
std::conditional_t<std::is_same_v<DictionaryAttributeType, String>, ColumnString,
std::conditional_t<IsDecimalNumber<DictionaryAttributeType>, ColumnDecimal<DictionaryAttributeType>,
ColumnVector<DictionaryAttributeType>>>>;
using ColumnPtr = typename ColumnType::MutablePtr;
static ColumnPtr getColumn(const DictionaryAttribute & dictionary_attribute, size_t size)
{
if constexpr (std::is_same_v<DictionaryAttributeType, Array>)
{
if (const auto * array_type = typeid_cast<const DataTypeArray *>(dictionary_attribute.nested_type.get()))
{
auto nested_column = array_type->getNestedType()->createColumn();
return ColumnArray::create(std::move(nested_column));
}
else
{
throw Exception(ErrorCodes::TYPE_MISMATCH, "Unsupported attribute type.");
}
}
if constexpr (std::is_same_v<DictionaryAttributeType, String>)
{
return ColumnType::create();
@ -280,18 +295,18 @@ public:
: default_value(std::move(attribute_default_value))
{
if (default_values_column_ == nullptr)
use_default_value_from_column = false;
use_attribute_default_value = true;
else
{
if (const auto * const default_col = checkAndGetColumn<DefaultColumnType>(*default_values_column_))
{
default_values_column = default_col;
use_default_value_from_column = true;
use_attribute_default_value = false;
}
else if (const auto * const default_col_const = checkAndGetColumnConst<DefaultColumnType>(default_values_column_.get()))
{
default_value = default_col_const->template getValue<DictionaryAttributeType>();
use_default_value_from_column = false;
use_attribute_default_value = true;
}
else
throw Exception(ErrorCodes::TYPE_MISMATCH, "Type of default column is not the same as dictionary attribute type.");
@ -300,12 +315,17 @@ public:
DefaultValueType operator[](size_t row)
{
if (!use_default_value_from_column)
if (use_attribute_default_value)
return static_cast<DefaultValueType>(default_value);
assert(default_values_column != nullptr);
if constexpr (std::is_same_v<DefaultColumnType, ColumnString>)
if constexpr (std::is_same_v<DefaultColumnType, ColumnArray>)
{
Field field = (*default_values_column)[row];
return field.get<Array>();
}
else if constexpr (std::is_same_v<DefaultColumnType, ColumnString>)
return default_values_column->getDataAt(row);
else
return default_values_column->getData()[row];
@ -313,7 +333,7 @@ public:
private:
DictionaryAttributeType default_value;
const DefaultColumnType * default_values_column = nullptr;
bool use_default_value_from_column = false;
bool use_attribute_default_value = false;
};
template <DictionaryKeyType key_type>

View File

@ -25,9 +25,10 @@ namespace
Block block;
if (dict_struct.id)
{
block.insert(ColumnWithTypeAndName{ColumnUInt64::create(1, 0), std::make_shared<DataTypeUInt64>(), dict_struct.id->name});
if (dict_struct.key)
}
else if (dict_struct.key)
{
for (const auto & attribute : *dict_struct.key)
{

View File

@ -79,9 +79,7 @@ AttributeUnderlyingType getAttributeUnderlyingType(const DataTypePtr & type)
case TypeIndex::String: return AttributeUnderlyingType::String;
// Temporary hack to allow arrays in keys, since they are never retrieved for polygon dictionaries.
// TODO: This should be fixed by fully supporting arrays in dictionaries.
case TypeIndex::Array: return AttributeUnderlyingType::String;
case TypeIndex::Array: return AttributeUnderlyingType::Array;
default: break;
}
@ -125,7 +123,7 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
id.emplace(config, structure_prefix + ".id");
else if (has_key)
{
key.emplace(getAttributes(config, structure_prefix + ".key", true));
key.emplace(getAttributes(config, structure_prefix + ".key", /*complex_key_attributes =*/ true));
if (key->empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Empty 'key' supplied");
}
@ -173,7 +171,7 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
has_expressions = true;
}
attributes = getAttributes(config, structure_prefix, false);
attributes = getAttributes(config, structure_prefix, /*complex_key_attributes =*/ false);
for (size_t i = 0; i < attributes.size(); ++i)
{
@ -375,17 +373,22 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
const auto type_string = config.getString(prefix + "type");
const auto initial_type = DataTypeFactory::instance().get(type_string);
auto type = initial_type;
bool is_array = false;
auto nested_type = initial_type;
bool is_nullable = false;
if (type->isNullable())
// while (const auto * array_type = typeid_cast<const DataTypeArray *>(nested_type.get()))
// {
// is_array = true;
// nested_type = array_type->getNestedType();
// }
if (nested_type->isNullable())
{
is_nullable = true;
type = removeNullable(type);
nested_type = removeNullable(nested_type);
}
const auto underlying_type = getAttributeUnderlyingType(type);
const auto underlying_type = getAttributeUnderlyingType(nested_type);
const auto expression = config.getString(prefix + "expression", "");
if (!expression.empty())
@ -399,13 +402,13 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
{
if (null_value_string.empty())
{
null_value = type->getDefault();
null_value = initial_type->getDefault();
}
else
{
ReadBufferFromString null_value_buffer{null_value_string};
auto column_with_null_value = type->createColumn();
type->getDefaultSerialization()->deserializeTextEscaped(*column_with_null_value, null_value_buffer, format_settings);
auto column_with_null_value = nested_type->createColumn();
initial_type->getDefaultSerialization()->deserializeTextEscaped(*column_with_null_value, null_value_buffer, format_settings);
null_value = (*column_with_null_value)[0];
}
}
@ -418,6 +421,8 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
}
}
std::cerr << "DictionaryStructure::getAttributes null value " << null_value.dump() << std::endl;
const auto hierarchical = config.getBool(prefix + "hierarchical", false);
const auto injective = config.getBool(prefix + "injective", false);
const auto is_object_id = config.getBool(prefix + "is_object_id", false);
@ -437,14 +442,13 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
underlying_type,
initial_type,
initial_type->getDefaultSerialization(),
type,
nested_type,
expression,
null_value,
hierarchical,
injective,
is_object_id,
is_nullable,
is_array});
is_nullable});
}
return res_attributes;

View File

@ -39,6 +39,7 @@
M(Decimal256) \
M(UUID) \
M(String) \
M(Array) \
namespace DB
@ -82,7 +83,6 @@ struct DictionaryAttribute final
const bool injective;
const bool is_object_id;
const bool is_nullable;
const bool is_array;
};
template <typename Type>
@ -92,7 +92,7 @@ struct DictionaryAttributeType
};
template <typename F>
void callOnDictionaryAttributeType(AttributeUnderlyingType type, F&& func)
void callOnDictionaryAttributeType(AttributeUnderlyingType type, F && func)
{
switch (type)
{

View File

@ -76,11 +76,21 @@ ColumnPtr FlatDictionary::getColumn(
auto column = ColumnProvider::getColumn(dictionary_attribute, size);
if constexpr (std::is_same_v<ValueType, StringRef>)
if constexpr (std::is_same_v<ValueType, Array>)
{
auto * out = column.get();
getItemsImpl<ValueType, ValueType>(
getItemsImpl<ValueType>(
attribute,
ids,
[&](const size_t, const Array & value) { out->insert(value); },
default_value_extractor);
}
else if constexpr (std::is_same_v<ValueType, StringRef>)
{
auto * out = column.get();
getItemsImpl<ValueType>(
attribute,
ids,
[&](const size_t, const StringRef value) { out->insertData(value.data, value.size); },
@ -90,7 +100,7 @@ ColumnPtr FlatDictionary::getColumn(
{
auto & out = column->getData();
getItemsImpl<ValueType, ValueType>(
getItemsImpl<ValueType>(
attribute,
ids,
[&](const size_t row, const auto value) { out[row] = value; },
@ -275,6 +285,7 @@ void FlatDictionary::blockToAttributes(const Block & block)
if (already_processed_keys.find(key) != nullptr)
continue;
already_processed_keys.insert(key);
setAttributeValue(attribute, key, attribute_column[i]);
@ -352,7 +363,18 @@ void FlatDictionary::calculateBytesAllocated()
using ValueType = DictionaryValueType<AttributeType>;
const auto & container = std::get<ContainerType<ValueType>>(attribute.container);
bytes_allocated += sizeof(PaddedPODArray<ValueType>) + container.allocated_bytes();
bytes_allocated += sizeof(ContainerType<ValueType>);
if constexpr (std::is_same_v<ValueType, Array>)
{
/// It is not accurate calculations
bytes_allocated += sizeof(Array) * container.size();
}
else
{
bytes_allocated += container.allocated_bytes();
}
bucket_count = container.capacity();
if constexpr (std::is_same_v<ValueType, StringRef>)
@ -396,7 +418,7 @@ FlatDictionary::Attribute FlatDictionary::createAttribute(const DictionaryAttrib
return attribute;
}
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultValueExtractor>
template <typename AttributeType, typename ValueSetter, typename DefaultValueExtractor>
void FlatDictionary::getItemsImpl(
const Attribute & attribute,
const PaddedPODArray<UInt64> & keys,
@ -414,7 +436,7 @@ void FlatDictionary::getItemsImpl(
if (key < loaded_keys.size() && loaded_keys[key])
{
set_value(row, static_cast<OutputType>(container[key]));
set_value(row, container[key]);
++keys_found;
}
else
@ -440,7 +462,11 @@ void FlatDictionary::resize(Attribute & attribute, UInt64 key)
{
const size_t elements_count = key + 1; //id=0 -> elements_count=1
loaded_keys.resize(elements_count, false);
container.resize_fill(elements_count, std::get<T>(attribute.null_values));
if constexpr (std::is_same_v<T, Array>)
container.resize(elements_count, std::get<T>(attribute.null_values));
else
container.resize_fill(elements_count, std::get<T>(attribute.null_values));
}
}
@ -461,13 +487,13 @@ void FlatDictionary::setAttributeValueImpl<String>(Attribute & attribute, UInt64
void FlatDictionary::setAttributeValue(Attribute & attribute, const UInt64 key, const Field & value)
{
auto type_call = [&](const auto &dictionary_attribute_type)
auto type_call = [&](const auto & dictionary_attribute_type)
{
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
using AttributeType = typename Type::AttributeType;
using ResizeType = std::conditional_t<std::is_same_v<AttributeType, String>, StringRef, AttributeType>;
using ValueType = DictionaryValueType<AttributeType>;
resize<ResizeType>(attribute, key);
resize<ValueType>(attribute, key);
if (attribute.nullable_set)
{

View File

@ -106,7 +106,7 @@ public:
private:
template <typename Value>
using ContainerType = PaddedPODArray<Value>;
using ContainerType = std::conditional_t<std::is_same_v<Value, Array>, std::vector<Value>, PaddedPODArray<Value>>;
using NullableSet = HashSet<UInt64, DefaultHash<UInt64>>;
@ -135,8 +135,10 @@ private:
Float32,
Float64,
UUID,
StringRef>
StringRef,
Array>
null_values;
std::variant<
ContainerType<UInt8>,
ContainerType<UInt16>,
@ -157,7 +159,8 @@ private:
ContainerType<Float32>,
ContainerType<Float64>,
ContainerType<UUID>,
ContainerType<StringRef>>
ContainerType<StringRef>,
ContainerType<Array>>
container;
std::unique_ptr<Arena> string_arena;
@ -172,7 +175,7 @@ private:
Attribute createAttribute(const DictionaryAttribute& attribute, const Field & null_value);
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultValueExtractor>
template <typename AttributeType, typename ValueSetter, typename DefaultValueExtractor>
void getItemsImpl(
const Attribute & attribute,
const PaddedPODArray<UInt64> & keys,

View File

@ -173,7 +173,8 @@ private:
Float32,
Float64,
UUID,
StringRef>
StringRef,
Array>
null_values;
std::variant<
@ -196,7 +197,8 @@ private:
CollectionType<Float32>,
CollectionType<Float64>,
CollectionType<UUID>,
CollectionType<StringRef>>
CollectionType<StringRef>,
CollectionType<Array>>
container;
std::unique_ptr<Arena> string_arena;

View File

@ -27,6 +27,7 @@ namespace ErrorCodes
extern const int DICTIONARY_IS_EMPTY;
extern const int LOGICAL_ERROR;
extern const int TYPE_MISMATCH;
extern const int UNSUPPORTED_METHOD;
}
namespace
@ -315,8 +316,15 @@ void IPAddressDictionary::createAttributes()
auto create_attributes_from_dictionary_attributes = [this](const std::vector<DictionaryAttribute> & dict_attrs)
{
attributes.reserve(attributes.size() + dict_attrs.size());
for (const auto & attribute : dict_attrs)
{
if (attribute.is_nullable)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"{}: array or nullable attributes not supported for dictionary of type {}",
full_name,
getTypeName());
attribute_index_by_name.emplace(attribute.name, attributes.size());
attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value));

View File

@ -116,7 +116,8 @@ private:
Float32,
Float64,
UUID,
String>
String,
Array>
null_values;
std::variant<
ContainerType<UInt8>,
@ -138,7 +139,8 @@ private:
ContainerType<Float32>,
ContainerType<Float64>,
ContainerType<UUID>,
ContainerType<StringRef>>
ContainerType<StringRef>,
ContainerType<Array>>
maps;
std::unique_ptr<Arena> string_arena;
};

View File

@ -52,7 +52,6 @@ ColumnPtr IPolygonDictionary::getColumn(
const auto requested_key_points = extractPoints(key_columns);
const auto & attribute = dict_struct.getAttribute(attribute_name, result_type);
bool complex_attribute = attribute.is_nullable || attribute.is_array;
DefaultValueProvider default_value_provider(attribute.null_value, default_values_column);
size_t attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second;
@ -66,7 +65,7 @@ ColumnPtr IPolygonDictionary::getColumn(
size_t keys_found = 0;
if (unlikely(complex_attribute))
if (unlikely(attribute.is_nullable))
{
for (size_t requested_key_index = 0; requested_key_index < requested_key_points.size(); ++requested_key_index)
{

View File

@ -49,6 +49,7 @@ public:
Array,
Tuple,
};
IPolygonDictionary(
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,

View File

@ -127,7 +127,8 @@ private:
Float32,
Float64,
UUID,
StringRef>
StringRef,
Array>
null_values;
std::variant<
Ptr<UInt8>,
@ -149,7 +150,8 @@ private:
Ptr<Float32>,
Ptr<Float64>,
Ptr<UUID>,
Ptr<StringRef>>
Ptr<StringRef>,
Ptr<Array>>
maps;
std::unique_ptr<Arena> string_arena;
};

View File

@ -0,0 +1,2 @@
Flat dictionary
[0,1,2]

View File

@ -0,0 +1,55 @@
DROP TABLE IF EXISTS dictionary_array_source_table;
CREATE TABLE dictionary_array_source_table
(
id UInt64,
array_value Array(Int) DEFAULT [0, 1, 3]
) ENGINE=TinyLog;
INSERT INTO dictionary_array_source_table VALUES (0, [0, 1, 2]);
DROP DICTIONARY IF EXISTS flat_dictionary;
CREATE DICTIONARY flat_dictionary
(
id UInt64,
array_value Array(Int)
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'dictionary_array_source_table'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(FLAT());
SELECT 'Flat dictionary';
SELECT dictGet('flat_dictionary', 'array_value', toUInt64(0));
DROP DICTIONARY flat_dictionary;
-- DROP DICTIONARY IF EXISTS hashed_dictionary;
-- CREATE DICTIONARY hashed_dictionary
-- (
-- id UInt64,
-- array_value Array(Int)
-- )
-- PRIMARY KEY id
-- SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'dictionary_array_source_table'))
-- LIFETIME(MIN 1 MAX 1000)
-- LAYOUT(HASHED());
-- SELECT 'Hashed dictionary';
-- SELECT dictGet('hashed_dictionary', 'array_value', toUInt64(1));
-- DROP DICTIONARY hashed_dictionary;
-- DROP DICTIONARY IF EXISTS cache_dictionary;
-- CREATE DICTIONARY cache_dictionary
-- (
-- id UInt64,
-- array_value Array(Int)
-- )
-- PRIMARY KEY id
-- SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'dictionary_array_source_table'))
-- LIFETIME(MIN 1 MAX 1000)
-- LAYOUT(CACHE(SIZE_IN_CELLS 10));
-- SELECT 'Cache dictionary';
-- SELECT dictGet('cache_dictionary', 'array_value', toUInt64(1));
-- DROP DICTIONARY cache_dictionary;
DROP TABLE dictionary_array_source_table;