mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
New tests and fixes for them
This commit is contained in:
parent
10f69eef03
commit
83dbfe9634
@ -102,13 +102,13 @@ void DatabaseDictionary::removeDictionary(
|
||||
}
|
||||
|
||||
void DatabaseDictionary::attachDictionary(
|
||||
const String & /*dictionary_name*/, const Context & /*context*/, bool /*load*/)
|
||||
const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/)
|
||||
{
|
||||
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
void DatabaseDictionary::detachDictionary(
|
||||
const String & /*dictionary_name*/, const Context & /*context*/)
|
||||
const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/)
|
||||
{
|
||||
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
@ -89,9 +89,9 @@ public:
|
||||
ASTPtr tryGetCreateDictionaryQuery(const Context & context, const String & table_name) const override;
|
||||
|
||||
|
||||
void attachDictionary(const String & dictionary_name, const Context & context, bool load) override;
|
||||
void attachDictionary(const String & dictionary_name, const Context & context, bool reload) override;
|
||||
|
||||
void detachDictionary(const String & dictionary_name, const Context & context) override;
|
||||
void detachDictionary(const String & dictionary_name, const Context & context, bool reload) override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
|
@ -129,7 +129,7 @@ void DatabaseLazy::attachDictionary(
|
||||
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
void DatabaseLazy::detachDictionary(const String & /*dictionary_name*/, const Context & /*context*/)
|
||||
void DatabaseLazy::detachDictionary(const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/)
|
||||
{
|
||||
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
@ -111,9 +111,9 @@ public:
|
||||
|
||||
StoragePtr detachTable(const String & table_name) override;
|
||||
|
||||
void attachDictionary(const String & dictionary_name, const Context & context, bool load) override;
|
||||
void attachDictionary(const String & dictionary_name, const Context & context, bool reload) override;
|
||||
|
||||
void detachDictionary(const String & dictionary_name, const Context & context) override;
|
||||
void detachDictionary(const String & dictionary_name, const Context & context, bool reload) override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
|
@ -35,7 +35,7 @@ void DatabaseMemory::createTable(
|
||||
}
|
||||
|
||||
|
||||
void DatabaseMemory::attachDictionary(const String & /*name*/, const Context & /*context*/, bool /*load*/)
|
||||
void DatabaseMemory::attachDictionary(const String & /*name*/, const Context & /*context*/, bool /*reload*/)
|
||||
{
|
||||
throw Exception("There is no ATTACH DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
@ -57,7 +57,7 @@ void DatabaseMemory::removeTable(
|
||||
}
|
||||
|
||||
|
||||
void DatabaseMemory::detachDictionary(const String & /*name*/, const Context & /*context*/)
|
||||
void DatabaseMemory::detachDictionary(const String & /*name*/, const Context & /*context*/, bool /*reload*/)
|
||||
{
|
||||
throw Exception("There is no DETACH DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
@ -41,7 +41,7 @@ public:
|
||||
void attachDictionary(
|
||||
const String & name,
|
||||
const Context & context,
|
||||
bool load) override;
|
||||
bool reload) override;
|
||||
|
||||
void removeTable(
|
||||
const Context & context,
|
||||
@ -53,7 +53,8 @@ public:
|
||||
|
||||
void detachDictionary(
|
||||
const String & name,
|
||||
const Context & context) override;
|
||||
const Context & context,
|
||||
bool reload) override;
|
||||
|
||||
time_t getObjectMetadataModificationTime(const Context & context, const String & table_name) override;
|
||||
|
||||
|
@ -64,7 +64,7 @@ public:
|
||||
throw Exception("MySQL database engine does not support detach table.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void detachDictionary(const String &, const Context &) override
|
||||
void detachDictionary(const String &, const Context &, bool) override
|
||||
{
|
||||
throw Exception("MySQL database engine does not support detach dictionary.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
@ -312,6 +312,7 @@ void DatabaseOnDisk::createDictionary(
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
database.detachDictionary(dictionary_name, context);
|
||||
Poco::File(dictionary_metadata_tmp_path).remove();
|
||||
throw;
|
||||
}
|
||||
|
@ -161,7 +161,7 @@ StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name)
|
||||
return res;
|
||||
}
|
||||
|
||||
void DatabaseWithOwnTablesBase::detachDictionary(const String & dictionary_name, const Context & context)
|
||||
void DatabaseWithOwnTablesBase::detachDictionary(const String & dictionary_name, const Context & context, bool reload)
|
||||
{
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
@ -171,7 +171,11 @@ void DatabaseWithOwnTablesBase::detachDictionary(const String & dictionary_name,
|
||||
dictionaries.erase(it);
|
||||
}
|
||||
|
||||
context.getExternalDictionariesLoader().reload(getDatabaseName() + "." + dictionary_name, true);
|
||||
if (reload)
|
||||
{
|
||||
bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true);
|
||||
context.getExternalDictionariesLoader().reload(getDatabaseName() + "." + dictionary_name, !lazy_load);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
@ -33,11 +33,11 @@ public:
|
||||
|
||||
void attachTable(const String & table_name, const StoragePtr & table) override;
|
||||
|
||||
void attachDictionary(const String & name, const Context & context, bool load) override;
|
||||
void attachDictionary(const String & name, const Context & context, bool reload) override;
|
||||
|
||||
StoragePtr detachTable(const String & table_name) override;
|
||||
|
||||
void detachDictionary(const String & name, const Context & context) override;
|
||||
void detachDictionary(const String & name, const Context & context, bool reload) override;
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
|
||||
|
||||
|
@ -165,13 +165,14 @@ public:
|
||||
virtual void attachTable(const String & name, const StoragePtr & table) = 0;
|
||||
|
||||
/// Add dictionary to the database, but do not add it to the metadata. The database may not support this method.
|
||||
virtual void attachDictionary(const String & name, const Context & context, bool load=true) = 0;
|
||||
/// load is false when we starting up and lazy_load is true, so we don't want to load dictionaries synchronously.
|
||||
virtual void attachDictionary(const String & name, const Context & context, bool reload = true) = 0;
|
||||
|
||||
/// Forget about the table without deleting it, and return it. The database may not support this method.
|
||||
virtual StoragePtr detachTable(const String & name) = 0;
|
||||
|
||||
/// Forget about the dictionary without deleting it, and return it. The database may not support this method.
|
||||
virtual void detachDictionary(const String & name, const Context & context) = 0;
|
||||
virtual void detachDictionary(const String & name, const Context & context, bool reload = true) = 0;
|
||||
|
||||
/// Rename the table and possibly move the table to another database.
|
||||
virtual void renameTable(
|
||||
|
@ -281,23 +281,25 @@ void buildPrimaryKeyConfiguration(
|
||||
/**
|
||||
* Transforms list of ASTDictionaryAttributeDeclarations to list of dictionary attributes
|
||||
*/
|
||||
void buildDictionaryAttributesConfiguration(
|
||||
std::unordered_set<std::string> buildDictionaryAttributesConfiguration(
|
||||
AutoPtr<Document> doc,
|
||||
AutoPtr<Element> root,
|
||||
const ASTExpressionList * dictionary_attributes,
|
||||
const Names & key_columns)
|
||||
{
|
||||
const auto & children = dictionary_attributes->children;
|
||||
std::unordered_set<std::string> dictionary_attributes_names;
|
||||
for (size_t i = 0; i < children.size(); ++i)
|
||||
{
|
||||
const ASTDictionaryAttributeDeclaration * dict_attr = children[i]->as<const ASTDictionaryAttributeDeclaration>();
|
||||
if (!dict_attr->type)
|
||||
throw Exception("Dictionary attribute must has type", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
dictionary_attributes_names.insert(dict_attr->name);
|
||||
if (std::find(key_columns.begin(), key_columns.end(), dict_attr->name) == key_columns.end())
|
||||
buildSingleAttribute(doc, root, dict_attr);
|
||||
|
||||
}
|
||||
return dictionary_attributes_names;
|
||||
}
|
||||
|
||||
/** Transform function with key-value arguments to configuration
|
||||
@ -371,23 +373,30 @@ void checkAST(const ASTCreateQuery & query)
|
||||
throw Exception("Cannot convert dictionary to configuration from non-dictionary AST.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
if (query.dictionary_attributes_list == nullptr || query.dictionary_attributes_list->children.empty())
|
||||
throw Exception("Dictionary AST missing attributes list.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
throw Exception("Cannot create dictionary with empty attributes list", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
if (query.dictionary->layout == nullptr)
|
||||
throw Exception("Cannot create dictionary with empty layout.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
throw Exception("Cannot create dictionary with empty layout", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
if (query.dictionary->lifetime == nullptr)
|
||||
throw Exception("Dictionary AST missing lifetime section", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
throw Exception("Cannot create dictionary with empty lifetime", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
if (query.dictionary->primary_key == nullptr)
|
||||
throw Exception("Dictionary AST missing primary key", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
throw Exception("Cannot create dictionary without primary key", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
if (query.dictionary->source == nullptr)
|
||||
throw Exception("Dictionary AST missing source", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
throw Exception("Cannot create dictionary with empty source", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
/// Range can be empty
|
||||
}
|
||||
|
||||
void checkPrimaryKey(const std::unordered_set<std::string> & all_attrs, const Names & key_attrs)
|
||||
{
|
||||
for (const auto & key_attr : key_attrs)
|
||||
if (all_attrs.count(key_attr) == 0)
|
||||
throw Exception("Unknown key attribute '" + key_attr + "'", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
@ -409,14 +418,15 @@ DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuer
|
||||
|
||||
AutoPtr<Element> structure_element(xml_document->createElement("structure"));
|
||||
current_dictionary->appendChild(structure_element);
|
||||
Names pk_columns = getPrimaryKeyColumns(query.dictionary->primary_key);
|
||||
Names pk_attrs = getPrimaryKeyColumns(query.dictionary->primary_key);
|
||||
auto dictionary_layout = query.dictionary->layout;
|
||||
|
||||
bool complex = DictionaryFactory::instance().isComplex(dictionary_layout->layout_type);
|
||||
|
||||
buildDictionaryAttributesConfiguration(xml_document, structure_element, query.dictionary_attributes_list, pk_columns);
|
||||
auto all_attr_names = buildDictionaryAttributesConfiguration(xml_document, structure_element, query.dictionary_attributes_list, pk_attrs);
|
||||
checkPrimaryKey(all_attr_names, pk_attrs);
|
||||
|
||||
buildPrimaryKeyConfiguration(xml_document, structure_element, complex, pk_columns, query.dictionary_attributes_list);
|
||||
buildPrimaryKeyConfiguration(xml_document, structure_element, complex, pk_attrs, query.dictionary_attributes_list);
|
||||
|
||||
buildLayoutConfiguration(xml_document, current_dictionary, dictionary_layout);
|
||||
buildSourceConfiguration(xml_document, current_dictionary, query.dictionary->source);
|
||||
|
@ -0,0 +1,7 @@
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
120
dbms/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.sh
Executable file
120
dbms/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.sh
Executable file
@ -0,0 +1,120 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP DICTIONARY IF EXISTS dict1"
|
||||
|
||||
# Simple layout, but with two keys
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
CREATE DICTIONARY dict1
|
||||
(
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY key1, key2
|
||||
LAYOUT(HASHED())
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
" 2>&1 | grep -c 'Primary key for simple dictionary must contain exactly one element'
|
||||
|
||||
|
||||
# Simple layout, but with non existing key
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
CREATE DICTIONARY dict1
|
||||
(
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY non_existing_column
|
||||
LAYOUT(HASHED())
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
" 2>&1 | grep -c "Unknown key attribute 'non_existing_column'"
|
||||
|
||||
# Complex layout, with non existing key
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
CREATE DICTIONARY dict1
|
||||
(
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY non_existing_column, key1
|
||||
LAYOUT(COMPLEX_KEY_HASHED())
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
" 2>&1 | grep -c "Unknown key attribute 'non_existing_column'"
|
||||
|
||||
# No layout
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
CREATE DICTIONARY dict1
|
||||
(
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY key2, key1
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
" 2>&1 | grep -c "Cannot create dictionary with empty layout"
|
||||
|
||||
# No PK
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
CREATE DICTIONARY dict1
|
||||
(
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
value String
|
||||
)
|
||||
LAYOUT(COMPLEX_KEY_HASHED())
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
" 2>&1 | grep -c "Cannot create dictionary without primary key"
|
||||
|
||||
# No lifetime
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
CREATE DICTIONARY dict1
|
||||
(
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY key2, key1
|
||||
LAYOUT(COMPLEX_KEY_HASHED())
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE'))
|
||||
" 2>&1 | grep -c "Cannot create dictionary with empty lifetime"
|
||||
|
||||
# No source
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
CREATE DICTIONARY dict1
|
||||
(
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY non_existing_column, key1
|
||||
LAYOUT(COMPLEX_KEY_HASHED())
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
" 2>&1 | grep -c "Cannot create dictionary with empty source"
|
||||
|
||||
|
||||
# Complex layout, but with one key
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
CREATE DICTIONARY dict1
|
||||
(
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY key1
|
||||
LAYOUT(COMPLEX_KEY_HASHED())
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
" || exit 1
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP DICTIONARY IF EXISTS dict1"
|
@ -66,6 +66,8 @@ SELECT '==DROP DICTIONARY';
|
||||
|
||||
DROP DICTIONARY IF EXISTS ordinary_db.dict1;
|
||||
|
||||
SYSTEM RELOAD DICTIONARY 'ordinary_db.dict1'; -- due to lazy_load at can persist for some time
|
||||
|
||||
SHOW DICTIONARIES FROM ordinary_db LIKE 'dict1';
|
||||
|
||||
EXISTS DICTIONARY ordinary_db.dict1;
|
||||
|
@ -41,6 +41,8 @@ SELECT count(distinct(dictGetUInt8('database_for_dict.dict1', 'second_column', t
|
||||
|
||||
DETACH DICTIONARY database_for_dict.dict1;
|
||||
|
||||
SYSTEM RELOAD DICTIONARY 'database_for_dict.dict1';
|
||||
|
||||
SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); -- {serverError 36}
|
||||
|
||||
ATTACH DICTIONARY database_for_dict.dict1;
|
||||
@ -49,6 +51,8 @@ SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11));
|
||||
|
||||
DROP DICTIONARY database_for_dict.dict1;
|
||||
|
||||
SYSTEM RELOAD DICTIONARY 'database_for_dict.dict1';
|
||||
|
||||
SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); -- {serverError 36}
|
||||
|
||||
CREATE DICTIONARY database_for_dict.dict1
|
||||
|
@ -0,0 +1,11 @@
|
||||
0.33
|
||||
0.42
|
||||
0.46
|
||||
0
|
||||
17501
|
||||
NP
|
||||
Moscow
|
||||
[3,2,1,10000]
|
||||
1
|
||||
1
|
||||
0
|
@ -0,0 +1,94 @@
|
||||
SET send_logs_level = 'none';
|
||||
|
||||
DROP DATABASE IF EXISTS database_for_dict;
|
||||
|
||||
CREATE DATABASE database_for_dict Engine = Ordinary;
|
||||
|
||||
CREATE TABLE database_for_dict.table_for_dict
|
||||
(
|
||||
CountryID UInt64,
|
||||
StartDate Date,
|
||||
EndDate Date,
|
||||
Tax Float64
|
||||
)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY CountryID;
|
||||
|
||||
INSERT INTO database_for_dict.table_for_dict VALUES(1, toDate('2019-05-05'), toDate('2019-05-20'), 0.33);
|
||||
INSERT INTO database_for_dict.table_for_dict VALUES(1, toDate('2019-05-21'), toDate('2019-05-30'), 0.42);
|
||||
INSERT INTO database_for_dict.table_for_dict VALUES(2, toDate('2019-05-21'), toDate('2019-05-30'), 0.46);
|
||||
|
||||
CREATE DICTIONARY database_for_dict.dict1
|
||||
(
|
||||
CountryID UInt64,
|
||||
StartDate Date,
|
||||
EndDate Date,
|
||||
Tax Float64
|
||||
)
|
||||
PRIMARY KEY CountryID
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' DB 'database_for_dict'))
|
||||
LIFETIME(MIN 1 MAX 1000)
|
||||
LAYOUT(RANGE_HASHED())
|
||||
RANGE(MIN StartDate MAX EndDate);
|
||||
|
||||
SELECT dictGetFloat64('database_for_dict.dict1', 'Tax', toUInt64(1), toDate('2019-05-15'));
|
||||
SELECT dictGetFloat64('database_for_dict.dict1', 'Tax', toUInt64(1), toDate('2019-05-29'));
|
||||
SELECT dictGetFloat64('database_for_dict.dict1', 'Tax', toUInt64(2), toDate('2019-05-29'));
|
||||
SELECT dictGetFloat64('database_for_dict.dict1', 'Tax', toUInt64(2), toDate('2019-05-31'));
|
||||
|
||||
CREATE TABLE database_for_dict.table_ip_trie
|
||||
(
|
||||
prefix String,
|
||||
asn UInt32,
|
||||
cca2 String
|
||||
)
|
||||
engine = TinyLog;
|
||||
|
||||
INSERT INTO database_for_dict.table_ip_trie VALUES ('202.79.32.0/20', 17501, 'NP'), ('2620:0:870::/48', 3856, 'US'), ('2a02:6b8:1::/48', 13238, 'RU'), ('2001:db8::/32', 65536, 'ZZ');
|
||||
|
||||
|
||||
CREATE DICTIONARY database_for_dict.dict_ip_trie
|
||||
(
|
||||
prefix String,
|
||||
asn UInt32,
|
||||
cca2 String
|
||||
)
|
||||
PRIMARY KEY prefix
|
||||
SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_ip_trie'))
|
||||
LAYOUT(IP_TRIE())
|
||||
LIFETIME(MIN 10 MAX 100);
|
||||
|
||||
SELECT dictGetUInt32('database_for_dict.dict_ip_trie', 'asn', tuple(IPv4StringToNum('202.79.32.0')));
|
||||
SELECT dictGetString('database_for_dict.dict_ip_trie', 'cca2', tuple(IPv4StringToNum('202.79.32.0')));
|
||||
|
||||
CREATE TABLE database_for_dict.table_with_hierarchy
|
||||
(
|
||||
RegionID UInt64,
|
||||
ParentRegionID UInt64,
|
||||
RegionName String
|
||||
)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY RegionID;
|
||||
|
||||
INSERT INTO database_for_dict.table_with_hierarchy VALUES (3, 2, 'Hamovniki'), (2, 1, 'Moscow'), (1, 10000, 'Russia') (7, 10000, 'Ulan-Ude');
|
||||
|
||||
|
||||
CREATE DICTIONARY database_for_dict.dictionary_with_hierarchy
|
||||
(
|
||||
RegionID UInt64,
|
||||
ParentRegionID UInt64 HIERARCHICAL,
|
||||
RegionName String
|
||||
)
|
||||
PRIMARY KEY RegionID
|
||||
SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_with_hierarchy'))
|
||||
LAYOUT(HASHED())
|
||||
LIFETIME(MIN 1 MAX 1000);
|
||||
|
||||
SELECT dictGetString('database_for_dict.dictionary_with_hierarchy', 'RegionName', toUInt64(2));
|
||||
SELECT dictGetHierarchy('database_for_dict.dictionary_with_hierarchy', toUInt64(3));
|
||||
SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(3), toUInt64(2));
|
||||
SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(7), toUInt64(10000));
|
||||
SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(1), toUInt64(5));
|
||||
|
||||
DROP DATABASE IF EXISTS database_for_dict;
|
||||
|
Loading…
Reference in New Issue
Block a user