mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Add a separate docker-compose with multiple servers for postgres and mysql
This commit is contained in:
parent
4e849f3b77
commit
03ce320926
@ -8,17 +8,3 @@ services:
|
||||
ports:
|
||||
- 3308:3306
|
||||
command: --server_id=100 --log-bin='mysql-bin-1.log' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency
|
||||
mysql2:
|
||||
image: mysql:5.7
|
||||
restart: always
|
||||
environment:
|
||||
MYSQL_ROOT_PASSWORD: clickhouse
|
||||
ports:
|
||||
- 3388:3306
|
||||
mysql3:
|
||||
image: mysql:5.7
|
||||
restart: always
|
||||
environment:
|
||||
MYSQL_ROOT_PASSWORD: clickhouse
|
||||
ports:
|
||||
- 3368:3306
|
||||
|
@ -11,17 +11,3 @@ services:
|
||||
default:
|
||||
aliases:
|
||||
- postgre-sql.local
|
||||
postgres2:
|
||||
image: postgres
|
||||
restart: always
|
||||
environment:
|
||||
POSTGRES_PASSWORD: mysecretpassword
|
||||
ports:
|
||||
- 5441:5432
|
||||
postgres3:
|
||||
image: postgres
|
||||
restart: always
|
||||
environment:
|
||||
POSTGRES_PASSWORD: mysecretpassword
|
||||
ports:
|
||||
- 5461:5432
|
||||
|
@ -139,7 +139,9 @@ class ClickHouseCluster:
|
||||
self.instances = {}
|
||||
self.with_zookeeper = False
|
||||
self.with_mysql = False
|
||||
self.with_mysql_cluster = False
|
||||
self.with_postgres = False
|
||||
self.with_postgres_cluster = False
|
||||
self.with_kafka = False
|
||||
self.with_kerberized_kafka = False
|
||||
self.with_rabbitmq = False
|
||||
@ -180,9 +182,9 @@ class ClickHouseCluster:
|
||||
|
||||
def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None,
|
||||
macros=None,
|
||||
with_zookeeper=False, with_mysql=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False,
|
||||
with_zookeeper=False, with_mysql=False, with_mysql_cluster=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False,
|
||||
clickhouse_path_dir=None,
|
||||
with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_kerberized_hdfs=False, with_mongo=False,
|
||||
with_odbc_drivers=False, with_postgres=False, with_postgres_cluster=False, with_hdfs=False, with_kerberized_hdfs=False, with_mongo=False,
|
||||
with_redis=False, with_minio=False, with_cassandra=False,
|
||||
hostname=None, env_variables=None, image="yandex/clickhouse-integration-test", tag=None,
|
||||
stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None,
|
||||
@ -223,6 +225,7 @@ class ClickHouseCluster:
|
||||
with_zookeeper=with_zookeeper,
|
||||
zookeeper_config_path=self.zookeeper_config_path,
|
||||
with_mysql=with_mysql,
|
||||
with_mysql_cluster=with_mysql_cluster,
|
||||
with_kafka=with_kafka,
|
||||
with_kerberized_kafka=with_kerberized_kafka,
|
||||
with_rabbitmq=with_rabbitmq,
|
||||
@ -274,6 +277,14 @@ class ClickHouseCluster:
|
||||
|
||||
cmds.append(self.base_mysql_cmd)
|
||||
|
||||
if with_mysql_cluster and not self.with_mysql_cluster:
|
||||
self.with_mysql_cluster = True
|
||||
self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql_cluster.yml')])
|
||||
self.base_mysql_cluster_cmd = ['docker-compose', '--project-name', self.project_name,
|
||||
'--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql_cluster.yml')]
|
||||
|
||||
cmds.append(self.base_mysql_cluster_cmd)
|
||||
|
||||
if with_postgres and not self.with_postgres:
|
||||
self.with_postgres = True
|
||||
self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')])
|
||||
@ -281,6 +292,13 @@ class ClickHouseCluster:
|
||||
'--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]
|
||||
cmds.append(self.base_postgres_cmd)
|
||||
|
||||
if with_postgres_cluster and not self.with_postgres_cluster:
|
||||
self.with_postgres_cluster = True
|
||||
self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')])
|
||||
self.base_postgres_cluster_cmd = ['docker-compose', '--project-name', self.project_name,
|
||||
'--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres_cluster.yml')]
|
||||
cmds.append(self.base_postgres_cluster_cmd)
|
||||
|
||||
if with_odbc_drivers and not self.with_odbc_drivers:
|
||||
self.with_odbc_drivers = True
|
||||
if not self.with_mysql:
|
||||
@ -449,11 +467,11 @@ class ClickHouseCluster:
|
||||
["bash", "-c", "echo {} | base64 --decode > {}".format(encodedStr, dest_path)],
|
||||
user='root')
|
||||
|
||||
def wait_mysql_to_start(self, timeout=60):
|
||||
def wait_mysql_to_start(self, timeout=60, port=3308):
|
||||
start = time.time()
|
||||
while time.time() - start < timeout:
|
||||
try:
|
||||
conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=3308)
|
||||
conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=port)
|
||||
conn.close()
|
||||
print("Mysql Started")
|
||||
return
|
||||
@ -464,11 +482,11 @@ class ClickHouseCluster:
|
||||
subprocess_call(['docker-compose', 'ps', '--services', '--all'])
|
||||
raise Exception("Cannot wait MySQL container")
|
||||
|
||||
def wait_postgres_to_start(self, timeout=60):
|
||||
def wait_postgres_to_start(self, timeout=60, port=5432):
|
||||
start = time.time()
|
||||
while time.time() - start < timeout:
|
||||
try:
|
||||
conn_string = "host='localhost' user='postgres' password='mysecretpassword'"
|
||||
conn_string = "host='localhost' port={} user='postgres' password='mysecretpassword'".format(port)
|
||||
conn = psycopg2.connect(conn_string)
|
||||
conn.close()
|
||||
print("Postgres Started")
|
||||
@ -650,11 +668,21 @@ class ClickHouseCluster:
|
||||
subprocess_check_call(self.base_mysql_cmd + common_opts)
|
||||
self.wait_mysql_to_start(120)
|
||||
|
||||
if self.with_mysql_cluster and self.base_mysql_cluster_cmd:
|
||||
print('Setup MySQL')
|
||||
subprocess_check_call(self.base_mysql_cluster_cmd + common_opts)
|
||||
self.wait_mysql_to_start(120, port=3348)
|
||||
|
||||
if self.with_postgres and self.base_postgres_cmd:
|
||||
print('Setup Postgres')
|
||||
subprocess_check_call(self.base_postgres_cmd + common_opts)
|
||||
self.wait_postgres_to_start(120)
|
||||
|
||||
if self.with_postgres_cluster and self.base_postgres_cluster_cmd:
|
||||
print('Setup Postgres')
|
||||
subprocess_check_call(self.base_postgres_cluster_cmd + common_opts)
|
||||
self.wait_postgres_to_start(120, port=5441)
|
||||
|
||||
if self.with_kafka and self.base_kafka_cmd:
|
||||
print('Setup Kafka')
|
||||
subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes'])
|
||||
@ -896,7 +924,7 @@ class ClickHouseInstance:
|
||||
def __init__(
|
||||
self, cluster, base_path, name, base_config_dir, custom_main_configs, custom_user_configs,
|
||||
custom_dictionaries,
|
||||
macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_kerberized_kafka, with_rabbitmq, with_kerberized_hdfs,
|
||||
macros, with_zookeeper, zookeeper_config_path, with_mysql, with_mysql_cluster, with_kafka, with_kerberized_kafka, with_rabbitmq, with_kerberized_hdfs,
|
||||
with_mongo, with_redis, with_minio,
|
||||
with_cassandra, server_bin_path, odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers,
|
||||
hostname=None, env_variables=None,
|
||||
@ -924,6 +952,7 @@ class ClickHouseInstance:
|
||||
self.odbc_bridge_bin_path = odbc_bridge_bin_path
|
||||
|
||||
self.with_mysql = with_mysql
|
||||
self.with_mysql_cluster = with_mysql_cluster
|
||||
self.with_kafka = with_kafka
|
||||
self.with_kerberized_kafka = with_kerberized_kafka
|
||||
self.with_rabbitmq = with_rabbitmq
|
||||
|
@ -8,6 +8,8 @@ from helpers.cluster import ClickHouseCluster
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml'], with_mysql=True)
|
||||
node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml'], with_mysql_cluster=True)
|
||||
|
||||
create_table_sql_template = """
|
||||
CREATE TABLE `clickhouse`.`{}` (
|
||||
`id` int(11) NOT NULL,
|
||||
@ -189,10 +191,11 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8('
|
||||
def test_mysql_distributed(started_cluster):
|
||||
table_name = 'test_replicas'
|
||||
|
||||
conn1 = get_mysql_conn(port=3308)
|
||||
conn1 = get_mysql_conn(port=3348)
|
||||
conn2 = get_mysql_conn(port=3388)
|
||||
conn3 = get_mysql_conn(port=3368)
|
||||
|
||||
create_mysql_db(conn1, 'clickhouse')
|
||||
create_mysql_db(conn2, 'clickhouse')
|
||||
create_mysql_db(conn3, 'clickhouse')
|
||||
|
||||
@ -201,18 +204,18 @@ def test_mysql_distributed(started_cluster):
|
||||
create_mysql_table(conn3, table_name)
|
||||
|
||||
# Storage with with 3 replicas
|
||||
node1.query('''
|
||||
node2.query('''
|
||||
CREATE TABLE test_replicas
|
||||
(id UInt32, name String, age UInt32, money UInt32)
|
||||
ENGINE = MySQL(`mysql{1|2|3}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''')
|
||||
ENGINE = MySQL(`mysql_{1|2|3}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''')
|
||||
|
||||
# Fill remote tables with different data to be able to check
|
||||
for i in range(1, 4):
|
||||
node1.query('''
|
||||
node2.query('''
|
||||
CREATE TABLE test_replica{}
|
||||
(id UInt32, name String, age UInt32, money UInt32)
|
||||
ENGINE = MySQL(`mysql{}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse');'''.format(i, i))
|
||||
node1.query("INSERT INTO test_replica{} (id, name) SELECT number, 'host{}' from numbers(10) ".format(i, i))
|
||||
ENGINE = MySQL(`mysql_{}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse');'''.format(i, i))
|
||||
node2.query("INSERT INTO test_replica{} (id, name) SELECT number, 'host{}' from numbers(10) ".format(i, i))
|
||||
|
||||
# check all replicas are traversed
|
||||
query = "SELECT * FROM ("
|
||||
@ -220,14 +223,14 @@ def test_mysql_distributed(started_cluster):
|
||||
query += "SELECT name FROM test_replicas UNION DISTINCT "
|
||||
query += "SELECT name FROM test_replicas)"
|
||||
|
||||
result = node1.query(query.format(t=table_name))
|
||||
result = node2.query(query.format(t=table_name))
|
||||
assert(result == 'host1\nhost2\nhost3\n')
|
||||
|
||||
# Storage with with two two shards, each has 2 replicas
|
||||
# Storage with with two shards, each has 2 replicas
|
||||
node1.query('''
|
||||
CREATE TABLE test_shards
|
||||
(id UInt32, name String, age UInt32, money UInt32)
|
||||
ENGINE = ExternalDistributed('MySQL', `mysql{1|2}:3306,mysql{3|4}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''')
|
||||
ENGINE = ExternalDistributed('MySQL', `mysql_{1|2}:3306,mysql_{3|4}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''')
|
||||
|
||||
# Check only one replica in each shard is used
|
||||
result = node1.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name")
|
||||
|
@ -10,6 +10,7 @@ from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node1 = cluster.add_instance('node1', main_configs=["configs/log_conf.xml"], with_postgres=True)
|
||||
node2 = cluster.add_instance('node2', main_configs=['configs/log_conf.xml'], with_postgres_cluster=True)
|
||||
|
||||
def get_postgres_conn(database=False, port=5432):
|
||||
if database == True:
|
||||
@ -35,6 +36,9 @@ def started_cluster():
|
||||
postgres_conn = get_postgres_conn(port=5432)
|
||||
create_postgres_db(postgres_conn, 'clickhouse')
|
||||
|
||||
postgres_conn = get_postgres_conn(port=5421)
|
||||
create_postgres_db(postgres_conn, 'clickhouse')
|
||||
|
||||
postgres_conn = get_postgres_conn(port=5441)
|
||||
create_postgres_db(postgres_conn, 'clickhouse')
|
||||
|
||||
@ -229,7 +233,7 @@ def test_concurrent_queries(started_cluster):
|
||||
|
||||
|
||||
def test_postgres_distributed(started_cluster):
|
||||
conn1 = get_postgres_conn(port=5432, database=True)
|
||||
conn1 = get_postgres_conn(port=5421, database=True)
|
||||
conn2 = get_postgres_conn(port=5441, database=True)
|
||||
conn3 = get_postgres_conn(port=5461, database=True)
|
||||
|
||||
@ -243,10 +247,10 @@ def test_postgres_distributed(started_cluster):
|
||||
cursors[i].execute("""INSERT INTO test_replicas select i, 'host{}' from generate_series(0, 99) as t(i);""".format(i + 1));
|
||||
|
||||
# Storage with with 3 replicas
|
||||
node1.query('''
|
||||
node2.query('''
|
||||
CREATE TABLE test_replicas
|
||||
(id UInt32, name String)
|
||||
ENGINE = PostgreSQL(`postgres{1|2|3}:5432`, 'clickhouse', 'test_replicas', 'postgres', 'mysecretpassword'); ''')
|
||||
ENGINE = PostgreSQL(`postgres_{1|2|3}:5432`, 'clickhouse', 'test_replicas', 'postgres', 'mysecretpassword'); ''')
|
||||
|
||||
# check all replicas are traversed
|
||||
query = "SELECT name FROM ("
|
||||
@ -254,17 +258,17 @@ def test_postgres_distributed(started_cluster):
|
||||
query += "SELECT name FROM test_replicas UNION DISTINCT "
|
||||
query += "SELECT name FROM test_replicas) ORDER BY name"
|
||||
|
||||
result = node1.query(query)
|
||||
result = node2.query(query)
|
||||
assert(result == 'host1\nhost2\nhost3\n')
|
||||
|
||||
# Storage with with two two shards, each has 2 replicas
|
||||
node1.query('''
|
||||
node2.query('''
|
||||
CREATE TABLE test_shards
|
||||
(id UInt32, name String, age UInt32, money UInt32)
|
||||
ENGINE = ExternalDistributed('PostgreSQL', `postgres{1|2}:5432,postgres{3|4}:5432`, 'clickhouse', 'test_replicas', 'postgres', 'mysecretpassword'); ''')
|
||||
ENGINE = ExternalDistributed('PostgreSQL', `postgres_{1|2}:5432,postgres_{3|4}:5432`, 'clickhouse', 'test_replicas', 'postgres', 'mysecretpassword'); ''')
|
||||
|
||||
# Check only one replica in each shard is used
|
||||
result = node1.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name")
|
||||
result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name")
|
||||
assert(result == 'host1\nhost3\n')
|
||||
|
||||
# check all replicas are traversed
|
||||
@ -273,7 +277,7 @@ def test_postgres_distributed(started_cluster):
|
||||
query += "SELECT name FROM test_shards UNION DISTINCT "
|
||||
query += "SELECT name FROM test_shards) ORDER BY name"
|
||||
|
||||
result = node1.query(query)
|
||||
result = node2.query(query)
|
||||
assert(result == 'host1\nhost2\nhost3\n')
|
||||
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user