mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Fixed minor issues
This commit is contained in:
parent
8138ced862
commit
a22ec65e8a
@ -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()
|
||||
|
@ -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>
|
||||
|
@ -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,
|
||||
|
@ -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);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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';
|
||||
|
@ -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';
|
||||
|
Loading…
Reference in New Issue
Block a user