Add explicit main_configs, user_configs and dictionaries in integration tests.

This commit is contained in:
Yatsishin Ilya 2020-08-12 11:55:04 +03:00
parent 9767d96274
commit 160776f183
117 changed files with 922 additions and 439 deletions

View File

@ -19,7 +19,8 @@ set -e
echo "Start tests" echo "Start tests"
export CLICKHOUSE_TESTS_SERVER_BIN_PATH=/clickhouse export CLICKHOUSE_TESTS_SERVER_BIN_PATH=/clickhouse
export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse
export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-config export CLICKHOUSE_TESTS_CONFIG_DIR=/clickhouse-config
export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-base-config
export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge
export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest}

View File

@ -18,7 +18,7 @@ if(MAKE_STATIC_LIBRARIES AND DOCKER_CMD)
if(NOT INTEGRATION_USE_RUNNER AND DOCKER_COMPOSE_CMD AND PYTEST_CMD) if(NOT INTEGRATION_USE_RUNNER AND DOCKER_COMPOSE_CMD AND PYTEST_CMD)
# To run one test with debug: # To run one test with debug:
# cmake . -DPYTEST_OPT="-ss;test_cluster_copier" # cmake . -DPYTEST_OPT="-ss;test_cluster_copier"
add_test(NAME integration-pytest WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} COMMAND env ${TEST_USE_BINARIES} "CLICKHOUSE_TESTS_BASE_CONFIG_DIR=${ClickHouse_SOURCE_DIR}/programs/server/" ${PYTEST_STARTER} ${PYTEST_CMD} ${PYTEST_OPT}) add_test(NAME integration-pytest WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} COMMAND env ${TEST_USE_BINARIES} "CLICKHOUSE_TESTS_BASE_CONFIG_DIR=${ClickHouse_SOURCE_DIR}/programs/server/" "CLICKHOUSE_TESTS_CONFIG_DIR=${ClickHouse_SOURCE_DIR}/tests/config/" ${PYTEST_STARTER} ${PYTEST_CMD} ${PYTEST_OPT})
message(STATUS "Using tests in docker DOCKER=${DOCKER_CMD}; DOCKER_COMPOSE=${DOCKER_COMPOSE_CMD}; PYTEST=${PYTEST_STARTER} ${PYTEST_CMD} ${PYTEST_OPT}") message(STATUS "Using tests in docker DOCKER=${DOCKER_CMD}; DOCKER_COMPOSE=${DOCKER_COMPOSE_CMD}; PYTEST=${PYTEST_STARTER} ${PYTEST_CMD} ${PYTEST_OPT}")
endif() endif()
endif() endif()

View File

