From 603c06d02ef2f61d6c5aa4a4df5adffccae2ab08 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 13 Jun 2021 12:56:22 +0000 Subject: [PATCH] Add test --- docker/test/integration/runner/Dockerfile | 2 + .../runner/compose/docker_compose_nginx.yml | 11 +++ tests/integration/helpers/cluster.py | 68 +++++++++++++++---- .../test_disk_over_web_server/__init__.py | 0 .../configs/storage_conf.xml | 27 ++++++++ .../test_disk_over_web_server/test.py | 50 ++++++++++++++ 6 files changed, 144 insertions(+), 14 deletions(-) create mode 100644 docker/test/integration/runner/compose/docker_compose_nginx.yml create mode 100644 tests/integration/test_disk_over_web_server/__init__.py create mode 100644 tests/integration/test_disk_over_web_server/configs/storage_conf.xml create mode 100644 tests/integration/test_disk_over_web_server/test.py diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 0665ab7560f..e0026e75f89 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -94,6 +94,8 @@ RUN set -x \ && echo 'dockremap:165536:65536' >> /etc/subuid \ && echo 'dockremap:165536:65536' >> /etc/subgid +RUN echo '127.0.0.1 localhost test.com' >> /etc/hosts + EXPOSE 2375 ENTRYPOINT ["dockerd-entrypoint.sh"] CMD ["sh", "-c", "pytest $PYTEST_OPTS"] diff --git a/docker/test/integration/runner/compose/docker_compose_nginx.yml b/docker/test/integration/runner/compose/docker_compose_nginx.yml new file mode 100644 index 00000000000..d0fb9fc1ff4 --- /dev/null +++ b/docker/test/integration/runner/compose/docker_compose_nginx.yml @@ -0,0 +1,11 @@ +version: '2.3' +services: + # nginx server to host static files. + # Accepts only PUT data by test.com/path and GET already existing data on test.com/path. + # Files will be put into /usr/share/nginx/files. + + nginx: + image: kssenii/nginx-test:1.1 + restart: always + ports: + - 80:80 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 27d8c05d8ed..ca66acac8fd 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -35,7 +35,7 @@ from helpers.test_tools import assert_eq_with_retry import docker from .client import Client -from .hdfs_api import HDFSApi +#from .hdfs_api import HDFSApi HELPERS_DIR = p.dirname(__file__) CLICKHOUSE_ROOT_DIR = p.join(p.dirname(__file__), "../../..") @@ -248,10 +248,11 @@ class ClickHouseCluster: self.with_redis = False self.with_cassandra = False self.with_jdbc_bridge = False + self.with_nginx = False self.with_minio = False self.minio_dir = os.path.join(self.instances_dir, "minio") - self.minio_certs_dir = None # source for certificates + self.minio_certs_dir = None # source for certificates self.minio_host = "minio1" self.minio_ip = None self.minio_bucket = "root" @@ -309,6 +310,11 @@ class ClickHouseCluster: self.rabbitmq_dir = p.abspath(p.join(self.instances_dir, "rabbitmq")) self.rabbitmq_logs_dir = os.path.join(self.rabbitmq_dir, "logs") + # available when with_rabbitmq == True + self.nginx_host = "nginx" + self.nginx_ip = None + self.nginx_port = 80 + self.nginx_id = self.get_instance_docker_id(self.nginx_host) # available when with_redis == True self.redis_host = "redis1" @@ -333,7 +339,7 @@ class ClickHouseCluster: # available when with_mysql_client == True self.mysql_client_host = "mysql_client" self.mysql_client_container = None - + # available when with_mysql == True self.mysql_host = "mysql57" self.mysql_port = 3306 @@ -532,7 +538,7 @@ class ClickHouseCluster: return self.base_mysql8_cmd - 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 env_variables['MYSQL_CLUSTER_PORT'] = str(self.mysql_port) env_variables['MYSQL_CLUSTER_ROOT_HOST'] = '%' @@ -556,7 +562,7 @@ class ClickHouseCluster: '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')] return self.base_postgres_cmd - def setup_postgres_cluster_cmd(self, instance, env_variables, docker_compose_yml_dir): + def setup_postgres_cluster_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_postgres_cluster = True env_variables['POSTGRES_PORT'] = str(self.postgres_port) env_variables['POSTGRES2_DIR'] = self.postgres2_logs_dir @@ -648,7 +654,7 @@ class ClickHouseCluster: return self.base_mongo_cmd def setup_minio_cmd(self, instance, env_variables, docker_compose_yml_dir): - self.with_minio = True + self.with_minio = True cert_d = p.join(self.minio_dir, "certs") env_variables['MINIO_CERTS_DIR'] = cert_d env_variables['MINIO_PORT'] = str(self.minio_port) @@ -674,10 +680,18 @@ class ClickHouseCluster: '--file', p.join(docker_compose_yml_dir, 'docker_compose_jdbc_bridge.yml')] return self.base_jdbc_bridge_cmd + def setup_nginx_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_nginx = True + + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_nginx.yml')]) + self.base_nginx_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_nginx.yml')] + return self.base_nginx_cmd + def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, with_zookeeper=False, with_zookeeper_secure=False, - with_mysql_client=False, with_mysql=False, with_mysql8=False, with_mysql_cluster=False, - with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, clickhouse_path_dir=None, + with_mysql_client=False, with_mysql=False, with_mysql8=False, with_mysql_cluster=False, + with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, with_nginx=False, clickhouse_path_dir=None, 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, with_jdbc_bridge=False, hostname=None, env_variables=None, image="yandex/clickhouse-integration-test", tag=None, @@ -730,6 +744,7 @@ class ClickHouseCluster: with_kafka=with_kafka, with_kerberized_kafka=with_kerberized_kafka, with_rabbitmq=with_rabbitmq, + with_nginx=with_nginx, with_kerberized_hdfs=with_kerberized_hdfs, with_mongo=with_mongo, with_redis=with_redis, @@ -810,6 +825,9 @@ class ClickHouseCluster: if with_rabbitmq and not self.with_rabbitmq: cmds.append(self.setup_rabbitmq_cmd(instance, env_variables, docker_compose_yml_dir)) + if with_nginx and not self.with_nginx: + cmds.append(self.setup_nginx_cmd(instance, env_variables, docker_compose_yml_dir)) + if with_hdfs and not self.with_hdfs: cmds.append(self.setup_hdfs_cmd(instance, env_variables, docker_compose_yml_dir)) @@ -833,7 +851,7 @@ class ClickHouseCluster: if self.minio_certs_dir is None: self.minio_certs_dir = minio_certs_dir else: - raise Exception("Overwriting minio certs dir") + raise Exception("Overwriting minio certs dir") if with_cassandra and not self.with_cassandra: cmds.append(self.setup_cassandra_cmd(instance, env_variables, docker_compose_yml_dir)) @@ -1095,6 +1113,20 @@ class ClickHouseCluster: raise Exception("Cannot wait RabbitMQ container") + def wait_nginx_to_start(self, timeout=60): + self.nginx_ip = self.get_instance_ip(self.nginx_host) + start = time.time() + while time.time() - start < timeout: + try: + self.exec_in_container(self.nginx_id, ["curl", "-X", "PUT", "-d", "Test", "http://test.com/test.txt"]) + res = self.exec_in_container(self.nginx_id, ["curl", "-X", "GET", "http://test.com/test.txt"]) + assert(res == 'Test') + print('nginx static files server is available') + return + except Exception as ex: + print("Can't connect to nginx: " + str(ex)) + time.sleep(0.5) + def wait_zookeeper_secure_to_start(self, timeout=20): logging.debug("Wait ZooKeeper Secure to start") start = time.time() @@ -1135,6 +1167,7 @@ class ClickHouseCluster: krb_conf = p.abspath(p.join(self.instances['node1'].path, "secrets/krb_long.conf")) self.hdfs_kerberized_ip = self.get_instance_ip(self.hdfs_kerberized_host) kdc_ip = self.get_instance_ip('hdfskerberos') + self.hdfs_api = HDFSApi(user="root", timeout=timeout, kerberized=True, @@ -1146,7 +1179,7 @@ class ClickHouseCluster: proxy_port=self.hdfs_kerberized_name_port, data_port=self.hdfs_kerberized_data_port, hdfs_ip=self.hdfs_kerberized_ip, - kdc_ip=kdc_ip) + kdc_ip=kdc_ip) else: self.hdfs_ip = self.get_instance_ip(self.hdfs_host) self.hdfs_api = HDFSApi(user="root", host=self.hdfs_host, data_port=self.hdfs_data_port, proxy_port=self.hdfs_name_port, hdfs_ip=self.hdfs_ip) @@ -1246,7 +1279,7 @@ class ClickHouseCluster: raise Exception("Can't wait Schema Registry to start") - + def wait_cassandra_to_start(self, timeout=180): self.cassandra_ip = self.get_instance_ip(self.cassandra_host) cass_client = cassandra.cluster.Cluster([self.cassandra_ip], port=self.cassandra_port, load_balancing_policy=RoundRobinPolicy()) @@ -1321,7 +1354,7 @@ class ClickHouseCluster: for dir in self.zookeeper_dirs_to_create: os.makedirs(dir) - + if self.use_keeper: # TODO: remove hardcoded paths from here for i in range(1,4): shutil.copy(os.path.join(HELPERS_DIR, f'keeper_config{i}.xml'), os.path.join(self.keeper_instance_dir_prefix + f"{i}", "config" )) @@ -1421,6 +1454,12 @@ class ClickHouseCluster: self.make_hdfs_api(kerberized=True) self.wait_hdfs_to_start() + if self.with_nginx and self.base_nginx_cmd: + logging.debug('Setup nginx') + subprocess_check_call(self.base_nginx_cmd + common_opts + ['--renew-anon-volumes']) + self.nginx_docker_id = self.get_instance_docker_id('nginx') + self.wait_nginx_to_start() + if self.with_mongo and self.base_mongo_cmd: logging.debug('Setup Mongo') run_and_check(self.base_mongo_cmd + common_opts) @@ -1651,7 +1690,7 @@ class ClickHouseInstance: self, cluster, base_path, name, base_config_dir, custom_main_configs, custom_user_configs, custom_dictionaries, macros, with_zookeeper, zookeeper_config_path, with_mysql_client, with_mysql, with_mysql8, with_mysql_cluster, with_kafka, with_kerberized_kafka, - with_rabbitmq, with_kerberized_hdfs, with_mongo, with_redis, with_minio, with_jdbc_bridge, + with_rabbitmq, with_nginx, with_kerberized_hdfs, with_mongo, with_redis, with_minio, with_jdbc_bridge, with_cassandra, server_bin_path, odbc_bridge_bin_path, library_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, with_postgres, with_postgres_cluster, clickhouse_start_command=CLICKHOUSE_START_COMMAND, main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True, @@ -1689,6 +1728,7 @@ class ClickHouseInstance: self.with_kafka = with_kafka self.with_kerberized_kafka = with_kerberized_kafka self.with_rabbitmq = with_rabbitmq + self.with_nginx = with_nginx self.with_kerberized_hdfs = with_kerberized_hdfs self.with_mongo = with_mongo self.with_redis = with_redis @@ -2173,7 +2213,7 @@ class ClickHouseInstance: depends_on.append("postgres2") depends_on.append("postgres3") depends_on.append("postgres4") - + if self.with_kafka: depends_on.append("kafka1") depends_on.append("schema-registry") diff --git a/tests/integration/test_disk_over_web_server/__init__.py b/tests/integration/test_disk_over_web_server/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_disk_over_web_server/configs/storage_conf.xml b/tests/integration/test_disk_over_web_server/configs/storage_conf.xml new file mode 100644 index 00000000000..b1b35627cef --- /dev/null +++ b/tests/integration/test_disk_over_web_server/configs/storage_conf.xml @@ -0,0 +1,27 @@ + + + + + + static + http://nginx:80/ + + + local + / + + + + + +
+ static +
+ + hdd + +
+
+
+
+
diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py new file mode 100644 index 00000000000..a743fc523c6 --- /dev/null +++ b/tests/integration/test_disk_over_web_server/test.py @@ -0,0 +1,50 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance("node", main_configs=["configs/storage_conf.xml"], with_nginx=True) + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_insert_select(cluster): + node = cluster.instances["node"] + node.query(""" + CREATE TABLE test1 (id Int32) + ENGINE = MergeTree() ORDER BY id + SETTINGS storage_policy = 'static'; + """) + + node.query("INSERT INTO test1 SELECT number FROM numbers(100)") + result = node.query("SELECT count() FROM test1") + assert(int(result) == 100) + + node.query("DETACH TABLE test1") + node.query("ATTACH TABLE test1") + result = node.query("SELECT count() FROM test1") + assert(int(result) == 100) + + node = cluster.instances["node"] + node.query(""" + CREATE TABLE test2 (id Int32) + ENGINE = MergeTree() ORDER BY id + SETTINGS storage_policy = 'static'; + """) + + node.query("INSERT INTO test2 SELECT number FROM numbers(500000)") + result = node.query("SELECT id FROM test2 ORDER BY id") + expected = node.query("SELECT number FROM numbers(500000)") + assert(result == expected) + + node.query("DETACH TABLE test2") + node.query("ATTACH TABLE test2") + result = node.query("SELECT count() FROM test2") + assert(int(result) == 500000)