Fix tests.

This commit is contained in:
Vitaly Baranov 2020-10-11 05:19:16 +03:00
parent 8e6c68f4f7
commit db9f762e73
7 changed files with 95 additions and 119 deletions

View File

@ -16,6 +16,7 @@ CONTAINER_NAME = "clickhouse_integration_tests"
CONFIG_DIR_IN_REPO = "programs/server" CONFIG_DIR_IN_REPO = "programs/server"
INTERGATION_DIR_IN_REPO = "tests/integration" INTERGATION_DIR_IN_REPO = "tests/integration"
SRC_DIR_IN_REPO = "src"
DIND_INTEGRATION_TESTS_IMAGE_NAME = "yandex/clickhouse-integration-tests-runner" DIND_INTEGRATION_TESTS_IMAGE_NAME = "yandex/clickhouse-integration-tests-runner"
@ -51,6 +52,13 @@ def check_args_and_update_paths(args):
args.cases_dir = os.path.abspath(os.path.join(CLICKHOUSE_ROOT, INTERGATION_DIR_IN_REPO)) args.cases_dir = os.path.abspath(os.path.join(CLICKHOUSE_ROOT, INTERGATION_DIR_IN_REPO))
logging.info("Cases dir is not set. Will use {}".format(args.cases_dir)) logging.info("Cases dir is not set. Will use {}".format(args.cases_dir))
if args.src_dir:
if not os.path.isabs(args.src_dir):
args.src_dir = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.src_dir))
else:
args.src_dir = os.path.abspath(os.path.join(CLICKHOUSE_ROOT, SRC_DIR_IN_REPO))
logging.info("src dir is not set. Will use {}".format(args.src_dir))
logging.info("base_configs_dir: {}, binary: {}, cases_dir: {} ".format(args.base_configs_dir, args.binary, args.cases_dir)) logging.info("base_configs_dir: {}, binary: {}, cases_dir: {} ".format(args.base_configs_dir, args.binary, args.cases_dir))
for path in [args.binary, args.bridge_binary, args.base_configs_dir, args.cases_dir, CLICKHOUSE_ROOT]: for path in [args.binary, args.bridge_binary, args.base_configs_dir, args.cases_dir, CLICKHOUSE_ROOT]:
@ -104,6 +112,11 @@ if __name__ == "__main__":
default=os.environ.get("CLICKHOUSE_TESTS_INTEGRATION_PATH"), default=os.environ.get("CLICKHOUSE_TESTS_INTEGRATION_PATH"),
help="Path to integration tests cases and configs directory. For example tests/integration in repository") help="Path to integration tests cases and configs directory. For example tests/integration in repository")
parser.add_argument(
"--src-dir",
default=os.environ.get("CLICKHOUSE_SRC_DIR"),
help="Path to the 'src' directory in repository. Used to provide schemas (e.g. *.proto) for some tests when those schemas are located in the 'src' directory")
parser.add_argument( parser.add_argument(
"--clickhouse-root", "--clickhouse-root",
help="Path to repository root folder. Used to take configuration from repository default paths.") help="Path to repository root folder. Used to take configuration from repository default paths.")
@ -174,6 +187,7 @@ if __name__ == "__main__":
cmd = "docker run {net} {tty} --rm --name {name} --privileged --volume={bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ cmd = "docker run {net} {tty} --rm --name {name} --privileged --volume={bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \
--volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \
--volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \
--volume={name}_volume:/var/lib/docker {env_tags} -e PYTEST_OPTS='{opts}' {img} {command}".format( --volume={name}_volume:/var/lib/docker {env_tags} -e PYTEST_OPTS='{opts}' {img} {command}".format(
net=net, net=net,
tty=tty, tty=tty,
@ -181,6 +195,7 @@ if __name__ == "__main__":
bridge_bin=args.bridge_binary, bridge_bin=args.bridge_binary,
base_cfg=args.base_configs_dir, base_cfg=args.base_configs_dir,
cases_dir=args.cases_dir, cases_dir=args.cases_dir,
src_dir=args.src_dir,
env_tags=env_tags, env_tags=env_tags,
opts=' '.join(args.pytest_args), opts=' '.join(args.pytest_args),
img=DIND_INTEGRATION_TESTS_IMAGE_NAME + ":" + args.docker_image_version, img=DIND_INTEGRATION_TESTS_IMAGE_NAME + ":" + args.docker_image_version,

View File

@ -0,0 +1 @@
_gen

View File

@ -1,20 +0,0 @@
<?xml version="1.0"?>
<yandex>
<logger>
<level>trace</level>
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
<size>1000M</size>
<count>10</count>
</logger>
<tcp_port>9000</tcp_port>
<grpc_port>9001</grpc_port>
<listen_host>127.0.0.1</listen_host>
<max_concurrent_queries>500</max_concurrent_queries>
<mark_cache_size>5368709120</mark_cache_size>
<path>./clickhouse/</path>
<users_config>users.xml</users_config>
</yandex>

View File

@ -0,0 +1,3 @@
<yandex>
<grpc_port>9001</grpc_port>
</yandex>

View File

@ -1,23 +0,0 @@
<?xml version="1.0"?>
<yandex>
<profiles>
<default>
</default>
</profiles>
<users>
<default>
<password>123</password>
<networks incl="networks" replace="replace">
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
</default>
</users>
<quotas>
<default>
</default>
</quotas>
</yandex>

View File

@ -1,92 +1,92 @@
# coding: utf-8
# proto file should be the same, as in server GRPC
import os import os
import pytest import pytest
import subprocess import subprocess
import sys
import grpc import grpc
from docker.models.containers import Container
from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseCluster
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
# Use grpcio-tools to generate *pb2.py files from *.proto.
proto_dir = os.path.join(SCRIPT_DIR, './protos') proto_dir = os.path.join(SCRIPT_DIR, './protos')
try: proto_gen_dir = os.path.join(SCRIPT_DIR, './_gen')
subprocess.check_call( os.makedirs(proto_gen_dir, exist_ok=True)
'python -m grpc_tools.protoc -I{proto_path} --python_out=. --grpc_python_out=. \ subprocess.check_call(
{proto_path}/GrpcConnection.proto'.format(proto_path=proto_dir), shell=True) 'python3 -m grpc_tools.protoc -I{proto_dir} --python_out={proto_gen_dir} --grpc_python_out={proto_gen_dir} \
except subprocess.CalledProcessError, e: {proto_dir}/clickhouse_grpc.proto'.format(proto_dir=proto_dir, proto_gen_dir=proto_gen_dir), shell=True)
print "Please, copy proto file, can be programs/server/grpc_proto/GrpcConnection.proto"
assert False
finally:
import GrpcConnection_pb2
import GrpcConnection_pb2_grpc
config_dir = os.path.join(SCRIPT_DIR, './configs') # Import everything from the generated *pb2.py files.
cluster = ClickHouseCluster(__file__) sys.path.append(proto_gen_dir)
import clickhouse_grpc_pb2
import clickhouse_grpc_pb2_grpc
node = cluster.add_instance('node', config_dir=config_dir, env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) config_dir = os.path.join(SCRIPT_DIR, './configs')
server_port = 9001 cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', main_configs=['configs/grpc_port.xml'])
server_port = 9001
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
def server_address(): def server_address():
cluster.start() cluster.start()
try: try:
yield cluster.get_instance_ip('node') yield cluster.get_instance_ip('node')
finally: finally:
cluster.shutdown() cluster.shutdown()
def Query(server_address_and_port, query, mode="output", insert_data=[]):
output = []
totals = []
data_stream = (len(insert_data) != 0)
with grpc.insecure_channel(server_address_and_port) as channel:
stub = GrpcConnection_pb2_grpc.GRPCStub(channel)
def write_query():
user_info = GrpcConnection_pb2.User(user="default", password='123', quota='default')
query_info = GrpcConnection_pb2.QuerySettings(query=query, query_id='123', data_stream=data_stream, format='TabSeparated')
yield GrpcConnection_pb2.QueryRequest(user_info=user_info, query_info=query_info)
if data_stream:
for data in insert_data:
yield GrpcConnection_pb2.QueryRequest(insert_data=data)
yield GrpcConnection_pb2.QueryRequest(insert_data="")
for response in stub.Query(write_query(), 10.0):
output += response.output.split()
totals += response.totals.split()
if mode == "output":
return output
elif mode == "totals":
return totals
def test_ordinary_query(server_address): def Query(server_address_and_port, query, mode="output", insert_data=[]):
server_address_and_port = server_address + ':' + str(server_port) output = []
assert Query(server_address_and_port, "SELECT 1") == [u'1'] totals = []
assert Query(server_address_and_port, "SELECT count() FROM numbers(100)") == [u'100'] data_stream = (len(insert_data) != 0)
with grpc.insecure_channel(server_address_and_port) as channel:
grpc.channel_ready_future(channel).result()
stub = clickhouse_grpc_pb2_grpc.GRPCStub(channel)
def write_query():
user_info = clickhouse_grpc_pb2.User(user="default", quota='default')
query_info = clickhouse_grpc_pb2.QuerySettings(query=query, query_id='123', data_stream=data_stream, format='TabSeparated')
yield clickhouse_grpc_pb2.QueryRequest(user_info=user_info, query_info=query_info)
if data_stream:
for data in insert_data:
yield clickhouse_grpc_pb2.QueryRequest(insert_data=data)
yield clickhouse_grpc_pb2.QueryRequest(insert_data="")
for response in stub.Query(write_query(), 10.0):
output += response.output.split()
totals += response.totals.split()
if mode == "output":
return output
elif mode == "totals":
return totals
def test_query_insert(server_address): def test_ordinary_query(server_address):
server_address_and_port = server_address + ':' + str(server_port) server_address_and_port = server_address + ':' + str(server_port)
assert Query(server_address_and_port, "CREATE TABLE t (a UInt8) ENGINE = Memory") == [] assert Query(server_address_and_port, "SELECT 1") == [u'1']
assert Query(server_address_and_port, "INSERT INTO t VALUES (1),(2),(3)") == [] assert Query(server_address_and_port, "SELECT count() FROM numbers(100)") == [u'100']
assert Query(server_address_and_port, "INSERT INTO t FORMAT TabSeparated 4\n5\n6\n") == []
assert Query(server_address_and_port, "INSERT INTO t FORMAT TabSeparated 10\n11\n12\n") == []
assert Query(server_address_and_port, "SELECT a FROM t ORDER BY a") == [u'1', u'2', u'3', u'4', u'5', u'6', u'10', u'11', u'12']
assert Query(server_address_and_port, "DROP TABLE t") == []
def test_handle_mistakes(server_address): def test_query_insert(server_address):
server_address_and_port = server_address + ':' + str(server_port) server_address_and_port = server_address + ':' + str(server_port)
assert Query(server_address_and_port, "") == [] assert Query(server_address_and_port, "CREATE TABLE t (a UInt8) ENGINE = Memory") == []
assert Query(server_address_and_port, "CREATE TABLE t (a UInt8) ENGINE = Memory") == [] assert Query(server_address_and_port, "INSERT INTO t VALUES (1),(2),(3)") == []
assert Query(server_address_and_port, "CREATE TABLE t (a UInt8) ENGINE = Memory") == [] assert Query(server_address_and_port, "INSERT INTO t FORMAT TabSeparated 4\n5\n6\n") == []
assert Query(server_address_and_port, "INSERT INTO t FORMAT TabSeparated 10\n11\n12\n") == []
assert Query(server_address_and_port, "SELECT a FROM t ORDER BY a") == [u'1', u'2', u'3', u'4', u'5', u'6', u'10', u'11', u'12']
assert Query(server_address_and_port, "DROP TABLE t") == []
def test_totals(server_address): def test_handle_mistakes(server_address):
server_address_and_port = server_address + ':' + str(server_port) server_address_and_port = server_address + ':' + str(server_port)
assert Query(server_address_and_port, "") == [] assert Query(server_address_and_port, "") == []
assert Query(server_address_and_port, "CREATE TABLE tabl (x UInt8, y UInt8) ENGINE = Memory;") == [] assert Query(server_address_and_port, "CREATE TABLE t (a UInt8) ENGINE = Memory") == []
assert Query(server_address_and_port, "INSERT INTO tabl VALUES (1, 2), (2, 4), (3, 2), (3, 3), (3, 4);") == [] assert Query(server_address_and_port, "CREATE TABLE t (a UInt8) ENGINE = Memory") == []
assert Query(server_address_and_port, "SELECT sum(x), y FROM tabl GROUP BY y WITH TOTALS") == [u'4', u'2', u'3', u'3', u'5', u'4']
assert Query(server_address_and_port, "SELECT sum(x), y FROM tabl GROUP BY y WITH TOTALS", mode="totals") == [u'12', u'0']
def test_query_insert(server_address): def test_totals(server_address):
server_address_and_port = server_address + ':' + str(server_port) server_address_and_port = server_address + ':' + str(server_port)
assert Query(server_address_and_port, "CREATE TABLE t (a UInt8) ENGINE = Memory") == [] assert Query(server_address_and_port, "") == []
assert Query(server_address_and_port, "INSERT INTO t VALUES", insert_data=["(1),(2),(3)", "(5),(4),(6)", "(8),(7),(9)"]) == [] assert Query(server_address_and_port, "CREATE TABLE tabl (x UInt8, y UInt8) ENGINE = Memory;") == []
assert Query(server_address_and_port, "SELECT a FROM t ORDER BY a") == [u'1', u'2', u'3', u'4', u'5', u'6', u'7', u'8', u'9'] assert Query(server_address_and_port, "INSERT INTO tabl VALUES (1, 2), (2, 4), (3, 2), (3, 3), (3, 4);") == []
assert Query(server_address_and_port, "DROP TABLE t") == [] assert Query(server_address_and_port, "SELECT sum(x), y FROM tabl GROUP BY y WITH TOTALS") == [u'4', u'2', u'3', u'3', u'5', u'4']
assert Query(server_address_and_port, "SELECT sum(x), y FROM tabl GROUP BY y WITH TOTALS", mode="totals") == [u'12', u'0']
def test_query_insert(server_address):
server_address_and_port = server_address + ':' + str(server_port)
assert Query(server_address_and_port, "CREATE TABLE t (a UInt8) ENGINE = Memory") == []
assert Query(server_address_and_port, "INSERT INTO t VALUES", insert_data=["(1),(2),(3)", "(5),(4),(6)", "(8),(7),(9)"]) == []
assert Query(server_address_and_port, "SELECT a FROM t ORDER BY a") == [u'1', u'2', u'3', u'4', u'5', u'6', u'7', u'8', u'9']
assert Query(server_address_and_port, "DROP TABLE t") == []