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