CI: integration tests to mysql80

#ci_set_integration
 #no_merge_commit
This commit is contained in:
Max Kainov 2024-02-13 12:15:46 +00:00
parent c1754d3cd1
commit 9f51e840db
24 changed files with 1334 additions and 212 deletions

View File

@ -1,7 +1,7 @@
version: '2.3' version: '2.3'
services: services:
mysql2: mysql2:
image: mysql:5.7 image: mysql:8.0
restart: always restart: always
environment: environment:
MYSQL_ROOT_PASSWORD: clickhouse MYSQL_ROOT_PASSWORD: clickhouse
@ -23,7 +23,7 @@ services:
source: ${MYSQL_CLUSTER_LOGS:-} source: ${MYSQL_CLUSTER_LOGS:-}
target: /mysql/ target: /mysql/
mysql3: mysql3:
image: mysql:5.7 image: mysql:8.0
restart: always restart: always
environment: environment:
MYSQL_ROOT_PASSWORD: clickhouse MYSQL_ROOT_PASSWORD: clickhouse
@ -45,7 +45,7 @@ services:
source: ${MYSQL_CLUSTER_LOGS:-} source: ${MYSQL_CLUSTER_LOGS:-}
target: /mysql/ target: /mysql/
mysql4: mysql4:
image: mysql:5.7 image: mysql:8.0
restart: always restart: always
environment: environment:
MYSQL_ROOT_PASSWORD: clickhouse MYSQL_ROOT_PASSWORD: clickhouse

View File