@ -0,0 +1,4 @@
<?xml version="1.0"?>
<yandex>
<dictionaries_config>/etc/clickhouse-server/dictionaries/*.xml</dictionaries_config>
</yandex>

View File

@ -71,7 +71,7 @@ class CommandRequest:
self.stderr_file = tempfile.TemporaryFile() self.stderr_file = tempfile.TemporaryFile()
self.ignore_error = ignore_error self.ignore_error = ignore_error
#print " ".join(command) print " ".join(command)
# we suppress stderror on client becase sometimes thread sanitizer # we suppress stderror on client becase sometimes thread sanitizer
# can print some debug information there # can print some debug information there

View File

@ -1,25 +1,25 @@
import base64 import base64
import cassandra.cluster
import distutils.dir_util import distutils.dir_util
import docker
import errno import errno
import httplib
import logging
import os import os
import os.path as p import os.path as p
import pprint
import psycopg2
import pwd import pwd
import pymongo
import pymysql
import re import re
import requests
import shutil import shutil
import socket import socket
import subprocess import subprocess
import time import time
import urllib import urllib
import httplib
import requests
import xml.dom.minidom import xml.dom.minidom
import logging
import docker
import pprint
import psycopg2
import pymongo
import pymysql
import cassandra.cluster
from dicttoxml import dicttoxml from dicttoxml import dicttoxml
from kazoo.client import KazooClient from kazoo.client import KazooClient
from kazoo.exceptions import KazooException from kazoo.exceptions import KazooException
@ -40,6 +40,7 @@ SANITIZER_SIGN = "=================="
def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME):
full_path = os.path.join(path, fname) full_path = os.path.join(path, fname)
with open(full_path, 'w') as f: with open(full_path, 'w') as f:
f.write('TSAN_OPTIONS="external_symbolizer_path=/usr/bin/llvm-symbolizer"\n')
for var, value in variables.items(): for var, value in variables.items():
f.write("=".join([var, value]) + "\n") f.write("=".join([var, value]) + "\n")
return full_path return full_path
@ -88,12 +89,16 @@ class ClickHouseCluster:
these directories will contain logs, database files, docker-compose config, ClickHouse configs etc. these directories will contain logs, database files, docker-compose config, ClickHouse configs etc.
""" """
def __init__(self, base_path, name=None, base_configs_dir=None, server_bin_path=None, client_bin_path=None, def __init__(self, base_path, name=None, base_config_dir=None, config_dir=None, server_bin_path=None, client_bin_path=None,
odbc_bridge_bin_path=None, zookeeper_config_path=None, custom_dockerd_host=None): odbc_bridge_bin_path=None, zookeeper_config_path=None, custom_dockerd_host=None):
for param in os.environ.keys():
print "ENV %40s %s" % (param,os.environ[param])
self.base_dir = p.dirname(base_path) self.base_dir = p.dirname(base_path)
self.name = name if name is not None else '' self.name = name if name is not None else ''
self.base_configs_dir = base_configs_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR', self.base_config_dir = base_config_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR',
'/etc/clickhouse-server/')
self.config_dir = config_dir or os.environ.get('CLICKHOUSE_TESTS_CONFIG_DIR',
'/etc/clickhouse-server/') '/etc/clickhouse-server/')
self.server_bin_path = p.realpath( self.server_bin_path = p.realpath(
server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse')) server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse'))
@ -154,6 +159,7 @@ class ClickHouseCluster:
self.docker_client = None self.docker_client = None
self.is_up = False self.is_up = False
print "CLUSTER INIT base_config_dir:{} config_dir:{}".format(self.base_config_dir, self.config_dir)
def get_client_cmd(self): def get_client_cmd(self):
cmd = self.client_bin_path cmd = self.client_bin_path
@ -161,7 +167,7 @@ class ClickHouseCluster:
cmd += " client" cmd += " client"
return cmd return cmd
def add_instance(self, name, config_dir=None, main_configs=None, user_configs=None, macros=None, def add_instance(self, name, base_config_dir=None, config_dir=None, main_configs=None, user_configs=None, dictionaries = None, macros=None,
with_zookeeper=False, with_mysql=False, with_kafka=False, with_rabbitmq=False, clickhouse_path_dir=None, with_zookeeper=False, with_mysql=False, with_kafka=False, with_rabbitmq=False, clickhouse_path_dir=None,
with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False,
with_redis=False, with_minio=False, with_cassandra=False, with_redis=False, with_minio=False, with_cassandra=False,
@ -172,6 +178,7 @@ class ClickHouseCluster:
name - the name of the instance directory and the value of the 'instance' macro in ClickHouse. name - the name of the instance directory and the value of the 'instance' macro in ClickHouse.
config_dir - a directory with config files which content will be copied to /etc/clickhouse-server/ directory config_dir - a directory with config files which content will be copied to /etc/clickhouse-server/ directory
base_config_dir - a directory with config.xml and users.xml files which will be copied to /etc/clickhouse-server/ directory
main_configs - a list of config files that will be added to config.d/ directory main_configs - a list of config files that will be added to config.d/ directory
user_configs - a list of config files that will be added to users.d/ directory user_configs - a list of config files that will be added to users.d/ directory
with_zookeeper - if True, add ZooKeeper configuration to configs and ZooKeeper instances to the cluster. with_zookeeper - if True, add ZooKeeper configuration to configs and ZooKeeper instances to the cluster.
@ -184,11 +191,11 @@ class ClickHouseCluster:
raise Exception("Can\'t add instance `%s': there is already an instance with the same name!" % name) raise Exception("Can\'t add instance `%s': there is already an instance with the same name!" % name)
instance = ClickHouseInstance( instance = ClickHouseInstance(
self, self.base_dir, name, config_dir, main_configs or [], user_configs or [], macros or {}, self, self.base_dir, name, base_config_dir if base_config_dir else self.base_config_dir,
with_zookeeper, config_dir if config_dir else self.config_dir, main_configs or [], user_configs or [], dictionaries or [],
macros or {}, with_zookeeper,
self.zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra, self.zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra,
self.base_configs_dir, self.server_bin_path, self.server_bin_path, self.odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=hostname,
self.odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=hostname,
env_variables=env_variables or {}, image=image, stay_alive=stay_alive, ipv4_address=ipv4_address, env_variables=env_variables or {}, image=image, stay_alive=stay_alive, ipv4_address=ipv4_address,
ipv6_address=ipv6_address, ipv6_address=ipv6_address,
with_installed_binary=with_installed_binary, tmpfs=tmpfs or []) with_installed_binary=with_installed_binary, tmpfs=tmpfs or [])
@ -458,19 +465,19 @@ class ClickHouseCluster:
try: try:
minio_client.list_buckets() minio_client.list_buckets()
logging.info("Connected to Minio.") print("Connected to Minio.")
if minio_client.bucket_exists(self.minio_bucket): if minio_client.bucket_exists(self.minio_bucket):
minio_client.remove_bucket(self.minio_bucket) minio_client.remove_bucket(self.minio_bucket)
minio_client.make_bucket(self.minio_bucket) minio_client.make_bucket(self.minio_bucket)
logging.info("S3 bucket '%s' created", self.minio_bucket) print("S3 bucket '%s' created", self.minio_bucket)
self.minio_client = minio_client self.minio_client = minio_client
return return
except Exception as ex: except Exception as ex:
logging.warning("Can't connect to Minio: %s", str(ex)) print("Can't connect to Minio: %s", str(ex))
time.sleep(1) time.sleep(1)
raise Exception("Can't wait Minio to start") raise Exception("Can't wait Minio to start")
@ -482,10 +489,10 @@ class ClickHouseCluster:
try: try:
sr_client._send_request(sr_client.url) sr_client._send_request(sr_client.url)
self.schema_registry_client = sr_client self.schema_registry_client = sr_client
logging.info("Connected to SchemaRegistry") print("Connected to SchemaRegistry")
return return
except Exception as ex: except Exception as ex:
logging.warning("Can't connect to SchemaRegistry: %s", str(ex)) print("Can't connect to SchemaRegistry: %s", str(ex))
time.sleep(1) time.sleep(1)
def wait_cassandra_to_start(self, timeout=30): def wait_cassandra_to_start(self, timeout=30):
@ -501,25 +508,27 @@ class ClickHouseCluster:
time.sleep(1) time.sleep(1)
def start(self, destroy_dirs=True): def start(self, destroy_dirs=True):
print "Cluster start called. is_up={}, destroy_dirs={}".format(self.is_up, destroy_dirs)
if self.is_up: if self.is_up:
return return
# Just in case kill unstopped containers from previous launch # Just in case kill unstopped containers from previous launch
try: try:
logging.info("Trying to kill unstopped containers...") print("Trying to kill unstopped containers...")
if not subprocess_call(['docker-compose', 'kill']): if not subprocess_call(['docker-compose', 'kill']):
subprocess_call(['docker-compose', 'down', '--volumes']) subprocess_call(['docker-compose', 'down', '--volumes'])
logging.info("Unstopped containers killed") print("Unstopped containers killed")
except: except:
pass pass
try: try:
if destroy_dirs and p.exists(self.instances_dir): if destroy_dirs and p.exists(self.instances_dir):
logging.info("Removing instances dir %s", self.instances_dir) print("Removing instances dir %s", self.instances_dir)
shutil.rmtree(self.instances_dir) shutil.rmtree(self.instances_dir)
for instance in self.instances.values(): for instance in self.instances.values():
print('Setup directory for instance: {} destroy_dirs: {}'.format(instance.name, destroy_dirs))
instance.create_dir(destroy_dir=destroy_dirs) instance.create_dir(destroy_dir=destroy_dirs)
self.docker_client = docker.from_env(version=self.docker_api_version) self.docker_client = docker.from_env(version=self.docker_api_version)
@ -527,6 +536,7 @@ class ClickHouseCluster:
common_opts = ['up', '-d', '--force-recreate'] common_opts = ['up', '-d', '--force-recreate']
if self.with_zookeeper and self.base_zookeeper_cmd: if self.with_zookeeper and self.base_zookeeper_cmd:
print('Setup ZooKeeper')
env = os.environ.copy() env = os.environ.copy()
if not self.zookeeper_use_tmpfs: if not self.zookeeper_use_tmpfs:
env['ZK_FS'] = 'bind' env['ZK_FS'] = 'bind'
@ -545,14 +555,17 @@ class ClickHouseCluster:
self.wait_zookeeper_to_start(120) self.wait_zookeeper_to_start(120)
if self.with_mysql and self.base_mysql_cmd: if self.with_mysql and self.base_mysql_cmd:
print('Setup MySQL')
subprocess_check_call(self.base_mysql_cmd + common_opts) subprocess_check_call(self.base_mysql_cmd + common_opts)
self.wait_mysql_to_start(120) self.wait_mysql_to_start(120)
if self.with_postgres and self.base_postgres_cmd: if self.with_postgres and self.base_postgres_cmd:
print('Setup Postgres')
subprocess_check_call(self.base_postgres_cmd + common_opts) subprocess_check_call(self.base_postgres_cmd + common_opts)
self.wait_postgres_to_start(120) self.wait_postgres_to_start(120)
if self.with_kafka and self.base_kafka_cmd: if self.with_kafka and self.base_kafka_cmd:
print('Setup Kafka')
subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes']) subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes'])
self.kafka_docker_id = self.get_instance_docker_id('kafka1') self.kafka_docker_id = self.get_instance_docker_id('kafka1')
self.wait_schema_registry_to_start(120) self.wait_schema_registry_to_start(120)
@ -562,14 +575,17 @@ class ClickHouseCluster:
self.rabbitmq_docker_id = self.get_instance_docker_id('rabbitmq1') self.rabbitmq_docker_id = self.get_instance_docker_id('rabbitmq1')
if self.with_hdfs and self.base_hdfs_cmd: if self.with_hdfs and self.base_hdfs_cmd:
print('Setup HDFS')
subprocess_check_call(self.base_hdfs_cmd + common_opts) subprocess_check_call(self.base_hdfs_cmd + common_opts)
self.wait_hdfs_to_start(120) self.wait_hdfs_to_start(120)
if self.with_mongo and self.base_mongo_cmd: if self.with_mongo and self.base_mongo_cmd:
print('Setup Mongo')
subprocess_check_call(self.base_mongo_cmd + common_opts) subprocess_check_call(self.base_mongo_cmd + common_opts)
self.wait_mongo_to_start(30) self.wait_mongo_to_start(30)
if self.with_redis and self.base_redis_cmd: if self.with_redis and self.base_redis_cmd:
print('Setup Redis')
subprocess_check_call(self.base_redis_cmd + ['up', '-d', '--force-recreate']) subprocess_check_call(self.base_redis_cmd + ['up', '-d', '--force-recreate'])
time.sleep(10) time.sleep(10)
@ -608,18 +624,19 @@ class ClickHouseCluster:
self.wait_cassandra_to_start() self.wait_cassandra_to_start()
clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate'] clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate']
logging.info("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd))) print("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd)))
subprocess_check_call(clickhouse_start_cmd) subprocess_check_call(clickhouse_start_cmd)
logging.info("ClickHouse instance created") print("ClickHouse instance created")
start_deadline = time.time() + 20.0 # seconds start_deadline = time.time() + 20.0 # seconds
for instance in self.instances.itervalues(): for instance in self.instances.itervalues():
instance.docker_client = self.docker_client instance.docker_client = self.docker_client
instance.ip_address = self.get_instance_ip(instance.name) instance.ip_address = self.get_instance_ip(instance.name)
logging.info("Waiting for ClickHouse start...") print("Waiting for ClickHouse start...")
instance.wait_for_start(start_deadline) instance.wait_for_start(start_deadline)
logging.info("ClickHouse started") print("ClickHouse started")
instance.client = Client(instance.ip_address, command=self.client_bin_path) instance.client = Client(instance.ip_address, command=self.client_bin_path)
@ -633,7 +650,10 @@ class ClickHouseCluster:
def shutdown(self, kill=True): def shutdown(self, kill=True):
sanitizer_assert_instance = None sanitizer_assert_instance = None
with open(self.docker_logs_path, "w+") as f: with open(self.docker_logs_path, "w+") as f:
try:
subprocess.check_call(self.base_cmd + ['logs'], stdout=f) subprocess.check_call(self.base_cmd + ['logs'], stdout=f)
except Exception as e:
print "Unable to get logs from docker."
f.seek(0) f.seek(0)
for line in f: for line in f:
if SANITIZER_SIGN in line: if SANITIZER_SIGN in line:
@ -641,8 +661,15 @@ class ClickHouseCluster:
break break
if kill: if kill:
try:
subprocess_check_call(self.base_cmd + ['kill']) subprocess_check_call(self.base_cmd + ['kill'])
except Exception as e:
print "Kill command failed durung shutdown. {}".format(repr(e))
try:
subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans']) subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans'])
except Exception as e:
print "Down + remove orphans failed durung shutdown. {}".format(repr(e))
self.is_up = False self.is_up = False
@ -707,7 +734,7 @@ services:
image: {image} image: {image}
hostname: {hostname} hostname: {hostname}
volumes: volumes:
- {configs_dir}:/etc/clickhouse-server/ - {instance_config_dir}:/etc/clickhouse-server/
- {db_dir}:/var/lib/clickhouse/ - {db_dir}:/var/lib/clickhouse/
- {logs_dir}:/var/log/clickhouse-server/ - {logs_dir}:/var/log/clickhouse-server/
{binary_volume} {binary_volume}
@ -723,6 +750,9 @@ services:
- {env_file} - {env_file}
security_opt: security_opt:
- label:disable - label:disable
dns_opt:
- timeout:1
- attempts:3
{networks} {networks}
{app_net} {app_net}
{ipv4_address} {ipv4_address}
@ -735,9 +765,9 @@ services:
class ClickHouseInstance: class ClickHouseInstance:
def __init__( def __init__(
self, cluster, base_path, name, custom_config_dir, custom_main_configs, custom_user_configs, macros, self, cluster, base_path, name, base_config_dir, config_dir, custom_main_configs, custom_user_configs, custom_dictionaries,
with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra, macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio,
base_configs_dir, server_bin_path, odbc_bridge_bin_path, with_cassandra, server_bin_path, odbc_bridge_bin_path,
clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables=None, clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables=None,
image="yandex/clickhouse-integration-test", image="yandex/clickhouse-integration-test",
stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None): stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None):
@ -749,15 +779,16 @@ class ClickHouseInstance:
self.hostname = hostname if hostname is not None else self.name self.hostname = hostname if hostname is not None else self.name
self.tmpfs = tmpfs or [] self.tmpfs = tmpfs or []
self.custom_config_dir = p.abspath(p.join(base_path, custom_config_dir)) if custom_config_dir else None self.base_config_dir = p.abspath(p.join(base_path, base_config_dir)) if base_config_dir else None
self.config_dir = p.abspath(p.join(base_path, config_dir)) if config_dir else None
self.custom_main_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_main_configs] self.custom_main_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_main_configs]
self.custom_user_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_user_configs] self.custom_user_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_user_configs]
self.custom_dictionaries_paths = [p.abspath(p.join(base_path, c)) for c in custom_dictionaries]
self.clickhouse_path_dir = p.abspath(p.join(base_path, clickhouse_path_dir)) if clickhouse_path_dir else None self.clickhouse_path_dir = p.abspath(p.join(base_path, clickhouse_path_dir)) if clickhouse_path_dir else None
self.macros = macros if macros is not None else {} self.macros = macros if macros is not None else {}
self.with_zookeeper = with_zookeeper self.with_zookeeper = with_zookeeper
self.zookeeper_config_path = zookeeper_config_path self.zookeeper_config_path = zookeeper_config_path
self.base_configs_dir = base_configs_dir
self.server_bin_path = server_bin_path self.server_bin_path = server_bin_path
self.odbc_bridge_bin_path = odbc_bridge_bin_path self.odbc_bridge_bin_path = odbc_bridge_bin_path
@ -773,7 +804,7 @@ class ClickHouseInstance:
self.docker_compose_path = p.join(self.path, 'docker_compose.yml') self.docker_compose_path = p.join(self.path, 'docker_compose.yml')
self.env_variables = env_variables or {} self.env_variables = env_variables or {}
if with_odbc_drivers: if with_odbc_drivers:
self.odbc_ini_path = os.path.dirname(self.docker_compose_path) + "/odbc.ini:/etc/odbc.ini" self.odbc_ini_path = self.path + "/odbc.ini:/etc/odbc.ini"
self.with_mysql = True self.with_mysql = True
else: else:
self.odbc_ini_path = "" self.odbc_ini_path = ""
@ -975,7 +1006,7 @@ class ClickHouseInstance:
time_left = deadline - current_time time_left = deadline - current_time
if deadline is not None and current_time >= deadline: if deadline is not None and current_time >= deadline:
raise Exception("Timed out while waiting for instance `{}' with ip address {} to start. " raise Exception("Timed out while waiting for instance `{}' with ip address {} to start. "
"Container status: {}".format(self.name, self.ip_address, status)) "Container status: {}, logs: {}".format(self.name, self.ip_address, status, handle.logs()))
# Repeatedly poll the instance address until there is something that listens there. # Repeatedly poll the instance address until there is something that listens there.
# Usually it means that ClickHouse is ready to accept queries. # Usually it means that ClickHouse is ready to accept queries.
@ -1057,40 +1088,50 @@ class ClickHouseInstance:
os.makedirs(self.path) os.makedirs(self.path)
configs_dir = p.abspath(p.join(self.path, 'configs')) instance_config_dir = p.abspath(p.join(self.path, 'configs'))
os.mkdir(configs_dir) os.makedirs(instance_config_dir)
shutil.copy(p.join(self.base_configs_dir, 'config.xml'), configs_dir) print "Copy common default production configuration from {}".format(self.base_config_dir)
shutil.copy(p.join(self.base_configs_dir, 'users.xml'), configs_dir) shutil.copyfile(p.join(self.base_config_dir, 'config.xml'), p.join(instance_config_dir, 'config.xml'))
shutil.copyfile(p.join(self.base_config_dir, 'users.xml'), p.join(instance_config_dir, 'users.xml'))
print "Create directory for configuration generated in this helper"
# used by all utils with any config # used by all utils with any config
conf_d_dir = p.abspath(p.join(configs_dir, 'conf.d')) conf_d_dir = p.abspath(p.join(instance_config_dir, 'conf.d'))
# used by server with main config.xml
self.config_d_dir = p.abspath(p.join(configs_dir, 'config.d'))
users_d_dir = p.abspath(p.join(configs_dir, 'users.d'))
os.mkdir(conf_d_dir) os.mkdir(conf_d_dir)
os.mkdir(self.config_d_dir)
os.mkdir(users_d_dir)
print "Create directory for common tests configuration"
# used by server with main config.xml
self.config_d_dir = p.abspath(p.join(instance_config_dir, 'config.d'))
os.mkdir(self.config_d_dir)
users_d_dir = p.abspath(p.join(instance_config_dir, 'users.d'))
os.mkdir(users_d_dir)
dictionaries_dir = p.abspath(p.join(instance_config_dir, 'dictionaries'))
os.mkdir(dictionaries_dir)
print "Copy common configuration from helpers"
# The file is named with 0_ prefix to be processed before other configuration overloads. # The file is named with 0_ prefix to be processed before other configuration overloads.
shutil.copy(p.join(HELPERS_DIR, '0_common_instance_config.xml'), self.config_d_dir) shutil.copy(p.join(HELPERS_DIR, '0_common_instance_config.xml'), self.config_d_dir)
shutil.copy(p.join(HELPERS_DIR, '0_common_instance_users.xml'), users_d_dir) shutil.copy(p.join(HELPERS_DIR, '0_common_instance_users.xml'), users_d_dir)
if len(self.custom_dictionaries_paths):
shutil.copy(p.join(HELPERS_DIR, '0_common_enable_dictionaries.xml'), self.config_d_dir)
# Generate and write macros file print "Generate and write macros file"
macros = self.macros.copy() macros = self.macros.copy()
macros['instance'] = self.name macros['instance'] = self.name
with open(p.join(self.config_d_dir, 'macros.xml'), 'w') as macros_config: with open(p.join(conf_d_dir, 'macros.xml'), 'w') as macros_config:
macros_config.write(self.dict_to_xml({"macros": macros})) macros_config.write(self.dict_to_xml({"macros": macros}))
# Put ZooKeeper config # Put ZooKeeper config
if self.with_zookeeper: if self.with_zookeeper:
shutil.copy(self.zookeeper_config_path, conf_d_dir) shutil.copy(self.zookeeper_config_path, conf_d_dir)
# Copy config dir # print "Copy config dir {} to {}".format(self.config_dir, instance_config_dir)
if self.custom_config_dir: # if self.config_dir:
distutils.dir_util.copy_tree(self.custom_config_dir, configs_dir) # distutils.dir_util.copy_tree(self.config_dir, instance_config_dir)
# Copy config.d configs # Copy config.d configs
print "Copy custom test config files {} to {}".format(self.custom_main_config_paths, self.config_d_dir)
for path in self.custom_main_config_paths: for path in self.custom_main_config_paths:
shutil.copy(path, self.config_d_dir) shutil.copy(path, self.config_d_dir)
@ -1098,12 +1139,21 @@ class ClickHouseInstance:
for path in self.custom_user_config_paths: for path in self.custom_user_config_paths:
shutil.copy(path, users_d_dir) shutil.copy(path, users_d_dir)
self.config_dir
# Copy dictionaries configs to configs/dictionaries
for path in self.custom_dictionaries_paths:
shutil.copy(path, dictionaries_dir)
db_dir = p.abspath(p.join(self.path, 'database')) db_dir = p.abspath(p.join(self.path, 'database'))
print "Setup database dir {}".format(db_dir)
os.mkdir(db_dir) os.mkdir(db_dir)
if self.clickhouse_path_dir is not None: if self.clickhouse_path_dir is not None:
print "Database files taken from {}".format(self.clickhouse_path_dir)
distutils.dir_util.copy_tree(self.clickhouse_path_dir, db_dir) distutils.dir_util.copy_tree(self.clickhouse_path_dir, db_dir)
logs_dir = p.abspath(p.join(self.path, 'logs')) logs_dir = p.abspath(p.join(self.path, 'logs'))
print "Setup logs dir {}".format(logs_dir)
os.mkdir(logs_dir) os.mkdir(logs_dir)
depends_on = [] depends_on = []
@ -1128,6 +1178,8 @@ class ClickHouseInstance:
env_file = _create_env_file(os.path.dirname(self.docker_compose_path), self.env_variables) env_file = _create_env_file(os.path.dirname(self.docker_compose_path), self.env_variables)
print "Env {} stored in {}".format(self.env_variables, env_file)
odbc_ini_path = "" odbc_ini_path = ""
if self.odbc_ini_path: if self.odbc_ini_path:
self._create_odbc_config_file() self._create_odbc_config_file()
@ -1138,6 +1190,8 @@ class ClickHouseInstance:
if self.stay_alive: if self.stay_alive:
entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND
print "Entrypoint cmd: {}".format(entrypoint_cmd)
networks = app_net = ipv4_address = ipv6_address = net_aliases = net_alias1 = "" networks = app_net = ipv4_address = ipv6_address = net_aliases = net_alias1 = ""
if self.ipv4_address is not None or self.ipv6_address is not None or self.hostname != self.name: if self.ipv4_address is not None or self.ipv6_address is not None or self.hostname != self.name:
networks = "networks:" networks = "networks:"
@ -1157,6 +1211,7 @@ class ClickHouseInstance:
binary_volume = "- " + self.server_bin_path + ":/usr/share/clickhouse_fresh" binary_volume = "- " + self.server_bin_path + ":/usr/share/clickhouse_fresh"
odbc_bridge_volume = "- " + self.odbc_bridge_bin_path + ":/usr/share/clickhouse-odbc-bridge_fresh" odbc_bridge_volume = "- " + self.odbc_bridge_bin_path + ":/usr/share/clickhouse-odbc-bridge_fresh"
with open(self.docker_compose_path, 'w') as docker_compose: with open(self.docker_compose_path, 'w') as docker_compose:
docker_compose.write(DOCKER_COMPOSE_TEMPLATE.format( docker_compose.write(DOCKER_COMPOSE_TEMPLATE.format(
image=self.image, image=self.image,
@ -1164,7 +1219,7 @@ class ClickHouseInstance:
hostname=self.hostname, hostname=self.hostname,
binary_volume=binary_volume, binary_volume=binary_volume,
odbc_bridge_volume=odbc_bridge_volume, odbc_bridge_volume=odbc_bridge_volume,
configs_dir=configs_dir, instance_config_dir=instance_config_dir,
config_d_dir=self.config_d_dir, config_d_dir=self.config_d_dir,
db_dir=db_dir, db_dir=db_dir,
tmpfs=str(self.tmpfs), tmpfs=str(self.tmpfs),

View File

@ -0,0 +1 @@
../../config/dict_examples/

View File

@ -11,6 +11,9 @@ class TSV:
raw_lines = contents.splitlines(True) raw_lines = contents.splitlines(True)
elif isinstance(contents, list): elif isinstance(contents, list):
raw_lines = ['\t'.join(map(str, l)) if isinstance(l, list) else str(l) for l in contents] raw_lines = ['\t'.join(map(str, l)) if isinstance(l, list) else str(l) for l in contents]
elif isinstance(contents, TSV):
self.lines = contents.lines
return
else: else:
raise TypeError("contents must be either file or string or list, actual type: " + type(contents).__name__) raise TypeError("contents must be either file or string or list, actual type: " + type(contents).__name__)
self.lines = [l.strip() for l in raw_lines if l.strip()] self.lines = [l.strip() for l in raw_lines if l.strip()]

View File

@ -53,7 +53,7 @@ def check_args_and_update_paths(args):
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.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]:
if not os.path.exists(path): if not os.path.exists(path):
raise Exception("Path {} doesn't exist".format(path)) raise Exception("Path {} doesn't exist".format(path))

View File

@ -4,9 +4,9 @@ from helpers.cluster import ClickHouseCluster
from helpers.client import QueryRuntimeException from helpers.client import QueryRuntimeException
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
ch1 = cluster.add_instance('ch1', config_dir="configs", with_zookeeper=True) ch1 = cluster.add_instance('ch1', main_configs=["configs/config.d/clusters.xml"], with_zookeeper=True)
ch2 = cluster.add_instance('ch2', config_dir="configs", with_zookeeper=True) ch2 = cluster.add_instance('ch2', main_configs=["configs/config.d/clusters.xml"], with_zookeeper=True)
ch3 = cluster.add_instance('ch3', config_dir="configs", with_zookeeper=True) ch3 = cluster.add_instance('ch3', main_configs=["configs/config.d/clusters.xml"], with_zookeeper=True)
@pytest.fixture(scope="module", autouse=True) @pytest.fixture(scope="module", autouse=True)
def started_cluster(): def started_cluster():

View File

@ -9,23 +9,23 @@ from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True)
node2 = cluster.add_instance('node2', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True)
node3 = cluster.add_instance('node3', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', with_installed_binary=True) node3 = cluster.add_instance('node3', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', with_installed_binary=True)
node4 = cluster.add_instance('node4', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) node4 = cluster.add_instance('node4', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True)
node5 = cluster.add_instance('node5', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', with_installed_binary=True) node5 = cluster.add_instance('node5', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', with_installed_binary=True)
node6 = cluster.add_instance('node6', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) node6 = cluster.add_instance('node6', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True)
node7 = cluster.add_instance('node7', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', stay_alive=True, with_installed_binary=True) node7 = cluster.add_instance('node7', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', stay_alive=True, with_installed_binary=True)
node8 = cluster.add_instance('node8', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) node8 = cluster.add_instance('node8', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True)
node9 = cluster.add_instance('node9', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/merge_tree_settings.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) node9 = cluster.add_instance('node9', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/merge_tree_settings.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True)
node10 = cluster.add_instance('node10', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/merge_tree_settings.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', stay_alive=True, with_installed_binary=True) node10 = cluster.add_instance('node10', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml', 'configs/merge_tree_settings.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.6.3.18', stay_alive=True, with_installed_binary=True)
node11 = cluster.add_instance('node11', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) node11 = cluster.add_instance('node11', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True)
node12 = cluster.add_instance('node12', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True) node12 = cluster.add_instance('node12', main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True, image='yandex/clickhouse-server:19.1.15', stay_alive=True, with_installed_binary=True)
def prepare_single_pair_with_setting(first_node, second_node, group): def prepare_single_pair_with_setting(first_node, second_node, group):

View File

@ -4,7 +4,7 @@ from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
server = cluster.add_instance('server', config_dir="configs") server = cluster.add_instance('server', user_configs=["configs/users.d/network.xml"])
clientA1 = cluster.add_instance('clientA1', hostname = 'clientA1.com') clientA1 = cluster.add_instance('clientA1', hostname = 'clientA1.com')
clientA2 = cluster.add_instance('clientA2', hostname = 'clientA2.com') clientA2 = cluster.add_instance('clientA2', hostname = 'clientA2.com')
@ -20,7 +20,12 @@ clientD2 = cluster.add_instance('clientD2', hostname = 'xxx.clientD0002.ru')
clientD3 = cluster.add_instance('clientD3', hostname = 'clientD0003.ru') clientD3 = cluster.add_instance('clientD3', hostname = 'clientD0003.ru')
def check_clickhouse_is_ok(client_node, server_node):
assert client_node.exec_in_container(["bash", "-c", "/usr/bin/curl -s {}:8123 ".format(server_node.hostname)]) == "Ok.\n"
def query_from_one_node_to_another(client_node, server_node, query): def query_from_one_node_to_another(client_node, server_node, query):
check_clickhouse_is_ok(client_node, server_node)
return client_node.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host {} --query {!r}".format(server_node.hostname, query)]) return client_node.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host {} --query {!r}".format(server_node.hostname, query)])
@ -56,5 +61,6 @@ def test_allowed_host():
for client_node in expected_to_fail: for client_node in expected_to_fail:
with pytest.raises(Exception) as e: with pytest.raises(Exception) as e:
query_from_one_node_to_another(client_node, server, "SELECT * FROM test_table") result = query_from_one_node_to_another(client_node, server, "SELECT * FROM test_table")
print("Client node: {} Server node: {} Result: {}".format(client_node, server_node, result))
assert "default: Authentication failed" in str(e) assert "default: Authentication failed" in str(e)

View File

@ -49,18 +49,18 @@ def test_config_without_allowed_hosts(start_cluster):
assert node4.query("CREATE TABLE table_test_4_4 (word String) Engine=URL('ftp://something.com', S3)") == "" assert node4.query("CREATE TABLE table_test_4_4 (word String) Engine=URL('ftp://something.com', S3)") == ""
def test_table_function_remote(start_cluster): def test_table_function_remote(start_cluster):
assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-{1|2}', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1})
assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-1,example01-02-1', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1})
assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remote('example01-0{1,2}-1', system, events", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1})
assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remote('example01-0{1,2}-{1|2}', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1})
assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-{01..02}-{1|2}', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1})
assert "not allowed" in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-1,example01-03-1', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1})
assert "not allowed" in node6.query_and_get_error("SELECT * FROM remote('example01-01-{1|3}', system, events)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1})
assert "not allowed" in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-0{1,3}-1', system, metrics)", settings={"connections_with_failover_max_tries":1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout":1})
assert node6.query("SELECT * FROM remote('localhost', system, events)") != "" assert node6.query("SELECT * FROM remote('localhost', system, events)") != ""
assert node6.query("SELECT * FROM remoteSecure('localhost', system, metrics)") != "" assert node6.query("SELECT * FROM remoteSecure('localhost', system, metrics)") != ""
assert "URL \"localhost:800\" is not allowed in config.xml" in node6.query_and_get_error("SELECT * FROM remoteSecure('localhost:800', system, events)") assert "URL \"localhost:800\" is not allowed in config.xml" in node6.query_and_get_error("SELECT * FROM remoteSecure('localhost:800', system, events)")
assert "URL \"localhost:800\" is not allowed in config.xml" in node6.query_and_get_error("SELECT * FROM remote('localhost:800', system, metrics)") assert "URL \"localhost:800\" is not allowed in config.xml" in node6.query_and_get_error("SELECT * FROM remote('localhost:800', system, metrics)")
assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-1,example01-02-1', system, events)")
assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remote('example01-0{1,2}-1', system, events")
assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-{1|2}', system, events)")
assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remote('example01-0{1,2}-{1|2}', system, events)")
assert "not allowed in config.xml" not in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-{01..02}-{1|2}', system, events)")
assert "not allowed" in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-01-1,example01-03-1', system, events)")
assert "not allowed" in node6.query_and_get_error("SELECT * FROM remote('example01-01-{1|3}', system, events)")
assert "not allowed" in node6.query_and_get_error("SELECT * FROM remoteSecure('example01-0{1,3}-1', system, metrics)")
def test_redirect(start_cluster): def test_redirect(start_cluster):
hdfs_api = HDFSApi("root") hdfs_api = HDFSApi("root")

View File

@ -6,7 +6,7 @@ from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', config_dir="configs", with_zookeeper=True) node1 = cluster.add_instance('node1', main_configs=["configs/config.d/zookeeper_session_timeout.xml", "configs/remote_servers.xml"], with_zookeeper=True)
@pytest.fixture(scope="module") @pytest.fixture(scope="module")

View File

@ -1,6 +1,6 @@
<?xml version="1.0"?>
<yandex> <yandex>
<remote_servers> <remote_servers>
<cluster0> <cluster0>
<shard> <shard>
<internal_replication>true</internal_replication> <internal_replication>true</internal_replication>
@ -21,7 +21,6 @@
</replica> </replica>
</shard> </shard>
</cluster0> </cluster0>
<cluster1> <cluster1>
<shard> <shard>
<internal_replication>true</internal_replication> <internal_replication>true</internal_replication>
@ -42,7 +41,6 @@
</replica> </replica>
</shard> </shard>
</cluster1> </cluster1>
<shard_0_0> <shard_0_0>
<shard> <shard>
<internal_replication>true</internal_replication> <internal_replication>true</internal_replication>
@ -56,7 +54,6 @@
</replica> </replica>
</shard> </shard>
</shard_0_0> </shard_0_0>
<source_trivial_cluster> <source_trivial_cluster>
<shard> <shard>
<replica> <replica>
@ -65,8 +62,6 @@
</replica> </replica>
</shard> </shard>
</source_trivial_cluster> </source_trivial_cluster>
<destination_trivial_cluster> <destination_trivial_cluster>
<shard> <shard>
<replica> <replica>
@ -75,6 +70,5 @@
</replica> </replica>
</shard> </shard>
</destination_trivial_cluster> </destination_trivial_cluster>
</remote_servers> </remote_servers>
</yandex> </yandex>

View File

@ -54,7 +54,8 @@ def started_cluster():
for replica_name in replicas: for replica_name in replicas:
name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name) name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name)
cluster.add_instance(name, cluster.add_instance(name,
config_dir="configs", main_configs=["configs/conf.d/query_log.xml", "configs/conf.d/ddl.xml", "configs/conf.d/clusters.xml"],
user_configs=["configs/users.xml"],
macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name}, macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name},
with_zookeeper=True) with_zookeeper=True)
@ -226,6 +227,7 @@ def execute_task(task, cmd_options):
zk.ensure_path(zk_task_path) zk.ensure_path(zk_task_path)
zk.create(zk_task_path + "/description", task.copier_task_config) zk.create(zk_task_path + "/description", task.copier_task_config)
# Run cluster-copier processes on each node # Run cluster-copier processes on each node
docker_api = docker.from_env().api docker_api = docker.from_env().api
copiers_exec_ids = [] copiers_exec_ids = []
@ -241,9 +243,11 @@ def execute_task(task, cmd_options):
for instance_name in copiers: for instance_name in copiers:
instance = cluster.instances[instance_name] instance = cluster.instances[instance_name]
container = instance.get_docker_handle() container = instance.get_docker_handle()
instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs/config-copier.xml"), "/etc/clickhouse-server/config-copier.xml")
print "Copied copier config to {}".format(instance.name)
exec_id = docker_api.exec_create(container.id, cmd, stderr=True) exec_id = docker_api.exec_create(container.id, cmd, stderr=True)
docker_api.exec_start(exec_id, detach=True) output = docker_api.exec_start(exec_id).decode('utf8')
print(output)
copiers_exec_ids.append(exec_id) copiers_exec_ids.append(exec_id)
print "Copier for {} ({}) has started".format(instance.name, instance.ip_address) print "Copier for {} ({}) has started".format(instance.name, instance.ip_address)

View File

@ -34,7 +34,7 @@ def started_cluster():
for replica_name in replicas: for replica_name in replicas:
name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name) name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name)
cluster.add_instance(name, cluster.add_instance(name,
config_dir="configs", main_configs=[], user_configs=[],
macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name}, macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name},
with_zookeeper=True) with_zookeeper=True)

View File

@ -4,10 +4,9 @@ import pytest
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__))
config_dir = os.path.join(SCRIPT_DIR, './configs')
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', config_dir = config_dir) node = cluster.add_instance('node', main_configs=["configs/config.d/bad.xml"])
caught_exception = "" caught_exception = ""
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
@ -19,4 +18,5 @@ def start_cluster():
caught_exception = str(e) caught_exception = str(e)
def test_work(start_cluster): def test_work(start_cluster):
print(caught_exception)
assert caught_exception.find("Root element doesn't have the corresponding root element as the config file.") != -1 assert caught_exception.find("Root element doesn't have the corresponding root element as the config file.") != -1

View File

@ -2,7 +2,7 @@ import pytest
from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', config_dir='configs') node = cluster.add_instance('node', main_configs=["configs/config.d/text_log.xml"], user_configs=["configs/users.d/custom_settings.xml"])
@pytest.fixture(scope="module", autouse=True) @pytest.fixture(scope="module", autouse=True)

View File

@ -0,0 +1,12 @@
<?xml version="1.0"?>
<yandex>
<openSSL>
<client>
<cacheSessions>true</cacheSessions>
<verificationMode>none</verificationMode>
<invalidCertificateHandler>
<name>AcceptCertificateHandler</name>
</invalidCertificateHandler>
</client>
</openSSL>
</yandex>

View File

@ -0,0 +1,4 @@
<?xml version="1.0"?>
<yandex>
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
</yandex>

View File

@ -181,12 +181,18 @@ def setup_module(module):
if not (field.is_key or field.is_range or field.is_range_key): if not (field.is_key or field.is_range or field.is_range_key):
DICTIONARIES_KV.append(get_dict(source, layout, field_keys + [field], field.name)) DICTIONARIES_KV.append(get_dict(source, layout, field_keys + [field], field.name))
cluster = ClickHouseCluster(__file__)
main_configs = [] main_configs = []
main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml'))
cluster.add_instance('clickhouse1', main_configs=main_configs)
dictionaries = []
for fname in os.listdir(dict_configs_path): for fname in os.listdir(dict_configs_path):
main_configs.append(os.path.join(dict_configs_path, fname)) dictionaries.append(os.path.join(dict_configs_path, fname))
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
node = cluster.add_instance('node', main_configs=main_configs, with_mysql=True, with_mongo=True, with_redis=True, with_cassandra=True) node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mysql=True, with_mongo=True, with_redis=True, with_cassandra=True)
cluster.add_instance('clickhouse1')
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
@ -238,8 +244,8 @@ def remove_mysql_dicts():
TODO remove this when open ssl will be fixed or thread sanitizer will be suppressed TODO remove this when open ssl will be fixed or thread sanitizer will be suppressed
""" """
global DICTIONARIES #global DICTIONARIES
DICTIONARIES = [d for d in DICTIONARIES if not d.name.startswith("MySQL")] #DICTIONARIES = [d for d in DICTIONARIES if not d.name.startswith("MySQL")]
@pytest.mark.parametrize("fold", list(range(10))) @pytest.mark.parametrize("fold", list(range(10)))

View File

@ -0,0 +1,5 @@
<?xml version="1.0"?>
<yandex>
<dictionaries_config>/etc/clickhouse-server/config.d/complex_key_cache_string.xml</dictionaries_config>
<dictionaries_config>/etc/clickhouse-server/config.d/ssd_complex_key_cache_string.xml</dictionaries_config>
</yandex>

View File

@ -5,13 +5,12 @@ from helpers.cluster import ClickHouseCluster
@pytest.fixture(scope="function") @pytest.fixture(scope="function")
def cluster(request): def cluster(request):
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) cluster = ClickHouseCluster(__file__)
try: try:
if request.param == "memory": if request.param == "memory":
node = cluster.add_instance('node', main_configs=['configs/dictionaries/complex_key_cache_string.xml']) node = cluster.add_instance('node', main_configs=['configs/enable_dictionaries.xml', 'configs/dictionaries/complex_key_cache_string.xml'])
if request.param == "ssd": if request.param == "ssd":
node = cluster.add_instance('node', main_configs=['configs/dictionaries/ssd_complex_key_cache_string.xml']) node = cluster.add_instance('node', main_configs=['configs/enable_dictionaries.xml', 'configs/dictionaries/ssd_complex_key_cache_string.xml'])
cluster.start() cluster.start()
node.query("create table radars_table (radar_id String, radar_ip String, client_id String) engine=MergeTree() order by radar_id") node.query("create table radars_table (radar_id String, radar_ip String, client_id String) engine=MergeTree() order by radar_id")

View File

@ -0,0 +1,5 @@
<yandex>
<remote_url_allow_hosts>
<host>node1</host>
</remote_url_allow_hosts>
</yandex>

View File

@ -0,0 +1,41 @@
<yandex>
<dictionary>
<name>test.conflicting_dictionary</name>
<source>
<clickhouse>
<host>localhost</host>
<port>9000</port>
<user>default</user>
<password></password>
<db>test</db>
<table>xml_dictionary_table</table>
</clickhouse>
</source>
<lifetime>
<min>0</min>
<max>0</max>
</lifetime>
<layout>
<cache><size_in_cells>128</size_in_cells></cache>
</layout>
<structure>
<id>
<name>id</name>
</id>
<attribute>
<name>SomeValue1</name>
<type>UInt8</type>
<null_value>1</null_value>
</attribute>
<attribute>
<name>SomeValue2</name>
<type>String</type>
<null_value>''</null_value>
</attribute>
</structure>
</dictionary>
</yandex>

View File

@ -0,0 +1,4 @@
<yandex>
<dictionaries_lazy_load>false</dictionaries_lazy_load>
</yandex>

View File

@ -0,0 +1,3 @@
<yandex>
<dictionaries_config>/etc/clickhouse-server/config.d/*dictionary.xml</dictionaries_config>
</yandex>

View File

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

View File

@ -7,10 +7,10 @@ import warnings
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', with_mysql=True, main_configs=['configs/dictionaries/simple_dictionary.xml']) node1 = cluster.add_instance('node1', with_mysql=True, main_configs=['configs/enable_dictionaries.xml','configs/dictionaries/simple_dictionary.xml'], user_configs=['configs/user_admin.xml'])
node2 = cluster.add_instance('node2', with_mysql=True, main_configs=['configs/dictionaries/simple_dictionary.xml', 'configs/dictionaries/lazy_load.xml']) node2 = cluster.add_instance('node2', with_mysql=True, main_configs=['configs/allow_remote_node.xml','configs/enable_dictionaries.xml','configs/dictionaries/simple_dictionary.xml', 'configs/dictionaries/lazy_load_dictionary.xml'], user_configs=['configs/user_admin.xml'])
node3 = cluster.add_instance('node3', main_configs=['configs/dictionaries/dictionary_with_conflict_name.xml']) node3 = cluster.add_instance('node3', main_configs=['configs/allow_remote_node.xml','configs/enable_dictionaries.xml','configs/dictionaries/conflict_name_dictionary.xml'], user_configs=['configs/user_admin.xml'])
def create_mysql_conn(user, password, hostname, port): def create_mysql_conn(user, password, hostname, port):
@ -49,7 +49,7 @@ def started_cluster():
(node2, 'complex_node2_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())'), (node2, 'complex_node2_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())'),
(node2, 'complex_node2_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'), (node2, 'complex_node2_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'),
]) ])
def test_crete_and_select_mysql(started_cluster, clickhouse, name, layout): def test_create_and_select_mysql(started_cluster, clickhouse, name, layout):
mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", 3308) mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", 3308)
execute_mysql_query(mysql_conn, "CREATE DATABASE IF NOT EXISTS clickhouse") execute_mysql_query(mysql_conn, "CREATE DATABASE IF NOT EXISTS clickhouse")
execute_mysql_query(mysql_conn, "CREATE TABLE clickhouse.{} (key_field1 int, key_field2 bigint, value1 text, value2 float, PRIMARY KEY (key_field1, key_field2))".format(name)) execute_mysql_query(mysql_conn, "CREATE TABLE clickhouse.{} (key_field1 int, key_field2 bigint, value1 text, value2 float, PRIMARY KEY (key_field1, key_field2))".format(name))
@ -93,8 +93,8 @@ def test_crete_and_select_mysql(started_cluster, clickhouse, name, layout):
for i in range(172, 200): for i in range(172, 200):
assert clickhouse.query("SELECT dictGetString('default.{}', 'value1', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)) == str(i) * 3 + '\n' assert clickhouse.query("SELECT dictGetString('default.{}', 'value1', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)) == str(i) * 3 + '\n'
stroka = clickhouse.query("SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)).strip() string = clickhouse.query("SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)).strip()
value = float(stroka) value = float(string)
assert int(value) == int(i * 2.718) assert int(value) == int(i * 2.718)
clickhouse.query("select dictGetUInt8('xml_dictionary', 'SomeValue1', toUInt64(17))") == "17\n" clickhouse.query("select dictGetUInt8('xml_dictionary', 'SomeValue1', toUInt64(17))") == "17\n"

View File

@ -0,0 +1,4 @@
<?xml version="1.0"?>
<yandex>
<dictionaries_config>/etc/clickhouse-server/config.d/dep_*.xml</dictionaries_config>
</yandex>

View File

@ -3,11 +3,11 @@ import os
from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry from helpers.test_tools import assert_eq_with_retry
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml']
DICTIONARY_FILES = ['configs/dictionaries/dep_x.xml', 'configs/dictionaries/dep_y.xml', 'configs/dictionaries/dep_z.xml'] DICTIONARY_FILES = ['configs/dictionaries/dep_x.xml', 'configs/dictionaries/dep_y.xml', 'configs/dictionaries/dep_z.xml']
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES) instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES,)
@pytest.fixture(scope="module") @pytest.fixture(scope="module")

View File

@ -0,0 +1,4 @@
<?xml version="1.0"?>
<yandex>
<dictionaries_config>/etc/clickhouse-server/config.d/mysql_dict*.xml</dictionaries_config>
</yandex>

View File

@ -8,10 +8,9 @@ import pymysql.cursors
from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry from helpers.test_tools import assert_eq_with_retry
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
CONFIG_FILES = ['configs/dictionaries/mysql_dict1.xml', 'configs/dictionaries/mysql_dict2.xml', 'configs/remote_servers.xml'] CONFIG_FILES = ['configs/dictionaries/mysql_dict1.xml', 'configs/dictionaries/mysql_dict2.xml', 'configs/remote_servers.xml']
CONFIG_FILES += ['configs/enable_dictionaries.xml']
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance', main_configs=CONFIG_FILES, with_mysql = True) instance = cluster.add_instance('instance', main_configs=CONFIG_FILES, with_mysql = True)
create_table_mysql_template = """ create_table_mysql_template = """

View File

@ -0,0 +1,4 @@
<?xml version="1.0"?>
<yandex>
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
</yandex>

View File

@ -3,11 +3,11 @@ import os
from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV, assert_eq_with_retry from helpers.test_tools import TSV, assert_eq_with_retry
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml']
DICTIONARY_FILES = ['configs/dictionaries/cache.xml'] DICTIONARY_FILES = ['configs/dictionaries/cache.xml']
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES) instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES)
@pytest.fixture(scope="module") @pytest.fixture(scope="module")

View File

@ -0,0 +1,4 @@
<?xml version="1.0"?>
<yandex>
<dictionaries_config>/etc/clickhouse-server/config.d/dictionary*.xml</dictionaries_config>
</yandex>

View File

@ -19,12 +19,12 @@ def setup_module(module):
structure = generate_structure() structure = generate_structure()
dictionary_files = generate_dictionaries(os.path.join(SCRIPT_DIR, 'configs/dictionaries'), structure) dictionary_files = generate_dictionaries(os.path.join(SCRIPT_DIR, 'configs/dictionaries'), structure)
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance', main_configs=dictionary_files) instance = cluster.add_instance('instance', main_configs=dictionary_files+['configs/enable_dictionaries.xml'])
test_table = DictionaryTestTable(os.path.join(SCRIPT_DIR, 'configs/dictionaries/source.tsv')) test_table = DictionaryTestTable(os.path.join(SCRIPT_DIR, 'configs/dictionaries/source.tsv'))
@pytest.fixture(scope="module") @pytest.fixture(scope="module", autouse=True)
def started_cluster(): def started_cluster():
try: try:
cluster.start() cluster.start()

View File

@ -0,0 +1,4 @@
<?xml version="1.0"?>
<yandex>
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
</yandex>

View File

@ -6,10 +6,11 @@ from helpers.client import QueryTimeoutExceedException
from helpers.test_tools import assert_eq_with_retry from helpers.test_tools import assert_eq_with_retry
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml']
DICTIONARY_FILES = ['configs/dictionaries/cache_xypairs.xml', 'configs/dictionaries/executable.xml', 'configs/dictionaries/file.xml', 'configs/dictionaries/file.txt', 'configs/dictionaries/slow.xml'] DICTIONARY_FILES = ['configs/dictionaries/cache_xypairs.xml', 'configs/dictionaries/executable.xml', 'configs/dictionaries/file.xml', 'configs/dictionaries/file.txt', 'configs/dictionaries/slow.xml']
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES) instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES)
@pytest.fixture(scope="module") @pytest.fixture(scope="module")

View File

@ -0,0 +1,4 @@
<?xml version="1.0"?>
<yandex>
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
</yandex>

View File

@ -8,11 +8,11 @@ from helpers.cluster import ClickHouseCluster
from helpers.cluster import ClickHouseKiller from helpers.cluster import ClickHouseKiller
from helpers.network import PartitionManager from helpers.network import PartitionManager
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) cluster = ClickHouseCluster(__file__)
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True)
main_node = cluster.add_instance('main_node', main_configs=['configs/dictionaries/cache_ints_dictionary.xml']) main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml',
'configs/dictionaries/cache_ints_dictionary.xml'])
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
def started_cluster(): def started_cluster():

View File

@ -9,10 +9,10 @@ from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV from helpers.test_tools import TSV
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) cluster = ClickHouseCluster(__file__)
dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True)
main_node = cluster.add_instance('main_node', main_configs=['configs/dictionaries/cache_strings_default_settings.xml']) main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml','configs/dictionaries/cache_ints_dictionary.xml','configs/dictionaries/cache_strings_default_settings.xml'])
def get_random_string(string_length=8): def get_random_string(string_length=8):

View File

@ -9,11 +9,10 @@ from helpers.cluster import ClickHouseKiller
from helpers.network import PartitionManager from helpers.network import PartitionManager
from helpers.network import PartitionManagerDisabler from helpers.network import PartitionManagerDisabler
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) cluster = ClickHouseCluster(__file__)
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True)
main_node = cluster.add_instance('main_node', main_configs=['configs/dictionaries/cache_ints_dictionary.xml']) main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', 'configs/dictionaries/cache_ints_dictionary.xml'])
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
def started_cluster(): def started_cluster():

View File

@ -8,11 +8,10 @@ from helpers.cluster import ClickHouseCluster
from helpers.cluster import ClickHouseKiller from helpers.cluster import ClickHouseKiller
from helpers.network import PartitionManager from helpers.network import PartitionManager
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) cluster = ClickHouseCluster(__file__)
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True)
main_node = cluster.add_instance('main_node', main_configs=['configs/dictionaries/cache_ints_dictionary.xml']) main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml','configs/dictionaries/cache_ints_dictionary.xml'])
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
def started_cluster(): def started_cluster():

View File

@ -0,0 +1,4 @@
<?xml version="1.0"?>
<yandex>
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
</yandex>

View File

@ -3,8 +3,7 @@ import pytest
from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseCluster
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml']
config_dir = os.path.join(SCRIPT_DIR, './configs')
DICTIONARY_FILES = [ DICTIONARY_FILES = [
'configs/dictionaries/FileSourceConfig.xml', 'configs/dictionaries/FileSourceConfig.xml',
'configs/dictionaries/ExecutableSourceConfig.xml', 'configs/dictionaries/ExecutableSourceConfig.xml',
@ -13,8 +12,8 @@ DICTIONARY_FILES = [
'configs/dictionaries/ClickHouseSourceConfig.xml' 'configs/dictionaries/ClickHouseSourceConfig.xml'
] ]
cluster = ClickHouseCluster(__file__, base_configs_dir=config_dir) cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('node', main_configs=DICTIONARY_FILES, config_dir=config_dir) instance = cluster.add_instance('node', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES)
def prepare(): def prepare():
node = instance node = instance

View File

@ -4,10 +4,10 @@ from helpers.cluster import ClickHouseCluster
from helpers.client import QueryRuntimeException from helpers.client import QueryRuntimeException
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
ch1 = cluster.add_instance('ch1', config_dir="configs", with_zookeeper=True) ch1 = cluster.add_instance('ch1', main_configs=["configs/config.d/clusters.xml", "configs/config.d/ddl.xml"], with_zookeeper=True)
ch2 = cluster.add_instance('ch2', config_dir="configs", with_zookeeper=True) ch2 = cluster.add_instance('ch2', main_configs=["configs/config.d/clusters.xml", "configs/config.d/ddl.xml"], with_zookeeper=True)
ch3 = cluster.add_instance('ch3', config_dir="configs", with_zookeeper=True) ch3 = cluster.add_instance('ch3', main_configs=["configs/config.d/clusters.xml", "configs/config.d/ddl.xml"], with_zookeeper=True)
ch4 = cluster.add_instance('ch4', config_dir="configs", with_zookeeper=True) ch4 = cluster.add_instance('ch4', main_configs=["configs/config.d/clusters.xml", "configs/config.d/ddl.xml"], with_zookeeper=True)
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
def started_cluster(): def started_cluster():

View File

@ -0,0 +1,16 @@
<?xml version="1.0"?>
<yandex>
<storage_configuration>
<disks>
<disk_s3>
<type>s3</type>
<endpoint>http://minio1:9001/root/data/</endpoint>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</disk_s3>
<disk_memory>
<type>memory</type>
</disk_memory>
</disks>
</storage_configuration>
</yandex>

View File

@ -1,3 +1,4 @@
import pytest import pytest
from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseCluster
@ -12,7 +13,7 @@ disk_types = {
def cluster(): def cluster():
try: try:
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
cluster.add_instance("node", config_dir="configs", with_minio=True) cluster.add_instance("node", main_configs=["configs/storage.xml"], with_minio=True)
cluster.start() cluster.start()
yield cluster yield cluster
finally: finally:

View File

@ -17,10 +17,23 @@ class ClickHouseClusterWithDDLHelpers(ClickHouseCluster):
def prepare(self, replace_hostnames_with_ips=True): def prepare(self, replace_hostnames_with_ips=True):
try: try:
main_configs = [os.path.join(self.test_config_dir, "config.d/clusters.xml"),
os.path.join(self.test_config_dir, "config.d/zookeeper_session_timeout.xml"),
os.path.join(self.test_config_dir, "config.d/macro.xml"),
os.path.join(self.test_config_dir, "config.d/query_log.xml"),
os.path.join(self.test_config_dir, "config.d/ddl.xml")]
user_configs = [os.path.join(self.test_config_dir, "users.d/restricted_user.xml"),
os.path.join(self.test_config_dir, "users.d/query_log.xml")]
if self.test_config_dir == "configs_secure":
main_configs += [os.path.join(self.test_config_dir, "server.crt"),
os.path.join(self.test_config_dir, "server.key"),
os.path.join(self.test_config_dir, "dhparam.pem"),
os.path.join(self.test_config_dir, "config.d/ssl_conf.xml")]
for i in xrange(4): for i in xrange(4):
self.add_instance( self.add_instance(
'ch{}'.format(i+1), 'ch{}'.format(i+1),
config_dir=self.test_config_dir, main_configs=main_configs,
user_configs=user_configs,
macros={"layer": 0, "shard": i/2 + 1, "replica": i%2 + 1}, macros={"layer": 0, "shard": i/2 + 1, "replica": i%2 + 1},
with_zookeeper=True) with_zookeeper=True)

View File

@ -1,8 +1,9 @@
<yandex> <yandex>
<openSSL> <openSSL>
<server> <server>
<certificateFile>/etc/clickhouse-server/server.crt</certificateFile> <certificateFile>/etc/clickhouse-server/config.d/server.crt</certificateFile>
<privateKeyFile>/etc/clickhouse-server/server.key</privateKeyFile> <privateKeyFile>/etc/clickhouse-server/config.d/server.key</privateKeyFile>
<dhParamsFile>/etc/clickhouse-server/config.d/dhparam.pem</dhParamsFile>
<verificationMode>none</verificationMode> <verificationMode>none</verificationMode>
<cacheSessions>true</cacheSessions> <cacheSessions>true</cacheSessions>
</server> </server>

View File

@ -6,12 +6,12 @@ from helpers.test_tools import assert_eq_with_retry
from helpers.client import QueryRuntimeException from helpers.client import QueryRuntimeException
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', config_dir="configs", with_zookeeper=True) node1 = cluster.add_instance('node1', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True)
node2 = cluster.add_instance('node2', config_dir="configs", with_zookeeper=True) node2 = cluster.add_instance('node2', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True)
node3 = cluster.add_instance('node3', config_dir="configs", with_zookeeper=True) node3 = cluster.add_instance('node3', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True)
node4 = cluster.add_instance('node4', config_dir="configs", with_zookeeper=True) node4 = cluster.add_instance('node4', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True)
node5 = cluster.add_instance('node5', config_dir="configs", with_zookeeper=True) node5 = cluster.add_instance('node5', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True)
node6 = cluster.add_instance('node6', config_dir="configs", with_zookeeper=True) node6 = cluster.add_instance('node6', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True)
@pytest.fixture(scope="module") @pytest.fixture(scope="module")

View File

@ -9,7 +9,7 @@ from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', config_dir="configs", main_configs=['configs/remote_servers.xml']) node = cluster.add_instance('node', main_configs=['configs/remote_servers.xml'])
cluster_param = pytest.mark.parametrize("cluster", [ cluster_param = pytest.mark.parametrize("cluster", [
('test_cluster'), ('test_cluster'),

View File

@ -1,8 +1,9 @@
<yandex> <yandex>
<openSSL> <openSSL>
<server> <server>
<certificateFile>/etc/clickhouse-server/server.crt</certificateFile> <certificateFile>/etc/clickhouse-server/config.d/server.crt</certificateFile>
<privateKeyFile>/etc/clickhouse-server/server.key</privateKeyFile> <privateKeyFile>/etc/clickhouse-server/config.d/server.key</privateKeyFile>
<dhParamsFile>/etc/clickhouse-server/config.d/dhparam.pem</dhParamsFile>
<verificationMode>none</verificationMode> <verificationMode>none</verificationMode>
<cacheSessions>true</cacheSessions> <cacheSessions>true</cacheSessions>
</server> </server>

View File

@ -1,6 +1,6 @@
import itertools import itertools
import timeit import timeit
import os.path
import pytest import pytest
from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseCluster
@ -91,8 +91,16 @@ def started_cluster(request):
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
cluster.__with_ssl_config = request.param == "configs_secure" cluster.__with_ssl_config = request.param == "configs_secure"
main_configs = []
main_configs += [os.path.join(request.param, "config.d/remote_servers.xml")]
if cluster.__with_ssl_config:
main_configs += [os.path.join(request.param, "server.crt")]
main_configs += [os.path.join(request.param, "server.key")]
main_configs += [os.path.join(request.param, "dhparam.pem")]
main_configs += [os.path.join(request.param, "config.d/ssl_conf.xml")]
user_configs = [os.path.join(request.param, "users.d/set_distributed_defaults.xml")]
for name in NODES: for name in NODES:
NODES[name] = cluster.add_instance(name, config_dir=request.param) NODES[name] = cluster.add_instance(name, main_configs=main_configs, user_configs=user_configs)
try: try:
cluster.start() cluster.start()

View File

@ -9,7 +9,7 @@ from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', node = cluster.add_instance('node',
config_dir='configs', main_configs=["configs/config.d/storage_configuration.xml"],
tmpfs=['/disk1:size=100M', '/disk2:size=100M']) tmpfs=['/disk1:size=100M', '/disk2:size=100M'])
@pytest.fixture(scope='module') @pytest.fixture(scope='module')

View File

@ -2,7 +2,7 @@ import pytest
from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance', config_dir="configs") instance = cluster.add_instance('instance', user_configs=["configs/users.d/extra_users.xml"])
@pytest.fixture(scope="module", autouse=True) @pytest.fixture(scope="module", autouse=True)
def started_cluster(): def started_cluster():

View File

@ -12,8 +12,8 @@ from helpers.client import QueryTimeoutExceedException
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', config_dir='configs', with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 1}) node1 = cluster.add_instance('node1', main_configs=["configs/conf.d/merge_tree.xml", "configs/conf.d/remote_servers.xml"], with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 1})
node2 = cluster.add_instance('node2', config_dir='configs', with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 2}) node2 = cluster.add_instance('node2', main_configs=["configs/conf.d/merge_tree.xml", "configs/conf.d/remote_servers.xml"], with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 2})
nodes = [node1, node2] nodes = [node1, node2]
@pytest.fixture(scope="module") @pytest.fixture(scope="module")

View File

@ -0,0 +1,8 @@
-----BEGIN DH PARAMETERS-----
MIIBCAKCAQEAua92DDli13gJ+//ZXyGaggjIuidqB0crXfhUlsrBk9BV1hH3i7fR
XGP9rUdk2ubnB3k2ejBStL5oBrkHm9SzUFSQHqfDjLZjKoUpOEmuDc4cHvX1XTR5
Pr1vf5cd0yEncJWG5W4zyUB8k++SUdL2qaeslSs+f491HBLDYn/h8zCgRbBvxhxb
9qeho1xcbnWeqkN6Kc9bgGozA16P9NLuuLttNnOblkH+lMBf42BSne/TWt3AlGZf
slKmmZcySUhF8aKfJnLKbkBCFqOtFRh8zBA9a7g+BT/lSANATCDPaAk1YVih2EKb
dpc3briTDbRsiqg2JKMI7+VdULY9bh3EawIBAg==
-----END DH PARAMETERS-----

View File

@ -1,8 +1,9 @@
<yandex> <yandex>
<openSSL> <openSSL>
<server> <server>
<certificateFile>/etc/clickhouse-server/server.crt</certificateFile> <certificateFile>/etc/clickhouse-server/config.d/server.crt</certificateFile>
<privateKeyFile>/etc/clickhouse-server/server.key</privateKeyFile> <privateKeyFile>/etc/clickhouse-server/config.d/server.key</privateKeyFile>
<dhParamsFile>/etc/clickhouse-server/config.d/dhparam.pem</dhParamsFile>
<verificationMode>none</verificationMode> <verificationMode>none</verificationMode>
<cacheSessions>true</cacheSessions> <cacheSessions>true</cacheSessions>
</server> </server>
@ -15,4 +16,5 @@
</client> </client>
</openSSL> </openSSL>
<interserver_https_port>9010</interserver_https_port> <interserver_https_port>9010</interserver_https_port>
<interserver_http_port remove="1"/>
</yandex> </yandex>

View File

@ -23,8 +23,8 @@ def _fill_nodes(nodes, shard):
'''.format(shard=shard, replica=node.name)) '''.format(shard=shard, replica=node.name))
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml'], with_zookeeper=True) node1 = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml', "configs/server.crt", "configs/server.key", "configs/dhparam.pem"], with_zookeeper=True)
node2 = cluster.add_instance('node2', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml'], with_zookeeper=True) node2 = cluster.add_instance('node2', main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml', "configs/server.crt", "configs/server.key", "configs/dhparam.pem"], with_zookeeper=True)
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
def both_https_cluster(): def both_https_cluster():
@ -78,8 +78,8 @@ def test_replication_after_partition(both_https_cluster):
node3 = cluster.add_instance('node3', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True) node3 = cluster.add_instance('node3', main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True)
node4 = cluster.add_instance('node4', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True) node4 = cluster.add_instance('node4', main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True)
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
def both_http_cluster(): def both_http_cluster():
@ -104,8 +104,8 @@ def test_both_http(both_http_cluster):
assert_eq_with_retry(node3, "SELECT id FROM test_table order by id", '111\n222') assert_eq_with_retry(node3, "SELECT id FROM test_table order by id", '111\n222')
assert_eq_with_retry(node4, "SELECT id FROM test_table order by id", '111\n222') assert_eq_with_retry(node4, "SELECT id FROM test_table order by id", '111\n222')
node5 = cluster.add_instance('node5', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml'], with_zookeeper=True) node5 = cluster.add_instance('node5', main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml', "configs/server.crt", "configs/server.key", "configs/dhparam.pem"], with_zookeeper=True)
node6 = cluster.add_instance('node6', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True) node6 = cluster.add_instance('node6', main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True)
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
def mixed_protocol_cluster(): def mixed_protocol_cluster():

View File

@ -0,0 +1,13 @@
<?xml version="1.0"?>
<yandex>
<storage_configuration>
<disks>
<default>
<type>s3</type>
<endpoint>http://minio1:9001/root/data/</endpoint>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</default>
</disks>
</storage_configuration>
</yandex>

View File

@ -0,0 +1,12 @@
<?xml version="1.0"?>
<yandex>
<openSSL>
<client>
<cacheSessions>true</cacheSessions>
<verificationMode>none</verificationMode>
<invalidCertificateHandler>
<name>AcceptCertificateHandler</name>
</invalidCertificateHandler>
</client>
</openSSL>
</yandex>

View File

@ -11,7 +11,7 @@ logging.getLogger().addHandler(logging.StreamHandler())
def cluster(): def cluster():
try: try:
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
cluster.add_instance("node", config_dir="configs", with_minio=True) cluster.add_instance("node", main_configs=["configs/minio.xml", "configs/ssl.xml", "configs/config.d/log_conf.xml"], with_minio=True)
logging.info("Starting cluster...") logging.info("Starting cluster...")
cluster.start() cluster.start()
logging.info("Cluster started") logging.info("Cluster started")

View File

@ -22,8 +22,8 @@ def _fill_nodes(nodes, shard, connections_count):
'''.format(shard=shard, replica=node.name, connections=connections_count)) '''.format(shard=shard, replica=node.name, connections=connections_count))
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) node1 = cluster.add_instance('node1', user_configs=[], main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True)
node2 = cluster.add_instance('node2', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) node2 = cluster.add_instance('node2', user_configs=[], main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True)
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
def start_small_cluster(): def start_small_cluster():
@ -68,9 +68,9 @@ def test_keepalive_timeout(start_small_cluster):
assert not node2.contains_in_log("No message received"), "Found 'No message received' in clickhouse-server.log" assert not node2.contains_in_log("No message received"), "Found 'No message received' in clickhouse-server.log"
node3 = cluster.add_instance('node3', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) node3 = cluster.add_instance('node3', user_configs=[], main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True)
node4 = cluster.add_instance('node4', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) node4 = cluster.add_instance('node4', user_configs=[], main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True)
node5 = cluster.add_instance('node5', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True) node5 = cluster.add_instance('node5', user_configs=[], main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True)
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
def start_big_cluster(): def start_big_cluster():

View File

@ -14,7 +14,7 @@ logging.getLogger().addHandler(logging.StreamHandler())
def cluster(): def cluster():
try: try:
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
cluster.add_instance("node", config_dir="configs", with_minio=True) cluster.add_instance("node", main_configs=["configs/config.d/storage_conf.xml", "configs/config.d/bg_processing_pool_conf.xml", "configs/config.d/log_conf.xml"], user_configs=[], with_minio=True)
logging.info("Starting cluster...") logging.info("Starting cluster...")
cluster.start() cluster.start()
logging.info("Cluster started") logging.info("Cluster started")

View File

@ -0,0 +1,9 @@
<?xml version="1.0"?>
<yandex>
<query_log>
<database>system</database>
<table>query_log</table>
<partition_by>toYYYYMM(event_date)</partition_by>
<flush_interval_milliseconds>1000</flush_interval_milliseconds>
</query_log>
</yandex>

View File

@ -0,0 +1,12 @@
<?xml version="1.0"?>
<yandex>
<openSSL>
<client>
<cacheSessions>true</cacheSessions>
<verificationMode>none</verificationMode>
<invalidCertificateHandler>
<name>AcceptCertificateHandler</name>
</invalidCertificateHandler>
</client>
</openSSL>
</yandex>

View File

@ -11,7 +11,9 @@ logging.getLogger().addHandler(logging.StreamHandler())
def cluster(): def cluster():
try: try:
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
cluster.add_instance("node", config_dir="configs", with_minio=True) cluster.add_instance("node", main_configs=["configs/config.d/log_conf.xml", "configs/config.d/storage_conf.xml",
"configs/config.d/ssl_conf.xml", "configs/config.d/query_log.xml"],
user_configs=["configs/config.d/users.xml"], with_minio=True)
logging.info("Starting cluster...") logging.info("Starting cluster...")
cluster.start() cluster.start()
logging.info("Cluster started") logging.info("Cluster started")

View File

@ -13,16 +13,14 @@ from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', node1 = cluster.add_instance('node1',
config_dir='configs', main_configs=['configs/logs_config.xml', 'configs/config.d/storage_configuration.xml', 'configs/config.d/cluster.xml'],
main_configs=['configs/logs_config.xml'],
with_zookeeper=True, with_zookeeper=True,
stay_alive=True, stay_alive=True,
tmpfs=['/jbod1:size=40M', '/jbod2:size=40M', '/external:size=200M'], tmpfs=['/jbod1:size=40M', '/jbod2:size=40M', '/external:size=200M'],
macros={"shard": 0, "replica": 1} ) macros={"shard": 0, "replica": 1} )
node2 = cluster.add_instance('node2', node2 = cluster.add_instance('node2',
config_dir='configs', main_configs=['configs/logs_config.xml', 'configs/config.d/storage_configuration.xml', 'configs/config.d/cluster.xml'],
main_configs=['configs/logs_config.xml'],
with_zookeeper=True, with_zookeeper=True,
stay_alive=True, stay_alive=True,
tmpfs=['/jbod1:size=40M', '/jbod2:size=40M', '/external:size=200M'], tmpfs=['/jbod1:size=40M', '/jbod2:size=40M', '/external:size=200M'],

View File

@ -127,7 +127,6 @@ def test_bad_arguments_for_mysql_database_engine(started_cluster):
with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node:
with pytest.raises(QueryRuntimeException) as exception: with pytest.raises(QueryRuntimeException) as exception:
mysql_node.query("CREATE DATABASE IF NOT EXISTS test_bad_arguments DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE DATABASE IF NOT EXISTS test_bad_arguments DEFAULT CHARACTER SET 'utf8'")
clickhouse_node.query("CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', test_bad_arguments, root, 'clickhouse')") clickhouse_node.query("CREATE DATABASE test_database_bad_arguments ENGINE = MySQL('mysql1:3306', test_bad_arguments, root, 'clickhouse')")
assert 'Database engine MySQL requested literal argument.' in str(exception.value) assert 'Database engine MySQL requested literal argument.' in str(exception.value)
mysql_node.query("DROP DATABASE test_bad_arguments") mysql_node.query("DROP DATABASE test_bad_arguments")

View File

@ -0,0 +1,10 @@
<?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>
</yandex>

View File

@ -0,0 +1,4 @@
<?xml version="1.0"?>
<yandex>
<mysql_port>9001</mysql_port>
</yandex>

View File

@ -0,0 +1,18 @@
<?xml version="1.0"?>
<yandex>
<!-- Used with https_port and tcp_port_secure. Full ssl options list: https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h#L71 -->
<openSSL>
<server> <!-- Used for https server AND secure tcp port -->
<!-- openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout /etc/clickhouse-server/server.key -out /etc/clickhouse-server/server.crt -->
<certificateFile>/etc/clickhouse-server/config.d/server.crt</certificateFile>
<privateKeyFile>/etc/clickhouse-server/config.d/server.key</privateKeyFile>
<!-- openssl dhparam -out /etc/clickhouse-server/dhparam.pem 4096 -->
<dhParamsFile>/etc/clickhouse-server/config.d/dhparam.pem</dhParamsFile>
<verificationMode>none</verificationMode>
<loadDefaultCAFile>true</loadDefaultCAFile>
<cacheSessions>true</cacheSessions>
<disableProtocols>sslv2,sslv3</disableProtocols>
<preferServerCiphers>true</preferServerCiphers>
</server>
</openSSL>
</yandex>

View File

@ -17,9 +17,10 @@ from helpers.cluster import ClickHouseCluster, get_docker_compose_path
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
DOCKER_COMPOSE_PATH = get_docker_compose_path() DOCKER_COMPOSE_PATH = get_docker_compose_path()
config_dir = os.path.join(SCRIPT_DIR, './configs')
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', config_dir=config_dir, env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) node = cluster.add_instance('node', main_configs=["configs/log_conf.xml", "configs/ssl_conf.xml", "configs/mysql.xml",
"configs/dhparam.pem", "configs/server.crt", "configs/server.key"],
user_configs=["configs/users.xml"], env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'})
server_port = 9001 server_port = 9001
@ -36,7 +37,7 @@ def server_address():
@pytest.fixture(scope='module') @pytest.fixture(scope='module')
def mysql_client(): def mysql_client():
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_client.yml') docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_client.yml')
subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build'])
yield docker.from_env().containers.get(cluster.project_name + '_mysql1_1') yield docker.from_env().containers.get(cluster.project_name + '_mysql1_1')
@ -62,28 +63,28 @@ def mysql_server(mysql_client):
@pytest.fixture(scope='module') @pytest.fixture(scope='module')
def golang_container(): def golang_container():
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_golang_client.yml') docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_golang_client.yml')
subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build'])
yield docker.from_env().containers.get(cluster.project_name + '_golang1_1') yield docker.from_env().containers.get(cluster.project_name + '_golang1_1')
@pytest.fixture(scope='module') @pytest.fixture(scope='module')
def php_container(): def php_container():
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_php_client.yml') docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_php_client.yml')
subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build'])
yield docker.from_env().containers.get(cluster.project_name + '_php1_1') yield docker.from_env().containers.get(cluster.project_name + '_php1_1')
@pytest.fixture(scope='module') @pytest.fixture(scope='module')
def nodejs_container(): def nodejs_container():
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_js_client.yml') docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_js_client.yml')
subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build'])
yield docker.from_env().containers.get(cluster.project_name + '_mysqljs1_1') yield docker.from_env().containers.get(cluster.project_name + '_mysqljs1_1')
@pytest.fixture(scope='module') @pytest.fixture(scope='module')
def java_container(): def java_container():
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_java_client.yml') docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_java_client.yml')
subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build'])
yield docker.from_env().containers.get(cluster.project_name + '_java1_1') yield docker.from_env().containers.get(cluster.project_name + '_java1_1')

View File

@ -0,0 +1,4 @@
<?xml version="1.0"?>
<yandex>
<dictionaries_config>/etc/clickhouse-server/config.d/*dictionary.xml</dictionaries_config>
</yandex>

View File

@ -0,0 +1,8 @@
<?xml version="1.0"?>
<yandex>
<logger>
<odbc_bridge_log>/var/log/clickhouse-server/clickhouse-odbc-bridge.log</odbc_bridge_log>
<odbc_bridge_errlog>/var/log/clickhouse-server/clickhouse-odbc-bridge.err.log</odbc_bridge_errlog>
<odbc_bridge_level>trace</odbc_bridge_level>
</logger>
</yandex>

View File

@ -0,0 +1,12 @@
<?xml version="1.0"?>
<yandex>
<openSSL>
<client>
<cacheSessions>true</cacheSessions>
<verificationMode>none</verificationMode>
<invalidCertificateHandler>
<name>AcceptCertificateHandler</name>
</invalidCertificateHandler>
</client>
</openSSL>
</yandex>

View File

@ -7,10 +7,9 @@ import psycopg2
from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT
from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseCluster
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, image='yandex/clickhouse-integration-test', main_configs=['configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml', 'configs/dictionaries/sqlite3_odbc_cached_dictionary.xml', 'configs/dictionaries/postgres_odbc_hashed_dictionary.xml'], stay_alive=True) node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, image='yandex/clickhouse-integration-test', main_configs=['configs/openssl.xml','configs/odbc_logging.xml','configs/enable_dictionaries.xml','configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml','configs/dictionaries/sqlite3_odbc_cached_dictionary.xml','configs/dictionaries/postgres_odbc_hashed_dictionary.xml'], stay_alive=True)
create_table_sql_template = """ create_table_sql_template = """
CREATE TABLE `clickhouse`.`{}` ( CREATE TABLE `clickhouse`.`{}` (

View File

@ -1,3 +1,4 @@
import time import time
import os import os
import pytest import pytest
@ -9,13 +10,13 @@ from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node18_14 = cluster.add_instance('node18_14', image='yandex/clickhouse-server:18.14.19', with_installed_binary=True, config_dir="configs") node18_14 = cluster.add_instance('node18_14', image='yandex/clickhouse-server:18.14.19', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"])
node19_1 = cluster.add_instance('node19_1', image='yandex/clickhouse-server:19.1.16', with_installed_binary=True, config_dir="configs") node19_1 = cluster.add_instance('node19_1', image='yandex/clickhouse-server:19.1.16', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"])
node19_4 = cluster.add_instance('node19_4', image='yandex/clickhouse-server:19.4.5.35', with_installed_binary=True, config_dir="configs") node19_4 = cluster.add_instance('node19_4', image='yandex/clickhouse-server:19.4.5.35', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"])
node19_8 = cluster.add_instance('node19_8', image='yandex/clickhouse-server:19.8.3.8', with_installed_binary=True, config_dir="configs") node19_8 = cluster.add_instance('node19_8', image='yandex/clickhouse-server:19.8.3.8', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"])
node19_11 = cluster.add_instance('node19_11', image='yandex/clickhouse-server:19.11.13.74', with_installed_binary=True, config_dir="configs") node19_11 = cluster.add_instance('node19_11', image='yandex/clickhouse-server:19.11.13.74', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"])
node19_13 = cluster.add_instance('node19_13', image='yandex/clickhouse-server:19.13.7.57', with_installed_binary=True, config_dir="configs") node19_13 = cluster.add_instance('node19_13', image='yandex/clickhouse-server:19.13.7.57', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"])
node19_16 = cluster.add_instance('node19_16', image='yandex/clickhouse-server:19.16.2.2', with_installed_binary=True, config_dir="configs") node19_16 = cluster.add_instance('node19_16', image='yandex/clickhouse-server:19.16.2.2', with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"])
old_nodes = [node18_14, node19_1, node19_4, node19_8, node19_11, node19_13, node19_16] old_nodes = [node18_14, node19_1, node19_4, node19_8, node19_11, node19_13, node19_16]
new_node = cluster.add_instance('node_new') new_node = cluster.add_instance('node_new')

View File

@ -53,21 +53,21 @@ def create_tables_old_format(name, nodes, shard):
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/{name}', '{repl}', date, id, 64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/{name}', '{repl}', date, id, 64)
'''.format(name=name, shard=shard, repl=i)) '''.format(name=name, shard=shard, repl=i))
node1 = cluster.add_instance('node1', config_dir="configs", with_zookeeper=True) node1 = cluster.add_instance('node1', main_configs=[], user_configs=["configs/users.d/not_optimize_count.xml"], with_zookeeper=True)
node2 = cluster.add_instance('node2', config_dir="configs", with_zookeeper=True) node2 = cluster.add_instance('node2', main_configs=[], user_configs=["configs/users.d/not_optimize_count.xml"], with_zookeeper=True)
settings_default = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0} settings_default = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0}
settings_compact_only = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 1000000, 'min_rows_for_compact_part' : 0} settings_compact_only = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 1000000, 'min_rows_for_compact_part' : 0}
settings_not_adaptive = {'index_granularity_bytes' : 0, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0} settings_not_adaptive = {'index_granularity_bytes' : 0, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0}
node3 = cluster.add_instance('node3', config_dir="configs", with_zookeeper=True) node3 = cluster.add_instance('node3', main_configs=[], user_configs=["configs/users.d/not_optimize_count.xml"], with_zookeeper=True)
node4 = cluster.add_instance('node4', config_dir="configs", main_configs=['configs/no_leader.xml'], with_zookeeper=True) node4 = cluster.add_instance('node4', user_configs=["configs/users.d/not_optimize_count.xml"], main_configs=['configs/no_leader.xml'], with_zookeeper=True)
settings_compact = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0} settings_compact = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0}
settings_wide = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 0, 'min_rows_for_compact_part' : 0} settings_wide = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 0, 'min_rows_for_compact_part' : 0}
node5 = cluster.add_instance('node5', config_dir='configs', main_configs=['configs/compact_parts.xml'], with_zookeeper=True) node5 = cluster.add_instance('node5', main_configs=['configs/compact_parts.xml'], with_zookeeper=True)
node6 = cluster.add_instance('node6', config_dir='configs', main_configs=['configs/compact_parts.xml'], with_zookeeper=True) node6 = cluster.add_instance('node6', main_configs=['configs/compact_parts.xml'], with_zookeeper=True)
settings_in_memory = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 256} settings_in_memory = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 256}
@ -213,8 +213,8 @@ def test_different_part_types_on_replicas(start_cluster, table, part_type):
"WHERE table = '{}' AND active GROUP BY part_type ORDER BY part_type".format(table))) == TSV(expected) "WHERE table = '{}' AND active GROUP BY part_type ORDER BY part_type".format(table))) == TSV(expected)
node7 = cluster.add_instance('node7', config_dir="configs_old", with_zookeeper=True, image='yandex/clickhouse-server:19.17.8.54', stay_alive=True, with_installed_binary=True) node7 = cluster.add_instance('node7', user_configs=["configs_old/users.d/not_optimize_count.xml"], with_zookeeper=True, image='yandex/clickhouse-server:19.17.8.54', stay_alive=True, with_installed_binary=True)
node8 = cluster.add_instance('node8', config_dir="configs", with_zookeeper=True) node8 = cluster.add_instance('node8', main_configs=[], user_configs=["configs/users.d/not_optimize_count.xml"], with_zookeeper=True)
settings7 = {'index_granularity_bytes' : 10485760} settings7 = {'index_granularity_bytes' : 10485760}
settings8 = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0} settings8 = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0}

View File

@ -0,0 +1,13 @@
<?xml version="1.0"?>
<yandex>
<profiles>
<default>
</default>
</profiles>
<users>
<default>
<password>123</password>
</default>
</users>
</yandex>

View File

@ -0,0 +1,10 @@
<?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>
</yandex>

View File

@ -0,0 +1,4 @@
<?xml version="1.0"?>
<yandex>
<postgresql_port>5433</postgresql_port>
</yandex>

View File

@ -0,0 +1,18 @@
<?xml version="1.0"?>
<yandex>
<!-- Used with https_port and tcp_port_secure. Full ssl options list: https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h#L71 -->
<openSSL>
<server> <!-- Used for https server AND secure tcp port -->
<!-- openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout /etc/clickhouse-server/server.key -out /etc/clickhouse-server/server.crt -->
<certificateFile>/etc/clickhouse-server/config.d/server.crt</certificateFile>
<privateKeyFile>/etc/clickhouse-server/config.d/server.key</privateKeyFile>
<!-- openssl dhparam -out /etc/clickhouse-server/dhparam.pem 4096 -->
<dhParamsFile>/etc/clickhouse-server/config.d/dhparam.pem</dhParamsFile>
<verificationMode>none</verificationMode>
<loadDefaultCAFile>true</loadDefaultCAFile>
<cacheSessions>true</cacheSessions>
<disableProtocols>sslv2,sslv3</disableProtocols>
<preferServerCiphers>true</preferServerCiphers>
</server>
</openSSL>
</yandex>

View File

@ -19,11 +19,12 @@ from helpers.cluster import ClickHouseCluster, get_docker_compose_path
psycopg2.extras.register_uuid() psycopg2.extras.register_uuid()
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
config_dir = os.path.join(SCRIPT_DIR, './configs')
DOCKER_COMPOSE_PATH = get_docker_compose_path() DOCKER_COMPOSE_PATH = get_docker_compose_path()
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', config_dir=config_dir, env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) node = cluster.add_instance('node', main_configs=["configs/postresql.xml", "configs/log.xml", "configs/ssl_conf.xml",
"configs/dhparam.pem", "configs/server.crt", "configs/server.key"],
user_configs=["configs/default_passwd.xml"], env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'})
server_port = 5433 server_port = 5433

View File

@ -0,0 +1,10 @@
<?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>
</yandex>

View File

@ -0,0 +1,9 @@
<?xml version="1.0"?>
<yandex>
<query_log>
<database>system</database>
<table>query_log</table>
<partition_by>toYYYYMM(event_date)</partition_by>
<flush_interval_milliseconds>1000</flush_interval_milliseconds>
</query_log>
</yandex>

View File

@ -0,0 +1,12 @@
<?xml version="1.0"?>
<yandex>
<openSSL>
<client>
<cacheSessions>true</cacheSessions>
<verificationMode>none</verificationMode>
<invalidCertificateHandler>
<name>AcceptCertificateHandler</name>
</invalidCertificateHandler>
</client>
</openSSL>
</yandex>

View File

@ -17,7 +17,7 @@ def cluster():
try: try:
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
cluster.add_instance("node", config_dir="configs", with_minio=True) cluster.add_instance("node", main_configs=["configs/config.d/storage_conf.xml", "configs/log.xml", "configs/query_log.xml", "configs/ssl_conf.xml"], with_minio=True)
logging.info("Starting cluster...") logging.info("Starting cluster...")
cluster.start() cluster.start()

View File

@ -7,18 +7,15 @@ from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
zero = cluster.add_instance("zero", zero = cluster.add_instance("zero", user_configs=["configs/users.d/settings.xml"],
config_dir="configs",
macros={"cluster": "anime", "shard": "0", "replica": "zero"}, macros={"cluster": "anime", "shard": "0", "replica": "zero"},
with_zookeeper=True) with_zookeeper=True)
first = cluster.add_instance("first", first = cluster.add_instance("first", user_configs=["configs/users.d/settings.xml"],
config_dir="configs",
macros={"cluster": "anime", "shard": "0", "replica": "first"}, macros={"cluster": "anime", "shard": "0", "replica": "first"},
with_zookeeper=True) with_zookeeper=True)
second = cluster.add_instance("second", second = cluster.add_instance("second", user_configs=["configs/users.d/settings.xml"],
config_dir="configs",
macros={"cluster": "anime", "shard": "0", "replica": "second"}, macros={"cluster": "anime", "shard": "0", "replica": "second"},
with_zookeeper=True) with_zookeeper=True)

View File

@ -6,28 +6,38 @@ import re
import time import time
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance', instance = cluster.add_instance('instance', user_configs=["configs/users.d/assign_myquota.xml", "configs/users.d/drop_default_quota.xml", "configs/users.d/quota.xml"])
config_dir="configs")
def system_quotas(): def check_system_quotas(canonical):
return TSV(instance.query("SELECT * FROM system.quotas ORDER BY name")) canonical_tsv = TSV(canonical)
r = TSV(instance.query("SELECT * FROM system.quotas ORDER BY name"))
print("system_quotas: {},\ncanonical: {}".format(r, TSV(canonical_tsv)))
assert r == canonical_tsv
def system_quota_limits(): def system_quota_limits(canonical):
return TSV(instance.query("SELECT * FROM system.quota_limits ORDER BY quota_name, duration")) canonical_tsv = TSV(canonical)
r = TSV(instance.query("SELECT * FROM system.quota_limits ORDER BY quota_name, duration"))
print("system_quota_limits: {},\ncanonical: {}".format(r, TSV(canonical_tsv)))
assert r == canonical_tsv
def system_quota_usage(): def system_quota_usage(canonical):
canonical_tsv = TSV(canonical)
query = "SELECT quota_name, quota_key, duration, queries, max_queries, errors, max_errors, result_rows, max_result_rows,"\ query = "SELECT quota_name, quota_key, duration, queries, max_queries, errors, max_errors, result_rows, max_result_rows,"\
"result_bytes, max_result_bytes, read_rows, max_read_rows, read_bytes, max_read_bytes, max_execution_time "\ "result_bytes, max_result_bytes, read_rows, max_read_rows, read_bytes, max_read_bytes, max_execution_time "\
"FROM system.quota_usage ORDER BY duration" "FROM system.quota_usage ORDER BY duration"
return TSV(instance.query(query)) r = TSV(instance.query(query))
print("system_quota_usage: {},\ncanonical: {}".format(r, TSV(canonical_tsv)))
assert r == canonical_tsv
def system_quotas_usage(): def system_quotas_usage(canonical):
canonical_tsv = TSV(canonical)
query = "SELECT quota_name, quota_key, is_current, duration, queries, max_queries, errors, max_errors, result_rows, max_result_rows, "\ query = "SELECT quota_name, quota_key, is_current, duration, queries, max_queries, errors, max_errors, result_rows, max_result_rows, "\
"result_bytes, max_result_bytes, read_rows, max_read_rows, read_bytes, max_read_bytes, max_execution_time "\ "result_bytes, max_result_bytes, read_rows, max_read_rows, read_bytes, max_read_bytes, max_execution_time "\
"FROM system.quotas_usage ORDER BY quota_name, quota_key, duration" "FROM system.quotas_usage ORDER BY quota_name, quota_key, duration"
return TSV(instance.query(query)) r = TSV(instance.query(query))
print("system_quotas_usage: {},\ncanonical: {}".format(r, TSV(canonical_tsv)))
assert r == canonical_tsv
def copy_quota_xml(local_file_name, reload_immediately = True): def copy_quota_xml(local_file_name, reload_immediately = True):
script_dir = os.path.dirname(os.path.realpath(__file__)) script_dir = os.path.dirname(os.path.realpath(__file__))
@ -61,141 +71,141 @@ def reset_quotas_and_usage_info():
def test_quota_from_users_xml(): def test_quota_from_users_xml():
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]] check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]])
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]])
assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]])
assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] system_quotas_usage([["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]])
instance.query("SELECT * from test_table") instance.query("SELECT * from test_table")
assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"]] system_quota_usage([["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"]])
instance.query("SELECT COUNT() from test_table") instance.query("SELECT COUNT() from test_table")
assert system_quota_usage() == [["myQuota", "default", 31556952, 2, 1000, 0, "\N", 51, "\N", 208, "\N", 50, 1000, 200, "\N", "\N"]] system_quota_usage([["myQuota", "default", 31556952, 2, 1000, 0, "\N", 51, "\N", 208, "\N", 50, 1000, 200, "\N", "\N"]])
def test_simpliest_quota(): def test_simpliest_quota():
# Simpliest quota doesn't even track usage. # Simpliest quota doesn't even track usage.
copy_quota_xml('simpliest.xml') copy_quota_xml('simpliest.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]] check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]])
assert system_quota_limits() == "" system_quota_limits("")
assert system_quota_usage() == [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]] system_quota_usage([["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]])
instance.query("SELECT * from test_table") instance.query("SELECT * from test_table")
assert system_quota_usage() == [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]] system_quota_usage([["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]])
def test_tracking_quota(): def test_tracking_quota():
# Now we're tracking usage. # Now we're tracking usage.
copy_quota_xml('tracking.xml') copy_quota_xml('tracking.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]])
assert system_quota_limits() == [["myQuota", 31556952, 0, "\N", "\N", "\N", "\N", "\N", "\N", "\N"]] system_quota_limits([["myQuota", 31556952, 0, "\N", "\N", "\N", "\N", "\N", "\N", "\N"]])
assert system_quota_usage() == [["myQuota", "default", 31556952, 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", "\N"]] system_quota_usage([["myQuota", "default", 31556952, 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", "\N"]])
instance.query("SELECT * from test_table") instance.query("SELECT * from test_table")
assert system_quota_usage() == [["myQuota", "default", 31556952, 1, "\N", 0, "\N", 50, "\N", 200, "\N", 50, "\N", 200, "\N", "\N"]] system_quota_usage([["myQuota", "default", 31556952, 1, "\N", 0, "\N", 50, "\N", 200, "\N", 50, "\N", 200, "\N", "\N"]])
instance.query("SELECT COUNT() from test_table") instance.query("SELECT COUNT() from test_table")
assert system_quota_usage() == [["myQuota", "default", 31556952, 2, "\N", 0, "\N", 51, "\N", 208, "\N", 50, "\N", 200, "\N", "\N"]] system_quota_usage([["myQuota", "default", 31556952, 2, "\N", 0, "\N", 51, "\N", 208, "\N", 50, "\N", 200, "\N", "\N"]])
def test_exceed_quota(): def test_exceed_quota():
# Change quota, now the limits are tiny so we will exceed the quota. # Change quota, now the limits are tiny so we will exceed the quota.
copy_quota_xml('tiny_limits.xml') copy_quota_xml('tiny_limits.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]])
assert system_quota_limits() == [["myQuota", 31556952, 0, 1, 1, 1, "\N", 1, "\N", "\N"]] system_quota_limits([["myQuota", 31556952, 0, 1, 1, 1, "\N", 1, "\N", "\N"]])
assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1, 0, 1, 0, 1, 0, "\N", 0, 1, 0, "\N", "\N"]] system_quota_usage([["myQuota", "default", 31556952, 0, 1, 0, 1, 0, 1, 0, "\N", 0, 1, 0, "\N", "\N"]])
assert re.search("Quota.*has\ been\ exceeded", instance.query_and_get_error("SELECT * from test_table")) assert re.search("Quota.*has\ been\ exceeded", instance.query_and_get_error("SELECT * from test_table"))
assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1, 1, 1, 0, 1, 0, "\N", 50, 1, 0, "\N", "\N"]] system_quota_usage([["myQuota", "default", 31556952, 1, 1, 1, 1, 0, 1, 0, "\N", 50, 1, 0, "\N", "\N"]])
# Change quota, now the limits are enough to execute queries. # Change quota, now the limits are enough to execute queries.
copy_quota_xml('normal_limits.xml') copy_quota_xml('normal_limits.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]])
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]])
assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1000, 1, "\N", 0, "\N", 0, "\N", 50, 1000, 0, "\N", "\N"]] system_quota_usage([["myQuota", "default", 31556952, 1, 1000, 1, "\N", 0, "\N", 0, "\N", 50, 1000, 0, "\N", "\N"]])
instance.query("SELECT * from test_table") instance.query("SELECT * from test_table")
assert system_quota_usage() == [["myQuota", "default", 31556952, 2, 1000, 1, "\N", 50, "\N", 200, "\N", 100, 1000, 200, "\N", "\N"]] system_quota_usage([["myQuota", "default", 31556952, 2, 1000, 1, "\N", 50, "\N", 200, "\N", 100, 1000, 200, "\N", "\N"]])
def test_add_remove_interval(): def test_add_remove_interval():
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]] check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]])
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]])
assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]])
# Add interval. # Add interval.
copy_quota_xml('two_intervals.xml') copy_quota_xml('two_intervals.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952,63113904]", 0, "['default']", "[]"]] check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952,63113904]", 0, "['default']", "[]"]])
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"], system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"],
["myQuota", 63113904, 1, "\N", "\N", "\N", 30000, "\N", 20000, 120]] ["myQuota", 63113904, 1, "\N", "\N", "\N", 30000, "\N", 20000, 120]])
assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"], system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"],
["myQuota", "default", 63113904, 0, "\N", 0, "\N", 0, "\N", 0, 30000, 0, "\N", 0, 20000, 120]] ["myQuota", "default", 63113904, 0, "\N", 0, "\N", 0, "\N", 0, 30000, 0, "\N", 0, 20000, 120]])
instance.query("SELECT * from test_table") instance.query("SELECT * from test_table")
assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"], system_quota_usage([["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"],
["myQuota", "default", 63113904, 1, "\N", 0, "\N", 50, "\N", 200, 30000, 50, "\N", 200, 20000, 120]] ["myQuota", "default", 63113904, 1, "\N", 0, "\N", 50, "\N", 200, 30000, 50, "\N", 200, 20000, 120]])
# Remove interval. # Remove interval.
copy_quota_xml('normal_limits.xml') copy_quota_xml('normal_limits.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]] check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]])
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]])
assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"]] system_quota_usage([["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"]])
instance.query("SELECT * from test_table") instance.query("SELECT * from test_table")
assert system_quota_usage() == [["myQuota", "default", 31556952, 2, 1000, 0, "\N", 100, "\N", 400, "\N", 100, 1000, 400, "\N", "\N"]] system_quota_usage([["myQuota", "default", 31556952, 2, 1000, 0, "\N", 100, "\N", 400, "\N", 100, 1000, 400, "\N", "\N"]])
# Remove all intervals. # Remove all intervals.
copy_quota_xml('simpliest.xml') copy_quota_xml('simpliest.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]] check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]])
assert system_quota_limits() == "" system_quota_limits("")
assert system_quota_usage() == [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]] system_quota_usage([["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]])
instance.query("SELECT * from test_table") instance.query("SELECT * from test_table")
assert system_quota_usage() == [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]] system_quota_usage([["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]])
# Add one interval back. # Add one interval back.
copy_quota_xml('normal_limits.xml') copy_quota_xml('normal_limits.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]] check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]])
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]])
assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]])
def test_add_remove_quota(): def test_add_remove_quota():
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]] check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]])
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]])
assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] system_quotas_usage([["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]])
# Add quota. # Add quota.
copy_quota_xml('two_quotas.xml') copy_quota_xml('two_quotas.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"], check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"],
["myQuota2", "4590510c-4d13-bf21-ec8a-c2187b092e73", "users.xml", "['client_key','user_name']", "[3600,2629746]", 0, "[]", "[]"]] ["myQuota2", "4590510c-4d13-bf21-ec8a-c2187b092e73", "users.xml", "['client_key','user_name']", "[3600,2629746]", 0, "[]", "[]"]])
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"], system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"],
["myQuota2", 3600, 1, "\N", "\N", 4000, 400000, 4000, 400000, 60], ["myQuota2", 3600, 1, "\N", "\N", 4000, 400000, 4000, 400000, 60],
["myQuota2", 2629746, 0, "\N", "\N", "\N", "\N", "\N", "\N", 1800]] ["myQuota2", 2629746, 0, "\N", "\N", "\N", "\N", "\N", "\N", 1800]])
assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] system_quotas_usage([["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]])
# Drop quota. # Drop quota.
copy_quota_xml('normal_limits.xml') copy_quota_xml('normal_limits.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]])
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]])
assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] system_quotas_usage([["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]])
# Drop all quotas. # Drop all quotas.
copy_quota_xml('no_quotas.xml') copy_quota_xml('no_quotas.xml')
assert system_quotas() == "" check_system_quotas("")
assert system_quota_limits() == "" system_quota_limits("")
assert system_quotas_usage() == "" system_quotas_usage("")
# Add one quota back. # Add one quota back.
copy_quota_xml('normal_limits.xml') copy_quota_xml('normal_limits.xml')
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]])
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]])
assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]] system_quotas_usage([["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]])
def test_reload_users_xml_by_timer(): def test_reload_users_xml_by_timer():
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]] check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]])
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]] system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]])
time.sleep(1) # The modification time of the 'quota.xml' file should be different, time.sleep(1) # The modification time of the 'quota.xml' file should be different,
# because config files are reload by timer only when the modification time is changed. # because config files are reload by timer only when the modification time is changed.

View File

@ -14,8 +14,8 @@ from helpers.client import CommandRequest
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', config_dir='configs', with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 1}) node1 = cluster.add_instance('node1', main_configs=["configs/conf.d/merge_tree.xml", "configs/conf.d/remote_servers.xml" ], with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 1})
node2 = cluster.add_instance('node2', config_dir='configs', with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 2}) node2 = cluster.add_instance('node2', main_configs=["configs/conf.d/merge_tree.xml", "configs/conf.d/remote_servers.xml" ], with_zookeeper=True, macros={"layer": 0, "shard": 0, "replica": 2})
nodes = [node1, node2] nodes = [node1, node2]
@pytest.fixture(scope="module") @pytest.fixture(scope="module")

View File

@ -0,0 +1,5 @@
<?xml version="1.0"?>
<yandex>
<max_table_size_to_drop>1</max_table_size_to_drop>
<max_partition_size_to_drop>1</max_partition_size_to_drop>
</yandex>

View File

@ -1,3 +1,4 @@
import time import time
import pytest import pytest
import os import os
@ -6,10 +7,10 @@ from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', config_dir="configs") node = cluster.add_instance('node', main_configs=["configs/max_table_size_to_drop.xml"])
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
CONFIG_PATH = os.path.join(SCRIPT_DIR, './_instances/node/configs/config.xml') CONFIG_PATH = os.path.join(SCRIPT_DIR, './_instances/node/configs/config.d/max_table_size_to_drop.xml')
@pytest.fixture(scope="module") @pytest.fixture(scope="module")

View File

@ -12,8 +12,9 @@ from helpers.test_tools import TSV
node_options = dict( node_options = dict(
with_zookeeper=True, with_zookeeper=True,
main_configs=['configs/remote_servers.xml'], main_configs=["configs/remote_servers.xml", "configs/config.d/instant_moves.xml",
config_dir='configs', "configs/config.d/part_log.xml", "configs/config.d/zookeeper_session_timeout.xml",
"configs/config.d/storage_configuration.xml"],
tmpfs=['/external:size=200M', '/internal:size=1M']) tmpfs=['/external:size=200M', '/internal:size=1M'])
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)

View File

@ -15,9 +15,9 @@ def cluster():
try: try:
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
cluster.add_instance("node1", config_dir="configs", macros={'cluster': 'test1'}, with_minio=True, with_zookeeper=True) cluster.add_instance("node1", main_configs=["configs/config.d/storage_conf.xml"], macros={'cluster': 'test1'}, with_minio=True, with_zookeeper=True)
cluster.add_instance("node2", config_dir="configs", macros={'cluster': 'test1'}, with_zookeeper=True) cluster.add_instance("node2", main_configs=["configs/config.d/storage_conf.xml"], macros={'cluster': 'test1'}, with_zookeeper=True)
cluster.add_instance("node3", config_dir="configs", macros={'cluster': 'test1'}, with_zookeeper=True) cluster.add_instance("node3", main_configs=["configs/config.d/storage_conf.xml"], macros={'cluster': 'test1'}, with_zookeeper=True)
logging.info("Starting cluster...") logging.info("Starting cluster...")
cluster.start() cluster.start()

View File

@ -0,0 +1,13 @@
<?xml version="1.0"?>
<yandex>
<users>
<another>
<password/>
<networks>
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
</another>
</users>
</yandex>

View File

@ -0,0 +1,8 @@
<?xml version="1.0"?>
<yandex>
<profiles>
<default>
<any_join_distinct_right_table_keys>1</any_join_distinct_right_table_keys>
</default>
</profiles>
</yandex>

View File

@ -6,8 +6,8 @@ import re
import time import time
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', config_dir="configs", with_zookeeper=True) node = cluster.add_instance('node', main_configs=["configs/config.d/remote_servers.xml"], user_configs=["configs/users.d/row_policy.xml", "configs/users.d/another_user.xml", "configs/users.d/any_join_distinct_right_table_keys.xml"], with_zookeeper=True)
node2 = cluster.add_instance('node2', config_dir="configs", with_zookeeper=True) node2 = cluster.add_instance('node2', main_configs=["configs/config.d/remote_servers.xml"], user_configs=["configs/users.d/row_policy.xml", "configs/users.d/another_user.xml", "configs/users.d/any_join_distinct_right_table_keys.xml"], with_zookeeper=True)
nodes = [node, node2] nodes = [node, node2]

Some files were not shown because too many files have changed in this diff Show More