Merge pull request #44880 from qoega/green-ci-4

Rewrite test_postgres_protocol test
This commit is contained in:
Ilya Yatsishin 2023-01-05 12:34:33 +01:00 committed by GitHub
commit c5f01a653c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 186 additions and 135 deletions

View File

@ -207,6 +207,15 @@ def check_kafka_is_available(kafka_id, kafka_port):
return p.returncode == 0
def check_postgresql_java_client_is_available(postgresql_java_client_id):
p = subprocess.Popen(
("docker", "exec", "-i", postgresql_java_client_id, "java", "-version"),
stdout=subprocess.PIPE,
)
p.communicate()
return p.returncode == 0
def check_rabbitmq_is_available(rabbitmq_id):
p = subprocess.Popen(
("docker", "exec", "-i", rabbitmq_id, "rabbitmqctl", "await_startup"),
@ -259,12 +268,8 @@ def enable_consistent_hash_plugin(rabbitmq_id):
def get_instances_dir(name):
instances_dir_name = "_instances"
worker_name = os.environ.get("PYTEST_XDIST_WORKER", "")
run_id = os.environ.get("INTEGRATION_TESTS_RUN_ID", "")
if worker_name:
instances_dir_name += "_" + worker_name
if name:
instances_dir_name += "_" + name
@ -386,6 +391,7 @@ class ClickHouseCluster:
self.with_mysql_cluster = False
self.with_postgres = False
self.with_postgres_cluster = False
self.with_postgresql_java_client = False
self.with_kafka = False
self.with_kerberized_kafka = False
self.with_rabbitmq = False
@ -404,9 +410,11 @@ class ClickHouseCluster:
self.with_hive = False
self.with_coredns = False
# available when with_minio == True
self.with_minio = False
self.minio_dir = os.path.join(self.instances_dir, "minio")
self.minio_certs_dir = None # source for certificates
self.minio_data_dir = p.join(self.minio_dir, "data")
self.minio_host = "minio1"
self.minio_ip = None
self.minio_bucket = "root"
@ -516,6 +524,13 @@ class ClickHouseCluster:
self.postgres2_logs_dir = os.path.join(self.postgres_dir, "postgres2")
self.postgres3_logs_dir = os.path.join(self.postgres_dir, "postgres3")
self.postgres4_logs_dir = os.path.join(self.postgres_dir, "postgres4")
self.postgres_id = self.get_instance_docker_id(self.postgres_host)
# available when with_postgresql_java_client = True
self.postgresql_java_client_host = "java"
self.postgresql_java_client_docker_id = self.get_instance_docker_id(
self.postgresql_java_client_host
)
# available when with_mysql_client == True
self.mysql_client_host = "mysql_client"
@ -573,7 +588,6 @@ class ClickHouseCluster:
if p.exists(self.instances_dir):
shutil.rmtree(self.instances_dir, ignore_errors=True)
logging.debug(f"Removed :{self.instances_dir}")
os.mkdir(self.instances_dir)
def print_all_docker_pieces(self):
res_networks = subprocess.check_output(
@ -964,6 +978,28 @@ class ClickHouseCluster:
p.join(docker_compose_yml_dir, "docker_compose_postgres_cluster.yml"),
]
def setup_postgresql_java_client_cmd(
self, instance, env_variables, docker_compose_yml_dir
):
self.with_postgresql_java_client = True
self.base_cmd.extend(
[
"--file",
p.join(
docker_compose_yml_dir, "docker_compose_postgresql_java_client.yml"
),
]
)
self.base_postgresql_java_client_cmd = [
"docker-compose",
"--env-file",
instance.env_file,
"--project-name",
self.project_name,
"--file",
p.join(docker_compose_yml_dir, "docker_compose_postgresql_java_client.yml"),
]
def setup_hdfs_cmd(self, instance, env_variables, docker_compose_yml_dir):
self.with_hdfs = True
env_variables["HDFS_HOST"] = self.hdfs_host
@ -1212,6 +1248,7 @@ class ClickHouseCluster:
self.with_minio = True
cert_d = p.join(self.minio_dir, "certs")
env_variables["MINIO_CERTS_DIR"] = cert_d
env_variables["MINIO_DATA_DIR"] = self.minio_data_dir
env_variables["MINIO_PORT"] = str(self.minio_port)
env_variables["SSL_CERT_FILE"] = p.join(self.base_dir, cert_d, "public.crt")
@ -1335,6 +1372,7 @@ class ClickHouseCluster:
with_odbc_drivers=False,
with_postgres=False,
with_postgres_cluster=False,
with_postgresql_java_client=False,
with_hdfs=False,
with_kerberized_hdfs=False,
with_mongo=False,
@ -1360,6 +1398,7 @@ class ClickHouseCluster:
tmpfs=None,
zookeeper_docker_compose_path=None,
minio_certs_dir=None,
minio_data_dir=None,
use_keeper=True,
main_config_name="config.xml",
users_config_name="users.xml",
@ -1440,6 +1479,7 @@ class ClickHouseCluster:
with_odbc_drivers=with_odbc_drivers,
with_postgres=with_postgres,
with_postgres_cluster=with_postgres_cluster,
with_postgresql_java_client=with_postgresql_java_client,
hostname=hostname,
env_variables=env_variables,
image=image,
@ -1526,6 +1566,13 @@ class ClickHouseCluster:
)
)
if with_postgresql_java_client and not self.with_postgresql_java_client:
cmds.append(
self.setup_postgresql_java_client_cmd(
instance, env_variables, docker_compose_yml_dir
)
)
if with_odbc_drivers and not self.with_odbc_drivers:
self.with_odbc_drivers = True
if not self.with_mysql:
@ -1634,6 +1681,12 @@ class ClickHouseCluster:
else:
raise Exception("Overwriting minio certs dir")
if minio_data_dir is not None:
if self.minio_data_dir is None:
self.minio_data_dir = minio_data_dir
else:
raise Exception("Overwriting minio data dir")
if with_cassandra and not self.with_cassandra:
cmds.append(
self.setup_cassandra_cmd(
@ -2007,6 +2060,21 @@ class ClickHouseCluster:
raise Exception("Cannot wait Postgres container")
def wait_postgresql_java_client(self, timeout=180):
start = time.time()
while time.time() - start < timeout:
try:
if check_postgresql_java_client_is_available(
self.postgresql_java_client_docker_id
):
logging.debug("PostgreSQL Java Client is available")
return True
time.sleep(0.5)
except Exception as ex:
logging.debug("Can't find PostgreSQL Java Client" + str(ex))
time.sleep(0.5)
raise Exception("Cannot wait PostgreSQL Java Client container")
def wait_rabbitmq_to_start(self, timeout=180, throw=True):
self.rabbitmq_ip = self.get_instance_ip(self.rabbitmq_host)
@ -2306,7 +2374,12 @@ class ClickHouseCluster:
def start(self):
pytest_xdist_logging_to_separate_files.setup()
logging.info("Running tests in {}".format(self.base_path))
if not os.path.exists(self.instances_dir):
os.mkdir(self.instances_dir)
else:
logging.warning(
"Instance directory already exists. Did you call cluster.start() for second time?"
)
logging.debug(f"Cluster start called. is_up={self.is_up}")
self.print_all_docker_pieces()
@ -2441,7 +2514,7 @@ class ClickHouseCluster:
self.wait_postgres_to_start()
if self.with_postgres_cluster and self.base_postgres_cluster_cmd:
print("Setup Postgres")
logging.debug("Setup Postgres")
os.makedirs(self.postgres2_logs_dir)
os.chmod(self.postgres2_logs_dir, stat.S_IRWXU | stat.S_IRWXO)
os.makedirs(self.postgres3_logs_dir)
@ -2452,6 +2525,17 @@ class ClickHouseCluster:
self.up_called = True
self.wait_postgres_cluster_to_start()
if (
self.with_postgresql_java_client
and self.base_postgresql_java_client_cmd
):
logging.debug("Setup Postgres Java Client")
subprocess_check_call(
self.base_postgresql_java_client_cmd + common_opts
)
self.up_called = True
self.wait_postgresql_java_client()
if self.with_kafka and self.base_kafka_cmd:
logging.debug("Setup Kafka")
subprocess_check_call(
@ -2575,6 +2659,8 @@ class ClickHouseCluster:
os.path.join(self.base_dir, self.minio_certs_dir),
os.path.join(self.minio_dir, "certs"),
)
os.mkdir(self.minio_data_dir)
os.chmod(self.minio_data_dir, stat.S_IRWXU | stat.S_IRWXO)
minio_start_cmd = self.base_minio_cmd + common_opts
@ -2892,6 +2978,7 @@ class ClickHouseInstance:
with_odbc_drivers,
with_postgres,
with_postgres_cluster,
with_postgresql_java_client,
clickhouse_start_command=CLICKHOUSE_START_COMMAND,
main_config_name="config.xml",
users_config_name="users.xml",
@ -2953,6 +3040,7 @@ class ClickHouseInstance:
self.with_mysql_cluster = with_mysql_cluster
self.with_postgres = with_postgres
self.with_postgres_cluster = with_postgres_cluster
self.with_postgresql_java_client = with_postgresql_java_client
self.with_kafka = with_kafka
self.with_kerberized_kafka = with_kerberized_kafka
self.with_rabbitmq = with_rabbitmq

View File

@ -0,0 +1 @@
SELECT 1 as a

View File

@ -0,0 +1 @@
SELECT 'колонка' as a

View File

@ -0,0 +1 @@
CREATE DATABASE x; USE x; CREATE TABLE table1 (column UInt32) ENGINE = Memory; INSERT INTO table1 VALUES (0), (1), (5); INSERT INTO table1 VALUES (0), (1), (5); SELECT * FROM table1 ORDER BY column; DROP DATABASE x;

View File

@ -0,0 +1 @@
CREATE TEMPORARY TABLE tmp (tmp_column UInt32); INSERT INTO tmp VALUES (0), (1); SELECT * FROM tmp ORDER BY tmp_column; DROP TABLE tmp;

View File

@ -3,11 +3,9 @@
import datetime
import decimal
import os
import sys
import time
import uuid
import logging
import docker
import psycopg2 as py_psql
import psycopg2.extras
import pytest
@ -19,7 +17,7 @@ SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
DOCKER_COMPOSE_PATH = get_docker_compose_path()
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance(
cluster.add_instance(
"node",
main_configs=[
"configs/postresql.xml",
@ -30,6 +28,8 @@ node = cluster.add_instance(
"configs/server.key",
],
user_configs=["configs/default_passwd.xml"],
with_postgres=True,
with_postgresql_java_client=True,
env_variables={"UBSAN_OPTIONS": "print_stacktrace=1"},
)
@ -37,135 +37,83 @@ server_port = 5433
@pytest.fixture(scope="module")
def server_address():
cluster.start()
def started_cluster():
try:
yield cluster.get_instance_ip("node")
cluster.start()
yield cluster
except Exception as ex:
logging.exception(ex)
raise ex
finally:
cluster.shutdown()
@pytest.fixture(scope="module")
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",
"--force-recreate",
"-d",
"--build",
]
)
yield docker.DockerClient(
base_url="unix:///var/run/docker.sock",
version=cluster.docker_api_version,
timeout=600,
).containers.get(cluster.project_name + "_psql_1")
def test_psql_client(started_cluster):
node = cluster.instances["node"]
@pytest.fixture(scope="module")
def psql_server(psql_client):
"""Return PostgreSQL container when it is healthy."""
retries = 30
for i in range(retries):
info = psql_client.client.api.inspect_container(psql_client.name)
if info["State"]["Health"]["Status"] == "healthy":
break
time.sleep(1)
else:
print(info["State"])
raise Exception(
"PostgreSQL server has not started after {} retries.".format(retries)
for query_file in ["query1.sql", "query2.sql", "query3.sql", "query4.sql"]:
started_cluster.copy_file_to_container(
started_cluster.postgres_id,
os.path.join(SCRIPT_DIR, "queries", query_file),
f"/{query_file}",
)
cmd_prefix = [
"/usr/bin/psql",
f"sslmode=require host={node.hostname} port={server_port} user=default dbname=default password=123",
]
cmd_prefix += ["--no-align", "--field-separator=' '"]
return psql_client
@pytest.fixture(scope="module")
def java_container():
docker_compose = os.path.join(
DOCKER_COMPOSE_PATH, "docker_compose_postgresql_java_client.yml"
res = started_cluster.exec_in_container(
started_cluster.postgres_id, cmd_prefix + ["-f", "/query1.sql"], shell=True
)
run_and_check(
logging.debug(res)
assert res == "\n".join(["a", "1", "(1 row)", ""])
res = started_cluster.exec_in_container(
started_cluster.postgres_id, cmd_prefix + ["-f", "/query2.sql"], shell=True
)
logging.debug(res)
assert res == "\n".join(["a", "колонка", "(1 row)", ""])
res = started_cluster.exec_in_container(
started_cluster.postgres_id, cmd_prefix + ["-f", "/query3.sql"], shell=True
)
logging.debug(res)
assert res == "\n".join(
[
"docker-compose",
"-p",
cluster.project_name,
"-f",
docker_compose,
"up",
"--force-recreate",
"-d",
"--build",
"SELECT 0",
"SELECT 0",
"SELECT 0",
"INSERT 0 0",
"INSERT 0 0",
"column",
"0",
"0",
"1",
"1",
"5",
"5",
"(6 rows)",
"SELECT 0\n",
]
)
yield docker.DockerClient(
base_url="unix:///var/run/docker.sock",
version=cluster.docker_api_version,
timeout=600,
).containers.get(cluster.project_name + "_java_1")
def test_psql_is_ready(psql_server):
pass
def test_psql_client(psql_client, server_address):
cmd_prefix = 'psql "sslmode=require host={server_address} port={server_port} user=default dbname=default password=123" '.format(
server_address=server_address, server_port=server_port
res = started_cluster.exec_in_container(
started_cluster.postgres_id, cmd_prefix + ["-f", "/query4.sql"], shell=True
)
cmd_prefix += "--no-align --field-separator=' ' "
code, (stdout, stderr) = psql_client.exec_run(
cmd_prefix + '-c "SELECT 1 as a"', demux=True
)
assert stdout.decode() == "\n".join(["a", "1", "(1 row)", ""])
code, (stdout, stderr) = psql_client.exec_run(
cmd_prefix + '''-c "SELECT 'колонка' as a"''', demux=True
)
assert stdout.decode() == "\n".join(["a", "колонка", "(1 row)", ""])
code, (stdout, stderr) = psql_client.exec_run(
cmd_prefix
+ "-c "
+ """
"CREATE DATABASE x;
USE x;
CREATE TABLE table1 (column UInt32) ENGINE = Memory;
INSERT INTO table1 VALUES (0), (1), (5);
INSERT INTO table1 VALUES (0), (1), (5);
SELECT * FROM table1 ORDER BY column;"
""",
demux=True,
)
assert stdout.decode() == "\n".join(
["column", "0", "0", "1", "1", "5", "5", "(6 rows)", ""]
logging.debug(res)
assert res == "\n".join(
["SELECT 0", "INSERT 0 0", "tmp_column", "0", "1", "(2 rows)", "SELECT 0\n"]
)
code, (stdout, stderr) = psql_client.exec_run(
cmd_prefix
+ "-c "
+ """
"DROP DATABASE x;
CREATE TEMPORARY TABLE tmp (tmp_column UInt32);
INSERT INTO tmp VALUES (0), (1);
SELECT * FROM tmp ORDER BY tmp_column;"
""",
demux=True,
)
assert stdout.decode() == "\n".join(["tmp_column", "0", "1", "(2 rows)", ""])
def test_python_client(started_cluster):
node = cluster.instances["node"]
def test_python_client(server_address):
with pytest.raises(py_psql.InternalError) as exc_info:
ch = py_psql.connect(
host=server_address,
host=node.ip_address,
port=server_port,
user="default",
password="123",
@ -179,7 +127,7 @@ def test_python_client(server_address):
)
ch = py_psql.connect(
host=server_address,
host=node.ip_address,
port=server_port,
user="default",
password="123",
@ -209,26 +157,37 @@ def test_python_client(server_address):
decimal.Decimal("0.3333330000"),
uuid.UUID("61f0c404-5cb3-11e7-907b-a6006ad3dba0"),
)
cur.execute("DROP DATABASE x")
def test_java_client(server_address, java_container):
def test_java_client(started_cluster):
node = cluster.instances["node"]
with open(os.path.join(SCRIPT_DIR, "java.reference")) as fp:
reference = fp.read()
# database not exists exception.
code, (stdout, stderr) = java_container.exec_run(
"java JavaConnectorTest --host {host} --port {port} --user default --database "
"abc".format(host=server_address, port=server_port),
demux=True,
)
assert code == 1
with pytest.raises(Exception) as exc:
res = started_cluster.exec_in_container(
started_cluster.postgresql_java_client_docker_id,
[
"bash",
"-c",
f"java JavaConnectorTest --host {node.hostname} --port {server_port} --user default --database abc",
],
)
assert (
"org.postgresql.util.PSQLException: ERROR: Invalid user or password"
in str(exc.value)
)
# non-empty password passed.
code, (stdout, stderr) = java_container.exec_run(
"java JavaConnectorTest --host {host} --port {port} --user default --password 123 --database "
"default".format(host=server_address, port=server_port),
demux=True,
res = started_cluster.exec_in_container(
started_cluster.postgresql_java_client_docker_id,
[
"bash",
"-c",
f"java JavaConnectorTest --host {node.hostname} --port {server_port} --user default --password 123 --database default",
],
)
print(stdout, stderr, file=sys.stderr)
assert code == 0
assert stdout.decode() == reference
assert res == reference