ClickHouse/tests/integration/test_cluster_copier/test_trivial.py

154 lines
5.3 KiB
Python
Raw Normal View History

2020-03-16 21:05:38 +00:00
import os
import sys
import time
2021-04-27 12:34:56 +00:00
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
import kazoo
import pytest
2021-04-27 12:34:56 +00:00
import docker
2020-03-16 21:05:38 +00:00
CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__))
sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR))
2021-04-27 12:34:56 +00:00
2020-03-16 21:05:38 +00:00
2021-04-29 22:56:41 +00:00
COPYING_FAIL_PROBABILITY = 0.1
2020-12-23 18:59:27 +00:00
MOVING_FAIL_PROBABILITY = 0.1
2020-03-16 21:05:38 +00:00
cluster = None
2020-04-21 17:37:40 +00:00
@pytest.fixture(scope="function")
2020-03-16 21:05:38 +00:00
def started_cluster():
global cluster
try:
clusters_schema = {
"0": {"0": ["0"]},
"1": {"0": ["0"]}
2020-03-16 21:05:38 +00:00
}
cluster = ClickHouseCluster(__file__)
2020-10-02 16:54:07 +00:00
for cluster_name, shards in clusters_schema.items():
for shard_name, replicas in shards.items():
2020-03-16 21:05:38 +00:00
for replica_name in replicas:
name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name)
cluster.add_instance(name,
main_configs=[], user_configs=[],
2020-03-16 21:05:38 +00:00
macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name},
with_zookeeper=True)
cluster.start()
yield cluster
finally:
cluster.shutdown()
class TaskTrivial:
2020-04-21 17:37:40 +00:00
def __init__(self, cluster, use_sample_offset):
2020-03-16 21:05:38 +00:00
self.cluster = cluster
2020-04-21 17:37:40 +00:00
if use_sample_offset:
self.zk_task_path = "/clickhouse-copier/task_trivial_use_sample_offset"
2020-04-21 17:37:40 +00:00
else:
self.zk_task_path = "/clickhouse-copier/task_trivial"
2020-03-16 21:05:38 +00:00
self.copier_task_config = open(os.path.join(CURRENT_TEST_DIR, 'task_trivial.xml'), 'r').read()
def start(self):
source = cluster.instances['s0_0_0']
destination = cluster.instances['s1_0_0']
for node in [source, destination]:
node.query("DROP DATABASE IF EXISTS default")
node.query("CREATE DATABASE IF NOT EXISTS default")
2020-03-16 21:05:38 +00:00
source.query("CREATE TABLE trivial (d UInt64, d1 UInt64 MATERIALIZED d+1) "
"ENGINE=ReplicatedMergeTree('/clickhouse/tables/source_trivial_cluster/1/trivial', '1') "
2020-04-21 17:37:40 +00:00
"PARTITION BY d % 5 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16")
2020-03-16 21:05:38 +00:00
source.query("INSERT INTO trivial SELECT * FROM system.numbers LIMIT 1002",
settings={"insert_distributed_sync": 1})
2020-03-16 21:05:38 +00:00
def check(self):
2021-04-29 19:16:51 +00:00
zk = cluster.get_kazoo_client('zoo1')
status_data, _ = zk.get(self.zk_task_path + "/status")
assert status_data == b'{"hits":{"all_partitions_count":5,"processed_partitions_count":5}}'
2020-03-16 21:05:38 +00:00
source = cluster.instances['s0_0_0']
destination = cluster.instances['s1_0_0']
assert TSV(source.query("SELECT count() FROM trivial")) == TSV("1002\n")
assert TSV(destination.query("SELECT count() FROM trivial")) == TSV("1002\n")
for node in [source, destination]:
node.query("DROP TABLE trivial")
def execute_task(task, cmd_options):
task.start()
zk = cluster.get_kazoo_client('zoo1')
2020-10-02 16:54:07 +00:00
print("Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1]))
2020-03-16 21:05:38 +00:00
2021-04-27 12:34:56 +00:00
try:
zk.delete("/clickhouse-copier", recursive=True)
except kazoo.exceptions.NoNodeError:
print("No node /clickhouse-copier. It is Ok in first test.")
2020-03-16 21:05:38 +00:00
zk_task_path = task.zk_task_path
zk.ensure_path(zk_task_path)
2021-04-27 12:34:56 +00:00
zk.create(zk_task_path + "/description", task.copier_task_config.encode())
2020-03-16 21:05:38 +00:00
# Run cluster-copier processes on each node
docker_api = docker.from_env().api
copiers_exec_ids = []
cmd = ['/usr/bin/clickhouse', 'copier',
'--config', '/etc/clickhouse-server/config-copier.xml',
'--task-path', zk_task_path,
'--base-dir', '/var/log/clickhouse-server/copier']
cmd += cmd_options
2021-04-27 12:34:56 +00:00
copiers = list(cluster.instances.keys())
2020-03-16 21:05:38 +00:00
2021-04-27 12:34:56 +00:00
for instance_name in copiers:
instance = cluster.instances[instance_name]
2020-03-16 21:05:38 +00:00
container = instance.get_docker_handle()
2021-04-27 12:34:56 +00:00
instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs/config-copier.xml"),
"/etc/clickhouse-server/config-copier.xml")
print("Copied copier config to {}".format(instance.name))
2020-03-16 21:05:38 +00:00
exec_id = docker_api.exec_create(container.id, cmd, stderr=True)
2021-04-27 12:34:56 +00:00
output = docker_api.exec_start(exec_id).decode('utf8')
print(output)
2020-03-16 21:05:38 +00:00
copiers_exec_ids.append(exec_id)
2020-10-02 16:54:07 +00:00
print("Copier for {} ({}) has started".format(instance.name, instance.ip_address))
2020-03-16 21:05:38 +00:00
# Wait for copiers stopping and check their return codes
2021-04-27 12:34:56 +00:00
for exec_id, instance_name in zip(copiers_exec_ids, copiers):
instance = cluster.instances[instance_name]
2020-03-16 21:05:38 +00:00
while True:
res = docker_api.exec_inspect(exec_id)
if not res['Running']:
break
2021-04-27 12:34:56 +00:00
time.sleep(0.5)
2020-03-16 21:05:38 +00:00
assert res['ExitCode'] == 0, "Instance: {} ({}). Info: {}".format(instance.name, instance.ip_address, repr(res))
try:
task.check()
finally:
zk.delete(zk_task_path, recursive=True)
# Tests
2021-04-27 12:34:56 +00:00
@pytest.mark.parametrize(('use_sample_offset'),[False,True])
2020-04-21 17:37:40 +00:00
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'])
else:
execute_task(TaskTrivial(started_cluster, use_sample_offset), [])