mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Add test for existing dictionary
This commit is contained in:
parent
c3519ff376
commit
7edd80c9b7
@ -186,14 +186,20 @@ void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const Sto
|
|||||||
|
|
||||||
void DatabaseWithOwnTablesBase::attachDictionary(const String & dictionary_name, const Context & context, bool load)
|
void DatabaseWithOwnTablesBase::attachDictionary(const String & dictionary_name, const Context & context, bool load)
|
||||||
{
|
{
|
||||||
|
const auto & external_loader = context.getExternalDictionariesLoader();
|
||||||
|
|
||||||
|
String full_name = getDatabaseName() + "." + dictionary_name;
|
||||||
{
|
{
|
||||||
std::lock_guard lock(mutex);
|
std::lock_guard lock(mutex);
|
||||||
if (!dictionaries.emplace(dictionary_name).second)
|
auto status = external_loader.getCurrentStatus(full_name);
|
||||||
throw Exception("Dictionary " + name + "." + dictionary_name + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS);
|
if (status != ExternalLoader::Status::NOT_EXIST || !dictionaries.emplace(dictionary_name).second)
|
||||||
|
throw Exception(
|
||||||
|
"Dictionary " + full_name + " already exists.",
|
||||||
|
ErrorCodes::DICTIONARY_ALREADY_EXISTS);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (load)
|
if (load)
|
||||||
context.getExternalDictionariesLoader().reload(getDatabaseName() + "." + dictionary_name, true);
|
external_loader.reload(full_name, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
void DatabaseWithOwnTablesBase::shutdown()
|
void DatabaseWithOwnTablesBase::shutdown()
|
||||||
|
@ -12,6 +12,7 @@ SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
|||||||
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
|
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
|
||||||
node1 = cluster.add_instance('node1', with_mysql=True, main_configs=['configs/dictionaries/simple_dictionary.xml'])
|
node1 = cluster.add_instance('node1', with_mysql=True, main_configs=['configs/dictionaries/simple_dictionary.xml'])
|
||||||
node2 = cluster.add_instance('node2', with_mysql=True, main_configs=['configs/dictionaries/simple_dictionary.xml', 'configs/dictionaries/lazy_load.xml'])
|
node2 = cluster.add_instance('node2', with_mysql=True, main_configs=['configs/dictionaries/simple_dictionary.xml', 'configs/dictionaries/lazy_load.xml'])
|
||||||
|
node3 = cluster.add_instance('node3', main_configs=['configs/dictionaries/dictionary_with_conflict_name.xml'])
|
||||||
|
|
||||||
|
|
||||||
def create_mysql_conn(user, password, hostname, port):
|
def create_mysql_conn(user, password, hostname, port):
|
||||||
@ -33,7 +34,7 @@ def execute_mysql_query(connection, query):
|
|||||||
def started_cluster():
|
def started_cluster():
|
||||||
try:
|
try:
|
||||||
cluster.start()
|
cluster.start()
|
||||||
for clickhouse in [node1, node2]:
|
for clickhouse in [node1, node2, node3]:
|
||||||
clickhouse.query("CREATE DATABASE test", user="admin")
|
clickhouse.query("CREATE DATABASE test", user="admin")
|
||||||
clickhouse.query("CREATE TABLE test.xml_dictionary_table (id UInt64, SomeValue1 UInt8, SomeValue2 String) ENGINE = MergeTree() ORDER BY id", user="admin")
|
clickhouse.query("CREATE TABLE test.xml_dictionary_table (id UInt64, SomeValue1 UInt8, SomeValue2 String) ENGINE = MergeTree() ORDER BY id", user="admin")
|
||||||
clickhouse.query("INSERT INTO test.xml_dictionary_table SELECT number, number % 23, hex(number) from numbers(1000)", user="admin")
|
clickhouse.query("INSERT INTO test.xml_dictionary_table SELECT number, number % 23, hex(number) from numbers(1000)", user="admin")
|
||||||
@ -161,3 +162,23 @@ def test_restricted_database(started_cluster):
|
|||||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_in_restricted_db' DB 'restricted_db'))
|
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_in_restricted_db' DB 'restricted_db'))
|
||||||
LIFETIME(MIN 1 MAX 10)
|
LIFETIME(MIN 1 MAX 10)
|
||||||
""")
|
""")
|
||||||
|
|
||||||
|
|
||||||
|
def test_conflicting_name(started_cluster):
|
||||||
|
assert node3.query("select dictGetUInt8('test.conflicting_dictionary', 'SomeValue1', toUInt64(17))") == '17\n'
|
||||||
|
|
||||||
|
with pytest.raises(QueryRuntimeException):
|
||||||
|
node3.query("""
|
||||||
|
CREATE DICTIONARY test.conflicting_dictionary(
|
||||||
|
id UInt64,
|
||||||
|
SomeValue1 UInt8,
|
||||||
|
SomeValue2 String
|
||||||
|
)
|
||||||
|
PRIMARY KEY id
|
||||||
|
LAYOUT(FLAT())
|
||||||
|
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'xml_dictionary_table' DB 'test'))
|
||||||
|
LIFETIME(MIN 1 MAX 10)
|
||||||
|
""")
|
||||||
|
|
||||||
|
# old version still works
|
||||||
|
node3.query("select dictGetUInt8('test.conflicting_dictionary', 'SomeValue1', toUInt64(17))") == '17\n'
|
||||||
|
Loading…
Reference in New Issue
Block a user