diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md
index efef91b4b09..6af22eb27dc 100644
--- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md
+++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md
@@ -320,8 +320,6 @@ Similar to `cache`, but stores data on SSD and index in RAM.
1048576
/var/lib/clickhouse/clickhouse_dictionaries/test_dict
-
- 1048576
```
@@ -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}
diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md
index 0fd4a85c46f..9b33a801973 100644
--- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md
+++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md
@@ -318,8 +318,6 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000))
1048576
/var/lib/clickhouse/clickhouse_dictionaries/test_dict
-
- 1048576
```
@@ -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}
diff --git a/src/Common/HashTable/LRUHashMap.h b/src/Common/HashTable/LRUHashMap.h
index bc5fd51d0e2..870fb219523 100644
--- a/src/Common/HashTable/LRUHashMap.h
+++ b/src/Common/HashTable/LRUHashMap.h
@@ -202,16 +202,6 @@ public:
return const_cast *>(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 *>(this)->findNoLRU(key);
- }
-
Value & ALWAYS_INLINE get(const Key & key)
{
auto it = find(key);
diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp
index 6c13f76132b..eedf4dd3d87 100644
--- a/src/Dictionaries/CacheDictionary.cpp
+++ b/src/Dictionaries/CacheDictionary.cpp
@@ -101,7 +101,7 @@ template
double CacheDictionary::getLoadFactor() const
{
const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs};
- return static_cast(cache_storage_ptr->getSize()) / cache_storage_ptr->getMaxSize();
+ return cache_storage_ptr->getLoadFactor();
}
template
@@ -332,16 +332,8 @@ Columns CacheDictionary::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);
}
diff --git a/src/Dictionaries/CacheDictionaryStorage.h b/src/Dictionaries/CacheDictionaryStorage.h
index 6b1200dd474..874796d879b 100644
--- a/src/Dictionaries/CacheDictionaryStorage.h
+++ b/src/Dictionaries/CacheDictionaryStorage.h
@@ -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(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)
{
diff --git a/src/Dictionaries/ICacheDictionaryStorage.h b/src/Dictionaries/ICacheDictionaryStorage.h
index 8a3202b5590..72b3ef76f11 100644
--- a/src/Dictionaries/ICacheDictionaryStorage.h
+++ b/src/Dictionaries/ICacheDictionaryStorage.h
@@ -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;
diff --git a/src/Dictionaries/SSDCacheDictionaryStorage.h b/src/Dictionaries/SSDCacheDictionaryStorage.h
index 5396846e383..f28f9ab37cd 100644
--- a/src/Dictionaries/SSDCacheDictionaryStorage.h
+++ b/src/Dictionaries/SSDCacheDictionaryStorage.h
@@ -17,7 +17,7 @@
#include
#include
#include
-#include
+#include
#include
#include
#include
@@ -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(block_data); }
+ inline size_t getCheckSum() const { return unalignedLoad(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(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(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(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
@@ -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(cell_is_expired);
+ result.found_keys_size += static_cast(!cell_is_expired);
switch (cell.state)
{
@@ -1016,13 +1029,19 @@ private:
case Cell::on_disk:
{
PaddedPODArray & 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 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)
+ complex_key_arena.free(const_cast(key.data), key.size);
+ }
+
SSDCacheDictionaryStorageConfiguration configuration;
SSDCacheFileBuffer file_buffer;
@@ -1325,31 +1353,17 @@ private:
pcg64 rnd_engine;
- class ArenaCellKeyDisposer
- {
- public:
- ArenaWithFreeLists & arena;
+ using SimpleKeyHashMap = HashMap;
+ using ComplexKeyHashMap = HashMapWithSavedHash;
- template
- void operator()(const Key & key, const Value &) const
- {
- /// In case of complex key we keep it in arena
- if constexpr (std::is_same_v)
- arena.free(const_cast(key.data), key.size);
- }
- };
-
- using SimpleKeyLRUHashMap = LRUHashMap;
- using ComplexKeyLRUHashMap = LRUHashMapWithSavedHash;
-
- 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;
diff --git a/src/Dictionaries/registerCacheDictionaries.cpp b/src/Dictionaries/registerCacheDictionaries.cpp
index 9f0f214e79b..b93a08acb76 100644
--- a/src/Dictionaries/registerCacheDictionaries.cpp
+++ b/src/Dictionaries/registerCacheDictionaries.cpp
@@ -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(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,
diff --git a/tests/integration/helpers/dictionary.py b/tests/integration/helpers/dictionary.py
index b3f7a729777..41d87180c8a 100644
--- a/tests/integration/helpers/dictionary.py
+++ b/tests/integration/helpers/dictionary.py
@@ -7,12 +7,12 @@ class Layout(object):
'flat': '',
'hashed': '',
'cache': '128',
- 'ssd_cache': '/etc/clickhouse/dictionaries/all128',
+ 'ssd_cache': '/etc/clickhouse/dictionaries/all',
'complex_key_hashed': '',
'complex_key_hashed_one_key': '',
'complex_key_hashed_two_keys': '',
'complex_key_cache': '128',
- 'complex_key_ssd_cache': '/etc/clickhouse/dictionaries/all128',
+ 'complex_key_ssd_cache': '/etc/clickhouse/dictionaries/all',
'range_hashed': '',
'direct': '',
'complex_key_direct': ''
diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml b/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml
index 85f811d2d85..c8fdbcbe0ef 100644
--- a/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml
+++ b/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/ssd_complex_key_cache_string.xml
@@ -42,7 +42,6 @@
131072
1048576
/etc/clickhouse/dictionaries/radars
- 1048576
1
diff --git a/tests/queries/0_stateless/01053_ssd_dictionary.sql b/tests/queries/0_stateless/01053_ssd_dictionary.sql
index a23ae7e5e96..23a369cc8a6 100644
--- a/tests/queries/0_stateless/01053_ssd_dictionary.sql
+++ b/tests/queries/0_stateless/01053_ssd_dictionary.sql
@@ -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
diff --git a/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sql b/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sql
index 50b34c4b18f..cd3e52c9691 100644
--- a/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sql
+++ b/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sql
@@ -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