未验证 提交 bbe1960e 编写于 作者: M Maksim Kita 提交者: GitHub

Merge pull request #21573 from kitaisreal/updated-cache-dictionary-storage

Added specialized CacheDictionaryStorage
......@@ -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}
......
......@@ -271,13 +271,13 @@ private:
};
template <typename Key, typename Mapped>
struct DefaultCellDisposer
struct DefaultLRUHashMapCellDisposer
{
void operator()(const Key &, const Mapped &) const {}
};
template <typename Key, typename Value, typename Disposer = DefaultCellDisposer<Key, Value>, typename Hash = DefaultHash<Key>>
template <typename Key, typename Value, typename Disposer = DefaultLRUHashMapCellDisposer<Key, Value>, typename Hash = DefaultHash<Key>>
using LRUHashMap = LRUHashMapImpl<Key, Value, Disposer, Hash, false>;
template <typename Key, typename Value, typename Disposer = DefaultCellDisposer<Key, Value>, typename Hash = DefaultHash<Key>>
template <typename Key, typename Value, typename Disposer = DefaultLRUHashMapCellDisposer<Key, Value>, typename Hash = DefaultHash<Key>>
using LRUHashMapWithSavedHash = LRUHashMapImpl<Key, Value, Disposer, Hash, true>;
......@@ -692,6 +692,30 @@ public:
assign(from.begin(), from.end());
}
void erase(const_iterator first, const_iterator last)
{
iterator first_no_const = const_cast<iterator>(first);
iterator last_no_const = const_cast<iterator>(last);
size_t items_to_move = end() - last;
while (items_to_move != 0)
{
*first_no_const = *last_no_const;
++first_no_const;
++last_no_const;
--items_to_move;
}
this->c_end = reinterpret_cast<char *>(first_no_const);
}
void erase(const_iterator pos)
{
this->erase(pos, pos + 1);
}
bool operator== (const PODArray & rhs) const
{
......
......@@ -92,3 +92,57 @@ TEST(Common, PODInsertElementSizeNotMultipleOfLeftPadding)
EXPECT_EQ(arr1_initially_empty.size(), items_to_insert_size);
}
TEST(Common, PODErase)
{
{
PaddedPODArray<UInt64> items {0,1,2,3,4,5,6,7,8,9};
PaddedPODArray<UInt64> expected;
expected = {0,1,2,3,4,5,6,7,8,9};
items.erase(items.begin(), items.begin());
EXPECT_EQ(items, expected);
items.erase(items.end(), items.end());
EXPECT_EQ(items, expected);
}
{
PaddedPODArray<UInt64> actual {0,1,2,3,4,5,6,7,8,9};
PaddedPODArray<UInt64> expected;
expected = {0,1,4,5,6,7,8,9};
actual.erase(actual.begin() + 2, actual.begin() + 4);
EXPECT_EQ(actual, expected);
expected = {0,1,4};
actual.erase(actual.begin() + 3, actual.end());
EXPECT_EQ(actual, expected);
expected = {};
actual.erase(actual.begin(), actual.end());
EXPECT_EQ(actual, expected);
for (size_t i = 0; i < 10; ++i)
actual.emplace_back(static_cast<UInt64>(i));
expected = {0,1,4,5,6,7,8,9};
actual.erase(actual.begin() + 2, actual.begin() + 4);
EXPECT_EQ(actual, expected);
expected = {0,1,4};
actual.erase(actual.begin() + 3, actual.end());
EXPECT_EQ(actual, expected);
expected = {};
actual.erase(actual.begin(), actual.end());
EXPECT_EQ(actual, expected);
}
{
PaddedPODArray<UInt64> actual {0,1,2,3,4,5,6,7,8,9};
PaddedPODArray<UInt64> expected;
expected = {1,2,3,4,5,6,7,8,9};
actual.erase(actual.begin());
EXPECT_EQ(actual, expected);
}
}
......@@ -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>
......@@ -333,9 +333,7 @@ Columns CacheDictionary<dictionary_key_type>::getColumnsImpl(
FetchResult result_of_fetch_from_storage;
{
/// Write lock on storage
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);
}
......
......@@ -12,9 +12,9 @@ struct KeyState
{
enum State: uint8_t
{
not_found = 2,
expired = 4,
found = 8,
not_found = 0,
expired = 1,
found = 2,
};
KeyState(State state_, size_t fetched_column_index_)
......@@ -31,9 +31,10 @@ struct KeyState
inline bool isNotFound() const { return state == State::not_found; }
inline bool isDefault() const { return is_default; }
inline void setDefault() { is_default = true; }
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;
......@@ -111,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;
......
clickhouse-client --query="DROP TABLE IF EXISTS simple_cache_dictionary_table_source";
clickhouse-client --query="CREATE TABLE simple_cache_dictionary_table_source (id UInt64, value1 String, value2 UInt64, value3 String, value4 Float64, value5 Decimal64(4)) ENGINE=TinyLog;"
clickhouse-client --query="INSERT INTO simple_cache_dictionary_table_source SELECT number, concat('Value1 ', toString(number)), number, concat('Value3 ', toString(number)), toFloat64(number), cast(number, 'Decimal64(4)') FROM system.numbers LIMIT 1000000;"
clickhouse-client --multiquery --query="CREATE DICTIONARY clickhouse_simple_cache_dictionary (
id UInt64,
value1 String,
value2 UInt64,
value3 String,
value4 Float64,
value5 Decimal64(4)
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_cache_dictionary_table_source' PASSWORD '' DB 'default'))
LIFETIME(MIN 300 MAX 300)
LAYOUT(CACHE(SIZE_IN_CELLS 100000));"
clickhouse-client --multiquery --query="CREATE DICTIONARY clickhouse_ssd_simple_cache_dictionary (
id UInt64,
value1 String,
value2 UInt64,
value3 String,
value4 Float64,
value5 Decimal64(4)
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_cache_dictionary_table_source' PASSWORD '' DB 'default'))
LIFETIME(MIN 300 MAX 300)
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576 WRITE_BUFFER_SIZE 327680 MAX_STORED_KEYS 1048576 PATH '/opt/mkita/ClickHouse/build_release/programs/ssd_cache'));"
clickhouse-client --multiquery --query="CREATE DICTIONARY clickhouse_dummy_simple_cache_dictionary (
id UInt64,
value1 String,
value2 UInt64,
value3 String,
value4 Float64,
value5 Decimal64(4)
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_cache_dictionary_table_source' PASSWORD '' DB 'default'))
LIFETIME(MIN 300 MAX 300)
LAYOUT(DUMMY_SIMPLE());"
./clickhouse-benchmark --query="SELECT
dictGet('default.clickhouse_dummy_simple_cache_dictionary', 'value1', number),
dictGet('default.clickhouse_dummy_simple_cache_dictionary', 'value2', number),
dictGet('default.clickhouse_dummy_simple_cache_dictionary', 'value3', number),
dictGet('default.clickhouse_dummy_simple_cache_dictionary', 'value4', number),
dictGet('default.clickhouse_dummy_simple_cache_dictionary', 'value5', number)
FROM system.numbers
LIMIT 10000
FORMAT Null"
./clickhouse-benchmark --query="SELECT
dictGet('default.clickhouse_simple_cache_dictionary', ('value1', 'value2', 'value3', 'value4', 'value5'), number)
FROM system.numbers
LIMIT 10000
FORMAT Null"
./clickhouse-benchmark --query="SELECT dictGet('default.clickhouse_ssd_simple_cache_dictionary', 'value1', number) FROM system.numbers_mt LIMIT 10000 FORMAT Null"
./clickhouse-benchmark --query="SELECT
dictGet('default.clickhouse_simple_cache_dictionary', 'value1', number),
dictGet('default.clickhouse_simple_cache_dictionary', 'value2', number),
dictGet('default.clickhouse_simple_cache_dictionary', 'value3', number),
dictGet('default.clickhouse_simple_cache_dictionary', 'value4', number),
dictGet('default.clickhouse_simple_cache_dictionary', 'value5', number)
FROM system.numbers
LIMIT 10000
FORMAT Null"
./clickhouse-benchmark --query="SELECT dictGet('default.clickhouse_ssd_simple_cache_dictionary', 'value1', number) FROM system.numbers_mt LIMIT 10000 FORMAT Null"
SELECT
dictGet('default.clickhouse_ssd_simple_cache_dictionary', 'value1', number),
dictGet('default.clickhouse_ssd_simple_cache_dictionary', 'value2', number),
dictGet('default.clickhouse_ssd_simple_cache_dictionary', 'value3', number),
dictGet('default.clickhouse_ssd_simple_cache_dictionary', 'value4', number),
dictGet('default.clickhouse_ssd_simple_cache_dictionary', 'value5', number)
FROM system.numbers
LIMIT 10000
FORMAT Null
SELECT dictGet('default.clickhouse_simple_cache_dictionary', ('value1', 'value2', 'value3', 'value4', 'value5'), number) FROM system.numbers LIMIT 10000 FORMAT Null
SELECT dictGet('default.clickhouse_ssd_simple_cache_dictionary', ('value1', 'value2', 'value3', 'value4', 'value5'), number) FROM system.numbers LIMIT 10000
FORMAT Null
SELECT
dictGet('default.clickhouse_simple_cache_dictionary', ('value1', 'value2', 'value3', 'value4', 'value5'), number)
FROM system.numbers
LIMIT 10000
FORMAT
Null
SELECT
dictGet('default.clickhouse_simple_cache_dictionary', 'value1', number),
dictGet('default.clickhouse_simple_cache_dictionary', 'value2', number),
dictGet('default.clickhouse_simple_cache_dictionary', 'value3', number),
dictGet('default.clickhouse_simple_cache_dictionary', 'value4', number),
dictGet('default.clickhouse_simple_cache_dictionary', 'value5', number)
FROM system.numbers
LIMIT 10000
FORMAT
Null
SELECT
dictGet('default.clickhouse_simple_cache_dictionary', 'value1', number),
dictGet('default.clickhouse_simple_cache_dictionary', 'value2', number)
FROM system.numbers
LIMIT 10000
FORMAT Null
SELECT
dictGet('clickhouse_simple_cache_dictionary', 'value1', number)
FROM system.numbers
LIMIT 100000
FORMAT Null
SELECT
dictGet('clickhouse_simple_cache_dictionary', 'value2', number)
FROM system.numbers
LIMIT 100000
FORMAT Null
SELECT
dictGet('clickhouse_simple_cache_dictionary', 'value3', number)
FROM system.numbers
LIMIT 100000
FORMAT Null
SELECT
dictGet('clickhouse_simple_cache_dictionary', 'value4', number)
FROM system.numbers
LIMIT 100000
FORMAT Null
SELECT
dictGet('clickhouse_simple_cache_dictionary', 'value5', number)
FROM system.numbers
LIMIT 100000
FORMAT Null
SELECT
dictGet('clickhouse_simple_cache_dictionary', 'value1', number),
dictGet('clickhouse_simple_cache_dictionary', 'value2', number),
dictGet('clickhouse_simple_cache_dictionary', 'value3', number),
dictGet('clickhouse_simple_cache_dictionary', 'value4', number),
dictGet('clickhouse_simple_cache_dictionary', 'value5', number)
FROM system.numbers
LIMIT 100000
FORMAT Null
SELECT * FROM clickhouse_simple_cache_dictionary_table;
\ No newline at end of file
#include "CacheDictionary.h"
#include "SSDCacheDictionaryStorage.h"
#include "CacheDictionaryStorage.h"
#include "SSDCacheDictionaryStorage.h"
#include <Dictionaries/DictionaryFactory.h>
namespace DB
......@@ -20,13 +20,13 @@ CacheDictionaryStorageConfiguration parseCacheStorageConfiguration(
const DictionaryLifetime & dict_lifetime,
DictionaryKeyType dictionary_key_type)
{
String dictionary_type_prefix = dictionary_key_type == DictionaryKeyType::complex ? ".complex_key_cache." : ".cache.";
String dictionary_type_prefix = (dictionary_key_type == DictionaryKeyType::complex) ? ".complex_key_cache." : ".cache.";
String dictionary_configuration_prefix = layout_prefix + dictionary_type_prefix;
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,
......@@ -194,7 +188,8 @@ DictionaryPtr createCacheDictionaryLayout(
const bool allow_read_expired_keys = config.getBool(layout_prefix + ".cache.allow_read_expired_keys", false);
auto storage_configuration = parseCacheStorageConfiguration(full_name, config, layout_prefix, dict_lifetime, dictionary_key_type);
auto storage = std::make_shared<CacheDictionaryStorage<dictionary_key_type>>(storage_configuration);
std::shared_ptr<ICacheDictionaryStorage> storage = std::make_shared<CacheDictionaryStorage<dictionary_key_type>>(dict_struct, storage_configuration);
auto update_queue_configuration = parseCacheDictionaryUpdateQueueConfiguration(full_name, config, layout_prefix, dictionary_key_type);
......
......@@ -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
......
......@@ -40,7 +40,7 @@ SELECT dictGetOrDefault('01681_database_for_cache_dictionary.cache_dictionary_si
SELECT 'dictHas';
SELECT dictHas('01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
SELECT 'select all values as input stream';
SELECT * FROM 01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes;
SELECT * FROM 01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes ORDER BY id;
DROP DICTIONARY 01681_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes;
DROP TABLE 01681_database_for_cache_dictionary.simple_key_simple_attributes_source_table;
......@@ -84,7 +84,7 @@ SELECT dictGetOrDefault('01681_database_for_cache_dictionary.cache_dictionary_si
SELECT 'dictHas';
SELECT dictHas('01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
SELECT 'select all values as input stream';
SELECT * FROM 01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes;
SELECT * FROM 01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes ORDER BY id;
DROP DICTIONARY 01681_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes;
DROP TABLE 01681_database_for_cache_dictionary.simple_key_complex_attributes_source_table;
......
......@@ -42,7 +42,7 @@ SELECT dictGetOrDefault('01682_database_for_cache_dictionary.cache_dictionary_co
SELECT 'dictHas';
SELECT dictHas('01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
SELECT 'select all values as input stream';
SELECT * FROM 01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes;
SELECT * FROM 01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes ORDER BY id;
DROP DICTIONARY 01682_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes;
DROP TABLE 01682_database_for_cache_dictionary.complex_key_simple_attributes_source_table;
......@@ -89,7 +89,7 @@ SELECT dictGetOrDefault('01682_database_for_cache_dictionary.cache_dictionary_co
SELECT 'dictHas';
SELECT dictHas('01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
SELECT 'select all values as input stream';
SELECT * FROM 01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes;
SELECT * FROM 01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes ORDER BY id;
DROP DICTIONARY 01682_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes;
DROP TABLE 01682_database_for_cache_dictionary.complex_key_complex_attributes_source_table;
......
......@@ -40,7 +40,7 @@ SELECT dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_si
SELECT 'dictHas';
SELECT dictHas('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
SELECT 'select all values as input stream';
SELECT * FROM 01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes;
SELECT * FROM 01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes ORDER BY id;
DROP DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes;
DROP TABLE 01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table;
......@@ -84,7 +84,7 @@ SELECT dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_si
SELECT 'dictHas';
SELECT dictHas('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
SELECT 'select all values as input stream';
SELECT * FROM 01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes;
SELECT * FROM 01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes ORDER BY id;
DROP DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes;
DROP TABLE 01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table;
......
......@@ -42,7 +42,7 @@ SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_co
SELECT 'dictHas';
SELECT dictHas('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
SELECT 'select all values as input stream';
SELECT * FROM 01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes;
SELECT * FROM 01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes ORDER BY id;
DROP DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes;
DROP TABLE 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table;
......@@ -89,10 +89,10 @@ SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_co
SELECT 'dictHas';
SELECT dictHas('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
SELECT 'select all values as input stream';
SELECT * FROM 01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes;
SELECT * FROM 01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes ORDER BY id;
DROP DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes;
DROP TABLE 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table;
DROP DATABASE 01685_database_for_cache_dictionary;
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册