Fixed ubsan test

This commit is contained in:
Maksim Kita 2021-02-27 23:39:34 +03:00
parent 1628c9b6c4
commit 92b79420a4
6 changed files with 32 additions and 29 deletions

View File

@ -301,7 +301,8 @@ Columns CacheDictionary<dictionary_key_type>::getColumns(
if (dictionary_key_type == DictionaryKeyType::complex)
dict_struct.validateKeyTypes(key_types);
DictionaryKeysExtractor<dictionary_key_type> extractor(key_columns);
Arena complex_keys_arena;
DictionaryKeysExtractor<dictionary_key_type> extractor(key_columns, complex_keys_arena);
auto & keys = extractor.getKeys();
return getColumnsImpl(attribute_names, key_columns, keys, default_values_columns);
@ -424,7 +425,8 @@ Columns CacheDictionary<dictionary_key_type>::getColumnsImpl(
template <DictionaryKeyType dictionary_key_type>
ColumnUInt8::Ptr CacheDictionary<dictionary_key_type>::hasKeys(const Columns & key_columns, const DataTypes &) const
{
DictionaryKeysExtractor<dictionary_key_type> extractor(key_columns);
Arena complex_keys_arena;
DictionaryKeysExtractor<dictionary_key_type> extractor(key_columns, complex_keys_arena);
const auto & keys = extractor.getKeys();
/// We make empty request just to fetch if keys exists
@ -707,7 +709,7 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
block_columns.erase(block_columns.begin());
}
DictionaryKeysExtractor<dictionary_key_type> keys_extractor(key_columns, *update_unit_ptr->complex_key_arena);
DictionaryKeysExtractor<dictionary_key_type> keys_extractor(key_columns, update_unit_ptr->complex_key_arena);
const auto & keys = keys_extractor.getKeys();
cache_storage_ptr->insertColumnsForKeys(keys, block_columns);

View File

@ -60,7 +60,6 @@ public:
: requested_complex_key_columns(requested_complex_key_columns_)
, requested_complex_key_rows(std::move(requested_complex_key_rows_))
, request(request_)
, complex_key_arena(std::make_shared<Arena>())
, alive_keys(CurrentMetrics::CacheDictionaryUpdateQueueKeys, requested_complex_key_rows.size())
{
fetched_columns_during_update = request.makeAttributesResultColumns();
@ -79,8 +78,8 @@ public:
HashMap<KeyType, size_t> requested_keys_to_fetched_columns_during_update_index;
MutableColumns fetched_columns_during_update;
/// Complex keys are serialized in this arena and added to map
const std::shared_ptr<Arena> complex_key_arena;
/// Complex keys are serialized in this arena
Arena complex_key_arena;
private:
template <DictionaryKeyType>

View File

@ -205,7 +205,7 @@ class DictionaryDefaultValueExtractor
public:
using DefaultValueType = DictionaryValueType<DictionaryAttributeType>;
DictionaryDefaultValueExtractor(DictionaryAttributeType attribute_default_value, ColumnPtr default_values_column_ = nullptr)
explicit DictionaryDefaultValueExtractor(DictionaryAttributeType attribute_default_value, ColumnPtr default_values_column_ = nullptr)
: default_value(std::move(attribute_default_value))
{
if (default_values_column_ == nullptr)
@ -252,16 +252,6 @@ public:
using KeyType = std::conditional_t<key_type == DictionaryKeyType::simple, UInt64, StringRef>;
static_assert(key_type != DictionaryKeyType::range, "Range key type is not supported by DictionaryKeysExtractor");
explicit DictionaryKeysExtractor(const Columns & key_columns)
{
assert(!key_columns.empty());
if constexpr (key_type == DictionaryKeyType::simple)
keys = getColumnVectorData(key_columns.front());
else
keys = deserializeKeyColumnsInArena(key_columns, complex_keys_temporary_arena);
}
explicit DictionaryKeysExtractor(const Columns & key_columns, Arena & existing_arena)
{
assert(!key_columns.empty());
@ -322,7 +312,7 @@ private:
}
PaddedPODArray<KeyType> keys;
Arena complex_keys_temporary_arena;
};
/**

View File

@ -125,7 +125,7 @@ ColumnPtr FlatDictionary::getColumn(
const auto & attribute = getAttribute(attribute_name);
const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type);
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;
@ -167,7 +167,7 @@ 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();
ColumnUInt8::Container & vec_null_map_to = col_null_map_to->getData();
for (size_t row = 0; row < ids.size(); ++row)
{

View File

@ -24,6 +24,7 @@
#include <Interpreters/Context.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/castColumn.h>
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionHelpers.h>
@ -338,17 +339,28 @@ public:
if (current_arguments_index >= arguments.size())
throw Exception{"Wrong argument count for function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
auto default_column_type = arguments[current_arguments_index].type;
const auto & column_before_cast = arguments[current_arguments_index];
if (!default_column_type->equals(*result_type))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Wrong argument for function ({}) default values column must be the same type as dictionary attribute type",
getName());
if (const DataTypeTuple * type_tuple = typeid_cast<const DataTypeTuple *>(column_before_cast.type.get()))
{
const DataTypes & nested_types = type_tuple->getElements();
for (const auto & nested_type : nested_types)
if (nested_type->isNullable())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Wrong argument for function ({}) default values column nullable is not supported", getName());
}
else if (column_before_cast.type->isNullable())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Wrong argument for function ({}) default values column nullable is not supported", getName());
auto result_type_no_nullable = removeNullable(result_type);
ColumnWithTypeAndName column_to_cast = {column_before_cast.column->convertToFullColumnIfConst(), column_before_cast.type, column_before_cast.name};
auto result = castColumnAccurate(column_to_cast, result_type_no_nullable);
if (attribute_names.size() > 1)
{
const auto * tuple_column = checkAndGetColumn<ColumnTuple>(arguments[current_arguments_index].column.get());
const auto * tuple_column = checkAndGetColumn<ColumnTuple>(result.get());
if (!tuple_column)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
@ -362,7 +374,7 @@ public:
default_cols = tuple_column->getColumnsCopy();
}
else
default_cols.emplace_back(arguments[current_arguments_index].column);
default_cols.emplace_back(result);
}
else
{

View File

@ -104,7 +104,7 @@ CREATE DICTIONARY 01681_database_for_flat_dictionary.flat_dictionary_simple_key_
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_hierarchy_table'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(CACHE(SIZE_IN_CELLS 10));
LAYOUT(FLAT());
SELECT 'Dictionary flat_dictionary_simple_key_hierarchy';
SELECT 'dictGet';