2019-10-18 15:44:32 +00:00
|
|
|
import os
|
|
|
|
import warnings
|
2020-09-15 15:42:46 +00:00
|
|
|
import time
|
2019-10-18 15:44:32 +00:00
|
|
|
|
2020-09-16 04:26:10 +00:00
|
|
|
import pymysql
|
|
|
|
import pytest
|
2021-02-15 09:35:45 +00:00
|
|
|
import logging
|
2020-09-16 04:26:10 +00:00
|
|
|
from helpers.client import QueryRuntimeException
|
|
|
|
from helpers.cluster import ClickHouseCluster
|
|
|
|
|
2019-10-18 15:44:32 +00:00
|
|
|
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
|
|
|
|
2020-08-12 08:55:04 +00:00
|
|
|
cluster = ClickHouseCluster(__file__)
|
2022-03-22 16:39:58 +00:00
|
|
|
node1 = cluster.add_instance(
|
|
|
|
"node1",
|
|
|
|
with_mysql=True,
|
|
|
|
dictionaries=["configs/dictionaries/simple_dictionary.xml"],
|
|
|
|
user_configs=["configs/user_admin.xml", "configs/user_default.xml"],
|
|
|
|
)
|
|
|
|
node2 = cluster.add_instance(
|
|
|
|
"node2",
|
|
|
|
with_mysql=True,
|
|
|
|
dictionaries=["configs/dictionaries/simple_dictionary.xml"],
|
|
|
|
main_configs=[
|
|
|
|
"configs/dictionaries/lazy_load.xml",
|
|
|
|
"configs/allow_remote_node.xml",
|
|
|
|
],
|
|
|
|
user_configs=["configs/user_admin.xml", "configs/user_default.xml"],
|
|
|
|
)
|
|
|
|
node3 = cluster.add_instance(
|
|
|
|
"node3",
|
|
|
|
main_configs=["configs/allow_remote_node.xml"],
|
|
|
|
dictionaries=[
|
|
|
|
"configs/dictionaries/dictionary_with_conflict_name.xml",
|
|
|
|
"configs/dictionaries/conflict_name_dictionary.xml",
|
|
|
|
],
|
|
|
|
user_configs=["configs/user_admin.xml"],
|
|
|
|
)
|
|
|
|
node4 = cluster.add_instance(
|
|
|
|
"node4", user_configs=["configs/user_admin.xml", "configs/config_password.xml"]
|
|
|
|
)
|
2019-10-18 15:44:32 +00:00
|
|
|
|
|
|
|
|
|
|
|
def create_mysql_conn(user, password, hostname, port):
|
2022-03-22 16:39:58 +00:00
|
|
|
logging.debug(
|
|
|
|
"Created MySQL connection user:{}, password:{}, host:{}, port{}".format(
|
|
|
|
user, password, hostname, port
|
|
|
|
)
|
|
|
|
)
|
|
|
|
return pymysql.connect(user=user, password=password, host=hostname, port=port)
|
2019-10-18 15:44:32 +00:00
|
|
|
|
2020-09-16 04:26:10 +00:00
|
|
|
|
2019-10-18 15:44:32 +00:00
|
|
|
def execute_mysql_query(connection, query):
|
2021-02-15 09:35:45 +00:00
|
|
|
logging.debug("Execute MySQL query:{}".format(query))
|
2019-10-18 15:44:32 +00:00
|
|
|
with warnings.catch_warnings():
|
|
|
|
warnings.simplefilter("ignore")
|
|
|
|
with connection.cursor() as cursor:
|
|
|
|
cursor.execute(query)
|
|
|
|
connection.commit()
|
|
|
|
|
|
|
|
|
|
|
|
@pytest.fixture(scope="module")
|
|
|
|
def started_cluster():
|
|
|
|
try:
|
|
|
|
cluster.start()
|
2020-07-29 05:57:46 +00:00
|
|
|
for clickhouse in [node1, node2, node3, node4]:
|
2019-10-18 15:44:32 +00:00
|
|
|
clickhouse.query("CREATE DATABASE test", user="admin")
|
2020-09-16 04:26:10 +00:00
|
|
|
clickhouse.query(
|
|
|
|
"CREATE TABLE test.xml_dictionary_table (id UInt64, SomeValue1 UInt8, SomeValue2 String) ENGINE = MergeTree() ORDER BY id",
|
2022-03-22 16:39:58 +00:00
|
|
|
user="admin",
|
|
|
|
)
|
2020-09-16 04:26:10 +00:00
|
|
|
clickhouse.query(
|
|
|
|
"INSERT INTO test.xml_dictionary_table SELECT number, number % 23, hex(number) from numbers(1000)",
|
2022-03-22 16:39:58 +00:00
|
|
|
user="admin",
|
|
|
|
)
|
2019-10-18 15:44:32 +00:00
|
|
|
yield cluster
|
|
|
|
|
|
|
|
finally:
|
|
|
|
cluster.shutdown()
|
|
|
|
|
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
@pytest.mark.parametrize(
|
|
|
|
"clickhouse,name,layout",
|
|
|
|
[
|
|
|
|
pytest.param(
|
|
|
|
node1,
|
|
|
|
"complex_node1_hashed",
|
|
|
|
"LAYOUT(COMPLEX_KEY_HASHED())",
|
|
|
|
id="complex_node1_hashed",
|
|
|
|
),
|
|
|
|
pytest.param(
|
|
|
|
node1,
|
|
|
|
"complex_node1_cache",
|
|
|
|
"LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))",
|
|
|
|
id="complex_node1_cache",
|
|
|
|
),
|
|
|
|
pytest.param(
|
|
|
|
node2,
|
|
|
|
"complex_node2_hashed",
|
|
|
|
"LAYOUT(COMPLEX_KEY_HASHED())",
|
|
|
|
id="complex_node2_hashed",
|
|
|
|
),
|
|
|
|
pytest.param(
|
|
|
|
node2,
|
|
|
|
"complex_node2_cache",
|
|
|
|
"LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))",
|
|
|
|
id="complex_node2_cache",
|
|
|
|
),
|
|
|
|
],
|
|
|
|
)
|
2020-08-12 08:55:04 +00:00
|
|
|
def test_create_and_select_mysql(started_cluster, clickhouse, name, layout):
|
2022-03-22 16:39:58 +00:00
|
|
|
mysql_conn = create_mysql_conn(
|
|
|
|
"root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port
|
|
|
|
)
|
2021-10-01 05:44:50 +00:00
|
|
|
execute_mysql_query(mysql_conn, "DROP DATABASE IF EXISTS create_and_select")
|
|
|
|
execute_mysql_query(mysql_conn, "CREATE DATABASE create_and_select")
|
2022-03-22 16:39:58 +00:00
|
|
|
execute_mysql_query(
|
|
|
|
mysql_conn,
|
|
|
|
"CREATE TABLE create_and_select.{} (key_field1 int, key_field2 bigint, value1 text, value2 float, PRIMARY KEY (key_field1, key_field2))".format(
|
|
|
|
name
|
|
|
|
),
|
|
|
|
)
|
2019-10-18 15:44:32 +00:00
|
|
|
values = []
|
|
|
|
for i in range(1000):
|
2022-03-22 16:39:58 +00:00
|
|
|
values.append(
|
|
|
|
"(" + ",".join([str(i), str(i * i), str(i) * 5, str(i * 3.14)]) + ")"
|
|
|
|
)
|
|
|
|
execute_mysql_query(
|
|
|
|
mysql_conn,
|
|
|
|
"INSERT INTO create_and_select.{} VALUES ".format(name) + ",".join(values),
|
|
|
|
)
|
2019-10-18 15:44:32 +00:00
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
clickhouse.query(
|
|
|
|
"""
|
2019-10-18 15:44:32 +00:00
|
|
|
CREATE DICTIONARY default.{} (
|
|
|
|
key_field1 Int32,
|
|
|
|
key_field2 Int64,
|
|
|
|
value1 String DEFAULT 'xxx',
|
2021-11-22 12:21:15 +00:00
|
|
|
value2 Float32 DEFAULT '42.42'
|
2019-10-18 15:44:32 +00:00
|
|
|
)
|
|
|
|
PRIMARY KEY key_field1, key_field2
|
|
|
|
SOURCE(MYSQL(
|
|
|
|
USER 'root'
|
|
|
|
PASSWORD 'clickhouse'
|
2021-10-01 05:44:50 +00:00
|
|
|
DB 'create_and_select'
|
2019-10-18 15:44:32 +00:00
|
|
|
TABLE '{}'
|
|
|
|
REPLICA(PRIORITY 1 HOST '127.0.0.1' PORT 3333)
|
2021-02-15 09:35:45 +00:00
|
|
|
REPLICA(PRIORITY 2 HOST 'mysql57' PORT 3306)
|
2019-10-18 15:44:32 +00:00
|
|
|
))
|
|
|
|
{}
|
|
|
|
LIFETIME(MIN 1 MAX 3)
|
2022-03-22 16:39:58 +00:00
|
|
|
""".format(
|
|
|
|
name, name, layout
|
|
|
|
)
|
|
|
|
)
|
2019-10-18 15:44:32 +00:00
|
|
|
|
|
|
|
for i in range(172, 200):
|
2022-03-22 16:39:58 +00:00
|
|
|
assert (
|
|
|
|
clickhouse.query(
|
|
|
|
"SELECT dictGetString('default.{}', 'value1', tuple(toInt32({}), toInt64({})))".format(
|
|
|
|
name, i, i * i
|
|
|
|
)
|
|
|
|
)
|
|
|
|
== str(i) * 5 + "\n"
|
|
|
|
)
|
2020-09-16 04:26:10 +00:00
|
|
|
stroka = clickhouse.query(
|
2022-03-22 16:39:58 +00:00
|
|
|
"SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(
|
|
|
|
name, i, i * i
|
|
|
|
)
|
|
|
|
).strip()
|
2019-10-18 15:44:32 +00:00
|
|
|
value = float(stroka)
|
|
|
|
assert int(value) == int(i * 3.14)
|
|
|
|
|
|
|
|
for i in range(1000):
|
2022-03-22 16:39:58 +00:00
|
|
|
values.append(
|
|
|
|
"(" + ",".join([str(i), str(i * i), str(i) * 3, str(i * 2.718)]) + ")"
|
|
|
|
)
|
|
|
|
execute_mysql_query(
|
|
|
|
mysql_conn,
|
|
|
|
"REPLACE INTO create_and_select.{} VALUES ".format(name) + ",".join(values),
|
|
|
|
)
|
2019-10-18 15:44:32 +00:00
|
|
|
|
|
|
|
clickhouse.query("SYSTEM RELOAD DICTIONARY 'default.{}'".format(name))
|
|
|
|
|
|
|
|
for i in range(172, 200):
|
2022-03-22 16:39:58 +00:00
|
|
|
assert (
|
|
|
|
clickhouse.query(
|
|
|
|
"SELECT dictGetString('default.{}', 'value1', tuple(toInt32({}), toInt64({})))".format(
|
|
|
|
name, i, i * i
|
|
|
|
)
|
|
|
|
)
|
|
|
|
== str(i) * 3 + "\n"
|
|
|
|
)
|
2020-09-16 04:26:10 +00:00
|
|
|
string = clickhouse.query(
|
2022-03-22 16:39:58 +00:00
|
|
|
"SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(
|
|
|
|
name, i, i * i
|
|
|
|
)
|
|
|
|
).strip()
|
2020-08-12 08:55:04 +00:00
|
|
|
value = float(string)
|
2019-10-18 15:44:32 +00:00
|
|
|
assert int(value) == int(i * 2.718)
|
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
clickhouse.query(
|
|
|
|
"select dictGetUInt8('xml_dictionary', 'SomeValue1', toUInt64(17))"
|
|
|
|
) == "17\n"
|
|
|
|
clickhouse.query(
|
|
|
|
"select dictGetString('xml_dictionary', 'SomeValue2', toUInt64(977))"
|
|
|
|
) == str(hex(977))[2:] + "\n"
|
2021-10-01 05:44:50 +00:00
|
|
|
clickhouse.query(f"drop dictionary default.{name}")
|
2019-10-18 15:44:32 +00:00
|
|
|
|
|
|
|
|
|
|
|
def test_restricted_database(started_cluster):
|
|
|
|
for node in [node1, node2]:
|
|
|
|
node.query("CREATE DATABASE IF NOT EXISTS restricted_db", user="admin")
|
2022-03-22 16:39:58 +00:00
|
|
|
node.query(
|
|
|
|
"CREATE TABLE restricted_db.table_in_restricted_db AS test.xml_dictionary_table",
|
|
|
|
user="admin",
|
|
|
|
)
|
2019-10-18 15:44:32 +00:00
|
|
|
|
|
|
|
with pytest.raises(QueryRuntimeException):
|
2022-03-22 16:39:58 +00:00
|
|
|
node1.query(
|
|
|
|
"""
|
2019-10-18 15:44:32 +00:00
|
|
|
CREATE DICTIONARY restricted_db.some_dict(
|
|
|
|
id UInt64,
|
|
|
|
SomeValue1 UInt8,
|
|
|
|
SomeValue2 String
|
|
|
|
)
|
|
|
|
PRIMARY KEY id
|
|
|
|
LAYOUT(FLAT())
|
|
|
|
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_in_restricted_db' DB 'restricted_db'))
|
|
|
|
LIFETIME(MIN 1 MAX 10)
|
2022-03-22 16:39:58 +00:00
|
|
|
"""
|
|
|
|
)
|
2019-10-18 15:44:32 +00:00
|
|
|
|
|
|
|
with pytest.raises(QueryRuntimeException):
|
2022-03-22 16:39:58 +00:00
|
|
|
node1.query(
|
|
|
|
"""
|
2019-10-18 15:44:32 +00:00
|
|
|
CREATE DICTIONARY default.some_dict(
|
|
|
|
id UInt64,
|
|
|
|
SomeValue1 UInt8,
|
|
|
|
SomeValue2 String
|
|
|
|
)
|
|
|
|
PRIMARY KEY id
|
|
|
|
LAYOUT(FLAT())
|
|
|
|
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_in_restricted_db' DB 'restricted_db'))
|
|
|
|
LIFETIME(MIN 1 MAX 10)
|
2022-03-22 16:39:58 +00:00
|
|
|
"""
|
|
|
|
)
|
2019-10-18 15:44:32 +00:00
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
node1.query(
|
|
|
|
"SELECT dictGetUInt8('default.some_dict', 'SomeValue1', toUInt64(17))"
|
|
|
|
) == "17\n"
|
2019-10-18 15:44:32 +00:00
|
|
|
|
|
|
|
# with lazy load we don't need query to get exception
|
|
|
|
with pytest.raises(QueryRuntimeException):
|
2022-03-22 16:39:58 +00:00
|
|
|
node2.query(
|
|
|
|
"""
|
2019-10-18 15:44:32 +00:00
|
|
|
CREATE DICTIONARY restricted_db.some_dict(
|
|
|
|
id UInt64,
|
|
|
|
SomeValue1 UInt8,
|
|
|
|
SomeValue2 String
|
|
|
|
)
|
|
|
|
PRIMARY KEY id
|
|
|
|
LAYOUT(FLAT())
|
|
|
|
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_in_restricted_db' DB 'restricted_db'))
|
|
|
|
LIFETIME(MIN 1 MAX 10)
|
2022-03-22 16:39:58 +00:00
|
|
|
"""
|
|
|
|
)
|
2019-10-18 15:44:32 +00:00
|
|
|
|
|
|
|
with pytest.raises(QueryRuntimeException):
|
2022-03-22 16:39:58 +00:00
|
|
|
node2.query(
|
|
|
|
"""
|
2019-10-18 15:44:32 +00:00
|
|
|
CREATE DICTIONARY default.some_dict(
|
|
|
|
id UInt64,
|
|
|
|
SomeValue1 UInt8,
|
|
|
|
SomeValue2 String
|
|
|
|
)
|
|
|
|
PRIMARY KEY id
|
|
|
|
LAYOUT(FLAT())
|
|
|
|
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_in_restricted_db' DB 'restricted_db'))
|
|
|
|
LIFETIME(MIN 1 MAX 10)
|
2022-03-22 16:39:58 +00:00
|
|
|
"""
|
|
|
|
)
|
2021-10-01 05:44:50 +00:00
|
|
|
for node in [node1, node2]:
|
2021-11-03 15:06:19 +00:00
|
|
|
node.query("DROP DICTIONARY IF EXISTS default.some_dict", user="admin")
|
2021-10-01 05:44:50 +00:00
|
|
|
node.query("DROP DATABASE restricted_db", user="admin")
|
2019-10-24 09:25:28 +00:00
|
|
|
|
|
|
|
|
|
|
|
def test_conflicting_name(started_cluster):
|
2022-03-22 16:39:58 +00:00
|
|
|
assert (
|
|
|
|
node3.query(
|
|
|
|
"select dictGetUInt8('test.conflicting_dictionary', 'SomeValue1', toUInt64(17))"
|
|
|
|
)
|
|
|
|
== "17\n"
|
|
|
|
)
|
2019-10-24 09:25:28 +00:00
|
|
|
|
|
|
|
with pytest.raises(QueryRuntimeException):
|
2022-03-22 16:39:58 +00:00
|
|
|
node3.query(
|
|
|
|
"""
|
2019-10-24 09:25:28 +00:00
|
|
|
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)
|
2022-03-22 16:39:58 +00:00
|
|
|
"""
|
|
|
|
)
|
2019-10-24 09:25:28 +00:00
|
|
|
|
|
|
|
# old version still works
|
2022-03-22 16:39:58 +00:00
|
|
|
node3.query(
|
|
|
|
"select dictGetUInt8('test.conflicting_dictionary', 'SomeValue1', toUInt64(17))"
|
|
|
|
) == "17\n"
|
2019-12-10 17:27:29 +00:00
|
|
|
|
2020-09-16 04:26:10 +00:00
|
|
|
|
2019-12-10 17:27:29 +00:00
|
|
|
def test_http_dictionary_restrictions(started_cluster):
|
|
|
|
try:
|
2022-03-22 16:39:58 +00:00
|
|
|
node3.query(
|
|
|
|
"""
|
2019-12-10 17:27:29 +00:00
|
|
|
CREATE DICTIONARY test.restricted_http_dictionary (
|
|
|
|
id UInt64,
|
|
|
|
value String
|
|
|
|
)
|
|
|
|
PRIMARY KEY id
|
|
|
|
LAYOUT(FLAT())
|
|
|
|
SOURCE(HTTP(URL 'http://somehost.net' FORMAT TabSeparated))
|
|
|
|
LIFETIME(1)
|
2022-03-22 16:39:58 +00:00
|
|
|
"""
|
|
|
|
)
|
|
|
|
node3.query(
|
|
|
|
"SELECT dictGetString('test.restricted_http_dictionary', 'value', toUInt64(1))"
|
|
|
|
)
|
2019-12-10 17:27:29 +00:00
|
|
|
except QueryRuntimeException as ex:
|
2022-03-22 16:39:58 +00:00
|
|
|
assert "is not allowed in configuration file" in str(ex)
|
2021-10-01 05:44:50 +00:00
|
|
|
node3.query("DROP DICTIONARY test.restricted_http_dictionary")
|
2019-12-10 17:27:29 +00:00
|
|
|
|
2020-09-16 04:26:10 +00:00
|
|
|
|
2019-12-10 17:27:29 +00:00
|
|
|
def test_file_dictionary_restrictions(started_cluster):
|
|
|
|
try:
|
2022-03-22 16:39:58 +00:00
|
|
|
node3.query(
|
|
|
|
"""
|
2019-12-10 17:27:29 +00:00
|
|
|
CREATE DICTIONARY test.restricted_file_dictionary (
|
|
|
|
id UInt64,
|
|
|
|
value String
|
|
|
|
)
|
|
|
|
PRIMARY KEY id
|
|
|
|
LAYOUT(FLAT())
|
|
|
|
SOURCE(FILE(PATH '/usr/bin/cat' FORMAT TabSeparated))
|
|
|
|
LIFETIME(1)
|
2022-03-22 16:39:58 +00:00
|
|
|
"""
|
|
|
|
)
|
|
|
|
node3.query(
|
|
|
|
"SELECT dictGetString('test.restricted_file_dictionary', 'value', toUInt64(1))"
|
|
|
|
)
|
2019-12-10 17:27:29 +00:00
|
|
|
except QueryRuntimeException as ex:
|
2022-03-22 16:39:58 +00:00
|
|
|
assert "is not inside" in str(ex)
|
2021-10-01 05:44:50 +00:00
|
|
|
node3.query("DROP DICTIONARY test.restricted_file_dictionary")
|
2020-03-18 16:47:33 +00:00
|
|
|
|
|
|
|
|
|
|
|
def test_dictionary_with_where(started_cluster):
|
2022-03-22 16:39:58 +00:00
|
|
|
mysql_conn = create_mysql_conn(
|
|
|
|
"root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port
|
|
|
|
)
|
|
|
|
execute_mysql_query(
|
|
|
|
mysql_conn, "CREATE DATABASE IF NOT EXISTS dictionary_with_where"
|
|
|
|
)
|
|
|
|
execute_mysql_query(
|
|
|
|
mysql_conn,
|
|
|
|
"CREATE TABLE dictionary_with_where.special_table (key_field1 int, value1 text, PRIMARY KEY (key_field1))",
|
|
|
|
)
|
|
|
|
execute_mysql_query(
|
|
|
|
mysql_conn,
|
|
|
|
"INSERT INTO dictionary_with_where.special_table VALUES (1, 'abcabc'), (2, 'qweqwe')",
|
|
|
|
)
|
2020-03-18 16:47:33 +00:00
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
node1.query(
|
|
|
|
"""
|
2020-03-18 16:47:33 +00:00
|
|
|
CREATE DICTIONARY default.special_dict (
|
|
|
|
key_field1 Int32,
|
|
|
|
value1 String DEFAULT 'xxx'
|
|
|
|
)
|
|
|
|
PRIMARY KEY key_field1
|
|
|
|
SOURCE(MYSQL(
|
|
|
|
USER 'root'
|
|
|
|
PASSWORD 'clickhouse'
|
2021-10-01 05:44:50 +00:00
|
|
|
DB 'dictionary_with_where'
|
2020-03-18 16:47:33 +00:00
|
|
|
TABLE 'special_table'
|
2021-02-15 09:35:45 +00:00
|
|
|
REPLICA(PRIORITY 1 HOST 'mysql57' PORT 3306)
|
2020-03-18 20:23:48 +00:00
|
|
|
WHERE 'value1 = \\'qweqwe\\' OR value1 = \\'\\\\u3232\\''
|
2020-03-18 16:47:33 +00:00
|
|
|
))
|
|
|
|
LAYOUT(FLAT())
|
|
|
|
LIFETIME(MIN 1 MAX 3)
|
2022-03-22 16:39:58 +00:00
|
|
|
"""
|
|
|
|
)
|
2020-03-18 16:47:33 +00:00
|
|
|
|
|
|
|
node1.query("SYSTEM RELOAD DICTIONARY default.special_dict")
|
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
assert (
|
|
|
|
node1.query(
|
|
|
|
"SELECT dictGetString('default.special_dict', 'value1', toUInt64(2))"
|
|
|
|
)
|
|
|
|
== "qweqwe\n"
|
|
|
|
)
|
2021-10-01 05:44:50 +00:00
|
|
|
node1.query("DROP DICTIONARY default.special_dict")
|
|
|
|
execute_mysql_query(mysql_conn, "DROP TABLE dictionary_with_where.special_table")
|
|
|
|
execute_mysql_query(mysql_conn, "DROP DATABASE dictionary_with_where")
|
2020-07-29 05:57:46 +00:00
|
|
|
|
2020-09-15 15:42:46 +00:00
|
|
|
|
2020-07-29 05:57:46 +00:00
|
|
|
def test_clickhouse_remote(started_cluster):
|
|
|
|
with pytest.raises(QueryRuntimeException):
|
2022-03-22 16:39:58 +00:00
|
|
|
node3.query(
|
|
|
|
"""
|
2020-07-29 05:57:46 +00:00
|
|
|
CREATE DICTIONARY test.clickhouse_remote(
|
|
|
|
id UInt64,
|
|
|
|
SomeValue1 UInt8,
|
|
|
|
SomeValue2 String
|
|
|
|
)
|
|
|
|
PRIMARY KEY id
|
|
|
|
LAYOUT(FLAT())
|
|
|
|
SOURCE(CLICKHOUSE(HOST 'node4' PORT 9000 USER 'default' TABLE 'xml_dictionary_table' DB 'test'))
|
|
|
|
LIFETIME(MIN 1 MAX 10)
|
2022-03-22 16:39:58 +00:00
|
|
|
"""
|
|
|
|
)
|
2020-09-15 15:42:46 +00:00
|
|
|
for i in range(5):
|
|
|
|
node3.query("system reload dictionary test.clickhouse_remote")
|
|
|
|
time.sleep(0.5)
|
2020-07-29 05:57:46 +00:00
|
|
|
|
|
|
|
node3.query("detach dictionary if exists test.clickhouse_remote")
|
2021-04-25 14:05:48 +00:00
|
|
|
|
|
|
|
with pytest.raises(QueryRuntimeException):
|
2022-03-22 16:39:58 +00:00
|
|
|
node3.query(
|
|
|
|
"""
|
2021-04-25 14:05:48 +00:00
|
|
|
CREATE DICTIONARY test.clickhouse_remote(
|
|
|
|
id UInt64,
|
|
|
|
SomeValue1 UInt8,
|
|
|
|
SomeValue2 String
|
|
|
|
)
|
|
|
|
PRIMARY KEY id
|
|
|
|
LAYOUT(FLAT())
|
|
|
|
SOURCE(CLICKHOUSE(HOST 'node4' PORT 9000 USER 'default' PASSWORD 'default' TABLE 'xml_dictionary_table' DB 'test'))
|
|
|
|
LIFETIME(MIN 1 MAX 10)
|
2022-03-22 16:39:58 +00:00
|
|
|
"""
|
|
|
|
)
|
2021-04-25 14:05:48 +00:00
|
|
|
|
|
|
|
node3.query("attach dictionary test.clickhouse_remote")
|
|
|
|
node3.query("drop dictionary test.clickhouse_remote")
|
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
node3.query(
|
|
|
|
"""
|
2020-07-29 05:57:46 +00:00
|
|
|
CREATE DICTIONARY test.clickhouse_remote(
|
|
|
|
id UInt64,
|
|
|
|
SomeValue1 UInt8,
|
|
|
|
SomeValue2 String
|
|
|
|
)
|
|
|
|
PRIMARY KEY id
|
|
|
|
LAYOUT(FLAT())
|
|
|
|
SOURCE(CLICKHOUSE(HOST 'node4' PORT 9000 USER 'default' PASSWORD 'default' TABLE 'xml_dictionary_table' DB 'test'))
|
|
|
|
LIFETIME(MIN 1 MAX 10)
|
2022-03-22 16:39:58 +00:00
|
|
|
"""
|
|
|
|
)
|
2020-07-29 05:57:46 +00:00
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
node3.query(
|
|
|
|
"select dictGetUInt8('test.clickhouse_remote', 'SomeValue1', toUInt64(17))"
|
|
|
|
) == "17\n"
|