ClickHouse/tests/integration/test_zookeeper_config/test.py

195 lines
7.7 KiB
Python
Raw Normal View History

2020-10-02 16:54:07 +00:00
import time
2021-03-04 13:07:40 +00:00
import threading
from os import path as p, unlink
from tempfile import NamedTemporaryFile
import helpers
import pytest
from helpers.cluster import ClickHouseCluster
2017-08-30 16:25:34 +00:00
def test_chroot_with_same_root():
2017-08-30 16:25:34 +00:00
cluster_1 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml')
cluster_2 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml')
node1 = cluster_1.add_instance('node1',
main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"],
with_zookeeper=True, zookeeper_use_tmpfs=False)
node2 = cluster_2.add_instance('node2',
main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"],
with_zookeeper=True, zookeeper_use_tmpfs=False)
2017-08-30 16:25:34 +00:00
nodes = [node1, node2]
def create_zk_root(zk):
zk.ensure_path('/root_a')
print(zk.get_children('/'))
cluster_1.add_zookeeper_startup_command(create_zk_root)
2017-08-30 16:25:34 +00:00
try:
cluster_1.start()
try:
cluster_2.start(destroy_dirs=False)
for i, node in enumerate(nodes):
node.query('''
CREATE TABLE simple (date Date, id UInt32)
2017-08-30 16:25:34 +00:00
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192);
'''.format(replica=node.name))
for j in range(2): # Second insert to test deduplication
node.query("INSERT INTO simple VALUES ({0}, {0})".format(i))
2017-08-30 16:25:34 +00:00
time.sleep(1)
2017-08-30 16:25:34 +00:00
assert node1.query('select count() from simple').strip() == '2'
assert node2.query('select count() from simple').strip() == '2'
finally:
cluster_2.shutdown()
finally:
cluster_1.shutdown()
def test_chroot_with_different_root():
cluster_1 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_a.xml')
cluster_2 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_root_b.xml')
node1 = cluster_1.add_instance('node1',
main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_a.xml"],
with_zookeeper=True, zookeeper_use_tmpfs=False)
node2 = cluster_2.add_instance('node2',
main_configs=["configs/remote_servers.xml", "configs/zookeeper_config_root_b.xml"],
with_zookeeper=True, zookeeper_use_tmpfs=False)
2017-08-30 16:25:34 +00:00
nodes = [node1, node2]
def create_zk_roots(zk):
zk.ensure_path('/root_a')
zk.ensure_path('/root_b')
print(zk.get_children('/'))
cluster_1.add_zookeeper_startup_command(create_zk_roots)
2017-08-30 16:25:34 +00:00
try:
cluster_1.start()
try:
cluster_2.start(destroy_dirs=False)
for i, node in enumerate(nodes):
node.query('''
CREATE TABLE simple (date Date, id UInt32)
2017-08-30 16:25:34 +00:00
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192);
'''.format(replica=node.name))
for j in range(2): # Second insert to test deduplication
node.query("INSERT INTO simple VALUES ({0}, {0})".format(i))
2017-08-30 16:25:34 +00:00
assert node1.query('select count() from simple').strip() == '1'
assert node2.query('select count() from simple').strip() == '1'
finally:
cluster_2.shutdown()
finally:
cluster_1.shutdown()
def test_identity():
cluster_1 = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_with_password.xml')
cluster_2 = ClickHouseCluster(__file__)
node1 = cluster_1.add_instance('node1', main_configs=["configs/remote_servers.xml",
"configs/zookeeper_config_with_password.xml"],
with_zookeeper=True, zookeeper_use_tmpfs=False)
node2 = cluster_2.add_instance('node2', main_configs=["configs/remote_servers.xml"], with_zookeeper=True,
zookeeper_use_tmpfs=False)
2017-08-30 16:25:34 +00:00
try:
cluster_1.start()
node1.query('''
CREATE TABLE simple (date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}', date, id, 8192);
'''.format(replica=node1.name))
2017-08-30 16:25:34 +00:00
with pytest.raises(Exception):
cluster_2.start(destroy_dirs=False)
node2.query('''
CREATE TABLE simple (date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '1', date, id, 8192);
''')
2017-08-30 16:25:34 +00:00
finally:
cluster_1.shutdown()
cluster_2.shutdown()
def test_secure_connection():
# We need absolute path in zookeeper volumes. Generate it dynamically.
TEMPLATE = '''
zoo{zoo_id}:
image: zookeeper:3.6.2
restart: always
environment:
ZOO_TICK_TIME: 500
ZOO_MY_ID: {zoo_id}
ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888;2181
ZOO_SECURE_CLIENT_PORT: 2281
volumes:
- {helpers_dir}/zookeeper-ssl-entrypoint.sh:/zookeeper-ssl-entrypoint.sh
- {configs_dir}:/clickhouse-config
command: ["zkServer.sh", "start-foreground"]
entrypoint: /zookeeper-ssl-entrypoint.sh
'''
configs_dir = p.abspath(p.join(p.dirname(__file__), 'configs_secure'))
helpers_dir = p.abspath(p.dirname(helpers.__file__))
cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_with_ssl.xml')
2020-10-02 16:54:07 +00:00
docker_compose = NamedTemporaryFile(mode='w+', delete=False)
docker_compose.write(
"version: '2.3'\nservices:\n" +
TEMPLATE.format(zoo_id=1, configs_dir=configs_dir, helpers_dir=helpers_dir) +
TEMPLATE.format(zoo_id=2, configs_dir=configs_dir, helpers_dir=helpers_dir) +
TEMPLATE.format(zoo_id=3, configs_dir=configs_dir, helpers_dir=helpers_dir)
)
docker_compose.close()
node1 = cluster.add_instance('node1', main_configs=["configs_secure/client.crt", "configs_secure/client.key",
"configs_secure/conf.d/remote_servers.xml",
"configs_secure/conf.d/ssl_conf.xml"],
with_zookeeper=True, zookeeper_docker_compose_path=docker_compose.name,
zookeeper_use_tmpfs=False)
node2 = cluster.add_instance('node2', main_configs=["configs_secure/client.crt", "configs_secure/client.key",
"configs_secure/conf.d/remote_servers.xml",
"configs_secure/conf.d/ssl_conf.xml"],
with_zookeeper=True, zookeeper_docker_compose_path=docker_compose.name,
zookeeper_use_tmpfs=False)
try:
cluster.start()
assert node1.query("SELECT count() FROM system.zookeeper WHERE path = '/'") == '2\n'
assert node2.query("SELECT count() FROM system.zookeeper WHERE path = '/'") == '2\n'
2021-03-04 13:07:40 +00:00
kThreadsNumber = 16
kIterations = 100
threads = []
for _ in range(kThreadsNumber):
threads.append(threading.Thread(target=(lambda:
[node1.query("SELECT count() FROM system.zookeeper WHERE path = '/'") for _ in range(kIterations)])))
2021-03-04 18:07:05 +00:00
for thread in threads:
thread.start()
2021-03-04 13:07:40 +00:00
for thread in threads:
thread.join()
finally:
cluster.shutdown()
unlink(docker_compose.name)