Fixed dicitionaries todo

This commit is contained in:
Maksim Kita 2021-01-21 17:42:50 +03:00
parent 498df53ebf
commit 7cb7d4dbce
18 changed files with 117 additions and 132 deletions

View File

@ -14,6 +14,7 @@
#include <ext/range.h>
#include <ext/size.h>
#include <Common/setThreadName.h>
#include <DataTypes/DataTypesDecimal.h>
#include "CacheDictionary.inc.h"
#include "DictionaryBlockInputStream.h"
#include "DictionaryFactory.h"
@ -251,7 +252,7 @@ void CacheDictionary::isInConstantVector(const Key child_id, const PaddedPODArra
ColumnPtr CacheDictionary::getColumn(
const std::string & attribute_name,
const DataTypePtr &,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes &,
const ColumnPtr default_untyped) const
@ -262,9 +263,7 @@ ColumnPtr CacheDictionary::getColumn(
const auto & ids = getColumnDataAsPaddedPODArray(this, key_columns.front(), backup_storage);
auto & attribute = getAttribute(attribute_name);
/// TODO: Check that attribute type is same as result type
/// TODO: Check if const will work as expected
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
auto type_call = [&](const auto &dictionary_attribute_type)
{
@ -311,8 +310,8 @@ ColumnPtr CacheDictionary::getColumn(
if constexpr (IsDecimalNumber<AttributeType>)
{
// auto scale = getDecimalScale(*attribute.type);
column = ColumnDecimal<AttributeType>::create(identifiers_size, 0);
auto scale = getDecimalScale(*dictionary_attribute.nested_type);
column = ColumnDecimal<AttributeType>::create(size, scale);
}
else if constexpr (IsNumber<AttributeType>)
column = ColumnVector<AttributeType>::create(identifiers_size);

View File

@ -11,7 +11,7 @@
#include "DictionaryBlockInputStream.h"
#include "DictionaryFactory.h"
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypesDecimal.h>
namespace ProfileEvents
{
@ -73,7 +73,7 @@ ComplexKeyCacheDictionary::ComplexKeyCacheDictionary(
ColumnPtr ComplexKeyCacheDictionary::getColumn(
const std::string & attribute_name,
const DataTypePtr &,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types,
const ColumnPtr default_untyped) const
@ -83,9 +83,7 @@ ColumnPtr ComplexKeyCacheDictionary::getColumn(
ColumnPtr result;
auto & attribute = getAttribute(attribute_name);
/// TODO: Check that attribute type is same as result type
/// TODO: Check if const will work as expected
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
auto keys_size = key_columns.front()->size();
@ -133,8 +131,8 @@ ColumnPtr ComplexKeyCacheDictionary::getColumn(
if constexpr (IsDecimalNumber<AttributeType>)
{
// auto scale = getDecimalScale(*attribute.type);
column = ColumnDecimal<AttributeType>::create(keys_size, 0);
auto scale = getDecimalScale(*dictionary_attribute.nested_type);
column = ColumnDecimal<AttributeType>::create(size, scale);
}
else if constexpr (IsNumber<AttributeType>)
column = ColumnVector<AttributeType>::create(keys_size);

View File

@ -5,6 +5,7 @@
#include <Core/Defines.h>
#include <Columns/ColumnNullable.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypesDecimal.h>
namespace DB
{
@ -35,7 +36,7 @@ ComplexKeyDirectDictionary::ComplexKeyDirectDictionary(
ColumnPtr ComplexKeyDirectDictionary::getColumn(
const std::string & attribute_name,
const DataTypePtr &,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types,
const ColumnPtr default_untyped) const
@ -45,9 +46,7 @@ ColumnPtr ComplexKeyDirectDictionary::getColumn(
ColumnPtr result;
const auto & attribute = getAttribute(attribute_name);
/// TODO: Check that attribute type is same as result type
/// TODO: Check if const will work as expected
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
auto size = key_columns.front()->size();
@ -146,8 +145,8 @@ ColumnPtr ComplexKeyDirectDictionary::getColumn(
if constexpr (IsDecimalNumber<AttributeType>)
{
// auto scale = getDecimalScale(*attribute.type);
column = ColumnDecimal<AttributeType>::create(size, 0);
auto scale = getDecimalScale(*dictionary_attribute.nested_type);
column = ColumnDecimal<AttributeType>::create(size, scale);
}
else if constexpr (IsNumber<AttributeType>)
column = ColumnVector<AttributeType>::create(size);

View File

@ -4,6 +4,7 @@
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnNullable.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypesDecimal.h>
#include "DictionaryBlockInputStream.h"
#include "DictionaryFactory.h"
@ -37,7 +38,7 @@ ComplexKeyHashedDictionary::ComplexKeyHashedDictionary(
ColumnPtr ComplexKeyHashedDictionary::getColumn(
const std::string & attribute_name,
const DataTypePtr &,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types,
const ColumnPtr default_untyped) const
@ -47,9 +48,7 @@ ColumnPtr ComplexKeyHashedDictionary::getColumn(
ColumnPtr result;
const auto & attribute = getAttribute(attribute_name);
/// TODO: Check that attribute type is same as result type
/// TODO: Check if const will work as expected
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
auto size = key_columns.front()->size();
@ -141,8 +140,8 @@ ColumnPtr ComplexKeyHashedDictionary::getColumn(
if constexpr (IsDecimalNumber<AttributeType>)
{
// auto scale = getDecimalScale(*attribute.type);
column = ColumnDecimal<AttributeType>::create(size, 0);
auto scale = getDecimalScale(*dictionary_attribute.nested_type);
column = ColumnDecimal<AttributeType>::create(size, scale);
}
else if constexpr (IsNumber<AttributeType>)
column = ColumnVector<AttributeType>::create(size);

View File

@ -208,16 +208,32 @@ void DictionaryStructure::validateKeyTypes(const DataTypes & key_types) const
for (const auto i : ext::range(0, key_types.size()))
{
const auto & expected_type = (*key)[i].type->getName();
const auto & actual_type = key_types[i]->getName();
const auto & expected_type = (*key)[i].type;
const auto & actual_type = key_types[i];
if (expected_type != actual_type)
throw Exception{"Key type at position " + std::to_string(i) + " does not match, expected " + expected_type + ", found "
+ actual_type,
ErrorCodes::TYPE_MISMATCH};
if (!areTypesEqual(expected_type, actual_type))
throw Exception{"Key type at position " + std::to_string(i) + " does not match, expected " + expected_type->getName() + ", found "
+ actual_type->getName(),
ErrorCodes::TYPE_MISMATCH};
}
}
const DictionaryAttribute & DictionaryStructure::getAttribute(const String& attribute_name, const DataTypePtr & type) const
{
auto find_iter
= std::find_if(attributes.begin(), attributes.end(), [&](const auto & attribute) { return attribute.name == attribute_name; });
if (find_iter == attributes.end())
throw Exception{"No such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
auto & attribute = *find_iter;
if (!areTypesEqual(attribute.type, type))
throw Exception{"Attribute type does not match, expected " + attribute.type->getName() + ", found " + type->getName(),
ErrorCodes::TYPE_MISMATCH};
return *find_iter;
}
std::string DictionaryStructure::getKeyDescription() const
{
@ -312,20 +328,12 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
continue;
/// TODO: Rewrite
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;
bool is_nullable = false;
// const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(type.get());
// if (array_type)
// {
// is_array = true;
// type = array_type->getNestedType();
// }
if (type->isNullable())
{
is_nullable = true;
@ -396,14 +404,4 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
return res_attributes;
}
const DictionaryAttribute & DictionaryStructure::getAttribute(const String& attribute_name) const
{
auto find_iter
= std::find_if(attributes.begin(), attributes.end(), [&](const auto & attribute) { return attribute.name == attribute_name; });
if (find_iter == attributes.end())
throw Exception{"No such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
return *find_iter;
}
}

View File

@ -154,11 +154,10 @@ struct DictionaryStructure final
DictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
void validateKeyTypes(const DataTypes & key_types) const;
const DictionaryAttribute &getAttribute(const String& attribute_name, const DataTypePtr & type) const;
std::string getKeyDescription() const;
bool isKeySizeFixed() const;
size_t getKeySize() const;
const DictionaryAttribute &getAttribute(const String& attribute_name) const;
private:
/// range_min and range_max have to be parsed before this function call
std::vector<DictionaryAttribute> getAttributes(

View File

@ -5,6 +5,7 @@
#include <Core/Defines.h>
#include <Functions/FunctionHelpers.h>
#include <Columns/ColumnNullable.h>
#include <DataTypes/DataTypesDecimal.h>
namespace DB
{
@ -130,8 +131,8 @@ void DirectDictionary::isInConstantVector(const Key child_id, const PaddedPODArr
}
ColumnPtr DirectDictionary::getColumn(
const std::string& attribute_name,
const DataTypePtr &,
const std::string & attribute_name,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes &,
const ColumnPtr default_untyped) const
@ -153,8 +154,7 @@ ColumnPtr DirectDictionary::getColumn(
vec_null_map_to = &col_null_map_to->getData();
}
/// TODO: Check that attribute type is same as result type
/// TODO: Check if const will work as expected
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
auto type_call = [&](const auto &dictionary_attribute_type)
{
@ -244,8 +244,8 @@ ColumnPtr DirectDictionary::getColumn(
if constexpr (IsDecimalNumber<AttributeType>)
{
// auto scale = getDecimalScale(*attribute.type);
column = ColumnDecimal<AttributeType>::create(size, 0);
auto scale = getDecimalScale(*dictionary_attribute.nested_type);
column = ColumnDecimal<AttributeType>::create(size, scale);
}
else if constexpr (IsNumber<AttributeType>)
column = ColumnVector<AttributeType>::create(size);

View File

@ -108,8 +108,8 @@ void FlatDictionary::isInConstantVector(const Key child_id, const PaddedPODArray
}
ColumnPtr FlatDictionary::getColumn(
const std::string& attribute_name,
const DataTypePtr &,
const std::string & attribute_name,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes &,
const ColumnPtr default_untyped) const
@ -122,18 +122,7 @@ ColumnPtr FlatDictionary::getColumn(
auto size = ids.size();
const auto & attribute = getAttribute(attribute_name);
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name);
ColumnUInt8::MutablePtr col_null_map_to;
ColumnUInt8::Container * vec_null_map_to = nullptr;
if (attribute.nullable_set)
{
col_null_map_to = ColumnUInt8::create(size, false);
vec_null_map_to = &col_null_map_to->getData();
}
/// TODO: Check that attribute type is same as result type
/// TODO: Check if const will work as expected
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
auto type_call = [&](const auto &dictionary_attribute_type)
{
@ -244,13 +233,15 @@ ColumnPtr FlatDictionary::getColumn(
if (attribute.nullable_set)
{
ColumnUInt8::MutablePtr col_null_map_to = ColumnUInt8::create(size, false);
ColumnUInt8::Container& vec_null_map_to = col_null_map_to->getData();
for (size_t row = 0; row < ids.size(); ++row)
{
auto id = ids[row];
if (attribute.nullable_set->find(id) != nullptr)
{
(*vec_null_map_to)[row] = true;
}
vec_null_map_to[row] = true;
}
result = ColumnNullable::create(result, std::move(col_null_map_to));

View File

@ -7,6 +7,7 @@
#include <Functions/FunctionHelpers.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnNullable.h>
#include <DataTypes/DataTypesDecimal.h>
namespace
{
@ -129,12 +130,11 @@ void HashedDictionary::isInConstantVector(const Key child_id, const PaddedPODArr
ColumnPtr HashedDictionary::getColumn(
const std::string & attribute_name,
const DataTypePtr &,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes &,
const ColumnPtr default_untyped) const
{
// dict_struct.validateKeyTypes(key_types);
ColumnPtr result;
PaddedPODArray<Key> backup_storage;
@ -143,17 +143,7 @@ ColumnPtr HashedDictionary::getColumn(
auto size = ids.size();
const auto & attribute = getAttribute(attribute_name);
ColumnUInt8::MutablePtr col_null_map_to;
ColumnUInt8::Container * vec_null_map_to = nullptr;
if (attribute.is_nullable)
{
col_null_map_to = ColumnUInt8::create(size, false);
vec_null_map_to = &col_null_map_to->getData();
}
/// TODO: Check that attribute type is same as result type
/// TODO: Check if const will work as expected
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
auto type_call = [&](const auto & dictionary_attribute_type)
{
@ -211,8 +201,8 @@ ColumnPtr HashedDictionary::getColumn(
if constexpr (IsDecimalNumber<AttributeType>)
{
// auto scale = getDecimalScale(*attribute.type);
column = ColumnDecimal<AttributeType>::create(size, 0);
auto scale = getDecimalScale(*dictionary_attribute.nested_type);
column = ColumnDecimal<AttributeType>::create(size, scale);
}
else if constexpr (IsNumber<AttributeType>)
column = ColumnVector<AttributeType>::create(size);
@ -262,15 +252,17 @@ ColumnPtr HashedDictionary::getColumn(
callOnDictionaryAttributeType(attribute.type, type_call);
if (attribute.is_nullable)
if (attribute.nullable_set)
{
ColumnUInt8::MutablePtr col_null_map_to = ColumnUInt8::create(size, false);
ColumnUInt8::Container& vec_null_map_to = col_null_map_to->getData();
for (size_t row = 0; row < ids.size(); ++row)
{
auto id = ids[row];
if (attribute.nullable_set->find(id) != nullptr)
{
(*vec_null_map_to)[row] = true;
}
vec_null_map_to[row] = true;
}
result = ColumnNullable::create(result, std::move(col_null_map_to));
@ -578,8 +570,8 @@ void HashedDictionary::createAttributeImpl<String>(Attribute & attribute, const
HashedDictionary::Attribute HashedDictionary::createAttribute(const DictionaryAttribute& attribute, const Field & null_value)
{
auto nullable_set = attribute.is_nullable ? std::make_unique<NullableSet>() : nullptr;
Attribute attr{attribute.underlying_type, attribute.is_nullable, std::move(nullable_set), {}, {}, {}, {}};
auto nullable_set = attribute.is_nullable ? std::make_optional<NullableSet>() : std::optional<NullableSet>{};
Attribute attr{attribute.underlying_type, std::move(nullable_set), {}, {}, {}, {}};
auto type_call = [&](const auto &dictionary_attribute_type)
{
@ -650,12 +642,11 @@ bool HashedDictionary::setAttributeValue(Attribute & attribute, const Key id, co
using Type = std::decay_t<decltype(dictionary_attribute_type)>;
using AttributeType = typename Type::AttributeType;
if (attribute.is_nullable)
if (attribute.nullable_set)
{
if (value.isNull())
{
attribute.nullable_set->insert(id);
result = true;
result = attribute.nullable_set->insert(id).second;
return;
}
else
@ -691,7 +682,7 @@ void HashedDictionary::has(const Attribute & attribute, const PaddedPODArray<Key
{
out[i] = attr.find(ids[i]) != nullptr;
if (attribute.is_nullable && !out[i])
if (attribute.nullable_set && !out[i])
out[i] = attribute.nullable_set->find(ids[i]) != nullptr;
}
}
@ -738,7 +729,7 @@ PaddedPODArray<HashedDictionary::Key> HashedDictionary::getIds() const
/// TODO: Check if order is satisfied
result = getIds<AttributeType>(attribute);
if (attribute.is_nullable)
if (attribute.nullable_set)
{
for (const auto& value: *attribute.nullable_set)
result.push_back(value.getKey());

View File

@ -3,6 +3,7 @@
#include <atomic>
#include <memory>
#include <variant>
#include <optional>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Core/Block.h>
@ -107,8 +108,7 @@ private:
struct Attribute final
{
AttributeUnderlyingType type;
bool is_nullable;
std::unique_ptr<NullableSet> nullable_set;
std::optional<NullableSet> nullable_set;
std::variant<
UInt8,

View File

@ -8,6 +8,7 @@
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesDecimal.h>
#include <IO/WriteIntText.h>
#include <Poco/ByteOrder.h>
#include <Common/formatIPv6.h>
@ -269,7 +270,7 @@ IPAddressDictionary::IPAddressDictionary(
ColumnPtr IPAddressDictionary::getColumn(
const std::string & attribute_name,
const DataTypePtr &,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types,
const ColumnPtr default_untyped) const
@ -279,9 +280,7 @@ ColumnPtr IPAddressDictionary::getColumn(
ColumnPtr result;
const auto & attribute = getAttribute(attribute_name);
/// TODO: Check that attribute type is same as result type
/// TODO: Check if const will work as expected
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
auto size = key_columns.front()->size();
@ -341,8 +340,8 @@ ColumnPtr IPAddressDictionary::getColumn(
if constexpr (IsDecimalNumber<AttributeType>)
{
// auto scale = getDecimalScale(*attribute.type);
column = ColumnDecimal<AttributeType>::create(size, 0);
auto scale = getDecimalScale(*dictionary_attribute.nested_type);
column = ColumnDecimal<AttributeType>::create(size, scale);
}
else if constexpr (IsNumber<AttributeType>)
column = ColumnVector<AttributeType>::create(size);

View File

@ -6,6 +6,7 @@
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypeArray.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypesDecimal.h>
#include <numeric>
@ -95,7 +96,7 @@ bool IPolygonDictionary::isInjective(const std::string &) const
ColumnPtr IPolygonDictionary::getColumn(
const std::string & attribute_name,
const DataTypePtr &,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes &,
const ColumnPtr default_untyped) const
@ -105,8 +106,7 @@ ColumnPtr IPolygonDictionary::getColumn(
ColumnPtr result;
const auto index = getAttributeIndex(attribute_name);
/// TODO: Check that attribute type is same as result type
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
auto size = key_columns.front()->size();
@ -166,8 +166,8 @@ ColumnPtr IPolygonDictionary::getColumn(
if constexpr (IsDecimalNumber<AttributeType>)
{
// auto scale = getDecimalScale(*attribute.type);
column = ColumnDecimal<AttributeType>::create(size, 0);
auto scale = getDecimalScale(*dictionary_attribute.nested_type);
column = ColumnDecimal<AttributeType>::create(size, scale);
}
else if constexpr (IsNumber<AttributeType>)
column = ColumnVector<AttributeType>::create(size);

View File

@ -6,6 +6,7 @@
#include "DictionaryFactory.h"
#include "RangeDictionaryBlockInputStream.h"
#include <Interpreters/castColumn.h>
#include <DataTypes/DataTypesDecimal.h>
namespace
{
@ -89,7 +90,7 @@ RangeHashedDictionary::RangeHashedDictionary(
ColumnPtr RangeHashedDictionary::getColumn(
const std::string & attribute_name,
const DataTypePtr &,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types,
const ColumnPtr default_untyped) const
@ -99,8 +100,7 @@ ColumnPtr RangeHashedDictionary::getColumn(
ColumnPtr result;
const auto & attribute = getAttribute(attribute_name);
/// TODO: Check that attribute type is same as result type
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
auto size = key_columns.front()->size();
@ -201,8 +201,8 @@ ColumnPtr RangeHashedDictionary::getColumn(
if constexpr (IsDecimalNumber<AttributeType>)
{
// auto scale = getDecimalScale(*attribute.type);
column = ColumnDecimal<AttributeType>::create(size, 0);
auto scale = getDecimalScale(*dictionary_attribute.nested_type);
column = ColumnDecimal<AttributeType>::create(size, scale);
}
else if constexpr (IsNumber<AttributeType>)
column = ColumnVector<AttributeType>::create(size);

View File

@ -23,6 +23,7 @@
#include <city.h>
#include <fcntl.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypesDecimal.h>
namespace ProfileEvents
{
@ -1329,7 +1330,7 @@ SSDCacheDictionary::SSDCacheDictionary(
ColumnPtr SSDCacheDictionary::getColumn(
const std::string & attribute_name,
const DataTypePtr &,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes &,
const ColumnPtr default_untyped) const
@ -1340,9 +1341,7 @@ ColumnPtr SSDCacheDictionary::getColumn(
const auto & ids = getColumnDataAsPaddedPODArray(this, key_columns.front(), backup_storage);
const auto index = getAttributeIndex(attribute_name);
/// TODO: Check that attribute type is same as result type
/// TODO: Check if const will work as expected
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
auto type_call = [&](const auto &dictionary_attribute_type)
{
@ -1389,8 +1388,8 @@ ColumnPtr SSDCacheDictionary::getColumn(
if constexpr (IsDecimalNumber<AttributeType>)
{
// auto scale = getDecimalScale(*attribute.type);
column = ColumnDecimal<AttributeType>::create(identifiers_size, 0);
auto scale = getDecimalScale(*dictionary_attribute.nested_type);
column = ColumnDecimal<AttributeType>::create(identifiers_size, scale);
}
else if constexpr (IsNumber<AttributeType>)
column = ColumnVector<AttributeType>::create(identifiers_size);

View File

@ -9,6 +9,7 @@
#include <Common/ProfilingScopedRWLock.h>
#include <Common/MemorySanitizer.h>
#include <DataStreams/IBlockInputStream.h>
#include <DataTypes/DataTypesDecimal.h>
#include "DictionaryBlockInputStream.h"
#include "DictionaryFactory.h"
#include <IO/AIO.h>
@ -1378,19 +1379,17 @@ SSDComplexKeyCacheDictionary::SSDComplexKeyCacheDictionary(
ColumnPtr SSDComplexKeyCacheDictionary::getColumn(
const std::string & attribute_name,
const DataTypePtr &,
const DataTypePtr & result_type,
const Columns & key_columns,
const DataTypes & key_types,
const ColumnPtr default_untyped) const
{
ColumnPtr result;
dict_struct.validateKeyTypes(key_types);
const auto index = getAttributeIndex(attribute_name);
ColumnPtr result;
/// TODO: Check that attribute type is same as result type
/// TODO: Check if const will work as expected
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
auto keys_size = key_columns.front()->size();
@ -1438,8 +1437,8 @@ ColumnPtr SSDComplexKeyCacheDictionary::getColumn(
if constexpr (IsDecimalNumber<AttributeType>)
{
// auto scale = getDecimalScale(*attribute.type);
column = ColumnDecimal<AttributeType>::create(keys_size, 0);
auto scale = getDecimalScale(*dictionary_attribute.nested_type);
column = ColumnDecimal<AttributeType>::create(keys_size, scale);
}
else if constexpr (IsNumber<AttributeType>)
column = ColumnVector<AttributeType>::create(keys_size);

View File

@ -212,4 +212,12 @@ checkAndGetNestedArrayOffset(const IColumn ** columns, size_t num_arguments)
return {nested_columns, offsets->data()};
}
bool areTypesEqual(const DataTypePtr & lhs, const DataTypePtr & rhs)
{
const auto & lhs_name = lhs->getName();
const auto & rhs_name = rhs->getName();
return lhs_name == rhs_name;
}
}

View File

@ -152,4 +152,8 @@ void validateFunctionArgumentTypes(const IFunction & func, const ColumnsWithType
std::pair<std::vector<const IColumn *>, const ColumnArray::Offset *>
checkAndGetNestedArrayOffset(const IColumn ** columns, size_t num_arguments);
/// Check if two types are equal
bool areTypesEqual(const DataTypePtr & lhs, const DataTypePtr & rhs);
}

View File

@ -58,6 +58,8 @@ namespace ErrorCodes
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_COLUMN;
extern const int BAD_ARGUMENTS;
extern const int TYPE_MISMATCH;
extern const int NOT_IMPLEMENTED;
}