@ -637,16 +637,8 @@ CI_CONFIG = CIConfig(
Labels.CI_SET_INTEGRATION: LabelConfig( Labels.CI_SET_INTEGRATION: LabelConfig(
run_jobs=[ run_jobs=[
JobNames.STYLE_CHECK, JobNames.STYLE_CHECK,
Build.PACKAGE_ASAN,
Build.PACKAGE_RELEASE, Build.PACKAGE_RELEASE,
Build.PACKAGE_TSAN,
Build.PACKAGE_AARCH64,
JobNames.INTEGRATION_TEST_ASAN,
JobNames.INTEGRATION_TEST_ARM,
JobNames.INTEGRATION_TEST, JobNames.INTEGRATION_TEST,
JobNames.INTEGRATION_TEST_ASAN_ANALYZER,
JobNames.INTEGRATION_TEST_TSAN,
JobNames.INTEGRATION_TEST_FLAKY,
] ]
), ),
Labels.CI_SET_REDUCED: LabelConfig( Labels.CI_SET_REDUCED: LabelConfig(

View File

@ -465,7 +465,7 @@ class ClickHouseCluster:
self.base_cmd += ["--project-name", self.project_name] self.base_cmd += ["--project-name", self.project_name]
self.base_zookeeper_cmd = None self.base_zookeeper_cmd = None
self.base_mysql_cmd = [] self.base_mysql57_cmd = []
self.base_kafka_cmd = [] self.base_kafka_cmd = []
self.base_kerberized_kafka_cmd = [] self.base_kerberized_kafka_cmd = []
self.base_kerberos_kdc_cmd = [] self.base_kerberos_kdc_cmd = []
@ -479,7 +479,7 @@ class ClickHouseCluster:
self.with_zookeeper = False self.with_zookeeper = False
self.with_zookeeper_secure = False self.with_zookeeper_secure = False
self.with_mysql_client = False self.with_mysql_client = False
self.with_mysql = False self.with_mysql57 = False
self.with_mysql8 = False self.with_mysql8 = False
self.with_mysql_cluster = False self.with_mysql_cluster = False
self.with_postgres = False self.with_postgres = False
@ -644,12 +644,19 @@ class ClickHouseCluster:
self.mysql_client_host = "mysql_client" self.mysql_client_host = "mysql_client"
self.mysql_client_container = None self.mysql_client_container = None
# available when with_mysql == True # available when with_mysql57 == True
self.mysql_host = "mysql57" self.mysql57_host = "mysql57"
self.mysql_port = 3306 self.mysql57_port = 3306
self.mysql_ip = None self.mysql57_ip = None
self.mysql_dir = p.abspath(p.join(self.instances_dir, "mysql")) self.mysql57_dir = p.abspath(p.join(self.instances_dir, "mysql"))
self.mysql_logs_dir = os.path.join(self.mysql_dir, "logs") self.mysql57_logs_dir = os.path.join(self.mysql57_dir, "logs")
# available when with_mysql8 == True
self.mysql8_host = "mysql80"
self.mysql8_port = 3306
self.mysql8_ip = None
self.mysql8_dir = p.abspath(p.join(self.instances_dir, "mysql8"))
self.mysql8_logs_dir = os.path.join(self.mysql8_dir, "logs")
# available when with_mysql_cluster == True # available when with_mysql_cluster == True
self.mysql2_host = "mysql2" self.mysql2_host = "mysql2"
@ -659,14 +666,7 @@ class ClickHouseCluster:
self.mysql3_ip = None self.mysql3_ip = None
self.mysql4_ip = None self.mysql4_ip = None
self.mysql_cluster_dir = p.abspath(p.join(self.instances_dir, "mysql")) self.mysql_cluster_dir = p.abspath(p.join(self.instances_dir, "mysql"))
self.mysql_cluster_logs_dir = os.path.join(self.mysql_dir, "logs") self.mysql_cluster_logs_dir = os.path.join(self.mysql8_dir, "logs")
# available when with_mysql8 == True
self.mysql8_host = "mysql80"
self.mysql8_port = 3306
self.mysql8_ip = None
self.mysql8_dir = p.abspath(p.join(self.instances_dir, "mysql8"))
self.mysql8_logs_dir = os.path.join(self.mysql8_dir, "logs")
# available when with_zookeper_secure == True # available when with_zookeper_secure == True
self.zookeeper_secure_port = 2281 self.zookeeper_secure_port = 2281
@ -1045,17 +1045,17 @@ class ClickHouseCluster:
return self.base_mysql_client_cmd return self.base_mysql_client_cmd
def setup_mysql_cmd(self, instance, env_variables, docker_compose_yml_dir): def setup_mysql57_cmd(self, instance, env_variables, docker_compose_yml_dir):
self.with_mysql = True self.with_mysql57 = True
env_variables["MYSQL_HOST"] = self.mysql_host env_variables["MYSQL_HOST"] = self.mysql57_host
env_variables["MYSQL_PORT"] = str(self.mysql_port) env_variables["MYSQL_PORT"] = str(self.mysql57_port)
env_variables["MYSQL_ROOT_HOST"] = "%" env_variables["MYSQL_ROOT_HOST"] = "%"
env_variables["MYSQL_LOGS"] = self.mysql_logs_dir env_variables["MYSQL_LOGS"] = self.mysql57_logs_dir
env_variables["MYSQL_LOGS_FS"] = "bind" env_variables["MYSQL_LOGS_FS"] = "bind"
self.base_cmd.extend( self.base_cmd.extend(
["--file", p.join(docker_compose_yml_dir, "docker_compose_mysql.yml")] ["--file", p.join(docker_compose_yml_dir, "docker_compose_mysql.yml")]
) )
self.base_mysql_cmd = [ self.base_mysql57_cmd = [
"docker-compose", "docker-compose",
"--env-file", "--env-file",
instance.env_file, instance.env_file,
@ -1065,7 +1065,7 @@ class ClickHouseCluster:
p.join(docker_compose_yml_dir, "docker_compose_mysql.yml"), p.join(docker_compose_yml_dir, "docker_compose_mysql.yml"),
] ]
return self.base_mysql_cmd return self.base_mysql57_cmd
def setup_mysql8_cmd(self, instance, env_variables, docker_compose_yml_dir): def setup_mysql8_cmd(self, instance, env_variables, docker_compose_yml_dir):
self.with_mysql8 = True self.with_mysql8 = True
@ -1091,7 +1091,7 @@ class ClickHouseCluster:
def setup_mysql_cluster_cmd(self, instance, env_variables, docker_compose_yml_dir): def setup_mysql_cluster_cmd(self, instance, env_variables, docker_compose_yml_dir):
self.with_mysql_cluster = True self.with_mysql_cluster = True
env_variables["MYSQL_CLUSTER_PORT"] = str(self.mysql_port) env_variables["MYSQL_CLUSTER_PORT"] = str(self.mysql8_port)
env_variables["MYSQL_CLUSTER_ROOT_HOST"] = "%" env_variables["MYSQL_CLUSTER_ROOT_HOST"] = "%"
env_variables["MYSQL_CLUSTER_LOGS"] = self.mysql_cluster_logs_dir env_variables["MYSQL_CLUSTER_LOGS"] = self.mysql_cluster_logs_dir
env_variables["MYSQL_CLUSTER_LOGS_FS"] = "bind" env_variables["MYSQL_CLUSTER_LOGS_FS"] = "bind"
@ -1572,7 +1572,7 @@ class ClickHouseCluster:
with_zookeeper=False, with_zookeeper=False,
with_zookeeper_secure=False, with_zookeeper_secure=False,
with_mysql_client=False, with_mysql_client=False,
with_mysql=False, with_mysql57=False,
with_mysql8=False, with_mysql8=False,
with_mysql_cluster=False, with_mysql_cluster=False,
with_kafka=False, with_kafka=False,
@ -1676,7 +1676,7 @@ class ClickHouseCluster:
with_zookeeper=with_zookeeper, with_zookeeper=with_zookeeper,
zookeeper_config_path=self.zookeeper_config_path, zookeeper_config_path=self.zookeeper_config_path,
with_mysql_client=with_mysql_client, with_mysql_client=with_mysql_client,
with_mysql=with_mysql, with_mysql57=with_mysql57,
with_mysql8=with_mysql8, with_mysql8=with_mysql8,
with_mysql_cluster=with_mysql_cluster, with_mysql_cluster=with_mysql_cluster,
with_kafka=with_kafka, with_kafka=with_kafka,
@ -1767,9 +1767,9 @@ class ClickHouseCluster:
) )
) )
if with_mysql and not self.with_mysql: if with_mysql57 and not self.with_mysql57:
cmds.append( cmds.append(
self.setup_mysql_cmd(instance, env_variables, docker_compose_yml_dir) self.setup_mysql57_cmd(instance, env_variables, docker_compose_yml_dir)
) )
if with_mysql8 and not self.with_mysql8: if with_mysql8 and not self.with_mysql8:
@ -1805,9 +1805,9 @@ class ClickHouseCluster:
if with_odbc_drivers and not self.with_odbc_drivers: if with_odbc_drivers and not self.with_odbc_drivers:
self.with_odbc_drivers = True self.with_odbc_drivers = True
if not self.with_mysql: if not self.with_mysql8:
cmds.append( cmds.append(
self.setup_mysql_cmd( self.setup_mysql8_cmd(
instance, env_variables, docker_compose_yml_dir instance, env_variables, docker_compose_yml_dir
) )
) )
@ -2148,8 +2148,8 @@ class ClickHouseCluster:
logging.error("Can't connect to MySQL Client:{}".format(errors)) logging.error("Can't connect to MySQL Client:{}".format(errors))
raise Exception("Cannot wait MySQL Client container") raise Exception("Cannot wait MySQL Client container")
def wait_mysql_to_start(self, timeout=180): def wait_mysql57_to_start(self, timeout=180):
self.mysql_ip = self.get_instance_ip("mysql57") self.mysql57_ip = self.get_instance_ip("mysql57")
start = time.time() start = time.time()
errors = [] errors = []
while time.time() - start < timeout: while time.time() - start < timeout:
@ -2157,8 +2157,8 @@ class ClickHouseCluster:
conn = pymysql.connect( conn = pymysql.connect(
user=mysql_user, user=mysql_user,
password=mysql_pass, password=mysql_pass,
host=self.mysql_ip, host=self.mysql57_ip,
port=self.mysql_port, port=self.mysql57_port,
) )
conn.close() conn.close()
logging.debug("Mysql Started") logging.debug("Mysql Started")
@ -2205,7 +2205,7 @@ class ClickHouseCluster:
user=mysql_user, user=mysql_user,
password=mysql_pass, password=mysql_pass,
host=ip, host=ip,
port=self.mysql_port, port=self.mysql8_port,
) )
conn.close() conn.close()
logging.debug(f"Mysql Started {ip}") logging.debug(f"Mysql Started {ip}")
@ -2752,15 +2752,15 @@ class ClickHouseCluster:
subprocess_check_call(self.base_mysql_client_cmd + common_opts) subprocess_check_call(self.base_mysql_client_cmd + common_opts)
self.wait_mysql_client_to_start() self.wait_mysql_client_to_start()
if self.with_mysql and self.base_mysql_cmd: if self.with_mysql57 and self.base_mysql57_cmd:
logging.debug("Setup MySQL") logging.debug("Setup MySQL")
if os.path.exists(self.mysql_dir): if os.path.exists(self.mysql57_dir):
shutil.rmtree(self.mysql_dir) shutil.rmtree(self.mysql57_dir)
os.makedirs(self.mysql_logs_dir) os.makedirs(self.mysql57_logs_dir)
os.chmod(self.mysql_logs_dir, stat.S_IRWXU | stat.S_IRWXO) os.chmod(self.mysql57_logs_dir, stat.S_IRWXU | stat.S_IRWXO)
subprocess_check_call(self.base_mysql_cmd + common_opts) subprocess_check_call(self.base_mysql57_cmd + common_opts)
self.up_called = True self.up_called = True
self.wait_mysql_to_start() self.wait_mysql57_to_start()
if self.with_mysql8 and self.base_mysql8_cmd: if self.with_mysql8 and self.base_mysql8_cmd:
logging.debug("Setup MySQL 8") logging.debug("Setup MySQL 8")
@ -2775,7 +2775,7 @@ class ClickHouseCluster:
print("Setup MySQL") print("Setup MySQL")
if os.path.exists(self.mysql_cluster_dir): if os.path.exists(self.mysql_cluster_dir):
shutil.rmtree(self.mysql_cluster_dir) shutil.rmtree(self.mysql_cluster_dir)
os.makedirs(self.mysql_cluster_logs_dir) os.makedirs(self.mysql_cluster_logs_dir, exist_ok=True)
os.chmod(self.mysql_cluster_logs_dir, stat.S_IRWXU | stat.S_IRWXO) os.chmod(self.mysql_cluster_logs_dir, stat.S_IRWXU | stat.S_IRWXO)
subprocess_check_call(self.base_mysql_cluster_cmd + common_opts) subprocess_check_call(self.base_mysql_cluster_cmd + common_opts)
@ -3239,7 +3239,7 @@ class ClickHouseInstance:
with_zookeeper, with_zookeeper,
zookeeper_config_path, zookeeper_config_path,
with_mysql_client, with_mysql_client,
with_mysql, with_mysql57,
with_mysql8, with_mysql8,
with_mysql_cluster, with_mysql_cluster,
with_kafka, with_kafka,
@ -3324,7 +3324,7 @@ class ClickHouseInstance:
self.library_bridge_bin_path = library_bridge_bin_path self.library_bridge_bin_path = library_bridge_bin_path
self.with_mysql_client = with_mysql_client self.with_mysql_client = with_mysql_client
self.with_mysql = with_mysql self.with_mysql57 = with_mysql57
self.with_mysql8 = with_mysql8 self.with_mysql8 = with_mysql8
self.with_mysql_cluster = with_mysql_cluster self.with_mysql_cluster = with_mysql_cluster
self.with_postgres = with_postgres self.with_postgres = with_postgres
@ -3368,7 +3368,7 @@ class ClickHouseInstance:
self.env_file = self.cluster.env_file self.env_file = self.cluster.env_file
if with_odbc_drivers: if with_odbc_drivers:
self.odbc_ini_path = self.path + "/odbc.ini:/etc/odbc.ini" self.odbc_ini_path = self.path + "/odbc.ini:/etc/odbc.ini"
self.with_mysql = True self.with_mysql8 = True
else: else:
self.odbc_ini_path = "" self.odbc_ini_path = ""
@ -4294,7 +4294,7 @@ class ClickHouseInstance:
"Database": odbc_mysql_db, "Database": odbc_mysql_db,
"Uid": odbc_mysql_uid, "Uid": odbc_mysql_uid,
"Pwd": odbc_mysql_pass, "Pwd": odbc_mysql_pass,
"Server": self.cluster.mysql_host, "Server": self.cluster.mysql8_host,
}, },
"PostgreSQL": { "PostgreSQL": {
"DSN": "postgresql_odbc", "DSN": "postgresql_odbc",
@ -4482,14 +4482,14 @@ class ClickHouseInstance:
if self.with_mysql_client: if self.with_mysql_client:
depends_on.append(self.cluster.mysql_client_host) depends_on.append(self.cluster.mysql_client_host)
if self.with_mysql: if self.with_mysql57:
depends_on.append("mysql57") depends_on.append("mysql57")
if self.with_mysql8: if self.with_mysql8:
depends_on.append("mysql80") depends_on.append("mysql80")
if self.with_mysql_cluster: if self.with_mysql_cluster:
depends_on.append("mysql57") depends_on.append("mysql80")
depends_on.append("mysql2") depends_on.append("mysql2")
depends_on.append("mysql3") depends_on.append("mysql3")
depends_on.append("mysql4") depends_on.append("mysql4")

View File

@ -119,7 +119,7 @@ class SourceMySQL(ExternalSource):
def prepare(self, structure, table_name, cluster): def prepare(self, structure, table_name, cluster):
if self.internal_hostname is None: if self.internal_hostname is None:
self.internal_hostname = cluster.mysql_ip self.internal_hostname = cluster.mysql8_ip
self.create_mysql_conn() self.create_mysql_conn()
self.execute_mysql_query( self.execute_mysql_query(
"create database if not exists test default character set 'utf8'" "create database if not exists test default character set 'utf8'"

View File

@ -29,9 +29,9 @@ def setup_module(module):
SOURCE = SourceMySQL( SOURCE = SourceMySQL(
"MySQL", "MySQL",
None, None,
cluster.mysql_port, cluster.mysql8_port,
cluster.mysql_host, cluster.mysql8_host,
cluster.mysql_port, cluster.mysql8_port,
"root", "root",
"clickhouse", "clickhouse",
) )
@ -53,7 +53,7 @@ def setup_module(module):
dictionaries = simple_tester.list_dictionaries() dictionaries = simple_tester.list_dictionaries()
node = cluster.add_instance( node = cluster.add_instance(
"node", main_configs=main_configs, dictionaries=dictionaries, with_mysql=True "node", main_configs=main_configs, dictionaries=dictionaries, with_mysql8=True
) )

View File

@ -13,7 +13,7 @@ SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance( node1 = cluster.add_instance(
"node1", "node1",
with_mysql=True, with_mysql8=True,
dictionaries=["configs/dictionaries/simple_dictionary.xml"], dictionaries=["configs/dictionaries/simple_dictionary.xml"],
main_configs=[ main_configs=[
"configs/ssl_conf.xml", "configs/ssl_conf.xml",
@ -26,7 +26,7 @@ node1 = cluster.add_instance(
) )
node2 = cluster.add_instance( node2 = cluster.add_instance(
"node2", "node2",
with_mysql=True, with_mysql8=True,
dictionaries=["configs/dictionaries/simple_dictionary.xml"], dictionaries=["configs/dictionaries/simple_dictionary.xml"],
main_configs=[ main_configs=[
"configs/dictionaries/lazy_load.xml", "configs/dictionaries/lazy_load.xml",
@ -117,7 +117,7 @@ def started_cluster():
) )
def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): def test_create_and_select_mysql(started_cluster, clickhouse, name, layout):
mysql_conn = create_mysql_conn( mysql_conn = create_mysql_conn(
"root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port "root", "clickhouse", started_cluster.mysql8_ip, started_cluster.mysql8_port
) )
execute_mysql_query(mysql_conn, "DROP DATABASE IF EXISTS create_and_select") execute_mysql_query(mysql_conn, "DROP DATABASE IF EXISTS create_and_select")
execute_mysql_query(mysql_conn, "CREATE DATABASE create_and_select") execute_mysql_query(mysql_conn, "CREATE DATABASE create_and_select")
@ -152,7 +152,7 @@ def test_create_and_select_mysql(started_cluster, clickhouse, name, layout):
DB 'create_and_select' DB 'create_and_select'
TABLE '{}' TABLE '{}'
REPLICA(PRIORITY 1 HOST '127.0.0.1' PORT 3333) REPLICA(PRIORITY 1 HOST '127.0.0.1' PORT 3333)
REPLICA(PRIORITY 2 HOST 'mysql57' PORT 3306) REPLICA(PRIORITY 2 HOST 'mysql80' PORT 3306)
)) ))
{} {}
LIFETIME(MIN 1 MAX 3) LIFETIME(MIN 1 MAX 3)
@ -367,7 +367,7 @@ def test_file_dictionary_restrictions(started_cluster):
def test_dictionary_with_where(started_cluster): def test_dictionary_with_where(started_cluster):
mysql_conn = create_mysql_conn( mysql_conn = create_mysql_conn(
"root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port "root", "clickhouse", started_cluster.mysql8_ip, started_cluster.mysql8_port
) )
execute_mysql_query( execute_mysql_query(
mysql_conn, "CREATE DATABASE IF NOT EXISTS dictionary_with_where" mysql_conn, "CREATE DATABASE IF NOT EXISTS dictionary_with_where"
@ -393,7 +393,7 @@ def test_dictionary_with_where(started_cluster):
PASSWORD 'clickhouse' PASSWORD 'clickhouse'
DB 'dictionary_with_where' DB 'dictionary_with_where'
TABLE 'special_table' TABLE 'special_table'
REPLICA(PRIORITY 1 HOST 'mysql57' PORT 3306) REPLICA(PRIORITY 1 HOST 'mysql80' PORT 3306)
WHERE 'value1 = \\'qweqwe\\' OR value1 = \\'\\\\u3232\\'' WHERE 'value1 = \\'qweqwe\\' OR value1 = \\'\\\\u3232\\''
)) ))
LAYOUT(FLAT()) LAYOUT(FLAT())

View File

@ -4,7 +4,7 @@
<source> <source>
<mysql > <mysql >
<db>test</db> <db>test</db>
<host>mysql57</host> <host>mysql80</host>
<port>3306</port> <port>3306</port>
<user>root</user> <user>root</user>
<password>clickhouse</password> <password>clickhouse</password>
@ -40,7 +40,7 @@
<source> <source>
<mysql > <mysql >
<db>test</db> <db>test</db>
<host>mysql57</host> <host>mysql80</host>
<port>3306</port> <port>3306</port>
<user>root</user> <user>root</user>
<password>clickhouse</password> <password>clickhouse</password>

View File

@ -4,7 +4,7 @@
<source> <source>
<mysql > <mysql >
<db>test</db> <db>test</db>
<host>mysql57</host> <host>mysql80</host>
<port>3306</port> <port>3306</port>
<user>root</user> <user>root</user>
<password>clickhouse</password> <password>clickhouse</password>
@ -41,7 +41,7 @@
<source> <source>
<mysql > <mysql >
<db>test</db> <db>test</db>
<host>mysql57</host> <host>mysql80</host>
<port>3306</port> <port>3306</port>
<user>root</user> <user>root</user>
<password>clickhouse</password> <password>clickhouse</password>
@ -77,7 +77,7 @@
<source> <source>
<mysql > <mysql >
<db>test</db> <db>test</db>
<host>mysql57</host> <host>mysql80</host>
<port>3306</port> <port>3306</port>
<user>root</user> <user>root</user>
<password>clickhouse</password> <password>clickhouse</password>

View File

@ -3,7 +3,7 @@
<mysql1> <mysql1>
<user>root</user> <user>root</user>
<password>clickhouse</password> <password>clickhouse</password>
<host>mysql57</host> <host>mysql80</host>
<port>3306</port> <port>3306</port>
<database>test</database> <database>test</database>
<table>test_table</table> <table>test_table</table>
@ -16,7 +16,7 @@
<mysql3> <mysql3>
<user>root</user> <user>root</user>
<password>clickhouse</password> <password>clickhouse</password>
<host>mysql57</host> <host>mysql80</host>
<port>1111</port> <port>1111</port>
<database>test</database> <database>test</database>
<table>test_table</table> <table>test_table</table>
@ -24,7 +24,7 @@
<mysql4> <mysql4>
<user>root</user> <user>root</user>
<password>clickhouse</password> <password>clickhouse</password>
<host>mysql57</host> <host>mysql80</host>
<port>3306</port> <port>3306</port>
<database>test</database> <database>test</database>
<table>test_table</table> <table>test_table</table>

View File

@ -14,7 +14,7 @@ instance = cluster.add_instance(
"instance", "instance",
main_configs=CONFIG_FILES, main_configs=CONFIG_FILES,
user_configs=USER_CONFIGS, user_configs=USER_CONFIGS,
with_mysql=True, with_mysql8=True,
dictionaries=DICTS, dictionaries=DICTS,
) )
@ -47,7 +47,7 @@ def started_cluster():
# Create database in ClickChouse using MySQL protocol (will be used for data insertion) # Create database in ClickChouse using MySQL protocol (will be used for data insertion)
instance.query( instance.query(
"CREATE DATABASE clickhouse_mysql ENGINE = MySQL('mysql57:3306', 'test', 'root', 'clickhouse')" "CREATE DATABASE clickhouse_mysql ENGINE = MySQL('mysql80:3306', 'test', 'root', 'clickhouse')"
) )
yield cluster yield cluster
@ -86,7 +86,7 @@ def test_mysql_dictionaries_custom_query_full_load(started_cluster):
PRIMARY KEY id PRIMARY KEY id
LAYOUT(FLAT()) LAYOUT(FLAT())
SOURCE(MYSQL( SOURCE(MYSQL(
HOST 'mysql57' HOST 'mysql80'
PORT 3306 PORT 3306
USER 'root' USER 'root'
PASSWORD 'clickhouse' PASSWORD 'clickhouse'
@ -135,7 +135,7 @@ def test_mysql_dictionaries_custom_query_partial_load_simple_key(started_cluster
PRIMARY KEY id PRIMARY KEY id
LAYOUT(DIRECT()) LAYOUT(DIRECT())
SOURCE(MYSQL( SOURCE(MYSQL(
HOST 'mysql57' HOST 'mysql80'
PORT 3306 PORT 3306
USER 'root' USER 'root'
PASSWORD 'clickhouse' PASSWORD 'clickhouse'
@ -186,7 +186,7 @@ def test_mysql_dictionaries_custom_query_partial_load_complex_key(started_cluste
PRIMARY KEY id, id_key PRIMARY KEY id, id_key
LAYOUT(COMPLEX_KEY_DIRECT()) LAYOUT(COMPLEX_KEY_DIRECT())
SOURCE(MYSQL( SOURCE(MYSQL(
HOST 'mysql57' HOST 'mysql80'
PORT 3306 PORT 3306
USER 'root' USER 'root'
PASSWORD 'clickhouse' PASSWORD 'clickhouse'
@ -372,13 +372,13 @@ def get_mysql_conn(started_cluster):
conn = pymysql.connect( conn = pymysql.connect(
user="root", user="root",
password="clickhouse", password="clickhouse",
host=started_cluster.mysql_ip, host=started_cluster.mysql8_ip,
port=started_cluster.mysql_port, port=started_cluster.mysql8_port,
) )
else: else:
conn.ping(reconnect=True) conn.ping(reconnect=True)
logging.debug( logging.debug(
f"MySQL Connection establised: {started_cluster.mysql_ip}:{started_cluster.mysql_port}" f"MySQL Connection establised: {started_cluster.mysql8_ip}:{started_cluster.mysql8_port}"
) )
return conn return conn
except Exception as e: except Exception as e:

View File

@ -11,7 +11,7 @@ from helpers.network import PartitionManager
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
clickhouse_node = cluster.add_instance( clickhouse_node = cluster.add_instance(
"node1", main_configs=["configs/remote_servers.xml"], with_mysql=True "node1", main_configs=["configs/remote_servers.xml"], with_mysql8=True
) )
@ -27,8 +27,8 @@ def started_cluster():
class MySQLNodeInstance: class MySQLNodeInstance:
def __init__(self, started_cluster, user="root", password="clickhouse"): def __init__(self, started_cluster, user="root", password="clickhouse"):
self.user = user self.user = user
self.port = cluster.mysql_port self.port = cluster.mysql8_port
self.hostname = cluster.mysql_ip self.hostname = cluster.mysql8_ip
self.password = password self.password = password
self.mysql_connection = None # lazy init self.mysql_connection = None # lazy init
@ -62,7 +62,7 @@ def test_disabled_mysql_server(started_cluster):
with PartitionManager() as pm: with PartitionManager() as pm:
clickhouse_node.query( clickhouse_node.query(
"CREATE DATABASE test_db_disabled ENGINE = MySQL('mysql57:3306', 'test_db_disabled', 'root', 'clickhouse')" "CREATE DATABASE test_db_disabled ENGINE = MySQL('mysql80:3306', 'test_db_disabled', 'root', 'clickhouse')"
) )
pm._add_rule( pm._add_rule(

View File

@ -175,7 +175,7 @@ def test_create_table():
password = new_password() password = new_password()
table_engines = [ table_engines = [
f"MySQL('mysql57:3306', 'mysql_db', 'mysql_table', 'mysql_user', '{password}')", f"MySQL('mysql80:3306', 'mysql_db', 'mysql_table', 'mysql_user', '{password}')",
f"PostgreSQL('postgres1:5432', 'postgres_db', 'postgres_table', 'postgres_user', '{password}')", f"PostgreSQL('postgres1:5432', 'postgres_db', 'postgres_table', 'postgres_user', '{password}')",
f"MongoDB('mongo1:27017', 'mongo_db', 'mongo_col', 'mongo_user', '{password}')", f"MongoDB('mongo1:27017', 'mongo_db', 'mongo_col', 'mongo_user', '{password}')",
f"S3('http://minio1:9001/root/data/test1.csv')", f"S3('http://minio1:9001/root/data/test1.csv')",
@ -183,9 +183,9 @@ def test_create_table():
f"S3('http://minio1:9001/root/data/test3.csv.gz', 'CSV', 'gzip')", f"S3('http://minio1:9001/root/data/test3.csv.gz', 'CSV', 'gzip')",
f"S3('http://minio1:9001/root/data/test4.csv', 'minio', '{password}', 'CSV')", f"S3('http://minio1:9001/root/data/test4.csv', 'minio', '{password}', 'CSV')",
f"S3('http://minio1:9001/root/data/test5.csv.gz', 'minio', '{password}', 'CSV', 'gzip')", f"S3('http://minio1:9001/root/data/test5.csv.gz', 'minio', '{password}', 'CSV', 'gzip')",
f"MySQL(named_collection_1, host = 'mysql57', port = 3306, database = 'mysql_db', table = 'mysql_table', user = 'mysql_user', password = '{password}')", f"MySQL(named_collection_1, host = 'mysql80', port = 3306, database = 'mysql_db', table = 'mysql_table', user = 'mysql_user', password = '{password}')",
f"MySQL(named_collection_2, database = 'mysql_db', host = 'mysql57', port = 3306, password = '{password}', table = 'mysql_table', user = 'mysql_user')", f"MySQL(named_collection_2, database = 'mysql_db', host = 'mysql80', port = 3306, password = '{password}', table = 'mysql_table', user = 'mysql_user')",
f"MySQL(named_collection_3, database = 'mysql_db', host = 'mysql57', port = 3306, table = 'mysql_table')", f"MySQL(named_collection_3, database = 'mysql_db', host = 'mysql80', port = 3306, table = 'mysql_table')",
f"PostgreSQL(named_collection_4, host = 'postgres1', port = 5432, database = 'postgres_db', table = 'postgres_table', user = 'postgres_user', password = '{password}')", f"PostgreSQL(named_collection_4, host = 'postgres1', port = 5432, database = 'postgres_db', table = 'postgres_table', user = 'postgres_user', password = '{password}')",
f"MongoDB(named_collection_5, host = 'mongo1', port = 5432, db = 'mongo_db', collection = 'mongo_col', user = 'mongo_user', password = '{password}')", f"MongoDB(named_collection_5, host = 'mongo1', port = 5432, db = 'mongo_db', collection = 'mongo_col', user = 'mongo_user', password = '{password}')",
f"S3(named_collection_6, url = 'http://minio1:9001/root/data/test8.csv', access_key_id = 'minio', secret_access_key = '{password}', format = 'CSV')", f"S3(named_collection_6, url = 'http://minio1:9001/root/data/test8.csv', access_key_id = 'minio', secret_access_key = '{password}', format = 'CSV')",
@ -219,7 +219,7 @@ def test_create_table():
assert ( assert (
node.query(f"SHOW CREATE TABLE table0 {show_secrets}={toggle}") node.query(f"SHOW CREATE TABLE table0 {show_secrets}={toggle}")
== "CREATE TABLE default.table0\\n(\\n `x` Int32\\n)\\n" == "CREATE TABLE default.table0\\n(\\n `x` Int32\\n)\\n"
"ENGINE = MySQL(\\'mysql57:3306\\', \\'mysql_db\\', " "ENGINE = MySQL(\\'mysql80:3306\\', \\'mysql_db\\', "
f"\\'mysql_table\\', \\'mysql_user\\', \\'{secret}\\')\n" f"\\'mysql_table\\', \\'mysql_user\\', \\'{secret}\\')\n"
) )
@ -228,16 +228,16 @@ def test_create_table():
) == TSV( ) == TSV(
[ [
[ [
"CREATE TABLE default.table0 (`x` Int32) ENGINE = MySQL(\\'mysql57:3306\\', \\'mysql_db\\', " "CREATE TABLE default.table0 (`x` Int32) ENGINE = MySQL(\\'mysql80:3306\\', \\'mysql_db\\', "
f"\\'mysql_table\\', \\'mysql_user\\', \\'{secret}\\')", f"\\'mysql_table\\', \\'mysql_user\\', \\'{secret}\\')",
f"MySQL(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'{secret}\\')", f"MySQL(\\'mysql80:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'{secret}\\')",
], ],
] ]
) )
check_logs( check_logs(
must_contain=[ must_contain=[
"CREATE TABLE table0 (`x` int) ENGINE = MySQL('mysql57:3306', 'mysql_db', 'mysql_table', 'mysql_user', '[HIDDEN]')", "CREATE TABLE table0 (`x` int) ENGINE = MySQL('mysql80:3306', 'mysql_db', 'mysql_table', 'mysql_user', '[HIDDEN]')",
"CREATE TABLE table1 (`x` int) ENGINE = PostgreSQL('postgres1:5432', 'postgres_db', 'postgres_table', 'postgres_user', '[HIDDEN]')", "CREATE TABLE table1 (`x` int) ENGINE = PostgreSQL('postgres1:5432', 'postgres_db', 'postgres_table', 'postgres_user', '[HIDDEN]')",
"CREATE TABLE table2 (`x` int) ENGINE = MongoDB('mongo1:27017', 'mongo_db', 'mongo_col', 'mongo_user', '[HIDDEN]')", "CREATE TABLE table2 (`x` int) ENGINE = MongoDB('mongo1:27017', 'mongo_db', 'mongo_col', 'mongo_user', '[HIDDEN]')",
"CREATE TABLE table3 (x int) ENGINE = S3('http://minio1:9001/root/data/test1.csv')", "CREATE TABLE table3 (x int) ENGINE = S3('http://minio1:9001/root/data/test1.csv')",
@ -245,9 +245,9 @@ def test_create_table():
"CREATE TABLE table5 (x int) ENGINE = S3('http://minio1:9001/root/data/test3.csv.gz', 'CSV', 'gzip')", "CREATE TABLE table5 (x int) ENGINE = S3('http://minio1:9001/root/data/test3.csv.gz', 'CSV', 'gzip')",
"CREATE TABLE table6 (`x` int) ENGINE = S3('http://minio1:9001/root/data/test4.csv', 'minio', '[HIDDEN]', 'CSV')", "CREATE TABLE table6 (`x` int) ENGINE = S3('http://minio1:9001/root/data/test4.csv', 'minio', '[HIDDEN]', 'CSV')",
"CREATE TABLE table7 (`x` int) ENGINE = S3('http://minio1:9001/root/data/test5.csv.gz', 'minio', '[HIDDEN]', 'CSV', 'gzip')", "CREATE TABLE table7 (`x` int) ENGINE = S3('http://minio1:9001/root/data/test5.csv.gz', 'minio', '[HIDDEN]', 'CSV', 'gzip')",
"CREATE TABLE table8 (`x` int) ENGINE = MySQL(named_collection_1, host = 'mysql57', port = 3306, database = 'mysql_db', table = 'mysql_table', user = 'mysql_user', password = '[HIDDEN]')", "CREATE TABLE table8 (`x` int) ENGINE = MySQL(named_collection_1, host = 'mysql80', port = 3306, database = 'mysql_db', table = 'mysql_table', user = 'mysql_user', password = '[HIDDEN]')",
"CREATE TABLE table9 (`x` int) ENGINE = MySQL(named_collection_2, database = 'mysql_db', host = 'mysql57', port = 3306, password = '[HIDDEN]', table = 'mysql_table', user = 'mysql_user')", "CREATE TABLE table9 (`x` int) ENGINE = MySQL(named_collection_2, database = 'mysql_db', host = 'mysql80', port = 3306, password = '[HIDDEN]', table = 'mysql_table', user = 'mysql_user')",
"CREATE TABLE table10 (x int) ENGINE = MySQL(named_collection_3, database = 'mysql_db', host = 'mysql57', port = 3306, table = 'mysql_table')", "CREATE TABLE table10 (x int) ENGINE = MySQL(named_collection_3, database = 'mysql_db', host = 'mysql80', port = 3306, table = 'mysql_table')",
"CREATE TABLE table11 (`x` int) ENGINE = PostgreSQL(named_collection_4, host = 'postgres1', port = 5432, database = 'postgres_db', table = 'postgres_table', user = 'postgres_user', password = '[HIDDEN]')", "CREATE TABLE table11 (`x` int) ENGINE = PostgreSQL(named_collection_4, host = 'postgres1', port = 5432, database = 'postgres_db', table = 'postgres_table', user = 'postgres_user', password = '[HIDDEN]')",
"CREATE TABLE table12 (`x` int) ENGINE = MongoDB(named_collection_5, host = 'mongo1', port = 5432, db = 'mongo_db', collection = 'mongo_col', user = 'mongo_user', password = '[HIDDEN]'", "CREATE TABLE table12 (`x` int) ENGINE = MongoDB(named_collection_5, host = 'mongo1', port = 5432, db = 'mongo_db', collection = 'mongo_col', user = 'mongo_user', password = '[HIDDEN]'",
"CREATE TABLE table13 (`x` int) ENGINE = S3(named_collection_6, url = 'http://minio1:9001/root/data/test8.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]', format = 'CSV')", "CREATE TABLE table13 (`x` int) ENGINE = S3(named_collection_6, url = 'http://minio1:9001/root/data/test8.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]', format = 'CSV')",
@ -320,7 +320,7 @@ def test_table_functions():
password = new_password() password = new_password()
table_functions = [ table_functions = [
f"mysql('mysql57:3306', 'mysql_db', 'mysql_table', 'mysql_user', '{password}')", f"mysql('mysql80:3306', 'mysql_db', 'mysql_table', 'mysql_user', '{password}')",
f"postgresql('postgres1:5432', 'postgres_db', 'postgres_table', 'postgres_user', '{password}')", f"postgresql('postgres1:5432', 'postgres_db', 'postgres_table', 'postgres_user', '{password}')",
f"mongodb('mongo1:27017', 'mongo_db', 'mongo_col', 'mongo_user', '{password}', 'x int')", f"mongodb('mongo1:27017', 'mongo_db', 'mongo_col', 'mongo_user', '{password}', 'x int')",
f"s3('http://minio1:9001/root/data/test1.csv')", f"s3('http://minio1:9001/root/data/test1.csv')",
@ -345,7 +345,7 @@ def test_table_functions():
f"remote('127.{{2..11}}', numbers(10), 'remote_user', '{password}', rand())", f"remote('127.{{2..11}}', numbers(10), 'remote_user', '{password}', rand())",
f"remoteSecure('127.{{2..11}}', 'default', 'remote_table', 'remote_user', '{password}')", f"remoteSecure('127.{{2..11}}', 'default', 'remote_table', 'remote_user', '{password}')",
f"remoteSecure('127.{{2..11}}', 'default', 'remote_table', 'remote_user', rand())", f"remoteSecure('127.{{2..11}}', 'default', 'remote_table', 'remote_user', rand())",
f"mysql(named_collection_1, host = 'mysql57', port = 3306, database = 'mysql_db', table = 'mysql_table', user = 'mysql_user', password = '{password}')", f"mysql(named_collection_1, host = 'mysql80', port = 3306, database = 'mysql_db', table = 'mysql_table', user = 'mysql_user', password = '{password}')",
f"postgresql(named_collection_2, password = '{password}', host = 'postgres1', port = 5432, database = 'postgres_db', table = 'postgres_table', user = 'postgres_user')", f"postgresql(named_collection_2, password = '{password}', host = 'postgres1', port = 5432, database = 'postgres_db', table = 'postgres_table', user = 'postgres_user')",
f"s3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '{password}')", f"s3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '{password}')",
f"remote(named_collection_6, addresses_expr = '127.{{2..11}}', database = 'default', table = 'remote_table', user = 'remote_user', password = '{password}', sharding_key = rand())", f"remote(named_collection_6, addresses_expr = '127.{{2..11}}', database = 'default', table = 'remote_table', user = 'remote_user', password = '{password}', sharding_key = rand())",
@ -377,7 +377,7 @@ def test_table_functions():
assert ( assert (
node.query(f"SHOW CREATE TABLE tablefunc0 {show_secrets}={toggle}") node.query(f"SHOW CREATE TABLE tablefunc0 {show_secrets}={toggle}")
== "CREATE TABLE default.tablefunc0\\n(\\n `x` Int32\\n) AS " == "CREATE TABLE default.tablefunc0\\n(\\n `x` Int32\\n) AS "
"mysql(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', " "mysql(\\'mysql80:3306\\', \\'mysql_db\\', \\'mysql_table\\', "
f"\\'mysql_user\\', \\'{secret}\\')\n" f"\\'mysql_user\\', \\'{secret}\\')\n"
) )
@ -387,7 +387,7 @@ def test_table_functions():
) == TSV( ) == TSV(
[ [
[ [
"CREATE TABLE default.tablefunc0 (`x` Int32) AS mysql(\\'mysql57:3306\\', " "CREATE TABLE default.tablefunc0 (`x` Int32) AS mysql(\\'mysql80:3306\\', "
f"\\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'{secret}\\')", f"\\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'{secret}\\')",
"", "",
], ],
@ -396,7 +396,7 @@ def test_table_functions():
check_logs( check_logs(
must_contain=[ must_contain=[
"CREATE TABLE tablefunc0 (`x` int) AS mysql('mysql57:3306', 'mysql_db', 'mysql_table', 'mysql_user', '[HIDDEN]')", "CREATE TABLE tablefunc0 (`x` int) AS mysql('mysql80:3306', 'mysql_db', 'mysql_table', 'mysql_user', '[HIDDEN]')",
"CREATE TABLE tablefunc1 (`x` int) AS postgresql('postgres1:5432', 'postgres_db', 'postgres_table', 'postgres_user', '[HIDDEN]')", "CREATE TABLE tablefunc1 (`x` int) AS postgresql('postgres1:5432', 'postgres_db', 'postgres_table', 'postgres_user', '[HIDDEN]')",
"CREATE TABLE tablefunc2 (`x` int) AS mongodb('mongo1:27017', 'mongo_db', 'mongo_col', 'mongo_user', '[HIDDEN]', 'x int')", "CREATE TABLE tablefunc2 (`x` int) AS mongodb('mongo1:27017', 'mongo_db', 'mongo_col', 'mongo_user', '[HIDDEN]', 'x int')",
"CREATE TABLE tablefunc3 (x int) AS s3('http://minio1:9001/root/data/test1.csv')", "CREATE TABLE tablefunc3 (x int) AS s3('http://minio1:9001/root/data/test1.csv')",
@ -421,7 +421,7 @@ def test_table_functions():
"CREATE TABLE tablefunc22 (`x` int) AS remote('127.{2..11}', numbers(10), 'remote_user', '[HIDDEN]', rand())", "CREATE TABLE tablefunc22 (`x` int) AS remote('127.{2..11}', numbers(10), 'remote_user', '[HIDDEN]', rand())",
"CREATE TABLE tablefunc23 (`x` int) AS remoteSecure('127.{2..11}', 'default', 'remote_table', 'remote_user', '[HIDDEN]')", "CREATE TABLE tablefunc23 (`x` int) AS remoteSecure('127.{2..11}', 'default', 'remote_table', 'remote_user', '[HIDDEN]')",
"CREATE TABLE tablefunc24 (x int) AS remoteSecure('127.{2..11}', 'default', 'remote_table', 'remote_user', rand())", "CREATE TABLE tablefunc24 (x int) AS remoteSecure('127.{2..11}', 'default', 'remote_table', 'remote_user', rand())",
"CREATE TABLE tablefunc25 (`x` int) AS mysql(named_collection_1, host = 'mysql57', port = 3306, database = 'mysql_db', table = 'mysql_table', user = 'mysql_user', password = '[HIDDEN]')", "CREATE TABLE tablefunc25 (`x` int) AS mysql(named_collection_1, host = 'mysql80', port = 3306, database = 'mysql_db', table = 'mysql_table', user = 'mysql_user', password = '[HIDDEN]')",
"CREATE TABLE tablefunc26 (`x` int) AS postgresql(named_collection_2, password = '[HIDDEN]', host = 'postgres1', port = 5432, database = 'postgres_db', table = 'postgres_table', user = 'postgres_user')", "CREATE TABLE tablefunc26 (`x` int) AS postgresql(named_collection_2, password = '[HIDDEN]', host = 'postgres1', port = 5432, database = 'postgres_db', table = 'postgres_table', user = 'postgres_user')",
"CREATE TABLE tablefunc27 (`x` int) AS s3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", "CREATE TABLE tablefunc27 (`x` int) AS s3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')",
"CREATE TABLE tablefunc28 (`x` int) AS remote(named_collection_6, addresses_expr = '127.{2..11}', database = 'default', table = 'remote_table', user = 'remote_user', password = '[HIDDEN]', sharding_key = rand())", "CREATE TABLE tablefunc28 (`x` int) AS remote(named_collection_6, addresses_expr = '127.{2..11}', database = 'default', table = 'remote_table', user = 'remote_user', password = '[HIDDEN]', sharding_key = rand())",
@ -567,31 +567,31 @@ def test_on_cluster():
password = new_password() password = new_password()
node.query( node.query(
f"CREATE TABLE table_oncl ON CLUSTER 'test_shard_localhost' (x int) ENGINE = MySQL('mysql57:3307', 'mysql_db', 'mysql_table', 'mysql_user', '{password}')" f"CREATE TABLE table_oncl ON CLUSTER 'test_shard_localhost' (x int) ENGINE = MySQL('mysql80:3307', 'mysql_db', 'mysql_table', 'mysql_user', '{password}')"
) )
check_logs( check_logs(
must_contain=[ must_contain=[
"CREATE TABLE table_oncl ON CLUSTER test_shard_localhost (`x` int) ENGINE = MySQL('mysql57:3307', 'mysql_db', 'mysql_table', 'mysql_user', '[HIDDEN]')", "CREATE TABLE table_oncl ON CLUSTER test_shard_localhost (`x` int) ENGINE = MySQL('mysql80:3307', 'mysql_db', 'mysql_table', 'mysql_user', '[HIDDEN]')",
], ],
must_not_contain=[password], must_not_contain=[password],
) )
# Check logs of DDLWorker during executing of this query. # Check logs of DDLWorker during executing of this query.
assert node.contains_in_log( assert node.contains_in_log(
"DDLWorker: Processing task .*CREATE TABLE default\\.table_oncl UUID '[0-9a-fA-F-]*' (\\`x\\` Int32) ENGINE = MySQL('mysql57:3307', 'mysql_db', 'mysql_table', 'mysql_user', '\\[HIDDEN\\]')" "DDLWorker: Processing task .*CREATE TABLE default\\.table_oncl UUID '[0-9a-fA-F-]*' (\\`x\\` Int32) ENGINE = MySQL('mysql80:3307', 'mysql_db', 'mysql_table', 'mysql_user', '\\[HIDDEN\\]')"
) )
assert node.contains_in_log( assert node.contains_in_log(
"DDLWorker: Executing query: .*CREATE TABLE default\\.table_oncl UUID '[0-9a-fA-F-]*' (\\`x\\` Int32) ENGINE = MySQL('mysql57:3307', 'mysql_db', 'mysql_table', 'mysql_user', '\\[HIDDEN\\]')" "DDLWorker: Executing query: .*CREATE TABLE default\\.table_oncl UUID '[0-9a-fA-F-]*' (\\`x\\` Int32) ENGINE = MySQL('mysql80:3307', 'mysql_db', 'mysql_table', 'mysql_user', '\\[HIDDEN\\]')"
) )
assert node.contains_in_log( assert node.contains_in_log(
"executeQuery: .*CREATE TABLE default\\.table_oncl UUID '[0-9a-fA-F-]*' (\\`x\\` Int32) ENGINE = MySQL('mysql57:3307', 'mysql_db', 'mysql_table', 'mysql_user', '\\[HIDDEN\\]')" "executeQuery: .*CREATE TABLE default\\.table_oncl UUID '[0-9a-fA-F-]*' (\\`x\\` Int32) ENGINE = MySQL('mysql80:3307', 'mysql_db', 'mysql_table', 'mysql_user', '\\[HIDDEN\\]')"
) )
assert node.contains_in_log( assert node.contains_in_log(
"DDLWorker: Executed query: .*CREATE TABLE default\\.table_oncl UUID '[0-9a-fA-F-]*' (\\`x\\` Int32) ENGINE = MySQL('mysql57:3307', 'mysql_db', 'mysql_table', 'mysql_user', '\\[HIDDEN\\]')" "DDLWorker: Executed query: .*CREATE TABLE default\\.table_oncl UUID '[0-9a-fA-F-]*' (\\`x\\` Int32) ENGINE = MySQL('mysql80:3307', 'mysql_db', 'mysql_table', 'mysql_user', '\\[HIDDEN\\]')"
) )
assert system_query_log_contains_search_pattern( assert system_query_log_contains_search_pattern(
"%CREATE TABLE default.table_oncl UUID \\'%\\' (`x` Int32) ENGINE = MySQL(\\'mysql57:3307\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')" "%CREATE TABLE default.table_oncl UUID \\'%\\' (`x` Int32) ENGINE = MySQL(\\'mysql80:3307\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')"
) )
node.query("DROP TABLE table_oncl") node.query("DROP TABLE table_oncl")

View File

@ -21,7 +21,7 @@ node_db = cluster.add_instance(
"node1", "node1",
main_configs=["configs/timezone_config.xml"], main_configs=["configs/timezone_config.xml"],
user_configs=["configs/users.xml"], user_configs=["configs/users.xml"],
with_mysql=True, with_mysql57=True,
with_mysql8=True, with_mysql8=True,
stay_alive=True, stay_alive=True,
) )
@ -29,14 +29,16 @@ node_disable_bytes_settings = cluster.add_instance(
"node2", "node2",
main_configs=["configs/timezone_config.xml"], main_configs=["configs/timezone_config.xml"],
user_configs=["configs/users_disable_bytes_settings.xml"], user_configs=["configs/users_disable_bytes_settings.xml"],
with_mysql=False, with_mysql57=False,
with_mysql8=False,
stay_alive=True, stay_alive=True,
) )
node_disable_rows_settings = cluster.add_instance( node_disable_rows_settings = cluster.add_instance(
"node3", "node3",
main_configs=["configs/timezone_config.xml"], main_configs=["configs/timezone_config.xml"],
user_configs=["configs/users_disable_rows_settings.xml"], user_configs=["configs/users_disable_rows_settings.xml"],
with_mysql=False, with_mysql57=False,
with_mysql8=False,
stay_alive=True, stay_alive=True,
) )
@ -125,7 +127,7 @@ class MySQLConnection:
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
def started_mysql_5_7(): def started_mysql_5_7():
mysql_node = MySQLConnection( mysql_node = MySQLConnection(
cluster.mysql_port, "root", "clickhouse", cluster.mysql_ip cluster.mysql57_port, "root", "clickhouse", cluster.mysql57_ip
) )
yield mysql_node yield mysql_node

View File

@ -0,0 +1,23 @@
<clickhouse>
<named_collections>
<mysql1>
<user>root</user>
<password>clickhouse</password>
<host>mysql57</host>
<port>3306</port>
<database>test_database</database>
</mysql1>
<mysql2>
<user>postgres</user>
<password>mysecretpassword</password>
<host>postgres1</host>
</mysql2>
<mysql3>
<user>root</user>
<password>clickhouse</password>
<host>mysql57</host>
<port>1111</port>
<database>clickhouse</database>
</mysql3>
</named_collections>
</clickhouse>

View File

@ -0,0 +1,12 @@
<clickhouse>
<remote_servers>
<test_cluster>
<shard>
<replica>
<host>node1</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
</remote_servers>
</clickhouse>

View File

@ -0,0 +1,10 @@
<clickhouse>
<users>
<default>
<password></password>
<profile>default</profile>
<quota>default</quota>
<named_collection_control>1</named_collection_control>
</default>
</users>
</clickhouse>

View File

@ -0,0 +1,9 @@
<clickhouse>
<users>
<default>
<password></password>
<profile>default</profile>
<named_collection_control>1</named_collection_control>
</default>
</users>
</clickhouse>

File diff suppressed because it is too large Load Diff

View File

@ -3,7 +3,7 @@
<mysql1> <mysql1>
<user>root</user> <user>root</user>
<password>clickhouse</password> <password>clickhouse</password>
<host>mysql57</host> <host>mysql80</host>
<port>3306</port> <port>3306</port>
<database>test_database</database> <database>test_database</database>
</mysql1> </mysql1>
@ -15,7 +15,7 @@
<mysql3> <mysql3>
<user>root</user> <user>root</user>
<password>clickhouse</password> <password>clickhouse</password>
<host>mysql57</host> <host>mysql80</host>
<port>1111</port> <port>1111</port>
<database>clickhouse</database> <database>clickhouse</database>
</mysql3> </mysql3>

View File

@ -13,7 +13,7 @@ clickhouse_node = cluster.add_instance(
"node1", "node1",
main_configs=["configs/remote_servers.xml", "configs/named_collections.xml"], main_configs=["configs/remote_servers.xml", "configs/named_collections.xml"],
user_configs=["configs/users.xml"], user_configs=["configs/users.xml"],
with_mysql=True, with_mysql8=True,
stay_alive=True, stay_alive=True,
) )
@ -70,7 +70,7 @@ class MySQLNodeInstance:
def test_mysql_ddl_for_mysql_database(started_cluster): def test_mysql_ddl_for_mysql_database(started_cluster):
with contextlib.closing( with contextlib.closing(
MySQLNodeInstance( MySQLNodeInstance(
"root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port "root", "clickhouse", started_cluster.mysql8_ip, started_cluster.mysql8_port
) )
) as mysql_node: ) as mysql_node:
mysql_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("DROP DATABASE IF EXISTS test_database")
@ -78,7 +78,7 @@ def test_mysql_ddl_for_mysql_database(started_cluster):
clickhouse_node.query("DROP DATABASE IF EXISTS test_database") clickhouse_node.query("DROP DATABASE IF EXISTS test_database")
clickhouse_node.query( clickhouse_node.query(
"CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', 'test_database', 'root', 'clickhouse')" "CREATE DATABASE test_database ENGINE = MySQL('mysql80:3306', 'test_database', 'root', 'clickhouse')"
) )
assert "test_database" in clickhouse_node.query("SHOW DATABASES") assert "test_database" in clickhouse_node.query("SHOW DATABASES")
@ -121,7 +121,7 @@ def test_mysql_ddl_for_mysql_database(started_cluster):
def test_clickhouse_ddl_for_mysql_database(started_cluster): def test_clickhouse_ddl_for_mysql_database(started_cluster):
with contextlib.closing( with contextlib.closing(
MySQLNodeInstance( MySQLNodeInstance(
"root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port "root", "clickhouse", started_cluster.mysql8_ip, started_cluster.mysql8_port
) )
) as mysql_node: ) as mysql_node:
mysql_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("DROP DATABASE IF EXISTS test_database")
@ -132,7 +132,7 @@ def test_clickhouse_ddl_for_mysql_database(started_cluster):
clickhouse_node.query("DROP DATABASE IF EXISTS test_database") clickhouse_node.query("DROP DATABASE IF EXISTS test_database")
clickhouse_node.query( clickhouse_node.query(
"CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', 'test_database', 'root', 'clickhouse')" "CREATE DATABASE test_database ENGINE = MySQL('mysql80:3306', 'test_database', 'root', 'clickhouse')"
) )
assert "test_table" in clickhouse_node.query("SHOW TABLES FROM test_database") assert "test_table" in clickhouse_node.query("SHOW TABLES FROM test_database")
@ -158,7 +158,7 @@ def test_clickhouse_ddl_for_mysql_database(started_cluster):
def test_clickhouse_dml_for_mysql_database(started_cluster): def test_clickhouse_dml_for_mysql_database(started_cluster):
with contextlib.closing( with contextlib.closing(
MySQLNodeInstance( MySQLNodeInstance(
"root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port "root", "clickhouse", started_cluster.mysql8_ip, started_cluster.mysql8_port
) )
) as mysql_node: ) as mysql_node:
mysql_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("DROP DATABASE IF EXISTS test_database")
@ -169,7 +169,7 @@ def test_clickhouse_dml_for_mysql_database(started_cluster):
clickhouse_node.query("DROP DATABASE IF EXISTS test_database") clickhouse_node.query("DROP DATABASE IF EXISTS test_database")
clickhouse_node.query( clickhouse_node.query(
"CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', test_database, 'root', 'clickhouse')" "CREATE DATABASE test_database ENGINE = MySQL('mysql80:3306', test_database, 'root', 'clickhouse')"
) )
assert ( assert (
@ -197,7 +197,7 @@ def test_clickhouse_dml_for_mysql_database(started_cluster):
def test_clickhouse_join_for_mysql_database(started_cluster): def test_clickhouse_join_for_mysql_database(started_cluster):
with contextlib.closing( with contextlib.closing(
MySQLNodeInstance( MySQLNodeInstance(
"root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port "root", "clickhouse", started_cluster.mysql8_ip, started_cluster.mysql8_port
) )
) as mysql_node: ) as mysql_node:
mysql_node.query("DROP DATABASE IF EXISTS test") mysql_node.query("DROP DATABASE IF EXISTS test")
@ -218,10 +218,10 @@ def test_clickhouse_join_for_mysql_database(started_cluster):
clickhouse_node.query("DROP TABLE IF EXISTS default.t1_remote_mysql SYNC") clickhouse_node.query("DROP TABLE IF EXISTS default.t1_remote_mysql SYNC")
clickhouse_node.query("DROP TABLE IF EXISTS default.t2_remote_mysql SYNC") clickhouse_node.query("DROP TABLE IF EXISTS default.t2_remote_mysql SYNC")
clickhouse_node.query( clickhouse_node.query(
"CREATE TABLE default.t1_remote_mysql AS mysql('mysql57:3306','test','t1_mysql_local','root','clickhouse')" "CREATE TABLE default.t1_remote_mysql AS mysql('mysql80:3306','test','t1_mysql_local','root','clickhouse')"
) )
clickhouse_node.query( clickhouse_node.query(
"CREATE TABLE default.t2_remote_mysql AS mysql('mysql57:3306','test','t2_mysql_local','root','clickhouse')" "CREATE TABLE default.t2_remote_mysql AS mysql('mysql80:3306','test','t2_mysql_local','root','clickhouse')"
) )
clickhouse_node.query( clickhouse_node.query(
"INSERT INTO `default`.`t1_remote_mysql` VALUES ('EN','A',''),('RU','B','AAA')" "INSERT INTO `default`.`t1_remote_mysql` VALUES ('EN','A',''),('RU','B','AAA')"
@ -248,8 +248,8 @@ def test_bad_arguments_for_mysql_database_engine(started_cluster):
MySQLNodeInstance( MySQLNodeInstance(
"root", "root",
"clickhouse", "clickhouse",
started_cluster.mysql_ip, started_cluster.mysql8_ip,
port=started_cluster.mysql_port, port=started_cluster.mysql8_port,
) )
) as mysql_node: ) as mysql_node:
with pytest.raises(QueryRuntimeException) as exception: with pytest.raises(QueryRuntimeException) as exception:
@ -257,7 +257,7 @@ def test_bad_arguments_for_mysql_database_engine(started_cluster):
"CREATE DATABASE IF NOT EXISTS test_bad_arguments DEFAULT CHARACTER SET 'utf8'" "CREATE DATABASE IF NOT EXISTS test_bad_arguments DEFAULT CHARACTER SET 'utf8'"
) )
clickhouse_node.query( clickhouse_node.query(
"CREATE DATABASE test_database_bad_arguments ENGINE = MySQL('mysql57:3306', test_bad_arguments, root, 'clickhouse')" "CREATE DATABASE test_database_bad_arguments ENGINE = MySQL('mysql80:3306', test_bad_arguments, root, 'clickhouse')"
) )
assert "Database engine MySQL requested literal argument." in str( assert "Database engine MySQL requested literal argument." in str(
exception.value exception.value
@ -268,7 +268,7 @@ def test_bad_arguments_for_mysql_database_engine(started_cluster):
def test_column_comments_for_mysql_database_engine(started_cluster): def test_column_comments_for_mysql_database_engine(started_cluster):
with contextlib.closing( with contextlib.closing(
MySQLNodeInstance( MySQLNodeInstance(
"root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port "root", "clickhouse", started_cluster.mysql8_ip, started_cluster.mysql8_port
) )
) as mysql_node: ) as mysql_node:
mysql_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("DROP DATABASE IF EXISTS test_database")
@ -276,7 +276,7 @@ def test_column_comments_for_mysql_database_engine(started_cluster):
clickhouse_node.query("DROP DATABASE IF EXISTS test_database") clickhouse_node.query("DROP DATABASE IF EXISTS test_database")
clickhouse_node.query( clickhouse_node.query(
"CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', 'test_database', 'root', 'clickhouse')" "CREATE DATABASE test_database ENGINE = MySQL('mysql80:3306', 'test_database', 'root', 'clickhouse')"
) )
assert "test_database" in clickhouse_node.query("SHOW DATABASES") assert "test_database" in clickhouse_node.query("SHOW DATABASES")
@ -304,7 +304,7 @@ def test_column_comments_for_mysql_database_engine(started_cluster):
def test_data_types_support_level_for_mysql_database_engine(started_cluster): def test_data_types_support_level_for_mysql_database_engine(started_cluster):
with contextlib.closing( with contextlib.closing(
MySQLNodeInstance( MySQLNodeInstance(
"root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port "root", "clickhouse", started_cluster.mysql8_ip, started_cluster.mysql8_port
) )
) as mysql_node: ) as mysql_node:
mysql_node.query("DROP DATABASE IF EXISTS test") mysql_node.query("DROP DATABASE IF EXISTS test")
@ -313,7 +313,7 @@ def test_data_types_support_level_for_mysql_database_engine(started_cluster):
) )
clickhouse_node.query("DROP DATABASE IF EXISTS test_database") clickhouse_node.query("DROP DATABASE IF EXISTS test_database")
clickhouse_node.query( clickhouse_node.query(
"CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', test, 'root', 'clickhouse')", "CREATE DATABASE test_database ENGINE = MySQL('mysql80:3306', test, 'root', 'clickhouse')",
settings={"mysql_datatypes_support_level": "decimal,datetime64"}, settings={"mysql_datatypes_support_level": "decimal,datetime64"},
) )
@ -331,7 +331,7 @@ def test_data_types_support_level_for_mysql_database_engine(started_cluster):
) )
clickhouse_node.query( clickhouse_node.query(
"CREATE DATABASE test_database_1 ENGINE = MySQL('mysql57:3306', test, 'root', 'clickhouse') SETTINGS mysql_datatypes_support_level = 'decimal,datetime64'", "CREATE DATABASE test_database_1 ENGINE = MySQL('mysql80:3306', test, 'root', 'clickhouse') SETTINGS mysql_datatypes_support_level = 'decimal,datetime64'",
settings={"mysql_datatypes_support_level": "decimal"}, settings={"mysql_datatypes_support_level": "decimal"},
) )
@ -693,7 +693,7 @@ def arryToString(expected_clickhouse_values):
pytest.param( pytest.param(
"timestamp_default", "timestamp_default",
"timestamp", "timestamp",
"DateTime", "Nullable(DateTime)",
timestamp_values, timestamp_values,
"", "",
"decimal,datetime64", "decimal,datetime64",
@ -702,7 +702,7 @@ def arryToString(expected_clickhouse_values):
pytest.param( pytest.param(
"timestamp_6", "timestamp_6",
"timestamp(6)", "timestamp(6)",
"DateTime64(6)", "Nullable(DateTime64(6))",
timestamp_values, timestamp_values,
"", "",
"decimal,datetime64", "decimal,datetime64",
@ -817,8 +817,8 @@ def test_mysql_types(
MySQLNodeInstance( MySQLNodeInstance(
"root", "root",
"clickhouse", "clickhouse",
started_cluster.mysql_ip, started_cluster.mysql8_ip,
port=started_cluster.mysql_port, port=started_cluster.mysql8_port,
) )
) as mysql_node: ) as mysql_node:
execute_query( execute_query(
@ -842,7 +842,7 @@ def test_mysql_types(
clickhouse_node, clickhouse_node,
[ [
"DROP TABLE IF EXISTS ${ch_mysql_table};", "DROP TABLE IF EXISTS ${ch_mysql_table};",
"CREATE TABLE ${ch_mysql_table} (value ${expected_ch_type}) ENGINE = MySQL('mysql57:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse')", "CREATE TABLE ${ch_mysql_table} (value ${expected_ch_type}) ENGINE = MySQL('mysql80:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse')",
], ],
settings=clickhouse_query_settings, settings=clickhouse_query_settings,
) )
@ -875,7 +875,7 @@ def test_mysql_types(
clickhouse_node, clickhouse_node,
[ [
"DROP DATABASE IF EXISTS ${ch_mysql_db}", "DROP DATABASE IF EXISTS ${ch_mysql_db}",
"CREATE DATABASE ${ch_mysql_db} ENGINE = MySQL('mysql57:3306', '${mysql_db}', 'root', 'clickhouse')", "CREATE DATABASE ${ch_mysql_db} ENGINE = MySQL('mysql80:3306', '${mysql_db}', 'root', 'clickhouse')",
], ],
settings=clickhouse_query_settings, settings=clickhouse_query_settings,
) )
@ -902,7 +902,7 @@ def test_mysql_types(
assert ( assert (
execute_query( execute_query(
clickhouse_node, clickhouse_node,
"SELECT toTypeName(value) FROM mysql('mysql57:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse') LIMIT 1", "SELECT toTypeName(value) FROM mysql('mysql80:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse') LIMIT 1",
settings=clickhouse_query_settings, settings=clickhouse_query_settings,
) )
== expected_ch_type == expected_ch_type
@ -911,7 +911,7 @@ def test_mysql_types(
# Validate values # Validate values
assert expected_format_clickhouse_values == execute_query( assert expected_format_clickhouse_values == execute_query(
clickhouse_node, clickhouse_node,
"SELECT value FROM mysql('mysql57:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse')", "SELECT value FROM mysql('mysql80:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse')",
settings=clickhouse_query_settings, settings=clickhouse_query_settings,
) )
@ -919,7 +919,7 @@ def test_mysql_types(
def test_predefined_connection_configuration(started_cluster): def test_predefined_connection_configuration(started_cluster):
with contextlib.closing( with contextlib.closing(
MySQLNodeInstance( MySQLNodeInstance(
"root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port "root", "clickhouse", started_cluster.mysql8_ip, started_cluster.mysql8_port
) )
) as mysql_node: ) as mysql_node:
mysql_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("DROP DATABASE IF EXISTS test_database")
@ -971,13 +971,13 @@ def test_predefined_connection_configuration(started_cluster):
def test_restart_server(started_cluster): def test_restart_server(started_cluster):
with contextlib.closing( with contextlib.closing(
MySQLNodeInstance( MySQLNodeInstance(
"root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port "root", "clickhouse", started_cluster.mysql8_ip, started_cluster.mysql8_port
) )
) as mysql_node: ) as mysql_node:
mysql_node.query("DROP DATABASE IF EXISTS test_restart") mysql_node.query("DROP DATABASE IF EXISTS test_restart")
clickhouse_node.query("DROP DATABASE IF EXISTS test_restart") clickhouse_node.query("DROP DATABASE IF EXISTS test_restart")
clickhouse_node.query_and_get_error( clickhouse_node.query_and_get_error(
"CREATE DATABASE test_restart ENGINE = MySQL('mysql57:3306', 'test_restart', 'root', 'clickhouse')" "CREATE DATABASE test_restart ENGINE = MySQL('mysql80:3306', 'test_restart', 'root', 'clickhouse')"
) )
assert "test_restart" not in clickhouse_node.query("SHOW DATABASES") assert "test_restart" not in clickhouse_node.query("SHOW DATABASES")
@ -986,7 +986,7 @@ def test_restart_server(started_cluster):
"CREATE TABLE `test_restart`.`test_table` ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;" "CREATE TABLE `test_restart`.`test_table` ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;"
) )
clickhouse_node.query( clickhouse_node.query(
"CREATE DATABASE test_restart ENGINE = MySQL('mysql57:3306', 'test_restart', 'root', 'clickhouse')" "CREATE DATABASE test_restart ENGINE = MySQL('mysql80:3306', 'test_restart', 'root', 'clickhouse')"
) )
assert "test_restart" in clickhouse_node.query("SHOW DATABASES") assert "test_restart" in clickhouse_node.query("SHOW DATABASES")
@ -1004,7 +1004,7 @@ def test_restart_server(started_cluster):
def test_memory_leak(started_cluster): def test_memory_leak(started_cluster):
with contextlib.closing( with contextlib.closing(
MySQLNodeInstance( MySQLNodeInstance(
"root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port "root", "clickhouse", started_cluster.mysql8_ip, started_cluster.mysql8_port
) )
) as mysql_node: ) as mysql_node:
mysql_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("DROP DATABASE IF EXISTS test_database")
@ -1015,7 +1015,7 @@ def test_memory_leak(started_cluster):
clickhouse_node.query("DROP DATABASE IF EXISTS test_database") clickhouse_node.query("DROP DATABASE IF EXISTS test_database")
clickhouse_node.query( clickhouse_node.query(
"CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', 'test_database', 'root', 'clickhouse') SETTINGS connection_auto_close = 1" "CREATE DATABASE test_database ENGINE = MySQL('mysql80:3306', 'test_database', 'root', 'clickhouse') SETTINGS connection_auto_close = 1"
) )
clickhouse_node.query("SELECT count() FROM `test_database`.`test_table`") clickhouse_node.query("SELECT count() FROM `test_database`.`test_table`")
@ -1026,7 +1026,7 @@ def test_memory_leak(started_cluster):
def test_password_leak(started_cluster): def test_password_leak(started_cluster):
with contextlib.closing( with contextlib.closing(
MySQLNodeInstance( MySQLNodeInstance(
"root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port "root", "clickhouse", started_cluster.mysql8_ip, started_cluster.mysql8_port
) )
) as mysql_node: ) as mysql_node:
mysql_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("DROP DATABASE IF EXISTS test_database")
@ -1037,7 +1037,7 @@ def test_password_leak(started_cluster):
clickhouse_node.query("DROP DATABASE IF EXISTS test_database") clickhouse_node.query("DROP DATABASE IF EXISTS test_database")
clickhouse_node.query( clickhouse_node.query(
"CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', 'test_database', 'root', 'clickhouse') SETTINGS connection_auto_close = 1" "CREATE DATABASE test_database ENGINE = MySQL('mysql80:3306', 'test_database', 'root', 'clickhouse') SETTINGS connection_auto_close = 1"
) )
assert "clickhouse" not in clickhouse_node.query( assert "clickhouse" not in clickhouse_node.query(
"SHOW CREATE test_database.test_table" "SHOW CREATE test_database.test_table"

View File

@ -14,7 +14,7 @@ cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance( node1 = cluster.add_instance(
"node1", "node1",
with_odbc_drivers=True, with_odbc_drivers=True,
with_mysql=True, with_mysql8=True,
with_postgres=True, with_postgres=True,
main_configs=["configs/openssl.xml", "configs/odbc_logging.xml"], main_configs=["configs/openssl.xml", "configs/odbc_logging.xml"],
dictionaries=[ dictionaries=[
@ -55,13 +55,13 @@ def get_mysql_conn():
conn = pymysql.connect( conn = pymysql.connect(
user="root", user="root",
password="clickhouse", password="clickhouse",
host=cluster.mysql_ip, host=cluster.mysql8_ip,
port=cluster.mysql_port, port=cluster.mysql8_port,
) )
else: else:
conn.ping(reconnect=True) conn.ping(reconnect=True)
logging.debug( logging.debug(
f"MySQL Connection establised: {cluster.mysql_ip}:{cluster.mysql_port}" f"MySQL Connection establised: {cluster.mysql8_ip}:{cluster.mysql8_port}"
) )
return conn return conn
except Exception as e: except Exception as e:
@ -230,7 +230,7 @@ def test_mysql_simple_select_works(started_cluster):
node1.query( node1.query(
""" """
CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, column_x Nullable(UInt32)) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, column_x Nullable(UInt32)) ENGINE = MySQL('mysql80:3306', 'clickhouse', '{}', 'root', 'clickhouse');
""".format( """.format(
table_name, table_name table_name, table_name
) )

View File

@ -3,7 +3,7 @@
<mysql1> <mysql1>
<user>root</user> <user>root</user>
<password>clickhouse</password> <password>clickhouse</password>
<host>mysql57</host> <host>mysql80</host>
<port>3306</port> <port>3306</port>
<database>clickhouse</database> <database>clickhouse</database>
<table>test_table</table> <table>test_table</table>
@ -16,7 +16,7 @@
<mysql3> <mysql3>
<user>root</user> <user>root</user>
<password>clickhouse</password> <password>clickhouse</password>
<host>mysql57</host> <host>mysql80</host>
<port>1111</port> <port>1111</port>
<database>clickhouse</database> <database>clickhouse</database>
<table>test_table</table> <table>test_table</table>
@ -24,7 +24,7 @@
<mysql4> <mysql4>
<user>root</user> <user>root</user>
<password>clickhouse</password> <password>clickhouse</password>
<host>mysql57</host> <host>mysql80</host>
<port>3306</port> <port>3306</port>
<database>clickhouse</database> <database>clickhouse</database>
<table>test_table</table> <table>test_table</table>
@ -33,7 +33,7 @@
<mysql_with_settings> <mysql_with_settings>
<user>root</user> <user>root</user>
<password>clickhouse</password> <password>clickhouse</password>
<host>mysql57</host> <host>mysql80</host>
<port>3306</port> <port>3306</port>
<database>clickhouse</database> <database>clickhouse</database>
<connection_pool_size>1</connection_pool_size> <connection_pool_size>1</connection_pool_size>

View File

@ -14,7 +14,7 @@ node1 = cluster.add_instance(
"node1", "node1",
main_configs=["configs/remote_servers.xml", "configs/named_collections.xml"], main_configs=["configs/remote_servers.xml", "configs/named_collections.xml"],
user_configs=["configs/users.xml"], user_configs=["configs/users.xml"],
with_mysql=True, with_mysql8=True,
) )
node2 = cluster.add_instance( node2 = cluster.add_instance(
"node2", main_configs=["configs/remote_servers.xml"], with_mysql_cluster=True "node2", main_configs=["configs/remote_servers.xml"], with_mysql_cluster=True
@ -23,7 +23,7 @@ node3 = cluster.add_instance(
"node3", "node3",
main_configs=["configs/remote_servers.xml"], main_configs=["configs/remote_servers.xml"],
user_configs=["configs/users.xml"], user_configs=["configs/users.xml"],
with_mysql=True, with_mysql8=True,
) )
create_table_sql_template = """ create_table_sql_template = """
@ -43,7 +43,7 @@ drop_table_sql_template = """
def get_mysql_conn(started_cluster, host): def get_mysql_conn(started_cluster, host):
conn = pymysql.connect( conn = pymysql.connect(
user="root", password="clickhouse", host=host, port=started_cluster.mysql_port user="root", password="clickhouse", host=host, port=started_cluster.mysql8_port
) )
return conn return conn
@ -69,7 +69,7 @@ def started_cluster():
try: try:
cluster.start() cluster.start()
conn = get_mysql_conn(cluster, cluster.mysql_ip) conn = get_mysql_conn(cluster, cluster.mysql8_ip)
create_mysql_db(conn, "clickhouse") create_mysql_db(conn, "clickhouse")
## create mysql db and table ## create mysql db and table
@ -85,13 +85,13 @@ def test_many_connections(started_cluster):
table_name = "test_many_connections" table_name = "test_many_connections"
node1.query(f"DROP TABLE IF EXISTS {table_name}") node1.query(f"DROP TABLE IF EXISTS {table_name}")
conn = get_mysql_conn(started_cluster, cluster.mysql_ip) conn = get_mysql_conn(started_cluster, cluster.mysql8_ip)
drop_mysql_table(conn, table_name) drop_mysql_table(conn, table_name)
create_mysql_table(conn, table_name) create_mysql_table(conn, table_name)
node1.query( node1.query(
""" """
CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql80:3306', 'clickhouse', '{}', 'root', 'clickhouse');
""".format( """.format(
table_name, table_name table_name, table_name
) )
@ -116,13 +116,13 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL
def test_insert_select(started_cluster): def test_insert_select(started_cluster):
table_name = "test_insert_select" table_name = "test_insert_select"
node1.query(f"DROP TABLE IF EXISTS {table_name}") node1.query(f"DROP TABLE IF EXISTS {table_name}")
conn = get_mysql_conn(started_cluster, cluster.mysql_ip) conn = get_mysql_conn(started_cluster, cluster.mysql8_ip)
drop_mysql_table(conn, table_name) drop_mysql_table(conn, table_name)
create_mysql_table(conn, table_name) create_mysql_table(conn, table_name)
node1.query( node1.query(
""" """
CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql80:3306', 'clickhouse', '{}', 'root', 'clickhouse');
""".format( """.format(
table_name, table_name table_name, table_name
) )
@ -142,13 +142,13 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL
def test_replace_select(started_cluster): def test_replace_select(started_cluster):
table_name = "test_replace_select" table_name = "test_replace_select"
node1.query(f"DROP TABLE IF EXISTS {table_name}") node1.query(f"DROP TABLE IF EXISTS {table_name}")
conn = get_mysql_conn(started_cluster, cluster.mysql_ip) conn = get_mysql_conn(started_cluster, cluster.mysql8_ip)
drop_mysql_table(conn, table_name) drop_mysql_table(conn, table_name)
create_mysql_table(conn, table_name) create_mysql_table(conn, table_name)
node1.query( node1.query(
""" """
CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql80:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1);
""".format( """.format(
table_name, table_name table_name, table_name
) )
@ -173,13 +173,13 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL
def test_insert_on_duplicate_select(started_cluster): def test_insert_on_duplicate_select(started_cluster):
table_name = "test_insert_on_duplicate_select" table_name = "test_insert_on_duplicate_select"
node1.query(f"DROP TABLE IF EXISTS {table_name}") node1.query(f"DROP TABLE IF EXISTS {table_name}")
conn = get_mysql_conn(started_cluster, cluster.mysql_ip) conn = get_mysql_conn(started_cluster, cluster.mysql8_ip)
drop_mysql_table(conn, table_name) drop_mysql_table(conn, table_name)
create_mysql_table(conn, table_name) create_mysql_table(conn, table_name)
node1.query( node1.query(
""" """
CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse', 0, 'update money = money + values(money)'); CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql80:3306', 'clickhouse', '{}', 'root', 'clickhouse', 0, 'update money = money + values(money)');
""".format( """.format(
table_name, table_name table_name, table_name
) )
@ -205,12 +205,12 @@ def test_where(started_cluster):
table_name = "test_where" table_name = "test_where"
node1.query(f"DROP TABLE IF EXISTS {table_name}") node1.query(f"DROP TABLE IF EXISTS {table_name}")
conn = get_mysql_conn(started_cluster, cluster.mysql_ip) conn = get_mysql_conn(started_cluster, cluster.mysql8_ip)
drop_mysql_table(conn, table_name) drop_mysql_table(conn, table_name)
create_mysql_table(conn, table_name) create_mysql_table(conn, table_name)
node1.query( node1.query(
""" """
CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql80:3306', 'clickhouse', '{}', 'root', 'clickhouse');
""".format( """.format(
table_name, table_name table_name, table_name
) )
@ -264,11 +264,11 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL
def test_table_function(started_cluster): def test_table_function(started_cluster):
conn = get_mysql_conn(started_cluster, cluster.mysql_ip) conn = get_mysql_conn(started_cluster, cluster.mysql8_ip)
drop_mysql_table(conn, "table_function") drop_mysql_table(conn, "table_function")
create_mysql_table(conn, "table_function") create_mysql_table(conn, "table_function")
table_function = ( table_function = (
"mysql('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format( "mysql('mysql80:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format(
"table_function" "table_function"
) )
) )
@ -309,7 +309,7 @@ def test_table_function(started_cluster):
def test_schema_inference(started_cluster): def test_schema_inference(started_cluster):
conn = get_mysql_conn(started_cluster, cluster.mysql_ip) conn = get_mysql_conn(started_cluster, cluster.mysql8_ip)
drop_mysql_table(conn, "inference_table") drop_mysql_table(conn, "inference_table")
with conn.cursor() as cursor: with conn.cursor() as cursor:
@ -317,7 +317,7 @@ def test_schema_inference(started_cluster):
"CREATE TABLE clickhouse.inference_table (id INT PRIMARY KEY, data BINARY(16) NOT NULL)" "CREATE TABLE clickhouse.inference_table (id INT PRIMARY KEY, data BINARY(16) NOT NULL)"
) )
parameters = "'mysql57:3306', 'clickhouse', 'inference_table', 'root', 'clickhouse'" parameters = "'mysql80:3306', 'clickhouse', 'inference_table', 'root', 'clickhouse'"
node1.query( node1.query(
f"CREATE TABLE mysql_schema_inference_engine ENGINE=MySQL({parameters})" f"CREATE TABLE mysql_schema_inference_engine ENGINE=MySQL({parameters})"
@ -335,7 +335,7 @@ def test_schema_inference(started_cluster):
def test_binary_type(started_cluster): def test_binary_type(started_cluster):
conn = get_mysql_conn(started_cluster, cluster.mysql_ip) conn = get_mysql_conn(started_cluster, cluster.mysql8_ip)
drop_mysql_table(conn, "binary_type") drop_mysql_table(conn, "binary_type")
with conn.cursor() as cursor: with conn.cursor() as cursor:
@ -343,7 +343,7 @@ def test_binary_type(started_cluster):
"CREATE TABLE clickhouse.binary_type (id INT PRIMARY KEY, data BINARY(16) NOT NULL)" "CREATE TABLE clickhouse.binary_type (id INT PRIMARY KEY, data BINARY(16) NOT NULL)"
) )
table_function = ( table_function = (
"mysql('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format( "mysql('mysql80:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format(
"binary_type" "binary_type"
) )
) )
@ -363,12 +363,12 @@ def test_enum_type(started_cluster):
table_name = "test_enum_type" table_name = "test_enum_type"
node1.query(f"DROP TABLE IF EXISTS {table_name}") node1.query(f"DROP TABLE IF EXISTS {table_name}")
conn = get_mysql_conn(started_cluster, cluster.mysql_ip) conn = get_mysql_conn(started_cluster, cluster.mysql8_ip)
drop_mysql_table(conn, table_name) drop_mysql_table(conn, table_name)
create_mysql_table(conn, table_name) create_mysql_table(conn, table_name)
node1.query( node1.query(
""" """
CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8('IP' = 1, 'URL' = 2)) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8('IP' = 1, 'URL' = 2)) ENGINE = MySQL('mysql80:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1);
""".format( """.format(
table_name, table_name table_name, table_name
) )
@ -388,7 +388,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8('
def test_mysql_distributed(started_cluster): def test_mysql_distributed(started_cluster):
table_name = "test_replicas" table_name = "test_replicas"
conn1 = get_mysql_conn(started_cluster, started_cluster.mysql_ip) conn1 = get_mysql_conn(started_cluster, started_cluster.mysql8_ip)
conn2 = get_mysql_conn(started_cluster, started_cluster.mysql2_ip) conn2 = get_mysql_conn(started_cluster, started_cluster.mysql2_ip)
conn3 = get_mysql_conn(started_cluster, started_cluster.mysql3_ip) conn3 = get_mysql_conn(started_cluster, started_cluster.mysql3_ip)
conn4 = get_mysql_conn(started_cluster, started_cluster.mysql4_ip) conn4 = get_mysql_conn(started_cluster, started_cluster.mysql4_ip)
@ -422,7 +422,7 @@ def test_mysql_distributed(started_cluster):
CREATE TABLE test_replica{} CREATE TABLE test_replica{}
(id UInt32, name String, age UInt32, money UInt32) (id UInt32, name String, age UInt32, money UInt32)
ENGINE = MySQL('mysql{}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse');""".format( ENGINE = MySQL('mysql{}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse');""".format(
i, 57 if i == 1 else i i, 80 if i == 1 else i
) )
) )
nodes[i - 1].query( nodes[i - 1].query(
@ -433,11 +433,11 @@ def test_mysql_distributed(started_cluster):
# test multiple ports parsing # test multiple ports parsing
result = node2.query( result = node2.query(
"""SELECT DISTINCT(name) FROM mysql('mysql{57|2|3}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """ """SELECT DISTINCT(name) FROM mysql('mysql{80|2|3}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """
) )
assert result == "host1\n" or result == "host2\n" or result == "host3\n" assert result == "host1\n" or result == "host2\n" or result == "host3\n"
result = node2.query( result = node2.query(
"""SELECT DISTINCT(name) FROM mysql('mysql57:3306|mysql2:3306|mysql3:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """ """SELECT DISTINCT(name) FROM mysql('mysql80:3306|mysql2:3306|mysql3:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """
) )
assert result == "host1\n" or result == "host2\n" or result == "host3\n" assert result == "host1\n" or result == "host2\n" or result == "host3\n"
@ -457,7 +457,7 @@ def test_mysql_distributed(started_cluster):
""" """
CREATE TABLE test_shards CREATE TABLE test_shards
(id UInt32, name String, age UInt32, money UInt32) (id UInt32, name String, age UInt32, money UInt32)
ENGINE = ExternalDistributed('MySQL', 'mysql{57|2}:3306,mysql{3|4}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """ ENGINE = ExternalDistributed('MySQL', 'mysql{80|2}:3306,mysql{3|4}:3306', 'clickhouse', 'test_replicas', 'root', 'clickhouse'); """
) )
# Check only one replica in each shard is used # Check only one replica in each shard is used
@ -472,24 +472,24 @@ def test_mysql_distributed(started_cluster):
result = node2.query(query) result = node2.query(query)
assert result == "host1\nhost2\nhost3\nhost4\n" assert result == "host1\nhost2\nhost3\nhost4\n"
# disconnect mysql57 # disconnect mysql
started_cluster.pause_container("mysql57") started_cluster.pause_container("mysql80")
result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name") result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name")
started_cluster.unpause_container("mysql57") started_cluster.unpause_container("mysql80")
assert result == "host2\nhost4\n" or result == "host3\nhost4\n" assert result == "host2\nhost4\n" or result == "host3\nhost4\n"
def test_external_settings(started_cluster): def test_external_settings(started_cluster):
table_name = "test_external_settings" table_name = "test_external_settings"
node1.query(f"DROP TABLE IF EXISTS {table_name}") node1.query(f"DROP TABLE IF EXISTS {table_name}")
conn = get_mysql_conn(started_cluster, started_cluster.mysql_ip) conn = get_mysql_conn(started_cluster, started_cluster.mysql8_ip)
drop_mysql_table(conn, table_name) drop_mysql_table(conn, table_name)
create_mysql_table(conn, table_name) create_mysql_table(conn, table_name)
node3.query(f"DROP TABLE IF EXISTS {table_name}") node3.query(f"DROP TABLE IF EXISTS {table_name}")
node3.query( node3.query(
""" """
CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql80:3306', 'clickhouse', '{}', 'root', 'clickhouse');
""".format( """.format(
table_name, table_name table_name, table_name
) )
@ -521,7 +521,7 @@ def test_settings_connection_wait_timeout(started_cluster):
node1.query(f"DROP TABLE IF EXISTS {table_name}") node1.query(f"DROP TABLE IF EXISTS {table_name}")
wait_timeout = 2 wait_timeout = 2
conn = get_mysql_conn(started_cluster, cluster.mysql_ip) conn = get_mysql_conn(started_cluster, cluster.mysql8_ip)
drop_mysql_table(conn, table_name) drop_mysql_table(conn, table_name)
create_mysql_table(conn, table_name) create_mysql_table(conn, table_name)
@ -534,7 +534,7 @@ def test_settings_connection_wait_timeout(started_cluster):
age UInt32, age UInt32,
money UInt32 money UInt32
) )
ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse') ENGINE = MySQL('mysql80:3306', 'clickhouse', '{}', 'root', 'clickhouse')
SETTINGS connection_wait_timeout={}, connection_pool_size=1 SETTINGS connection_wait_timeout={}, connection_pool_size=1
""".format( """.format(
table_name, table_name, wait_timeout table_name, table_name, wait_timeout
@ -584,7 +584,7 @@ def test_settings_connection_wait_timeout(started_cluster):
def test_predefined_connection_configuration(started_cluster): def test_predefined_connection_configuration(started_cluster):
conn = get_mysql_conn(started_cluster, started_cluster.mysql_ip) conn = get_mysql_conn(started_cluster, started_cluster.mysql8_ip)
table_name = "test_table" table_name = "test_table"
drop_mysql_table(conn, table_name) drop_mysql_table(conn, table_name)
create_mysql_table(conn, table_name) create_mysql_table(conn, table_name)
@ -671,7 +671,7 @@ def test_mysql_in(started_cluster):
table_name = "test_mysql_in" table_name = "test_mysql_in"
node1.query(f"DROP TABLE IF EXISTS {table_name}") node1.query(f"DROP TABLE IF EXISTS {table_name}")
conn = get_mysql_conn(started_cluster, cluster.mysql_ip) conn = get_mysql_conn(started_cluster, cluster.mysql8_ip)
drop_mysql_table(conn, table_name) drop_mysql_table(conn, table_name)
create_mysql_table(conn, table_name) create_mysql_table(conn, table_name)
@ -684,7 +684,7 @@ def test_mysql_in(started_cluster):
age UInt32, age UInt32,
money UInt32 money UInt32
) )
ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse') ENGINE = MySQL('mysql80:3306', 'clickhouse', '{}', 'root', 'clickhouse')
""".format( """.format(
table_name, table_name table_name, table_name
) )
@ -714,7 +714,7 @@ def test_mysql_null(started_cluster):
table_name = "test_mysql_in" table_name = "test_mysql_in"
node1.query(f"DROP TABLE IF EXISTS {table_name}") node1.query(f"DROP TABLE IF EXISTS {table_name}")
conn = get_mysql_conn(started_cluster, cluster.mysql_ip) conn = get_mysql_conn(started_cluster, cluster.mysql8_ip)
drop_mysql_table(conn, table_name) drop_mysql_table(conn, table_name)
with conn.cursor() as cursor: with conn.cursor() as cursor:
cursor.execute( cursor.execute(
@ -735,7 +735,7 @@ def test_mysql_null(started_cluster):
id UInt32, id UInt32,
money Nullable(UInt32) money Nullable(UInt32)
) )
ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse') ENGINE = MySQL('mysql80:3306', 'clickhouse', '{}', 'root', 'clickhouse')
""".format( """.format(
table_name, table_name table_name, table_name
) )
@ -780,7 +780,7 @@ def test_settings(started_cluster):
connect_timeout = 10123002 connect_timeout = 10123002
connection_pool_size = 1 connection_pool_size = 1
conn = get_mysql_conn(started_cluster, cluster.mysql_ip) conn = get_mysql_conn(started_cluster, cluster.mysql8_ip)
drop_mysql_table(conn, table_name) drop_mysql_table(conn, table_name)
create_mysql_table(conn, table_name) create_mysql_table(conn, table_name)
@ -793,7 +793,7 @@ def test_settings(started_cluster):
age UInt32, age UInt32,
money UInt32 money UInt32
) )
ENGINE = MySQL('mysql57:3306', 'clickhouse', '{table_name}', 'root', 'clickhouse') ENGINE = MySQL('mysql80:3306', 'clickhouse', '{table_name}', 'root', 'clickhouse')
SETTINGS connection_wait_timeout={wait_timeout}, connect_timeout={connect_timeout}, read_write_timeout={rw_timeout}, connection_pool_size={connection_pool_size} SETTINGS connection_wait_timeout={wait_timeout}, connect_timeout={connect_timeout}, read_write_timeout={rw_timeout}, connection_pool_size={connection_pool_size}
""" """
) )
@ -815,7 +815,7 @@ def test_settings(started_cluster):
node1.query( node1.query(
f""" f"""
SELECT * SELECT *
FROM mysql('mysql57:3306', 'clickhouse', '{table_name}', 'root', 'clickhouse', FROM mysql('mysql80:3306', 'clickhouse', '{table_name}', 'root', 'clickhouse',
SETTINGS SETTINGS
connection_wait_timeout={wait_timeout}, connection_wait_timeout={wait_timeout},
connect_timeout={connect_timeout}, connect_timeout={connect_timeout},
@ -843,7 +843,7 @@ def test_settings(started_cluster):
connect_timeout = 50123002 connect_timeout = 50123002
node1.query( node1.query(
f""" f"""
CREATE DATABASE mm ENGINE = MySQL('mysql57:3306', 'clickhouse', 'root', 'clickhouse') CREATE DATABASE mm ENGINE = MySQL('mysql80:3306', 'clickhouse', 'root', 'clickhouse')
SETTINGS SETTINGS
connection_wait_timeout={wait_timeout}, connection_wait_timeout={wait_timeout},
connect_timeout={connect_timeout}, connect_timeout={connect_timeout},
@ -863,7 +863,7 @@ def test_mysql_point(started_cluster):
table_name = "test_mysql_point" table_name = "test_mysql_point"
node1.query(f"DROP TABLE IF EXISTS {table_name}") node1.query(f"DROP TABLE IF EXISTS {table_name}")
conn = get_mysql_conn(started_cluster, cluster.mysql_ip) conn = get_mysql_conn(started_cluster, cluster.mysql8_ip)
drop_mysql_table(conn, table_name) drop_mysql_table(conn, table_name)
with conn.cursor() as cursor: with conn.cursor() as cursor:
cursor.execute( cursor.execute(
@ -882,25 +882,25 @@ def test_mysql_point(started_cluster):
conn.commit() conn.commit()
result = node1.query( result = node1.query(
f"DESCRIBE mysql('mysql57:3306', 'clickhouse', '{table_name}', 'root', 'clickhouse')" f"DESCRIBE mysql('mysql80:3306', 'clickhouse', '{table_name}', 'root', 'clickhouse')"
) )
assert result.strip() == "id\tInt32\t\t\t\t\t\npoint\tPoint" assert result.strip() == "id\tInt32\t\t\t\t\t\npoint\tPoint"
assert 1 == int( assert 1 == int(
node1.query( node1.query(
f"SELECT count() FROM mysql('mysql57:3306', 'clickhouse', '{table_name}', 'root', 'clickhouse')" f"SELECT count() FROM mysql('mysql80:3306', 'clickhouse', '{table_name}', 'root', 'clickhouse')"
) )
) )
assert ( assert (
"(15,20)" "(15,20)"
== node1.query( == node1.query(
f"SELECT point FROM mysql('mysql57:3306', 'clickhouse', '{table_name}', 'root', 'clickhouse')" f"SELECT point FROM mysql('mysql80:3306', 'clickhouse', '{table_name}', 'root', 'clickhouse')"
).strip() ).strip()
) )
node1.query("DROP TABLE IF EXISTS test") node1.query("DROP TABLE IF EXISTS test")
node1.query( node1.query(
f"CREATE TABLE test (id Int32, point Point) Engine=MySQL('mysql57:3306', 'clickhouse', '{table_name}', 'root', 'clickhouse')" f"CREATE TABLE test (id Int32, point Point) Engine=MySQL('mysql80:3306', 'clickhouse', '{table_name}', 'root', 'clickhouse')"
) )
assert "(15,20)" == node1.query(f"SELECT point FROM test").strip() assert "(15,20)" == node1.query(f"SELECT point FROM test").strip()