Fixed minor issues

This commit is contained in:
Maksim Kita 2021-02-19 13:30:53 +03:00
parent 8138ced862
commit a22ec65e8a
9 changed files with 64 additions and 39 deletions

View File

@ -229,8 +229,8 @@ public:
bool ALWAYS_INLINE erase(const Key & key)
{
auto hash = Base::hash(key);
auto it = Base::find(key, hash);
auto key_hash = Base::hash(key);
auto it = Base::find(key, key_hash);
if (!it)
return false;
@ -239,7 +239,7 @@ public:
lru_list.erase(lru_list.iterator_to(*it));
return Base::erase(key, hash);
return Base::erase(key, key_hash);
}
void ALWAYS_INLINE clear()

View File

@ -149,15 +149,17 @@ private:
if (it)
{
/// Columns values for key are serialized in cache now deserialize them
auto & cell = it->getMapped();
const auto & cell = it->getMapped();
if (now > cell.deadline + std::chrono::seconds(configuration.strict_max_lifetime_seconds))
bool has_deadline = cellHasDeadline(cell);
if (has_deadline && now > cell.deadline + std::chrono::seconds(configuration.strict_max_lifetime_seconds))
{
result.not_found_or_expired_keys.emplace_back(key);
result.not_found_or_expired_keys_indexes.emplace_back(key_index);
continue;
}
else if (now > cell.deadline)
else if (has_deadline && now > cell.deadline)
{
result.expired_keys_to_fetched_columns_index[key] = fetched_columns_index;
result.not_found_or_expired_keys.emplace_back(key);
@ -265,15 +267,21 @@ private:
char * place_for_serialized_columns;
};
inline static bool cellHasDeadline(const Cell & cell)
{
return cell.deadline != std::chrono::system_clock::from_time_t(0);
}
inline void setCellDeadline(Cell & cell, TimePoint now)
{
/// TODO: Fix zero dictionary lifetime deadlines
if (configuration.lifetime.min_sec == 0 && configuration.lifetime.max_sec == 0)
cell.deadline = std::chrono::system_clock::from_time_t(0);
size_t min_sec_lifetime = configuration.lifetime.min_sec;
size_t max_sec_lifetime = configuration.lifetime.max_sec;
std::uniform_int_distribution<UInt64> distribution{min_sec_lifetime, max_sec_lifetime};
cell.deadline = now + std::chrono::seconds{distribution(rnd_engine)};
cell.deadline = now + std::chrono::seconds(distribution(rnd_engine));
}
template <typename>

View File

@ -147,7 +147,7 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
id.emplace(config, structure_prefix + ".id");
else if (has_key)
{
key.emplace(getAttributes(config, structure_prefix + ".key", false, false));
key.emplace(getAttributes(config, structure_prefix + ".key", true));
if (key->empty())
throw Exception{"Empty 'key' supplied", ErrorCodes::BAD_ARGUMENTS};
}
@ -196,7 +196,13 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
has_expressions = true;
}
attributes = getAttributes(config, structure_prefix);
attributes = getAttributes(config, structure_prefix, false);
for (size_t i = 0; i < attributes.size(); ++i)
{
const auto & attribute_name = attributes[i].name;
attribute_name_to_index[attribute_name] = i;
}
if (attributes.empty())
throw Exception{"Dictionary has no attributes defined", ErrorCodes::BAD_ARGUMENTS};
@ -223,23 +229,22 @@ void DictionaryStructure::validateKeyTypes(const DataTypes & key_types) const
}
}
size_t DictionaryStructure::getAttributeIndex(const std::string & attribute_name) const
const DictionaryAttribute & DictionaryStructure::getAttribute(const std::string & attribute_name) const
{
auto it = attribute_name_to_index.find(attribute_name);
if (it == attribute_name_to_index.end())
throw Exception{"No such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
{
if (!access_to_key_from_attributes)
throw Exception{"No such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS};
size_t index = it->second;
for (const auto & key_attribute : *key)
if (key_attribute.name == attribute_name)
return key_attribute;
}
return index;
}
const DictionaryAttribute & DictionaryStructure::getAttribute(const std::string & attribute_name) const
{
size_t index = getAttributeIndex(attribute_name);
return attributes[index];
size_t attribute_index = it->second;
return attributes[attribute_index];
}
const DictionaryAttribute & DictionaryStructure::getAttribute(const std::string & attribute_name, const DataTypePtr & type) const
@ -339,9 +344,12 @@ static void checkAttributeKeys(const Poco::Util::AbstractConfiguration::Keys & k
std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const bool hierarchy_allowed,
const bool allow_null_values)
bool complex_key_attributes)
{
/// If we request complex key attributes they does not support hierarchy and does not allow null values
const bool hierarchy_allowed = !complex_key_attributes;
const bool allow_null_values = !complex_key_attributes;
Poco::Util::AbstractConfiguration::Keys config_elems;
config.keys(config_prefix, config_elems);
auto has_hierarchy = false;
@ -368,7 +376,6 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
if ((range_min && name == range_min->name) || (range_max && name == range_max->name))
continue;
const auto type_string = config.getString(prefix + "type");
const auto initial_type = DataTypeFactory::instance().get(type_string);
auto type = initial_type;
@ -428,7 +435,6 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
has_hierarchy = has_hierarchy || hierarchical;
attribute_name_to_index[name] = res_attributes.size();
res_attributes.emplace_back(DictionaryAttribute{
name,
underlying_type,

View File

@ -159,7 +159,6 @@ struct DictionaryStructure final
void validateKeyTypes(const DataTypes & key_types) const;
size_t getAttributeIndex(const std::string & attribute_name) const;
const DictionaryAttribute & getAttribute(const std::string & attribute_name) const;
const DictionaryAttribute & getAttribute(const std::string & attribute_name, const DataTypePtr & type) const;
size_t getKeysSize() const;
@ -174,8 +173,7 @@ private:
std::vector<DictionaryAttribute> getAttributes(
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const bool hierarchy_allowed = true,
const bool allow_null_values = true);
bool complex_key_attributes);
};
}

View File

@ -290,7 +290,6 @@ ColumnPtr IPAddressDictionary::getColumn(
auto column = ColumnProvider::getColumn(dictionary_attribute, size);
if constexpr (std::is_same_v<AttributeType, String>)
{
auto * out = column.get();

View File

@ -670,7 +670,9 @@ public:
throw Exception("AIO failed to read file " + file_path + ".", ErrorCodes::AIO_READ_ERROR);
char * request_buffer = getRequestBuffer(request);
__msan_unpoison(buf_ptr, block_size);
// Unpoison the memory returned from an uninstrumented system function.
__msan_unpoison(request_buffer, block_size);
SSDCacheBlock block(block_size);
block.reset(request_buffer);
@ -941,13 +943,15 @@ private:
{
const auto & cell = it->getMapped();
if (now > cell.deadline + std::chrono::seconds(configuration.strict_max_lifetime_seconds))
bool has_deadline = cellHasDeadline(cell);
if (has_deadline && now > cell.deadline + std::chrono::seconds(configuration.strict_max_lifetime_seconds))
{
result.not_found_or_expired_keys.emplace_back(key);
result.not_found_or_expired_keys_indexes.emplace_back(key_index);
continue;
}
else if (now > cell.deadline)
else if (has_deadline && now > cell.deadline)
{
if (cell.in_memory)
{
@ -1190,9 +1194,15 @@ private:
}
}
inline static bool cellHasDeadline(const Cell & cell)
{
return cell.deadline != std::chrono::system_clock::from_time_t(0);
}
inline void setCellDeadline(Cell & cell, TimePoint now)
{
/// TODO: Fix zero dictionary lifetime deadlines
if (configuration.lifetime.min_sec == 0 && configuration.lifetime.max_sec == 0)
cell.deadline = std::chrono::system_clock::from_time_t(0);
size_t min_sec_lifetime = configuration.lifetime.min_sec;
size_t max_sec_lifetime = configuration.lifetime.max_sec;

View File

@ -276,8 +276,12 @@ public:
throw Exception{"Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName()
+ ", expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
std::cerr << "Attribute name to get " << attribute_name << std::endl;
/// We're extracting the return type from the dictionary's config, without loading the dictionary.
return helper.getDictionaryStructure(dictionary_name).getAttribute(attribute_name).type;
auto attribute = helper.getDictionaryStructure(dictionary_name).getAttribute(attribute_name);
std::cerr << "FunctionDictGetNoType " << attribute.name << " " << attribute.type->getName() << std::endl;
return attribute.type;
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override

View File

@ -22,7 +22,7 @@ CREATE DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_ke
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_simple_attributes_source_table'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '/home/maksim-kita/ClickHouseClang/build_debug/programs/0d'));
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/0d'));
SELECT 'Dictionary cache_dictionary_simple_key_simple_attributes';
SELECT 'dictGet existing value';
@ -66,7 +66,7 @@ CREATE DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_ke
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_complex_attributes_source_table'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '/home/maksim-kita/ClickHouseClang/build_debug/programs/0d'));
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/1d'));
SELECT 'Dictionary cache_dictionary_simple_key_complex_attributes';
SELECT 'dictGet existing value';
@ -108,7 +108,7 @@ CREATE DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_ke
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_hierarchy_table'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '/home/maksim-kita/ClickHouseClang/build_debug/programs/0d'));
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/2d'));
SELECT 'Dictionary cache_dictionary_simple_key_hierarchy';
SELECT 'dictGet';

View File

@ -24,7 +24,7 @@ CREATE DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_k
PRIMARY KEY id, id_key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_simple_attributes_source_table' DB '01685_database_for_cache_dictionary'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(COMPLEX_KEY_SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '/home/maksim-kita/ClickHouseClang/build_debug/programs/0d'));
LAYOUT(COMPLEX_KEY_SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/0d'));
SELECT 'Dictionary cache_dictionary_complex_key_simple_attributes';
SELECT 'dictGet existing value';
@ -71,7 +71,7 @@ CREATE DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_k
PRIMARY KEY id, id_key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_complex_attributes_source_table' DB '01685_database_for_cache_dictionary'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(COMPLEX_KEY_SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '/home/maksim-kita/ClickHouseClang/build_debug/programs/0d'));
LAYOUT(COMPLEX_KEY_SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/1d'));
SELECT 'Dictionary cache_dictionary_complex_key_complex_attributes';
SELECT 'dictGet existing value';