mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
SSDCacheDictionary remove max_stored_keys option
This commit is contained in:
parent
d5a1b50fd2
commit
18ed9c5c67
@ -320,8 +320,6 @@ Similar to `cache`, but stores data on SSD and index in RAM.
|
||||
<write_buffer_size>1048576</write_buffer_size>
|
||||
<!-- Path where cache file will be stored. -->
|
||||
<path>/var/lib/clickhouse/clickhouse_dictionaries/test_dict</path>
|
||||
<!-- Max number on stored keys in the cache. Rounded up to a power of two. -->
|
||||
<max_stored_keys>1048576</max_stored_keys>
|
||||
</ssd_cache>
|
||||
</layout>
|
||||
```
|
||||
@ -329,8 +327,8 @@ Similar to `cache`, but stores data on SSD and index in RAM.
|
||||
or
|
||||
|
||||
``` sql
|
||||
LAYOUT(CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576
|
||||
PATH /var/lib/clickhouse/clickhouse_dictionaries/test_dict MAX_STORED_KEYS 1048576))
|
||||
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576
|
||||
PATH /var/lib/clickhouse/clickhouse_dictionaries/test_dict))
|
||||
```
|
||||
|
||||
### complex_key_ssd_cache {#complex-key-ssd-cache}
|
||||
|
@ -318,8 +318,6 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000))
|
||||
<write_buffer_size>1048576</write_buffer_size>
|
||||
<!-- Path where cache file will be stored. -->
|
||||
<path>/var/lib/clickhouse/clickhouse_dictionaries/test_dict</path>
|
||||
<!-- Max number on stored keys in the cache. Rounded up to a power of two. -->
|
||||
<max_stored_keys>1048576</max_stored_keys>
|
||||
</ssd_cache>
|
||||
</layout>
|
||||
```
|
||||
@ -327,8 +325,8 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000))
|
||||
или
|
||||
|
||||
``` sql
|
||||
LAYOUT(CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576
|
||||
PATH /var/lib/clickhouse/clickhouse_dictionaries/test_dict MAX_STORED_KEYS 1048576))
|
||||
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576
|
||||
PATH /var/lib/clickhouse/clickhouse_dictionaries/test_dict))
|
||||
```
|
||||
|
||||
### complex_key_ssd_cache {#complex-key-ssd-cache}
|
||||
|
@ -202,16 +202,6 @@ public:
|
||||
return const_cast<std::decay_t<decltype(*this)> *>(this)->find(key);
|
||||
}
|
||||
|
||||
LookupResult ALWAYS_INLINE findNoLRU(const Key & key)
|
||||
{
|
||||
return Base::find(key);
|
||||
}
|
||||
|
||||
ConstLookupResult ALWAYS_INLINE findNoLRU(const Key & key) const
|
||||
{
|
||||
return const_cast<std::decay_t<decltype(*this)> *>(this)->findNoLRU(key);
|
||||
}
|
||||
|
||||
Value & ALWAYS_INLINE get(const Key & key)
|
||||
{
|
||||
auto it = find(key);
|
||||
|
@ -101,7 +101,7 @@ template <DictionaryKeyType dictionary_key_type>
|
||||
double CacheDictionary<dictionary_key_type>::getLoadFactor() const
|
||||
{
|
||||
const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs};
|
||||
return static_cast<double>(cache_storage_ptr->getSize()) / cache_storage_ptr->getMaxSize();
|
||||
return cache_storage_ptr->getLoadFactor();
|
||||
}
|
||||
|
||||
template <DictionaryKeyType dictionary_key_type>
|
||||
@ -332,16 +332,8 @@ Columns CacheDictionary<dictionary_key_type>::getColumnsImpl(
|
||||
|
||||
FetchResult result_of_fetch_from_storage;
|
||||
|
||||
bool can_perform_fetch_without_write_lock = cache_storage_ptr->canPerformFetchByMultipleThreadsWithoutLock();
|
||||
|
||||
if (can_perform_fetch_without_write_lock)
|
||||
{
|
||||
const ProfilingScopedReadRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
|
||||
result_of_fetch_from_storage = cache_storage_ptr->fetchColumnsForKeys(keys, request);
|
||||
}
|
||||
else
|
||||
{
|
||||
const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
|
||||
const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
|
||||
result_of_fetch_from_storage = cache_storage_ptr->fetchColumnsForKeys(keys, request);
|
||||
}
|
||||
|
||||
|
@ -58,8 +58,6 @@ public:
|
||||
|
||||
bool returnsFetchedColumnsInOrderOfRequestedKeys() const override { return true; }
|
||||
|
||||
bool canPerformFetchByMultipleThreadsWithoutLock() const override { return true; }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
@ -142,7 +140,7 @@ public:
|
||||
|
||||
size_t getSize() const override { return size; }
|
||||
|
||||
size_t getMaxSize() const override { return configuration.max_size_in_cells; }
|
||||
double getLoadFactor() const override { return static_cast<double>(size) / configuration.max_size_in_cells; }
|
||||
|
||||
size_t getBytesAllocated() const override
|
||||
{
|
||||
@ -654,7 +652,7 @@ private:
|
||||
return std::make_pair(KeyState::found, cell_place_value);
|
||||
}
|
||||
|
||||
return std::make_pair(KeyState::not_found, place_value);
|
||||
return std::make_pair(KeyState::not_found, place_value & size_overlap_mask);
|
||||
}
|
||||
|
||||
inline size_t getCellIndexForInsert(const KeyType & key) const
|
||||
@ -674,7 +672,7 @@ private:
|
||||
return cell_place_value;
|
||||
|
||||
if (cell.key == key)
|
||||
return place_value;
|
||||
return cell_place_value;
|
||||
|
||||
if (cell.deadline < oldest_time)
|
||||
{
|
||||
|
@ -34,7 +34,7 @@ struct KeyState
|
||||
inline void setDefaultValue(bool is_default_value) { is_default = is_default_value; }
|
||||
/// Valid only if keyState is found or expired
|
||||
inline size_t getFetchedColumnIndex() const { return fetched_column_index; }
|
||||
|
||||
inline void setFetchedColumnIndex(size_t fetched_column_index_value) { fetched_column_index = fetched_column_index_value; }
|
||||
private:
|
||||
State state = not_found;
|
||||
size_t fetched_column_index = 0;
|
||||
@ -72,8 +72,6 @@ public:
|
||||
/// Necessary if all keys are found we can return result to client without additional aggregation
|
||||
virtual bool returnsFetchedColumnsInOrderOfRequestedKeys() const = 0;
|
||||
|
||||
virtual bool canPerformFetchByMultipleThreadsWithoutLock() const = 0;
|
||||
|
||||
/// Name of storage
|
||||
virtual String getName() const = 0;
|
||||
|
||||
@ -114,8 +112,8 @@ public:
|
||||
/// Return size of keys in storage
|
||||
virtual size_t getSize() const = 0;
|
||||
|
||||
/// Return maximum size of keys in storage
|
||||
virtual size_t getMaxSize() const = 0;
|
||||
/// Returns storage load factor
|
||||
virtual double getLoadFactor() const = 0;
|
||||
|
||||
/// Return bytes allocated in storage
|
||||
virtual size_t getBytesAllocated() const = 0;
|
||||
|
@ -17,7 +17,7 @@
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/ArenaWithFreeLists.h>
|
||||
#include <Common/MemorySanitizer.h>
|
||||
#include <Common/HashTable/LRUHashMap.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <IO/AIO.h>
|
||||
#include <Dictionaries/DictionaryStructure.h>
|
||||
#include <Dictionaries/ICacheDictionaryStorage.h>
|
||||
@ -56,7 +56,6 @@ struct SSDCacheDictionaryStorageConfiguration
|
||||
|
||||
const std::string file_path;
|
||||
const size_t max_partitions_count;
|
||||
const size_t max_stored_keys;
|
||||
const size_t block_size;
|
||||
const size_t file_blocks_size;
|
||||
const size_t read_buffer_blocks_size;
|
||||
@ -127,7 +126,7 @@ public:
|
||||
|
||||
/// Reset block with new block_data
|
||||
/// block_data must be filled with zeroes if it is new block
|
||||
ALWAYS_INLINE inline void reset(char * new_block_data)
|
||||
inline void reset(char * new_block_data)
|
||||
{
|
||||
block_data = new_block_data;
|
||||
current_block_offset = block_header_size;
|
||||
@ -135,13 +134,13 @@ public:
|
||||
}
|
||||
|
||||
/// Check if it is enough place to write key in block
|
||||
ALWAYS_INLINE inline bool enoughtPlaceToWriteKey(const SSDCacheSimpleKey & cache_key) const
|
||||
inline bool enoughtPlaceToWriteKey(const SSDCacheSimpleKey & cache_key) const
|
||||
{
|
||||
return (current_block_offset + (sizeof(cache_key.key) + sizeof(cache_key.size) + cache_key.size)) <= block_size;
|
||||
}
|
||||
|
||||
/// Check if it is enough place to write key in block
|
||||
ALWAYS_INLINE inline bool enoughtPlaceToWriteKey(const SSDCacheComplexKey & cache_key) const
|
||||
inline bool enoughtPlaceToWriteKey(const SSDCacheComplexKey & cache_key) const
|
||||
{
|
||||
const StringRef & key = cache_key.key;
|
||||
size_t complex_key_size = sizeof(key.size) + key.size;
|
||||
@ -152,7 +151,7 @@ public:
|
||||
/// Write key and returns offset in ssd cache block where data is written
|
||||
/// It is client responsibility to check if there is enough place in block to write key
|
||||
/// Returns true if key was written and false if there was not enough place to write key
|
||||
ALWAYS_INLINE inline bool writeKey(const SSDCacheSimpleKey & cache_key, size_t & offset_in_block)
|
||||
inline bool writeKey(const SSDCacheSimpleKey & cache_key, size_t & offset_in_block)
|
||||
{
|
||||
assert(cache_key.size > 0);
|
||||
|
||||
@ -181,7 +180,7 @@ public:
|
||||
return true;
|
||||
}
|
||||
|
||||
ALWAYS_INLINE inline bool writeKey(const SSDCacheComplexKey & cache_key, size_t & offset_in_block)
|
||||
inline bool writeKey(const SSDCacheComplexKey & cache_key, size_t & offset_in_block)
|
||||
{
|
||||
assert(cache_key.size > 0);
|
||||
|
||||
@ -216,20 +215,20 @@ public:
|
||||
return true;
|
||||
}
|
||||
|
||||
ALWAYS_INLINE inline size_t getKeysSize() const { return keys_size; }
|
||||
inline size_t getKeysSize() const { return keys_size; }
|
||||
|
||||
/// Write keys size into block header
|
||||
ALWAYS_INLINE inline void writeKeysSize()
|
||||
inline void writeKeysSize()
|
||||
{
|
||||
char * keys_size_offset_data = block_data + block_header_check_sum_size;
|
||||
std::memcpy(keys_size_offset_data, &keys_size, sizeof(size_t));
|
||||
}
|
||||
|
||||
/// Get check sum from block header
|
||||
ALWAYS_INLINE inline size_t getCheckSum() const { return unalignedLoad<size_t>(block_data); }
|
||||
inline size_t getCheckSum() const { return unalignedLoad<size_t>(block_data); }
|
||||
|
||||
/// Calculate check sum in block
|
||||
ALWAYS_INLINE inline size_t calculateCheckSum() const
|
||||
inline size_t calculateCheckSum() const
|
||||
{
|
||||
size_t calculated_check_sum = static_cast<size_t>(CityHash_v1_0_2::CityHash64(block_data + block_header_check_sum_size, block_size - block_header_check_sum_size));
|
||||
|
||||
@ -237,7 +236,7 @@ public:
|
||||
}
|
||||
|
||||
/// Check if check sum from block header matched calculated check sum in block
|
||||
ALWAYS_INLINE inline bool checkCheckSum() const
|
||||
inline bool checkCheckSum() const
|
||||
{
|
||||
size_t calculated_check_sum = calculateCheckSum();
|
||||
size_t check_sum = getCheckSum();
|
||||
@ -246,16 +245,16 @@ public:
|
||||
}
|
||||
|
||||
/// Write check sum in block header
|
||||
ALWAYS_INLINE inline void writeCheckSum()
|
||||
inline void writeCheckSum()
|
||||
{
|
||||
size_t check_sum = static_cast<size_t>(CityHash_v1_0_2::CityHash64(block_data + block_header_check_sum_size, block_size - block_header_check_sum_size));
|
||||
std::memcpy(block_data, &check_sum, sizeof(size_t));
|
||||
}
|
||||
|
||||
ALWAYS_INLINE inline size_t getBlockSize() const { return block_size; }
|
||||
inline size_t getBlockSize() const { return block_size; }
|
||||
|
||||
/// Returns block data
|
||||
ALWAYS_INLINE inline char * getBlockData() const { return block_data; }
|
||||
inline char * getBlockData() const { return block_data; }
|
||||
|
||||
/// Read keys that were serialized in block
|
||||
/// It is client responsibility to ensure that simple or complex keys were written in block
|
||||
@ -753,7 +752,7 @@ private:
|
||||
int fd = -1;
|
||||
};
|
||||
|
||||
ALWAYS_INLINE inline static int preallocateDiskSpace(int fd, size_t offset, size_t len)
|
||||
inline static int preallocateDiskSpace(int fd, size_t offset, size_t len)
|
||||
{
|
||||
#if defined(__FreeBSD__)
|
||||
return posix_fallocate(fd, offset, len);
|
||||
@ -762,7 +761,7 @@ private:
|
||||
#endif
|
||||
}
|
||||
|
||||
ALWAYS_INLINE inline static char * getRequestBuffer(const iocb & request)
|
||||
inline static char * getRequestBuffer(const iocb & request)
|
||||
{
|
||||
char * result = nullptr;
|
||||
|
||||
@ -775,7 +774,7 @@ private:
|
||||
return result;
|
||||
}
|
||||
|
||||
ALWAYS_INLINE inline static ssize_t eventResult(io_event & event)
|
||||
inline static ssize_t eventResult(io_event & event)
|
||||
{
|
||||
ssize_t bytes_written;
|
||||
|
||||
@ -809,15 +808,12 @@ public:
|
||||
: configuration(configuration_)
|
||||
, file_buffer(configuration_.file_path, configuration.block_size, configuration.file_blocks_size)
|
||||
, rnd_engine(randomSeed())
|
||||
, index(configuration.max_stored_keys, false, { complex_key_arena })
|
||||
{
|
||||
memory_buffer_partitions.emplace_back(configuration.block_size, configuration.write_buffer_blocks_size);
|
||||
}
|
||||
|
||||
bool returnsFetchedColumnsInOrderOfRequestedKeys() const override { return false; }
|
||||
|
||||
bool canPerformFetchByMultipleThreadsWithoutLock() const override { return true; }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
if (dictionary_key_type == DictionaryKeyType::simple)
|
||||
@ -900,14 +896,31 @@ public:
|
||||
|
||||
size_t getSize() const override { return index.size(); }
|
||||
|
||||
size_t getMaxSize() const override {return index.getMaxSize(); }
|
||||
double getLoadFactor() const override
|
||||
{
|
||||
size_t partitions_size = memory_buffer_partitions.size();
|
||||
|
||||
if (partitions_size == configuration.max_partitions_count)
|
||||
return 1.0;
|
||||
|
||||
auto & current_memory_partition = memory_buffer_partitions[current_partition_index];
|
||||
|
||||
size_t full_partitions = partitions_size - 1;
|
||||
size_t blocks_in_memory = (full_partitions * configuration.write_buffer_blocks_size) + current_memory_partition.getCurrentBlockIndex();
|
||||
size_t blocks_on_disk = file_buffer.getCurrentBlockIndex();
|
||||
|
||||
size_t max_blocks_size = (configuration.file_blocks_size + configuration.write_buffer_blocks_size) * configuration.max_partitions_count;
|
||||
|
||||
double load_factor = static_cast<double>(blocks_in_memory + blocks_on_disk) / max_blocks_size;
|
||||
return load_factor;
|
||||
}
|
||||
|
||||
size_t getBytesAllocated() const override
|
||||
{
|
||||
size_t memory_partitions_bytes_size = memory_buffer_partitions.size() * configuration.write_buffer_blocks_size * configuration.block_size;
|
||||
size_t file_partitions_bytes_size = memory_buffer_partitions.size() * configuration.file_blocks_size * configuration.block_size;
|
||||
|
||||
return index.getSizeInBytes() + memory_partitions_bytes_size + file_partitions_bytes_size;
|
||||
return index.getBufferSizeInBytes() + memory_partitions_bytes_size + file_partitions_bytes_size;
|
||||
}
|
||||
|
||||
private:
|
||||
@ -935,13 +948,12 @@ private:
|
||||
|
||||
struct KeyToBlockOffset
|
||||
{
|
||||
KeyToBlockOffset(size_t key_index_, size_t offset_in_block_, bool is_expired_)
|
||||
: key_index(key_index_), offset_in_block(offset_in_block_), is_expired(is_expired_)
|
||||
KeyToBlockOffset(size_t key_index_, size_t offset_in_block_)
|
||||
: key_index(key_index_), offset_in_block(offset_in_block_)
|
||||
{}
|
||||
|
||||
size_t key_index = 0;
|
||||
size_t offset_in_block = 0;
|
||||
bool is_expired = false;
|
||||
};
|
||||
|
||||
template <typename Result>
|
||||
@ -952,7 +964,7 @@ private:
|
||||
Result result;
|
||||
|
||||
result.fetched_columns = fetch_request.makeAttributesResultColumns();
|
||||
result.key_index_to_state.resize_fill(keys.size(), {KeyState::not_found});
|
||||
result.key_index_to_state.resize_fill(keys.size());
|
||||
|
||||
const time_t now = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now());
|
||||
|
||||
@ -974,7 +986,7 @@ private:
|
||||
{
|
||||
auto key = keys[key_index];
|
||||
|
||||
const auto * it = index.findNoLRU(key);
|
||||
const auto * it = index.find(key);
|
||||
|
||||
if (!it)
|
||||
{
|
||||
@ -984,7 +996,7 @@ private:
|
||||
|
||||
const auto & cell = it->getMapped();
|
||||
|
||||
if (now > cell.deadline + strict_max_lifetime_seconds)
|
||||
if (unlikely(now > cell.deadline + strict_max_lifetime_seconds))
|
||||
{
|
||||
++result.not_found_keys_size;
|
||||
continue;
|
||||
@ -999,7 +1011,8 @@ private:
|
||||
key_state = KeyState::expired;
|
||||
}
|
||||
|
||||
result.expired_keys_size += cell_is_expired;
|
||||
result.expired_keys_size += static_cast<size_t>(cell_is_expired);
|
||||
result.found_keys_size += static_cast<size_t>(!cell_is_expired);
|
||||
|
||||
switch (cell.state)
|
||||
{
|
||||
@ -1016,13 +1029,19 @@ private:
|
||||
case Cell::on_disk:
|
||||
{
|
||||
PaddedPODArray<KeyToBlockOffset> & keys_block = block_to_keys_map[cell.index.block_index];
|
||||
keys_block.emplace_back(key_index, cell.index.offset_in_block, cell_is_expired);
|
||||
keys_block.emplace_back(key_index, cell.index.offset_in_block);
|
||||
|
||||
if (!unique_blocks_to_request.contains(cell.index.block_index))
|
||||
{
|
||||
KeyState::State state = cell_is_expired ? KeyState::expired : KeyState::found;
|
||||
|
||||
/// Fetched column index will be set later during fetch blocks
|
||||
result.key_index_to_state[key_index] = {state, 0};
|
||||
|
||||
auto insert_result = unique_blocks_to_request.insert(cell.index.block_index);
|
||||
bool was_inserted = insert_result.second;
|
||||
|
||||
if (was_inserted)
|
||||
blocks_to_request.emplace_back(cell.index.block_index);
|
||||
unique_blocks_to_request.insert(cell.index.block_index);
|
||||
}
|
||||
|
||||
break;
|
||||
}
|
||||
case Cell::default_value:
|
||||
@ -1038,8 +1057,6 @@ private:
|
||||
}
|
||||
}
|
||||
|
||||
result.found_keys_size = keys_size - (result.not_found_keys_size + result.expired_keys_size);
|
||||
|
||||
/// Sort blocks by offset before start async io requests
|
||||
std::sort(blocks_to_request.begin(), blocks_to_request.end());
|
||||
|
||||
@ -1052,8 +1069,7 @@ private:
|
||||
char * key_data = block_data + key_in_block.offset_in_block;
|
||||
deserializeAndInsertIntoColumns(result.fetched_columns, fetch_request, key_data);
|
||||
|
||||
KeyState::State state = key_in_block.is_expired ? KeyState::expired : KeyState::found;
|
||||
result.key_index_to_state[key_in_block.key_index] = {state, fetched_columns_index};
|
||||
result.key_index_to_state[key_in_block.key_index].setFetchedColumnIndex(fetched_columns_index);
|
||||
|
||||
++fetched_columns_index;
|
||||
}
|
||||
@ -1091,7 +1107,7 @@ private:
|
||||
throw Exception("Serialized columns size is greater than allowed block size and metadata", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
|
||||
/// We cannot reuse place that is already allocated in file or memory cache so we erase key from index
|
||||
index.erase(key);
|
||||
eraseKeyFromIndex(key);
|
||||
|
||||
Cell cell;
|
||||
setCellDeadline(cell, now);
|
||||
@ -1118,8 +1134,7 @@ private:
|
||||
|
||||
for (auto key : keys)
|
||||
{
|
||||
/// We cannot reuse place that is already allocated in file or memory cache so we erase key from index
|
||||
index.erase(key);
|
||||
eraseKeyFromIndex(key);
|
||||
|
||||
Cell cell;
|
||||
|
||||
@ -1139,7 +1154,7 @@ private:
|
||||
key = updated_key;
|
||||
}
|
||||
|
||||
index.insert(key, cell);
|
||||
index[key] = cell;
|
||||
}
|
||||
}
|
||||
|
||||
@ -1192,7 +1207,7 @@ private:
|
||||
cell.index = cache_index;
|
||||
cell.in_memory_partition_index = current_partition_index;
|
||||
|
||||
index.insert(ssd_cache_key.key, cell);
|
||||
index[ssd_cache_key.key] = cell;
|
||||
break;
|
||||
}
|
||||
else
|
||||
@ -1222,7 +1237,7 @@ private:
|
||||
if (old_key_cell.isOnDisk() &&
|
||||
old_key_block >= block_index_in_file_before_write &&
|
||||
old_key_block < file_read_end_block_index)
|
||||
index.erase(old_key);
|
||||
eraseKeyFromIndex(old_key);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1275,7 +1290,7 @@ private:
|
||||
cell.index = cache_index;
|
||||
cell.in_memory_partition_index = current_partition_index;
|
||||
|
||||
index.insert(ssd_cache_key.key, cell);
|
||||
index[ssd_cache_key.key] = cell;
|
||||
break;
|
||||
}
|
||||
else
|
||||
@ -1313,10 +1328,23 @@ private:
|
||||
size_t max_sec_lifetime = configuration.lifetime.max_sec;
|
||||
|
||||
std::uniform_int_distribution<UInt64> distribution{min_sec_lifetime, max_sec_lifetime};
|
||||
auto deadline = now + std::chrono::seconds{distribution(rnd_engine)};
|
||||
auto deadline = now + std::chrono::seconds(distribution(rnd_engine));
|
||||
cell.deadline = std::chrono::system_clock::to_time_t(deadline);
|
||||
}
|
||||
|
||||
inline void eraseKeyFromIndex(KeyType key)
|
||||
{
|
||||
auto it = index.find(key);
|
||||
|
||||
if (it == nullptr)
|
||||
return;
|
||||
|
||||
index.erase(key);
|
||||
|
||||
if constexpr(std::is_same_v<KeyType, StringRef>)
|
||||
complex_key_arena.free(const_cast<char *>(key.data), key.size);
|
||||
}
|
||||
|
||||
SSDCacheDictionaryStorageConfiguration configuration;
|
||||
|
||||
SSDCacheFileBuffer<SSDCacheKeyType> file_buffer;
|
||||
@ -1325,31 +1353,17 @@ private:
|
||||
|
||||
pcg64 rnd_engine;
|
||||
|
||||
class ArenaCellKeyDisposer
|
||||
{
|
||||
public:
|
||||
ArenaWithFreeLists & arena;
|
||||
using SimpleKeyHashMap = HashMap<UInt64, Cell>;
|
||||
using ComplexKeyHashMap = HashMapWithSavedHash<StringRef, Cell>;
|
||||
|
||||
template <typename Key, typename Value>
|
||||
void operator()(const Key & key, const Value &) const
|
||||
{
|
||||
/// In case of complex key we keep it in arena
|
||||
if constexpr (std::is_same_v<Key, StringRef>)
|
||||
arena.free(const_cast<char *>(key.data), key.size);
|
||||
}
|
||||
};
|
||||
|
||||
using SimpleKeyLRUHashMap = LRUHashMap<UInt64, Cell, ArenaCellKeyDisposer>;
|
||||
using ComplexKeyLRUHashMap = LRUHashMapWithSavedHash<StringRef, Cell, ArenaCellKeyDisposer>;
|
||||
|
||||
using CacheLRUHashMap = std::conditional_t<
|
||||
using CacheMap = std::conditional_t<
|
||||
dictionary_key_type == DictionaryKeyType::simple,
|
||||
SimpleKeyLRUHashMap,
|
||||
ComplexKeyLRUHashMap>;
|
||||
SimpleKeyHashMap,
|
||||
ComplexKeyHashMap>;
|
||||
|
||||
ArenaWithFreeLists complex_key_arena;
|
||||
|
||||
CacheLRUHashMap index;
|
||||
CacheMap index;
|
||||
|
||||
size_t current_partition_index = 0;
|
||||
|
||||
|
@ -26,7 +26,7 @@ CacheDictionaryStorageConfiguration parseCacheStorageConfiguration(
|
||||
const size_t size = config.getUInt64(dictionary_configuration_prefix + "size_in_cells");
|
||||
if (size == 0)
|
||||
throw Exception(ErrorCodes::TOO_SMALL_BUFFER_SIZE,
|
||||
"({}: cache dictionary cannot have 0 cells",
|
||||
"({}): cache dictionary cannot have 0 cells",
|
||||
full_name);
|
||||
|
||||
size_t dict_lifetime_seconds = static_cast<size_t>(dict_lifetime.max_sec);
|
||||
@ -59,7 +59,6 @@ SSDCacheDictionaryStorageConfiguration parseSSDCacheStorageConfiguration(
|
||||
static constexpr size_t DEFAULT_READ_BUFFER_SIZE_BYTES = 16 * DEFAULT_SSD_BLOCK_SIZE_BYTES;
|
||||
static constexpr size_t DEFAULT_WRITE_BUFFER_SIZE_BYTES = DEFAULT_SSD_BLOCK_SIZE_BYTES;
|
||||
|
||||
static constexpr size_t DEFAULT_MAX_STORED_KEYS = 100000;
|
||||
static constexpr size_t DEFAULT_PARTITIONS_COUNT = 16;
|
||||
|
||||
const size_t max_partitions_count
|
||||
@ -94,16 +93,11 @@ SSDCacheDictionaryStorageConfiguration parseSSDCacheStorageConfiguration(
|
||||
if (directory_path.at(0) != '/')
|
||||
directory_path = std::filesystem::path{config.getString("path")}.concat(directory_path).string();
|
||||
|
||||
const size_t max_stored_keys_in_partition
|
||||
= config.getInt64(dictionary_configuration_prefix + "max_stored_keys", DEFAULT_MAX_STORED_KEYS);
|
||||
const size_t rounded_size = roundUpToPowerOfTwoOrZero(max_stored_keys_in_partition);
|
||||
|
||||
SSDCacheDictionaryStorageConfiguration configuration{
|
||||
strict_max_lifetime_seconds,
|
||||
dict_lifetime,
|
||||
directory_path,
|
||||
max_partitions_count,
|
||||
rounded_size,
|
||||
block_size,
|
||||
file_size / block_size,
|
||||
read_buffer_size / block_size,
|
||||
|
@ -7,12 +7,12 @@ class Layout(object):
|
||||
'flat': '<flat/>',
|
||||
'hashed': '<hashed/>',
|
||||
'cache': '<cache><size_in_cells>128</size_in_cells></cache>',
|
||||
'ssd_cache': '<ssd_cache><path>/etc/clickhouse/dictionaries/all</path><max_stored_keys>128</max_stored_keys></ssd_cache>',
|
||||
'ssd_cache': '<ssd_cache><path>/etc/clickhouse/dictionaries/all</path></ssd_cache>',
|
||||
'complex_key_hashed': '<complex_key_hashed/>',
|
||||
'complex_key_hashed_one_key': '<complex_key_hashed/>',
|
||||
'complex_key_hashed_two_keys': '<complex_key_hashed/>',
|
||||
'complex_key_cache': '<complex_key_cache><size_in_cells>128</size_in_cells></complex_key_cache>',
|
||||
'complex_key_ssd_cache': '<complex_key_ssd_cache><path>/etc/clickhouse/dictionaries/all</path><max_stored_keys>128</max_stored_keys></complex_key_ssd_cache>',
|
||||
'complex_key_ssd_cache': '<complex_key_ssd_cache><path>/etc/clickhouse/dictionaries/all</path></complex_key_ssd_cache>',
|
||||
'range_hashed': '<range_hashed/>',
|
||||
'direct': '<direct/>',
|
||||
'complex_key_direct': '<complex_key_direct/>'
|
||||
|
@ -42,7 +42,6 @@
|
||||
<read_buffer_size>131072</read_buffer_size>
|
||||
<write_buffer_size>1048576</write_buffer_size>
|
||||
<path>/etc/clickhouse/dictionaries/radars</path>
|
||||
<max_stored_keys>1048576</max_stored_keys>
|
||||
</complex_key_ssd_cache>
|
||||
</layout>
|
||||
<lifetime>1</lifetime>
|
||||
|
@ -76,7 +76,7 @@ CREATE DICTIONARY 01053_db.ssd_dict
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01053_db'))
|
||||
LIFETIME(MIN 1000 MAX 2000)
|
||||
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/1d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 4096 MAX_STORED_KEYS 1000000));
|
||||
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/1d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 4096));
|
||||
|
||||
SELECT 'UPDATE DICTIONARY';
|
||||
-- 118
|
||||
@ -142,7 +142,7 @@ CREATE DICTIONARY 01053_db.ssd_dict
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01053_db'))
|
||||
LIFETIME(MIN 1000 MAX 2000)
|
||||
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/2d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 1024 MAX_STORED_KEYS 10));
|
||||
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/2d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 1024));
|
||||
|
||||
SELECT 'UPDATE DICTIONARY (MT)';
|
||||
-- 118
|
||||
|
@ -98,7 +98,7 @@ CREATE DICTIONARY 01280_db.ssd_dict
|
||||
PRIMARY KEY k1, k2
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01280_db'))
|
||||
LIFETIME(MIN 1000 MAX 2000)
|
||||
LAYOUT(COMPLEX_KEY_SSD_CACHE(FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/1d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 4096 MAX_STORED_KEYS 1000000));
|
||||
LAYOUT(COMPLEX_KEY_SSD_CACHE(FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/1d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 4096));
|
||||
|
||||
SELECT 'UPDATE DICTIONARY';
|
||||
-- 118
|
||||
|
Loading…
Reference in New Issue
Block a user