mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
wip
This commit is contained in:
parent
534500ef84
commit
e6f90298bb
@ -40,7 +40,6 @@ ENV TZ=Europe/Moscow
|
||||
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
|
||||
|
||||
ENV DOCKER_CHANNEL stable
|
||||
ENV DOCKER_VERSION 5:19.03.13~3-0~ubuntu-bionic
|
||||
RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add -
|
||||
RUN add-apt-repository "deb [arch=amd64] https://download.docker.com/linux/ubuntu $(lsb_release -c -s) ${DOCKER_CHANNEL}"
|
||||
|
||||
|
@ -12,3 +12,10 @@ services:
|
||||
--gtid-mode="ON"
|
||||
--enforce-gtid-consistency
|
||||
--log-error-verbosity=3
|
||||
--log-error=/var/log/mysqld/error.log
|
||||
--general-log=ON
|
||||
--general-log-file=/var/log/mysqld/general.log
|
||||
volumes:
|
||||
- type: ${MYSQL_LOGS_FS:-tmpfs}
|
||||
source: ${MYSQL_LOGS:-}
|
||||
target: /var/log/mysqld/
|
@ -5,9 +5,19 @@ services:
|
||||
restart: always
|
||||
environment:
|
||||
MYSQL_ROOT_PASSWORD: clickhouse
|
||||
ports:
|
||||
- ${MYSQL8_EXTERNAL_PORT}:${MYSQL8_INTERNAL_PORT}
|
||||
MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST}
|
||||
DATADIR: /mysql/
|
||||
expose:
|
||||
- ${MYSQL8_PORT}
|
||||
command: --server_id=100 --log-bin='mysql-bin-1.log'
|
||||
--default_authentication_plugin='mysql_native_password'
|
||||
--default-time-zone='+3:00' --gtid-mode="ON"
|
||||
--enforce-gtid-consistency
|
||||
--log-error-verbosity=3
|
||||
--log-error=/mysql/error.log
|
||||
--general-log=ON
|
||||
--general-log-file=/mysql/general.log
|
||||
volumes:
|
||||
- type: ${MYSQL8_LOGS_FS:-tmpfs}
|
||||
source: ${MYSQL8_LOGS:-}
|
||||
target: /mysql/
|
@ -1,7 +1,7 @@
|
||||
#!/bin/bash
|
||||
set -e
|
||||
|
||||
dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 &>/var/log/somefile &
|
||||
dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=10.0.0.0/8,size=24 &>/var/log/somefile &
|
||||
|
||||
set +e
|
||||
reties=0
|
||||
|
@ -249,7 +249,8 @@ class ClickHouseCluster:
|
||||
|
||||
# available when with_mysql8 == True
|
||||
self.mysql8_host = "mysql80"
|
||||
self.mysql8_port = get_open_port()
|
||||
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")
|
||||
|
||||
@ -281,8 +282,8 @@ class ClickHouseCluster:
|
||||
def setup_mysql8_cmd(self, instance, env_variables, docker_compose_yml_dir):
|
||||
self.with_mysql8 = True
|
||||
env_variables['MYSQL8_HOST'] = self.mysql8_host
|
||||
env_variables['MYSQL8_EXTERNAL_PORT'] = str(self.mysql8_port)
|
||||
env_variables['MYSQL8_INTERNAL_PORT'] = "3306"
|
||||
env_variables['MYSQL8_PORT'] = str(self.mysql8_port)
|
||||
env_variables['MYSQL8_ROOT_HOST'] = '%'
|
||||
env_variables['MYSQL8_LOGS'] = self.mysql8_logs_dir
|
||||
env_variables['MYSQL8_LOGS_FS'] = "bind"
|
||||
self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql_8_0.yml')])
|
||||
@ -578,7 +579,7 @@ class ClickHouseCluster:
|
||||
run_and_check(self.base_cmd + ["up", "--force-recreate", "--no-deps", "-d", node.name])
|
||||
node.ip_address = self.get_instance_ip(node.name)
|
||||
node.client = Client(node.ip_address, command=self.client_bin_path)
|
||||
start_deadline = time.time() + 20.0 # seconds
|
||||
start_deadline = time.time() + 60.0 # seconds
|
||||
node.wait_for_start(start_deadline)
|
||||
return node
|
||||
|
||||
@ -876,6 +877,8 @@ class ClickHouseCluster:
|
||||
|
||||
if self.with_mysql and self.base_mysql_cmd:
|
||||
logging.debug('Setup MySQL')
|
||||
if os.path.exists(self.mysql_dir):
|
||||
shutil.rmtree(self.mysql_dir)
|
||||
os.makedirs(self.mysql_logs_dir)
|
||||
os.chmod(self.mysql_logs_dir, stat.S_IRWXO)
|
||||
subprocess_check_call(self.base_mysql_cmd + common_opts)
|
||||
@ -883,7 +886,10 @@ class ClickHouseCluster:
|
||||
|
||||
if self.with_mysql8 and self.base_mysql8_cmd:
|
||||
logging.debug('Setup MySQL 8')
|
||||
if os.path.exists(self.mysql8_dir):
|
||||
shutil.rmtree(self.mysql8_dir)
|
||||
os.makedirs(self.mysql8_logs_dir)
|
||||
os.chmod(self.mysql8_logs_dir, stat.S_IRWXO)
|
||||
subprocess_check_call(self.base_mysql8_cmd + common_opts)
|
||||
self.wait_mysql8_to_start(120)
|
||||
|
||||
@ -958,7 +964,7 @@ class ClickHouseCluster:
|
||||
subprocess_check_call(clickhouse_start_cmd)
|
||||
logging.debug("ClickHouse instance created")
|
||||
|
||||
start_deadline = time.time() + 30.0 # seconds
|
||||
start_deadline = time.time() + 60.0 # seconds
|
||||
for instance in self.instances.values():
|
||||
instance.docker_client = self.docker_client
|
||||
instance.ip_address = self.get_instance_ip(instance.name)
|
||||
@ -1372,7 +1378,7 @@ class ClickHouseInstance:
|
||||
return None
|
||||
return None
|
||||
|
||||
def restart_with_latest_version(self, stop_start_wait_sec=15, callback_onstop=None, signal=15):
|
||||
def restart_with_latest_version(self, stop_start_wait_sec=15, callback_onstop=None, signal=60):
|
||||
if not self.stay_alive:
|
||||
raise Exception("Cannot restart not stay alive container")
|
||||
self.exec_in_container(["bash", "-c", "pkill -{} clickhouse".format(signal)], user='root')
|
||||
|
@ -159,12 +159,12 @@ class _NetworkManager:
|
||||
|
||||
def __init__(
|
||||
self,
|
||||
container_expire_timeout=50, container_exit_timeout=60):
|
||||
container_expire_timeout=50, container_exit_timeout=60, docker_api_version=os.environ.get("DOCKER_API_VERSION")):
|
||||
|
||||
self.container_expire_timeout = container_expire_timeout
|
||||
self.container_exit_timeout = container_exit_timeout
|
||||
|
||||
self._docker_client = docker.from_env(version=os.environ.get("DOCKER_API_VERSION"))
|
||||
self._docker_client = docker.DockerClient(base_url='unix:///var/run/docker.sock', version=docker_api_version, timeout=180)
|
||||
|
||||
self._container = None
|
||||
|
||||
|
@ -199,7 +199,7 @@ if __name__ == "__main__":
|
||||
cmd = "docker run {net} {tty} --rm --name {name} --privileged --volume={bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \
|
||||
--volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \
|
||||
--volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \
|
||||
--volume={name}_volume:/var/lib/docker -e DOCKER_CLIENT_TIMEOUT=120 -e COMPOSE_HTTP_TIMEOUT=120 \
|
||||
--volume={name}_volume:/var/lib/docker -e DOCKER_CLIENT_TIMEOUT=120 -e COMPOSE_HTTP_TIMEOUT=180 \
|
||||
{env_tags} -e PYTEST_OPTS='{parallel} {opts}' {img} {command}".format(
|
||||
net=net,
|
||||
tty=tty,
|
||||
|
@ -1,4 +1,4 @@
|
||||
<yandex>
|
||||
<merge_tree>
|
||||
<merge_tree replace="replace">
|
||||
</merge_tree>
|
||||
</yandex>
|
||||
|
@ -268,6 +268,11 @@ def test_version_single_node_update(start_dynamic_cluster, n, tables):
|
||||
]
|
||||
)
|
||||
def test_mixed_granularity_single_node(start_dynamic_cluster, node):
|
||||
assert node.name == "node9" or node.name == "node10"
|
||||
assert_eq_with_retry(node,
|
||||
"SELECT value FROM system.merge_tree_settings WHERE name='enable_mixed_granularity_parts'",
|
||||
'0') # check that enable_mixed_granularity_parts is off by default
|
||||
|
||||
node.query(
|
||||
"INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)")
|
||||
node.query(
|
||||
|
@ -252,10 +252,10 @@ class Task_non_partitioned_table:
|
||||
instance.query("DROP TABLE copier_test1_1")
|
||||
|
||||
|
||||
def execute_task(task, cmd_options):
|
||||
def execute_task(started_cluster, task, cmd_options):
|
||||
task.start()
|
||||
|
||||
zk = cluster.get_kazoo_client('zoo1')
|
||||
zk = started_cluster.get_kazoo_client('zoo1')
|
||||
print("Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1]))
|
||||
|
||||
try:
|
||||
@ -268,7 +268,7 @@ def execute_task(task, cmd_options):
|
||||
zk.create(zk_task_path + "/description", task.copier_task_config.encode())
|
||||
|
||||
# Run cluster-copier processes on each node
|
||||
docker_api = docker.from_env().api
|
||||
docker_api = started_cluster.docker_client.api
|
||||
copiers_exec_ids = []
|
||||
|
||||
cmd = ['/usr/bin/clickhouse', 'copier',
|
||||
@ -280,7 +280,7 @@ def execute_task(task, cmd_options):
|
||||
copiers = random.sample(list(cluster.instances.keys()), 3)
|
||||
|
||||
for instance_name in copiers:
|
||||
instance = cluster.instances[instance_name]
|
||||
instance = started_cluster.instances[instance_name]
|
||||
container = instance.get_docker_handle()
|
||||
instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs/config-copier.xml"),
|
||||
"/etc/clickhouse-server/config-copier.xml")
|
||||
@ -293,7 +293,7 @@ def execute_task(task, cmd_options):
|
||||
|
||||
# Wait for copiers stopping and check their return codes
|
||||
for exec_id, instance_name in zip(copiers_exec_ids, copiers):
|
||||
instance = cluster.instances[instance_name]
|
||||
instance = started_cluster.instances[instance_name]
|
||||
while True:
|
||||
res = docker_api.exec_inspect(exec_id)
|
||||
if not res['Running']:
|
||||
@ -319,9 +319,9 @@ def execute_task(task, cmd_options):
|
||||
)
|
||||
def test_copy_simple(started_cluster, use_sample_offset):
|
||||
if use_sample_offset:
|
||||
execute_task(Task1(started_cluster), ['--experimental-use-sample-offset', '1'])
|
||||
execute_task(started_cluster, Task1(started_cluster), ['--experimental-use-sample-offset', '1'])
|
||||
else:
|
||||
execute_task(Task1(started_cluster), [])
|
||||
execute_task(started_cluster, Task1(started_cluster), [])
|
||||
|
||||
|
||||
@pytest.mark.parametrize(
|
||||
@ -333,10 +333,10 @@ def test_copy_simple(started_cluster, use_sample_offset):
|
||||
)
|
||||
def test_copy_with_recovering(started_cluster, use_sample_offset):
|
||||
if use_sample_offset:
|
||||
execute_task(Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY),
|
||||
execute_task(started_cluster, Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY),
|
||||
'--experimental-use-sample-offset', '1'])
|
||||
else:
|
||||
execute_task(Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)])
|
||||
execute_task(started_cluster, Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)])
|
||||
|
||||
|
||||
@pytest.mark.parametrize(
|
||||
@ -348,40 +348,40 @@ def test_copy_with_recovering(started_cluster, use_sample_offset):
|
||||
)
|
||||
def test_copy_with_recovering_after_move_faults(started_cluster, use_sample_offset):
|
||||
if use_sample_offset:
|
||||
execute_task(Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY),
|
||||
execute_task(started_cluster, Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY),
|
||||
'--experimental-use-sample-offset', '1'])
|
||||
else:
|
||||
execute_task(Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)])
|
||||
execute_task(started_cluster, Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)])
|
||||
|
||||
|
||||
@pytest.mark.timeout(600)
|
||||
def test_copy_month_to_week_partition(started_cluster):
|
||||
execute_task(Task2(started_cluster, "test1"), [])
|
||||
execute_task(started_cluster, Task2(started_cluster, "test1"), [])
|
||||
|
||||
|
||||
@pytest.mark.timeout(600)
|
||||
def test_copy_month_to_week_partition_with_recovering(started_cluster):
|
||||
execute_task(Task2(started_cluster, "test2"), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)])
|
||||
execute_task(started_cluster, Task2(started_cluster, "test2"), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)])
|
||||
|
||||
|
||||
@pytest.mark.timeout(600)
|
||||
def test_copy_month_to_week_partition_with_recovering_after_move_faults(started_cluster):
|
||||
execute_task(Task2(started_cluster, "test3"), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)])
|
||||
execute_task(started_cluster, Task2(started_cluster, "test3"), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)])
|
||||
|
||||
|
||||
def test_block_size(started_cluster):
|
||||
execute_task(Task_test_block_size(started_cluster), [])
|
||||
execute_task(started_cluster, Task_test_block_size(started_cluster), [])
|
||||
|
||||
|
||||
def test_no_index(started_cluster):
|
||||
execute_task(Task_no_index(started_cluster), [])
|
||||
execute_task(started_cluster, Task_no_index(started_cluster), [])
|
||||
|
||||
|
||||
def test_no_arg(started_cluster):
|
||||
execute_task(Task_no_arg(started_cluster), [])
|
||||
execute_task(started_cluster, Task_no_arg(started_cluster), [])
|
||||
|
||||
def test_non_partitioned_table(started_cluster):
|
||||
execute_task(Task_non_partitioned_table(started_cluster), [])
|
||||
execute_task(started_cluster, Task_non_partitioned_table(started_cluster), [])
|
||||
|
||||
if __name__ == '__main__':
|
||||
with contextmanager(started_cluster)() as cluster:
|
||||
|
@ -79,10 +79,10 @@ class TaskTrivial:
|
||||
node.query("DROP TABLE trivial")
|
||||
|
||||
|
||||
def execute_task(task, cmd_options):
|
||||
def execute_task(started_cluster, task, cmd_options):
|
||||
task.start()
|
||||
|
||||
zk = cluster.get_kazoo_client('zoo1')
|
||||
zk = started_cluster.get_kazoo_client('zoo1')
|
||||
print("Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1]))
|
||||
|
||||
zk_task_path = task.zk_task_path
|
||||
@ -90,7 +90,7 @@ def execute_task(task, cmd_options):
|
||||
zk.create(zk_task_path + "/description", task.copier_task_config)
|
||||
|
||||
# Run cluster-copier processes on each node
|
||||
docker_api = docker.from_env().api
|
||||
docker_api = started_cluster.docker_client.api
|
||||
copiers_exec_ids = []
|
||||
|
||||
cmd = ['/usr/bin/clickhouse', 'copier',
|
||||
@ -101,7 +101,7 @@ def execute_task(task, cmd_options):
|
||||
|
||||
print(cmd)
|
||||
|
||||
for instance_name, instance in cluster.instances.items():
|
||||
for instance_name, instance in started_cluster.instances.items():
|
||||
container = instance.get_docker_handle()
|
||||
exec_id = docker_api.exec_create(container.id, cmd, stderr=True)
|
||||
docker_api.exec_start(exec_id, detach=True)
|
||||
@ -110,7 +110,7 @@ def execute_task(task, cmd_options):
|
||||
print("Copier for {} ({}) has started".format(instance.name, instance.ip_address))
|
||||
|
||||
# Wait for copiers stopping and check their return codes
|
||||
for exec_id, instance in zip(copiers_exec_ids, iter(cluster.instances.values())):
|
||||
for exec_id, instance in zip(copiers_exec_ids, iter(started_cluster.instances.values())):
|
||||
while True:
|
||||
res = docker_api.exec_inspect(exec_id)
|
||||
if not res['Running']:
|
||||
@ -137,10 +137,10 @@ def execute_task(task, cmd_options):
|
||||
)
|
||||
def test_trivial_copy(started_cluster, use_sample_offset):
|
||||
if use_sample_offset:
|
||||
execute_task(TaskTrivial(started_cluster, use_sample_offset), ['--experimental-use-sample-offset', '1'])
|
||||
execute_task(started_cluster, TaskTrivial(started_cluster, use_sample_offset), ['--experimental-use-sample-offset', '1'])
|
||||
else:
|
||||
print("AAAAA")
|
||||
execute_task(TaskTrivial(started_cluster, use_sample_offset), [])
|
||||
execute_task(started_cluster, TaskTrivial(started_cluster, use_sample_offset), [])
|
||||
|
||||
|
||||
@pytest.mark.parametrize(
|
||||
@ -152,10 +152,10 @@ def test_trivial_copy(started_cluster, use_sample_offset):
|
||||
)
|
||||
def test_trivial_copy_with_copy_fault(started_cluster, use_sample_offset):
|
||||
if use_sample_offset:
|
||||
execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY),
|
||||
execute_task(started_cluster, TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY),
|
||||
'--experimental-use-sample-offset', '1'])
|
||||
else:
|
||||
execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)])
|
||||
execute_task(started_cluster, TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)])
|
||||
|
||||
|
||||
@pytest.mark.parametrize(
|
||||
@ -167,10 +167,10 @@ def test_trivial_copy_with_copy_fault(started_cluster, use_sample_offset):
|
||||
)
|
||||
def test_trivial_copy_with_move_fault(started_cluster, use_sample_offset):
|
||||
if use_sample_offset:
|
||||
execute_task(TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY),
|
||||
execute_task(started_cluster, TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY),
|
||||
'--experimental-use-sample-offset', '1'])
|
||||
else:
|
||||
execute_task(TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)])
|
||||
execute_task(started_cluster, TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)])
|
||||
|
||||
|
||||
if __name__ == '__main__':
|
||||
|
@ -26,7 +26,7 @@ def setup_module(module):
|
||||
|
||||
cluster = ClickHouseCluster(__file__, name=test_name)
|
||||
|
||||
SOURCE = SourceMySQL("MySQL", "localhost", cluster.mysql_port, cluster.mysql_host, "3306", "root", "clickhouse")
|
||||
SOURCE = SourceMySQL("MySQL", None, cluster.mysql_port, cluster.mysql_host, cluster.mysql_port, "root", "clickhouse")
|
||||
|
||||
simple_tester = SimpleLayoutTester(test_name)
|
||||
simple_tester.cleanup()
|
||||
@ -55,7 +55,7 @@ def teardown_module(module):
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
|
||||
simple_tester.prepare(cluster)
|
||||
complex_tester.prepare(cluster)
|
||||
ranged_tester.prepare(cluster)
|
||||
|
@ -20,7 +20,7 @@ def test_different_user():
|
||||
|
||||
cluster.start()
|
||||
|
||||
docker_api = docker.from_env().api
|
||||
docker_api = cluster.docker_client.api
|
||||
container = node.get_docker_handle()
|
||||
container.stop()
|
||||
container.start()
|
||||
|
@ -21,7 +21,7 @@ def started_cluster():
|
||||
|
||||
|
||||
class MySQLNodeInstance:
|
||||
def __init__(self, user='root', password='clickhouse', hostname='127.0.0.1', port=cluster.mysql_port):
|
||||
def __init__(self, user, password, hostname, port):
|
||||
self.user = user
|
||||
self.port = port
|
||||
self.hostname = hostname
|
||||
@ -55,7 +55,7 @@ class MySQLNodeInstance:
|
||||
|
||||
|
||||
def test_mysql_ddl_for_mysql_database(started_cluster):
|
||||
with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node:
|
||||
with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node:
|
||||
mysql_node.query("DROP DATABASE IF EXISTS test_database")
|
||||
mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'")
|
||||
|
||||
@ -89,7 +89,7 @@ def test_mysql_ddl_for_mysql_database(started_cluster):
|
||||
|
||||
|
||||
def test_clickhouse_ddl_for_mysql_database(started_cluster):
|
||||
with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node:
|
||||
with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node:
|
||||
mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'")
|
||||
mysql_node.query(
|
||||
'CREATE TABLE `test_database`.`test_table` ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;')
|
||||
@ -114,7 +114,7 @@ def test_clickhouse_ddl_for_mysql_database(started_cluster):
|
||||
|
||||
|
||||
def test_clickhouse_dml_for_mysql_database(started_cluster):
|
||||
with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node:
|
||||
with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node:
|
||||
mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'")
|
||||
mysql_node.query(
|
||||
'CREATE TABLE `test_database`.`test_table` ( `i``d` int(11) NOT NULL, PRIMARY KEY (`i``d`)) ENGINE=InnoDB;')
|
||||
@ -132,7 +132,7 @@ def test_clickhouse_dml_for_mysql_database(started_cluster):
|
||||
|
||||
|
||||
def test_clickhouse_join_for_mysql_database(started_cluster):
|
||||
with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node:
|
||||
with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node:
|
||||
mysql_node.query("CREATE DATABASE IF NOT EXISTS test DEFAULT CHARACTER SET 'utf8'")
|
||||
mysql_node.query("CREATE TABLE test.t1_mysql_local ("
|
||||
"pays VARCHAR(55) DEFAULT 'FRA' NOT NULL,"
|
||||
@ -155,7 +155,7 @@ def test_clickhouse_join_for_mysql_database(started_cluster):
|
||||
|
||||
|
||||
def test_bad_arguments_for_mysql_database_engine(started_cluster):
|
||||
with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node:
|
||||
with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, port=started_cluster.mysql_port)) as mysql_node:
|
||||
with pytest.raises(QueryRuntimeException) as exception:
|
||||
mysql_node.query("CREATE DATABASE IF NOT EXISTS test_bad_arguments DEFAULT CHARACTER SET 'utf8'")
|
||||
clickhouse_node.query(
|
||||
@ -165,7 +165,7 @@ def test_bad_arguments_for_mysql_database_engine(started_cluster):
|
||||
|
||||
|
||||
def test_data_types_support_level_for_mysql_database_engine(started_cluster):
|
||||
with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node:
|
||||
with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node:
|
||||
mysql_node.query("CREATE DATABASE IF NOT EXISTS test DEFAULT CHARACTER SET 'utf8'")
|
||||
clickhouse_node.query("CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', test, 'root', 'clickhouse')",
|
||||
settings={"mysql_datatypes_support_level": "decimal,datetime64"})
|
||||
@ -307,7 +307,7 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m
|
||||
else:
|
||||
return [do_execute(q) for q in query]
|
||||
|
||||
with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node:
|
||||
with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, port=started_cluster.mysql_port)) as mysql_node:
|
||||
execute_query(mysql_node, [
|
||||
"DROP DATABASE IF EXISTS ${mysql_db}",
|
||||
"CREATE DATABASE ${mysql_db} DEFAULT CHARACTER SET 'utf8'",
|
||||
|
@ -5,6 +5,7 @@ import math
|
||||
import os
|
||||
import time
|
||||
|
||||
import logging
|
||||
import docker
|
||||
import pymysql.connections
|
||||
import pytest
|
||||
@ -36,7 +37,7 @@ def mysql_client():
|
||||
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_client.yml')
|
||||
run_and_check(
|
||||
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build'])
|
||||
yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysql1_1')
|
||||
yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysql1_1')
|
||||
|
||||
|
||||
@pytest.fixture(scope='module')
|
||||
@ -96,7 +97,7 @@ def test_mysql_client(mysql_client, server_address):
|
||||
mysql --protocol tcp -h {host} -P {port} default -u user_with_double_sha1 --password=abacaba
|
||||
-e "SELECT 1;"
|
||||
'''.format(host=server_address, port=server_port), demux=True)
|
||||
|
||||
logging.debug(f"test_mysql_client code:{code} stdout:{stdout}, stderr:{stderr}")
|
||||
assert stdout.decode() == '\n'.join(['1', '1', ''])
|
||||
|
||||
code, (stdout, stderr) = mysql_client.exec_run('''
|
||||
|
@ -41,7 +41,7 @@ def psql_client():
|
||||
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql.yml')
|
||||
run_and_check(
|
||||
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build'])
|
||||
yield docker.from_env().containers.get(cluster.project_name + '_psql_1')
|
||||
yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_psql_1')
|
||||
|
||||
|
||||
@pytest.fixture(scope='module')
|
||||
@ -65,7 +65,7 @@ def java_container():
|
||||
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql_java_client.yml')
|
||||
run_and_check(
|
||||
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build'])
|
||||
yield docker.from_env().containers.get(cluster.project_name + '_java_1')
|
||||
yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java_1')
|
||||
|
||||
|
||||
def test_psql_is_ready(psql_server):
|
||||
|
Loading…
Reference in New Issue
Block a user