mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Add explicit main_configs, user_configs and dictionaries in integration tests.
This commit is contained in:
parent
9767d96274
commit
160776f183
@ -19,7 +19,8 @@ set -e
|
||||
echo "Start tests"
|
||||
export CLICKHOUSE_TESTS_SERVER_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 DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest}
|
||||
|
@ -18,7 +18,7 @@ if(MAKE_STATIC_LIBRARIES AND DOCKER_CMD)
|
||||
if(NOT INTEGRATION_USE_RUNNER AND DOCKER_COMPOSE_CMD AND PYTEST_CMD)
|
||||
# To run one test with debug:
|
||||
# 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}")
|
||||
endif()
|
||||
endif()
|
||||
|
@ -0,0 +1,4 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<dictionaries_config>/etc/clickhouse-server/dictionaries/*.xml</dictionaries_config>
|
||||
</yandex>
|
@ -71,7 +71,7 @@ class CommandRequest:
|
||||
self.stderr_file = tempfile.TemporaryFile()
|
||||
self.ignore_error = ignore_error
|
||||
|
||||
#print " ".join(command)
|
||||
print " ".join(command)
|
||||
|
||||
# we suppress stderror on client becase sometimes thread sanitizer
|
||||
# can print some debug information there
|
||||
|
@ -1,25 +1,25 @@
|
||||
import base64
|
||||
import cassandra.cluster
|
||||
import distutils.dir_util
|
||||
import docker
|
||||
import errno
|
||||
import httplib
|
||||
import logging
|
||||
import os
|
||||
import os.path as p
|
||||
import pprint
|
||||
import psycopg2
|
||||
import pwd
|
||||
import pymongo
|
||||
import pymysql
|
||||
import re
|
||||
import requests
|
||||
import shutil
|
||||
import socket
|
||||
import subprocess
|
||||
import time
|
||||
import urllib
|
||||
import httplib
|
||||
import requests
|
||||
import xml.dom.minidom
|
||||
import logging
|
||||
import docker
|
||||
import pprint
|
||||
import psycopg2
|
||||
import pymongo
|
||||
import pymysql
|
||||
import cassandra.cluster
|
||||
from dicttoxml import dicttoxml
|
||||
from kazoo.client import KazooClient
|
||||
from kazoo.exceptions import KazooException
|
||||
@ -40,6 +40,7 @@ SANITIZER_SIGN = "=================="
|
||||
def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME):
|
||||
full_path = os.path.join(path, fname)
|
||||
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():
|
||||
f.write("=".join([var, value]) + "\n")
|
||||
return full_path
|
||||
@ -88,12 +89,16 @@ class ClickHouseCluster:
|
||||
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):
|
||||
for param in os.environ.keys():
|
||||
print "ENV %40s %s" % (param,os.environ[param])
|
||||
self.base_dir = p.dirname(base_path)
|
||||
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/')
|
||||
self.server_bin_path = p.realpath(
|
||||
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.is_up = False
|
||||
print "CLUSTER INIT base_config_dir:{} config_dir:{}".format(self.base_config_dir, self.config_dir)
|
||||
|
||||
def get_client_cmd(self):
|
||||
cmd = self.client_bin_path
|
||||
@ -161,7 +167,7 @@ class ClickHouseCluster:
|
||||
cmd += " client"
|
||||
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_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=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.
|
||||
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
|
||||
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.
|
||||
@ -184,11 +191,11 @@ class ClickHouseCluster:
|
||||
raise Exception("Can\'t add instance `%s': there is already an instance with the same name!" % name)
|
||||
|
||||
instance = ClickHouseInstance(
|
||||
self, self.base_dir, name, config_dir, main_configs or [], user_configs or [], macros or {},
|
||||
with_zookeeper,
|
||||
self, self.base_dir, name, base_config_dir if base_config_dir else self.base_config_dir,
|
||||
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.base_configs_dir, self.server_bin_path,
|
||||
self.odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=hostname,
|
||||
self.server_bin_path, 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,
|
||||
ipv6_address=ipv6_address,
|
||||
with_installed_binary=with_installed_binary, tmpfs=tmpfs or [])
|
||||
@ -458,19 +465,19 @@ class ClickHouseCluster:
|
||||
try:
|
||||
minio_client.list_buckets()
|
||||
|
||||
logging.info("Connected to Minio.")
|
||||
print("Connected to Minio.")
|
||||
|
||||
if minio_client.bucket_exists(self.minio_bucket):
|
||||
minio_client.remove_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
|
||||
return
|
||||
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)
|
||||
|
||||
raise Exception("Can't wait Minio to start")
|
||||
@ -482,10 +489,10 @@ class ClickHouseCluster:
|
||||
try:
|
||||
sr_client._send_request(sr_client.url)
|
||||
self.schema_registry_client = sr_client
|
||||
logging.info("Connected to SchemaRegistry")
|
||||
print("Connected to SchemaRegistry")
|
||||
return
|
||||
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)
|
||||
|
||||
def wait_cassandra_to_start(self, timeout=30):
|
||||
@ -501,25 +508,27 @@ class ClickHouseCluster:
|
||||
time.sleep(1)
|
||||
|
||||
def start(self, destroy_dirs=True):
|
||||
print "Cluster start called. is_up={}, destroy_dirs={}".format(self.is_up, destroy_dirs)
|
||||
if self.is_up:
|
||||
return
|
||||
|
||||
# Just in case kill unstopped containers from previous launch
|
||||
try:
|
||||
logging.info("Trying to kill unstopped containers...")
|
||||
print("Trying to kill unstopped containers...")
|
||||
|
||||
if not subprocess_call(['docker-compose', 'kill']):
|
||||
subprocess_call(['docker-compose', 'down', '--volumes'])
|
||||
logging.info("Unstopped containers killed")
|
||||
print("Unstopped containers killed")
|
||||
except:
|
||||
pass
|
||||
|
||||
try:
|
||||
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)
|
||||
|
||||
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)
|
||||
|
||||
self.docker_client = docker.from_env(version=self.docker_api_version)
|
||||
@ -527,6 +536,7 @@ class ClickHouseCluster:
|
||||
common_opts = ['up', '-d', '--force-recreate']
|
||||
|
||||
if self.with_zookeeper and self.base_zookeeper_cmd:
|
||||
print('Setup ZooKeeper')
|
||||
env = os.environ.copy()
|
||||
if not self.zookeeper_use_tmpfs:
|
||||
env['ZK_FS'] = 'bind'
|
||||
@ -545,14 +555,17 @@ class ClickHouseCluster:
|
||||
self.wait_zookeeper_to_start(120)
|
||||
|
||||
if self.with_mysql and self.base_mysql_cmd:
|
||||
print('Setup MySQL')
|
||||
subprocess_check_call(self.base_mysql_cmd + common_opts)
|
||||
self.wait_mysql_to_start(120)
|
||||
|
||||
if self.with_postgres and self.base_postgres_cmd:
|
||||
print('Setup Postgres')
|
||||
subprocess_check_call(self.base_postgres_cmd + common_opts)
|
||||
self.wait_postgres_to_start(120)
|
||||
|
||||
if self.with_kafka and self.base_kafka_cmd:
|
||||
print('Setup Kafka')
|
||||
subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes'])
|
||||
self.kafka_docker_id = self.get_instance_docker_id('kafka1')
|
||||
self.wait_schema_registry_to_start(120)
|
||||
@ -562,14 +575,17 @@ class ClickHouseCluster:
|
||||
self.rabbitmq_docker_id = self.get_instance_docker_id('rabbitmq1')
|
||||
|
||||
if self.with_hdfs and self.base_hdfs_cmd:
|
||||
print('Setup HDFS')
|
||||
subprocess_check_call(self.base_hdfs_cmd + common_opts)
|
||||
self.wait_hdfs_to_start(120)
|
||||
|
||||
if self.with_mongo and self.base_mongo_cmd:
|
||||
print('Setup Mongo')
|
||||
subprocess_check_call(self.base_mongo_cmd + common_opts)
|
||||
self.wait_mongo_to_start(30)
|
||||
|
||||
if self.with_redis and self.base_redis_cmd:
|
||||
print('Setup Redis')
|
||||
subprocess_check_call(self.base_redis_cmd + ['up', '-d', '--force-recreate'])
|
||||
time.sleep(10)
|
||||
|
||||
@ -608,18 +624,19 @@ class ClickHouseCluster:
|
||||
self.wait_cassandra_to_start()
|
||||
|
||||
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)
|
||||
logging.info("ClickHouse instance created")
|
||||
print("ClickHouse instance created")
|
||||
|
||||
|
||||
start_deadline = time.time() + 20.0 # seconds
|
||||
for instance in self.instances.itervalues():
|
||||
instance.docker_client = self.docker_client
|
||||
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)
|
||||
logging.info("ClickHouse started")
|
||||
print("ClickHouse started")
|
||||
|
||||
instance.client = Client(instance.ip_address, command=self.client_bin_path)
|
||||
|
||||
@ -633,7 +650,10 @@ class ClickHouseCluster:
|
||||
def shutdown(self, kill=True):
|
||||
sanitizer_assert_instance = None
|
||||
with open(self.docker_logs_path, "w+") as f:
|
||||
subprocess.check_call(self.base_cmd + ['logs'], stdout=f)
|
||||
try:
|
||||
subprocess.check_call(self.base_cmd + ['logs'], stdout=f)
|
||||
except Exception as e:
|
||||
print "Unable to get logs from docker."
|
||||
f.seek(0)
|
||||
for line in f:
|
||||
if SANITIZER_SIGN in line:
|
||||
@ -641,8 +661,15 @@ class ClickHouseCluster:
|
||||
break
|
||||
|
||||
if kill:
|
||||
subprocess_check_call(self.base_cmd + ['kill'])
|
||||
subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans'])
|
||||
try:
|
||||
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'])
|
||||
except Exception as e:
|
||||
print "Down + remove orphans failed durung shutdown. {}".format(repr(e))
|
||||
|
||||
self.is_up = False
|
||||
|
||||
@ -707,7 +734,7 @@ services:
|
||||
image: {image}
|
||||
hostname: {hostname}
|
||||
volumes:
|
||||
- {configs_dir}:/etc/clickhouse-server/
|
||||
- {instance_config_dir}:/etc/clickhouse-server/
|
||||
- {db_dir}:/var/lib/clickhouse/
|
||||
- {logs_dir}:/var/log/clickhouse-server/
|
||||
{binary_volume}
|
||||
@ -723,6 +750,9 @@ services:
|
||||
- {env_file}
|
||||
security_opt:
|
||||
- label:disable
|
||||
dns_opt:
|
||||
- timeout:1
|
||||
- attempts:3
|
||||
{networks}
|
||||
{app_net}
|
||||
{ipv4_address}
|
||||
@ -735,9 +765,9 @@ services:
|
||||
class ClickHouseInstance:
|
||||
|
||||
def __init__(
|
||||
self, cluster, base_path, name, custom_config_dir, custom_main_configs, custom_user_configs, macros,
|
||||
with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra,
|
||||
base_configs_dir, server_bin_path, odbc_bridge_bin_path,
|
||||
self, cluster, base_path, name, base_config_dir, config_dir, custom_main_configs, custom_user_configs, custom_dictionaries,
|
||||
macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio,
|
||||
with_cassandra, server_bin_path, odbc_bridge_bin_path,
|
||||
clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables=None,
|
||||
image="yandex/clickhouse-integration-test",
|
||||
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.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_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.macros = macros if macros is not None else {}
|
||||
self.with_zookeeper = with_zookeeper
|
||||
self.zookeeper_config_path = zookeeper_config_path
|
||||
|
||||
self.base_configs_dir = base_configs_dir
|
||||
self.server_bin_path = server_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.env_variables = env_variables or {}
|
||||
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
|
||||
else:
|
||||
self.odbc_ini_path = ""
|
||||
@ -975,7 +1006,7 @@ class ClickHouseInstance:
|
||||
time_left = deadline - current_time
|
||||
if deadline is not None and current_time >= deadline:
|
||||
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.
|
||||
# Usually it means that ClickHouse is ready to accept queries.
|
||||
@ -1057,40 +1088,50 @@ class ClickHouseInstance:
|
||||
|
||||
os.makedirs(self.path)
|
||||
|
||||
configs_dir = p.abspath(p.join(self.path, 'configs'))
|
||||
os.mkdir(configs_dir)
|
||||
instance_config_dir = p.abspath(p.join(self.path, 'configs'))
|
||||
os.makedirs(instance_config_dir)
|
||||
|
||||
shutil.copy(p.join(self.base_configs_dir, 'config.xml'), configs_dir)
|
||||
shutil.copy(p.join(self.base_configs_dir, 'users.xml'), configs_dir)
|
||||
print "Copy common default production configuration from {}".format(self.base_config_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
|
||||
conf_d_dir = p.abspath(p.join(configs_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'))
|
||||
conf_d_dir = p.abspath(p.join(instance_config_dir, 'conf.d'))
|
||||
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.
|
||||
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)
|
||||
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['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}))
|
||||
|
||||
# Put ZooKeeper config
|
||||
if self.with_zookeeper:
|
||||
shutil.copy(self.zookeeper_config_path, conf_d_dir)
|
||||
|
||||
# Copy config dir
|
||||
if self.custom_config_dir:
|
||||
distutils.dir_util.copy_tree(self.custom_config_dir, configs_dir)
|
||||
# print "Copy config dir {} to {}".format(self.config_dir, instance_config_dir)
|
||||
# if self.config_dir:
|
||||
# distutils.dir_util.copy_tree(self.config_dir, instance_config_dir)
|
||||
|
||||
# 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:
|
||||
shutil.copy(path, self.config_d_dir)
|
||||
|
||||
@ -1098,12 +1139,21 @@ class ClickHouseInstance:
|
||||
for path in self.custom_user_config_paths:
|
||||
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'))
|
||||
print "Setup database dir {}".format(db_dir)
|
||||
os.mkdir(db_dir)
|
||||
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)
|
||||
|
||||
logs_dir = p.abspath(p.join(self.path, 'logs'))
|
||||
print "Setup logs dir {}".format(logs_dir)
|
||||
os.mkdir(logs_dir)
|
||||
|
||||
depends_on = []
|
||||
@ -1128,6 +1178,8 @@ class ClickHouseInstance:
|
||||
|
||||
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 = ""
|
||||
if self.odbc_ini_path:
|
||||
self._create_odbc_config_file()
|
||||
@ -1138,6 +1190,8 @@ class ClickHouseInstance:
|
||||
if self.stay_alive:
|
||||
entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND
|
||||
|
||||
print "Entrypoint cmd: {}".format(entrypoint_cmd)
|
||||
|
||||
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:
|
||||
networks = "networks:"
|
||||
@ -1157,6 +1211,7 @@ class ClickHouseInstance:
|
||||
binary_volume = "- " + self.server_bin_path + ":/usr/share/clickhouse_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:
|
||||
docker_compose.write(DOCKER_COMPOSE_TEMPLATE.format(
|
||||
image=self.image,
|
||||
@ -1164,7 +1219,7 @@ class ClickHouseInstance:
|
||||
hostname=self.hostname,
|
||||
binary_volume=binary_volume,
|
||||
odbc_bridge_volume=odbc_bridge_volume,
|
||||
configs_dir=configs_dir,
|
||||
instance_config_dir=instance_config_dir,
|
||||
config_d_dir=self.config_d_dir,
|
||||
db_dir=db_dir,
|
||||
tmpfs=str(self.tmpfs),
|
||||
|
1
tests/integration/helpers/dictonaries
Symbolic link
1
tests/integration/helpers/dictonaries
Symbolic link
@ -0,0 +1 @@
|
||||
../../config/dict_examples/
|
@ -11,6 +11,9 @@ class TSV:
|
||||
raw_lines = contents.splitlines(True)
|
||||
elif isinstance(contents, list):
|
||||
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:
|
||||
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()]
|
||||
|
@ -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))
|
||||
|
||||
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):
|
||||
raise Exception("Path {} doesn't exist".format(path))
|
||||
|
||||
|
@ -4,9 +4,9 @@ from helpers.cluster import ClickHouseCluster
|
||||
from helpers.client import QueryRuntimeException
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
ch1 = cluster.add_instance('ch1', config_dir="configs", with_zookeeper=True)
|
||||
ch2 = cluster.add_instance('ch2', config_dir="configs", with_zookeeper=True)
|
||||
ch3 = cluster.add_instance('ch3', 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', main_configs=["configs/config.d/clusters.xml"], with_zookeeper=True)
|
||||
ch3 = cluster.add_instance('ch3', main_configs=["configs/config.d/clusters.xml"], with_zookeeper=True)
|
||||
|
||||
@pytest.fixture(scope="module", autouse=True)
|
||||
def started_cluster():
|
||||
|
@ -9,23 +9,23 @@ from helpers.test_tools import assert_eq_with_retry
|
||||
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node1 = cluster.add_instance('node1', config_dir="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)
|
||||
node1 = cluster.add_instance('node1', 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)
|
||||
node4 = cluster.add_instance('node4', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=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', 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)
|
||||
node6 = cluster.add_instance('node6', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=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', 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)
|
||||
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)
|
||||
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', 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)
|
||||
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)
|
||||
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', 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)
|
||||
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)
|
||||
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', 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):
|
||||
|
@ -4,7 +4,7 @@ from helpers.cluster import ClickHouseCluster
|
||||
|
||||
|
||||
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')
|
||||
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')
|
||||
|
||||
|
||||
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):
|
||||
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)])
|
||||
|
||||
|
||||
@ -56,5 +61,6 @@ def test_allowed_host():
|
||||
|
||||
for client_node in expected_to_fail:
|
||||
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)
|
||||
|
@ -40,7 +40,7 @@ def test_config_with_only_regexp_hosts(start_cluster):
|
||||
assert node3.query("CREATE TABLE table_test_3_1 (word String) Engine=URL('https://host:80', HDFS)") == ""
|
||||
assert node3.query("CREATE TABLE table_test_3_2 (word String) Engine=URL('https://yandex.ru', CSV)") == ""
|
||||
assert "not allowed" in node3.query_and_get_error("CREATE TABLE table_test_3_3 (word String) Engine=URL('https://host', CSV)")
|
||||
assert "not allowed" in node3.query_and_get_error("CREATE TABLE table_test_3_4 (word String) Engine=URL('https://yandex2.ru', S3)")
|
||||
assert "not allowed" in node3.query_and_get_error("CREATE TABLE table_test_3_4 (word String) Engine=URL('https://yandex2.ru', S3)")
|
||||
|
||||
def test_config_without_allowed_hosts(start_cluster):
|
||||
assert node4.query("CREATE TABLE table_test_4_1 (word String) Engine=URL('https://host:80', CSV)") == ""
|
||||
@ -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)") == ""
|
||||
|
||||
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 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 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):
|
||||
hdfs_api = HDFSApi("root")
|
||||
|
@ -6,7 +6,7 @@ from helpers.cluster import ClickHouseCluster
|
||||
|
||||
|
||||
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")
|
||||
|
@ -1,80 +1,74 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<remote_servers>
|
||||
|
||||
<cluster0>
|
||||
<shard>
|
||||
<internal_replication>true</internal_replication>
|
||||
<replica>
|
||||
<host>s0_0_0</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>s0_0_1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
<shard>
|
||||
<internal_replication>true</internal_replication>
|
||||
<replica>
|
||||
<host>s0_1_0</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</cluster0>
|
||||
|
||||
<cluster1>
|
||||
<shard>
|
||||
<internal_replication>true</internal_replication>
|
||||
<replica>
|
||||
<host>s1_0_0</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>s1_0_1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
<shard>
|
||||
<internal_replication>true</internal_replication>
|
||||
<replica>
|
||||
<host>s1_1_0</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</cluster1>
|
||||
|
||||
<shard_0_0>
|
||||
<shard>
|
||||
<internal_replication>true</internal_replication>
|
||||
<replica>
|
||||
<host>s0_0_0</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>s0_0_1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</shard_0_0>
|
||||
|
||||
<source_trivial_cluster>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>s0_0_0</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</source_trivial_cluster>
|
||||
|
||||
|
||||
<destination_trivial_cluster>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>s1_0_0</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</destination_trivial_cluster>
|
||||
|
||||
</remote_servers>
|
||||
<remote_servers>
|
||||
<cluster0>
|
||||
<shard>
|
||||
<internal_replication>true</internal_replication>
|
||||
<replica>
|
||||
<host>s0_0_0</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>s0_0_1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
<shard>
|
||||
<internal_replication>true</internal_replication>
|
||||
<replica>
|
||||
<host>s0_1_0</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</cluster0>
|
||||
<cluster1>
|
||||
<shard>
|
||||
<internal_replication>true</internal_replication>
|
||||
<replica>
|
||||
<host>s1_0_0</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>s1_0_1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
<shard>
|
||||
<internal_replication>true</internal_replication>
|
||||
<replica>
|
||||
<host>s1_1_0</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</cluster1>
|
||||
<shard_0_0>
|
||||
<shard>
|
||||
<internal_replication>true</internal_replication>
|
||||
<replica>
|
||||
<host>s0_0_0</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>s0_0_1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</shard_0_0>
|
||||
<source_trivial_cluster>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>s0_0_0</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</source_trivial_cluster>
|
||||
<destination_trivial_cluster>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>s1_0_0</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</destination_trivial_cluster>
|
||||
</remote_servers>
|
||||
</yandex>
|
||||
|
@ -54,7 +54,8 @@ def started_cluster():
|
||||
for replica_name in replicas:
|
||||
name = "s{}_{}_{}".format(cluster_name, shard_name, replica_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},
|
||||
with_zookeeper=True)
|
||||
|
||||
@ -226,6 +227,7 @@ def execute_task(task, cmd_options):
|
||||
zk.ensure_path(zk_task_path)
|
||||
zk.create(zk_task_path + "/description", task.copier_task_config)
|
||||
|
||||
|
||||
# Run cluster-copier processes on each node
|
||||
docker_api = docker.from_env().api
|
||||
copiers_exec_ids = []
|
||||
@ -241,9 +243,11 @@ def execute_task(task, cmd_options):
|
||||
for instance_name in copiers:
|
||||
instance = cluster.instances[instance_name]
|
||||
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)
|
||||
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)
|
||||
print "Copier for {} ({}) has started".format(instance.name, instance.ip_address)
|
||||
|
||||
|
@ -34,7 +34,7 @@ def started_cluster():
|
||||
for replica_name in replicas:
|
||||
name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name)
|
||||
cluster.add_instance(name,
|
||||
config_dir="configs",
|
||||
main_configs=[], user_configs=[],
|
||||
macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name},
|
||||
with_zookeeper=True)
|
||||
|
||||
|
@ -4,10 +4,9 @@ import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
config_dir = os.path.join(SCRIPT_DIR, './configs')
|
||||
|
||||
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 = ""
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
@ -19,4 +18,5 @@ def start_cluster():
|
||||
caught_exception = str(e)
|
||||
|
||||
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
|
||||
|
@ -2,7 +2,7 @@ import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
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)
|
||||
|
@ -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>
|
@ -0,0 +1,4 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
|
||||
</yandex>
|
@ -181,12 +181,18 @@ def setup_module(module):
|
||||
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))
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
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):
|
||||
main_configs.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)
|
||||
cluster.add_instance('clickhouse1')
|
||||
dictionaries.append(os.path.join(dict_configs_path, fname))
|
||||
|
||||
node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mysql=True, with_mongo=True, with_redis=True, with_cassandra=True)
|
||||
|
||||
|
||||
@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
|
||||
"""
|
||||
|
||||
global DICTIONARIES
|
||||
DICTIONARIES = [d for d in DICTIONARIES if not d.name.startswith("MySQL")]
|
||||
#global DICTIONARIES
|
||||
#DICTIONARIES = [d for d in DICTIONARIES if not d.name.startswith("MySQL")]
|
||||
|
||||
|
||||
@pytest.mark.parametrize("fold", list(range(10)))
|
||||
|
@ -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>
|
@ -5,13 +5,12 @@ from helpers.cluster import ClickHouseCluster
|
||||
@pytest.fixture(scope="function")
|
||||
def cluster(request):
|
||||
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:
|
||||
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":
|
||||
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()
|
||||
node.query("create table radars_table (radar_id String, radar_ip String, client_id String) engine=MergeTree() order by radar_id")
|
||||
|
||||
|
@ -0,0 +1,5 @@
|
||||
<yandex>
|
||||
<remote_url_allow_hosts>
|
||||
<host>node1</host>
|
||||
</remote_url_allow_hosts>
|
||||
</yandex>
|
@ -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>
|
@ -0,0 +1,4 @@
|
||||
<yandex>
|
||||
<dictionaries_lazy_load>false</dictionaries_lazy_load>
|
||||
</yandex>
|
||||
|
@ -0,0 +1,3 @@
|
||||
<yandex>
|
||||
<dictionaries_config>/etc/clickhouse-server/config.d/*dictionary.xml</dictionaries_config>
|
||||
</yandex>
|
@ -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>
|
@ -7,10 +7,10 @@ import warnings
|
||||
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
|
||||
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
|
||||
node1 = cluster.add_instance('node1', with_mysql=True, main_configs=['configs/dictionaries/simple_dictionary.xml'])
|
||||
node2 = cluster.add_instance('node2', with_mysql=True, main_configs=['configs/dictionaries/simple_dictionary.xml', 'configs/dictionaries/lazy_load.xml'])
|
||||
node3 = cluster.add_instance('node3', main_configs=['configs/dictionaries/dictionary_with_conflict_name.xml'])
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
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/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/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):
|
||||
@ -49,7 +49,7 @@ def started_cluster():
|
||||
(node2, 'complex_node2_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())'),
|
||||
(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)
|
||||
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))
|
||||
@ -93,8 +93,8 @@ def test_crete_and_select_mysql(started_cluster, clickhouse, name, layout):
|
||||
|
||||
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'
|
||||
stroka = clickhouse.query("SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)).strip()
|
||||
value = float(stroka)
|
||||
string = clickhouse.query("SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)).strip()
|
||||
value = float(string)
|
||||
assert int(value) == int(i * 2.718)
|
||||
|
||||
clickhouse.query("select dictGetUInt8('xml_dictionary', 'SomeValue1', toUInt64(17))") == "17\n"
|
||||
|
@ -0,0 +1,4 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<dictionaries_config>/etc/clickhouse-server/config.d/dep_*.xml</dictionaries_config>
|
||||
</yandex>
|
@ -3,11 +3,11 @@ import os
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
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']
|
||||
|
||||
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
|
||||
instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES)
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES,)
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
|
@ -0,0 +1,4 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<dictionaries_config>/etc/clickhouse-server/config.d/mysql_dict*.xml</dictionaries_config>
|
||||
</yandex>
|
@ -8,10 +8,9 @@ import pymysql.cursors
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
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']
|
||||
|
||||
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
|
||||
CONFIG_FILES += ['configs/enable_dictionaries.xml']
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance('instance', main_configs=CONFIG_FILES, with_mysql = True)
|
||||
|
||||
create_table_mysql_template = """
|
||||
|
@ -0,0 +1,4 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
|
||||
</yandex>
|
@ -3,11 +3,11 @@ import os
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
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']
|
||||
|
||||
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
|
||||
instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES)
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES)
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
|
@ -0,0 +1,4 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<dictionaries_config>/etc/clickhouse-server/config.d/dictionary*.xml</dictionaries_config>
|
||||
</yandex>
|
@ -19,12 +19,12 @@ def setup_module(module):
|
||||
structure = generate_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'))
|
||||
instance = cluster.add_instance('instance', main_configs=dictionary_files)
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
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'))
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
@pytest.fixture(scope="module", autouse=True)
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
|
@ -0,0 +1,4 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
|
||||
</yandex>
|
@ -6,10 +6,11 @@ from helpers.client import QueryTimeoutExceedException
|
||||
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/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'))
|
||||
instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES)
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance('instance', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES)
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
|
@ -0,0 +1,4 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
|
||||
</yandex>
|
@ -8,11 +8,11 @@ from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseKiller
|
||||
from helpers.network import PartitionManager
|
||||
|
||||
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)
|
||||
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")
|
||||
def started_cluster():
|
||||
|
@ -9,10 +9,10 @@ from helpers.cluster import ClickHouseCluster
|
||||
from helpers.test_tools import TSV
|
||||
|
||||
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)
|
||||
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):
|
||||
@ -26,7 +26,7 @@ def started_cluster():
|
||||
dictionary_node.query("CREATE DATABASE IF NOT EXISTS test;")
|
||||
dictionary_node.query("DROP TABLE IF EXISTS test.strings;")
|
||||
dictionary_node.query("""
|
||||
CREATE TABLE test.strings
|
||||
CREATE TABLE test.strings
|
||||
(key UInt64, value String)
|
||||
ENGINE = Memory;
|
||||
""")
|
||||
|
@ -9,11 +9,10 @@ from helpers.cluster import ClickHouseKiller
|
||||
from helpers.network import PartitionManager
|
||||
from helpers.network import PartitionManagerDisabler
|
||||
|
||||
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)
|
||||
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")
|
||||
def started_cluster():
|
||||
|
@ -8,11 +8,10 @@ from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseKiller
|
||||
from helpers.network import PartitionManager
|
||||
|
||||
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)
|
||||
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")
|
||||
def started_cluster():
|
||||
|
@ -0,0 +1,4 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
|
||||
</yandex>
|
@ -3,8 +3,7 @@ import pytest
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
config_dir = os.path.join(SCRIPT_DIR, './configs')
|
||||
ENABLE_DICT_CONFIG = ['configs/enable_dictionaries.xml']
|
||||
DICTIONARY_FILES = [
|
||||
'configs/dictionaries/FileSourceConfig.xml',
|
||||
'configs/dictionaries/ExecutableSourceConfig.xml',
|
||||
@ -13,8 +12,8 @@ DICTIONARY_FILES = [
|
||||
'configs/dictionaries/ClickHouseSourceConfig.xml'
|
||||
]
|
||||
|
||||
cluster = ClickHouseCluster(__file__, base_configs_dir=config_dir)
|
||||
instance = cluster.add_instance('node', main_configs=DICTIONARY_FILES, config_dir=config_dir)
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance('node', main_configs=ENABLE_DICT_CONFIG+DICTIONARY_FILES)
|
||||
|
||||
def prepare():
|
||||
node = instance
|
||||
|
@ -4,10 +4,10 @@ from helpers.cluster import ClickHouseCluster
|
||||
from helpers.client import QueryRuntimeException
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
ch1 = cluster.add_instance('ch1', config_dir="configs", with_zookeeper=True)
|
||||
ch2 = cluster.add_instance('ch2', config_dir="configs", with_zookeeper=True)
|
||||
ch3 = cluster.add_instance('ch3', config_dir="configs", with_zookeeper=True)
|
||||
ch4 = cluster.add_instance('ch4', 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', main_configs=["configs/config.d/clusters.xml", "configs/config.d/ddl.xml"], 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', main_configs=["configs/config.d/clusters.xml", "configs/config.d/ddl.xml"], with_zookeeper=True)
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
|
16
tests/integration/test_disk_types/configs/storage.xml
Normal file
16
tests/integration/test_disk_types/configs/storage.xml
Normal 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>
|
@ -1,3 +1,4 @@
|
||||
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
@ -12,7 +13,7 @@ disk_types = {
|
||||
def cluster():
|
||||
try:
|
||||
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()
|
||||
yield cluster
|
||||
finally:
|
||||
|
@ -17,10 +17,23 @@ class ClickHouseClusterWithDDLHelpers(ClickHouseCluster):
|
||||
|
||||
def prepare(self, replace_hostnames_with_ips=True):
|
||||
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):
|
||||
self.add_instance(
|
||||
'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},
|
||||
with_zookeeper=True)
|
||||
|
||||
|
@ -1,8 +1,9 @@
|
||||
<yandex>
|
||||
<openSSL>
|
||||
<server>
|
||||
<certificateFile>/etc/clickhouse-server/server.crt</certificateFile>
|
||||
<privateKeyFile>/etc/clickhouse-server/server.key</privateKeyFile>
|
||||
<certificateFile>/etc/clickhouse-server/config.d/server.crt</certificateFile>
|
||||
<privateKeyFile>/etc/clickhouse-server/config.d/server.key</privateKeyFile>
|
||||
<dhParamsFile>/etc/clickhouse-server/config.d/dhparam.pem</dhParamsFile>
|
||||
<verificationMode>none</verificationMode>
|
||||
<cacheSessions>true</cacheSessions>
|
||||
</server>
|
||||
|
@ -6,12 +6,12 @@ from helpers.test_tools import assert_eq_with_retry
|
||||
from helpers.client import QueryRuntimeException
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node1 = cluster.add_instance('node1', config_dir="configs", with_zookeeper=True)
|
||||
node2 = cluster.add_instance('node2', config_dir="configs", with_zookeeper=True)
|
||||
node3 = cluster.add_instance('node3', config_dir="configs", with_zookeeper=True)
|
||||
node4 = cluster.add_instance('node4', config_dir="configs", with_zookeeper=True)
|
||||
node5 = cluster.add_instance('node5', config_dir="configs", with_zookeeper=True)
|
||||
node6 = cluster.add_instance('node6', 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', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], 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', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], 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', main_configs=["configs/config.d/clusters.xml"], user_configs=["configs/users.d/default_with_password.xml"], with_zookeeper=True)
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
|
@ -9,7 +9,7 @@ from helpers.test_tools import assert_eq_with_retry
|
||||
|
||||
|
||||
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", [
|
||||
('test_cluster'),
|
||||
|
@ -1,8 +1,9 @@
|
||||
<yandex>
|
||||
<openSSL>
|
||||
<server>
|
||||
<certificateFile>/etc/clickhouse-server/server.crt</certificateFile>
|
||||
<privateKeyFile>/etc/clickhouse-server/server.key</privateKeyFile>
|
||||
<certificateFile>/etc/clickhouse-server/config.d/server.crt</certificateFile>
|
||||
<privateKeyFile>/etc/clickhouse-server/config.d/server.key</privateKeyFile>
|
||||
<dhParamsFile>/etc/clickhouse-server/config.d/dhparam.pem</dhParamsFile>
|
||||
<verificationMode>none</verificationMode>
|
||||
<cacheSessions>true</cacheSessions>
|
||||
</server>
|
||||
|
@ -1,6 +1,6 @@
|
||||
import itertools
|
||||
import timeit
|
||||
|
||||
import os.path
|
||||
import pytest
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
@ -91,8 +91,16 @@ def started_cluster(request):
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
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:
|
||||
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:
|
||||
cluster.start()
|
||||
|
||||
|
@ -9,7 +9,7 @@ from helpers.cluster import ClickHouseCluster
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
node = cluster.add_instance('node',
|
||||
config_dir='configs',
|
||||
main_configs=["configs/config.d/storage_configuration.xml"],
|
||||
tmpfs=['/disk1:size=100M', '/disk2:size=100M'])
|
||||
|
||||
@pytest.fixture(scope='module')
|
||||
|
@ -2,7 +2,7 @@ import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
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)
|
||||
def started_cluster():
|
||||
|
@ -12,8 +12,8 @@ from helpers.client import QueryTimeoutExceedException
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
node1 = cluster.add_instance('node1', config_dir='configs', 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})
|
||||
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', 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]
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
|
@ -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-----
|
@ -1,8 +1,9 @@
|
||||
<yandex>
|
||||
<openSSL>
|
||||
<server>
|
||||
<certificateFile>/etc/clickhouse-server/server.crt</certificateFile>
|
||||
<privateKeyFile>/etc/clickhouse-server/server.key</privateKeyFile>
|
||||
<certificateFile>/etc/clickhouse-server/config.d/server.crt</certificateFile>
|
||||
<privateKeyFile>/etc/clickhouse-server/config.d/server.key</privateKeyFile>
|
||||
<dhParamsFile>/etc/clickhouse-server/config.d/dhparam.pem</dhParamsFile>
|
||||
<verificationMode>none</verificationMode>
|
||||
<cacheSessions>true</cacheSessions>
|
||||
</server>
|
||||
@ -15,4 +16,5 @@
|
||||
</client>
|
||||
</openSSL>
|
||||
<interserver_https_port>9010</interserver_https_port>
|
||||
<interserver_http_port remove="1"/>
|
||||
</yandex>
|
||||
|
@ -23,8 +23,8 @@ def _fill_nodes(nodes, shard):
|
||||
'''.format(shard=shard, replica=node.name))
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node1 = cluster.add_instance('node1', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/ssl_conf.xml'], with_zookeeper=True)
|
||||
node2 = cluster.add_instance('node2', 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', 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")
|
||||
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)
|
||||
node4 = cluster.add_instance('node4', 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', main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True)
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
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(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)
|
||||
node6 = cluster.add_instance('node6', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/no_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', main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True)
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def mixed_protocol_cluster():
|
||||
|
13
tests/integration/test_log_family_s3/configs/minio.xml
Normal file
13
tests/integration/test_log_family_s3/configs/minio.xml
Normal 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>
|
12
tests/integration/test_log_family_s3/configs/ssl.xml
Normal file
12
tests/integration/test_log_family_s3/configs/ssl.xml
Normal 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>
|
@ -11,7 +11,7 @@ logging.getLogger().addHandler(logging.StreamHandler())
|
||||
def cluster():
|
||||
try:
|
||||
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...")
|
||||
cluster.start()
|
||||
logging.info("Cluster started")
|
||||
|
@ -22,8 +22,8 @@ def _fill_nodes(nodes, shard, connections_count):
|
||||
'''.format(shard=shard, replica=node.name, connections=connections_count))
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node1 = cluster.add_instance('node1', config_dir="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)
|
||||
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', user_configs=[], main_configs=['configs/remote_servers.xml', 'configs/log_conf.xml'], with_zookeeper=True)
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
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"
|
||||
|
||||
node3 = cluster.add_instance('node3', config_dir="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)
|
||||
node5 = cluster.add_instance('node5', 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', user_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")
|
||||
def start_big_cluster():
|
||||
|
@ -14,7 +14,7 @@ logging.getLogger().addHandler(logging.StreamHandler())
|
||||
def cluster():
|
||||
try:
|
||||
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...")
|
||||
cluster.start()
|
||||
logging.info("Cluster started")
|
||||
@ -55,7 +55,7 @@ def create_table(cluster, table_name, additional_settings=None):
|
||||
ORDER BY (dt, id)
|
||||
SETTINGS
|
||||
storage_policy='s3',
|
||||
old_parts_lifetime=0,
|
||||
old_parts_lifetime=0,
|
||||
index_granularity=512
|
||||
""".format(table_name)
|
||||
|
||||
|
@ -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>
|
@ -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>
|
@ -11,7 +11,9 @@ logging.getLogger().addHandler(logging.StreamHandler())
|
||||
def cluster():
|
||||
try:
|
||||
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...")
|
||||
cluster.start()
|
||||
logging.info("Cluster started")
|
||||
|
@ -13,16 +13,14 @@ from helpers.test_tools import TSV
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
node1 = cluster.add_instance('node1',
|
||||
config_dir='configs',
|
||||
main_configs=['configs/logs_config.xml'],
|
||||
main_configs=['configs/logs_config.xml', 'configs/config.d/storage_configuration.xml', 'configs/config.d/cluster.xml'],
|
||||
with_zookeeper=True,
|
||||
stay_alive=True,
|
||||
tmpfs=['/jbod1:size=40M', '/jbod2:size=40M', '/external:size=200M'],
|
||||
macros={"shard": 0, "replica": 1} )
|
||||
|
||||
node2 = cluster.add_instance('node2',
|
||||
config_dir='configs',
|
||||
main_configs=['configs/logs_config.xml'],
|
||||
main_configs=['configs/logs_config.xml', 'configs/config.d/storage_configuration.xml', 'configs/config.d/cluster.xml'],
|
||||
with_zookeeper=True,
|
||||
stay_alive=True,
|
||||
tmpfs=['/jbod1:size=40M', '/jbod2:size=40M', '/external:size=200M'],
|
||||
|
@ -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 pytest.raises(QueryRuntimeException) as exception:
|
||||
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)
|
||||
mysql_node.query("DROP DATABASE test_bad_arguments")
|
||||
|
10
tests/integration/test_mysql_protocol/configs/log_conf.xml
Normal file
10
tests/integration/test_mysql_protocol/configs/log_conf.xml
Normal 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>
|
4
tests/integration/test_mysql_protocol/configs/mysql.xml
Normal file
4
tests/integration/test_mysql_protocol/configs/mysql.xml
Normal file
@ -0,0 +1,4 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<mysql_port>9001</mysql_port>
|
||||
</yandex>
|
18
tests/integration/test_mysql_protocol/configs/ssl_conf.xml
Normal file
18
tests/integration/test_mysql_protocol/configs/ssl_conf.xml
Normal 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>
|
@ -17,9 +17,10 @@ from helpers.cluster import ClickHouseCluster, get_docker_compose_path
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
DOCKER_COMPOSE_PATH = get_docker_compose_path()
|
||||
|
||||
config_dir = os.path.join(SCRIPT_DIR, './configs')
|
||||
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
|
||||
|
||||
@ -36,7 +37,7 @@ def server_address():
|
||||
@pytest.fixture(scope='module')
|
||||
def mysql_client():
|
||||
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')
|
||||
|
||||
|
||||
@ -62,28 +63,28 @@ def mysql_server(mysql_client):
|
||||
@pytest.fixture(scope='module')
|
||||
def golang_container():
|
||||
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')
|
||||
|
||||
|
||||
@pytest.fixture(scope='module')
|
||||
def php_container():
|
||||
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')
|
||||
|
||||
|
||||
@pytest.fixture(scope='module')
|
||||
def nodejs_container():
|
||||
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')
|
||||
|
||||
|
||||
@pytest.fixture(scope='module')
|
||||
def java_container():
|
||||
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')
|
||||
|
||||
|
||||
@ -329,7 +330,7 @@ def test_python_client(server_address):
|
||||
|
||||
def test_golang_client(server_address, golang_container):
|
||||
# type: (str, Container) -> None
|
||||
with open(os.path.join(SCRIPT_DIR,'golang.reference')) as fp:
|
||||
with open(os.path.join(SCRIPT_DIR, 'golang.reference')) as fp:
|
||||
reference = fp.read()
|
||||
|
||||
code, (stdout, stderr) = golang_container.exec_run('./main --host {host} --port {port} --user default --password 123 --database '
|
||||
|
@ -0,0 +1,4 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<dictionaries_config>/etc/clickhouse-server/config.d/*dictionary.xml</dictionaries_config>
|
||||
</yandex>
|
@ -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>
|
12
tests/integration/test_odbc_interaction/configs/openssl.xml
Normal file
12
tests/integration/test_odbc_interaction/configs/openssl.xml
Normal 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>
|
@ -7,10 +7,9 @@ import psycopg2
|
||||
from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT
|
||||
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'))
|
||||
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)
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
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 `clickhouse`.`{}` (
|
||||
|
@ -1,3 +1,4 @@
|
||||
|
||||
import time
|
||||
import os
|
||||
import pytest
|
||||
@ -9,13 +10,13 @@ from helpers.test_tools import assert_eq_with_retry
|
||||
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node18_14 = cluster.add_instance('node18_14', image='yandex/clickhouse-server:18.14.19', 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, config_dir="configs")
|
||||
node19_4 = cluster.add_instance('node19_4', image='yandex/clickhouse-server:19.4.5.35', 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, config_dir="configs")
|
||||
node19_11 = cluster.add_instance('node19_11', image='yandex/clickhouse-server:19.11.13.74', 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, config_dir="configs")
|
||||
node19_16 = cluster.add_instance('node19_16', image='yandex/clickhouse-server:19.16.2.2', 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, 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, 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, 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, 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, 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, main_configs=["configs/config.d/test_cluster.xml"])
|
||||
old_nodes = [node18_14, node19_1, node19_4, node19_8, node19_11, node19_13, node19_16]
|
||||
new_node = cluster.add_instance('node_new')
|
||||
|
||||
|
@ -53,21 +53,21 @@ def create_tables_old_format(name, nodes, shard):
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/{name}', '{repl}', date, id, 64)
|
||||
'''.format(name=name, shard=shard, repl=i))
|
||||
|
||||
node1 = cluster.add_instance('node1', config_dir="configs", with_zookeeper=True)
|
||||
node2 = cluster.add_instance('node2', 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', 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_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}
|
||||
|
||||
node3 = cluster.add_instance('node3', config_dir="configs", with_zookeeper=True)
|
||||
node4 = cluster.add_instance('node4', config_dir="configs", main_configs=['configs/no_leader.xml'], 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', 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_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)
|
||||
node6 = cluster.add_instance('node6', 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', 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}
|
||||
|
||||
@ -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)
|
||||
|
||||
|
||||
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)
|
||||
node8 = cluster.add_instance('node8', config_dir="configs", with_zookeeper=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', main_configs=[], user_configs=["configs/users.d/not_optimize_count.xml"], with_zookeeper=True)
|
||||
|
||||
settings7 = {'index_granularity_bytes' : 10485760}
|
||||
settings8 = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0}
|
||||
|
@ -0,0 +1,13 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<profiles>
|
||||
<default>
|
||||
</default>
|
||||
</profiles>
|
||||
|
||||
<users>
|
||||
<default>
|
||||
<password>123</password>
|
||||
</default>
|
||||
</users>
|
||||
</yandex>
|
10
tests/integration/test_postgresql_protocol/configs/log.xml
Normal file
10
tests/integration/test_postgresql_protocol/configs/log.xml
Normal 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>
|
@ -0,0 +1,4 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<postgresql_port>5433</postgresql_port>
|
||||
</yandex>
|
@ -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>
|
@ -19,11 +19,12 @@ from helpers.cluster import ClickHouseCluster, get_docker_compose_path
|
||||
psycopg2.extras.register_uuid()
|
||||
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
config_dir = os.path.join(SCRIPT_DIR, './configs')
|
||||
DOCKER_COMPOSE_PATH = get_docker_compose_path()
|
||||
|
||||
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
|
||||
|
||||
|
10
tests/integration/test_profile_events_s3/configs/log.xml
Normal file
10
tests/integration/test_profile_events_s3/configs/log.xml
Normal 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>
|
@ -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>
|
@ -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>
|
@ -17,7 +17,7 @@ def cluster():
|
||||
try:
|
||||
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...")
|
||||
cluster.start()
|
||||
|
@ -7,18 +7,15 @@ from helpers.cluster import ClickHouseCluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
zero = cluster.add_instance("zero",
|
||||
config_dir="configs",
|
||||
zero = cluster.add_instance("zero", user_configs=["configs/users.d/settings.xml"],
|
||||
macros={"cluster": "anime", "shard": "0", "replica": "zero"},
|
||||
with_zookeeper=True)
|
||||
|
||||
first = cluster.add_instance("first",
|
||||
config_dir="configs",
|
||||
first = cluster.add_instance("first", user_configs=["configs/users.d/settings.xml"],
|
||||
macros={"cluster": "anime", "shard": "0", "replica": "first"},
|
||||
with_zookeeper=True)
|
||||
|
||||
second = cluster.add_instance("second",
|
||||
config_dir="configs",
|
||||
second = cluster.add_instance("second", user_configs=["configs/users.d/settings.xml"],
|
||||
macros={"cluster": "anime", "shard": "0", "replica": "second"},
|
||||
with_zookeeper=True)
|
||||
|
||||
|
@ -6,28 +6,38 @@ import re
|
||||
import time
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance('instance',
|
||||
config_dir="configs")
|
||||
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"])
|
||||
|
||||
|
||||
def system_quotas():
|
||||
return TSV(instance.query("SELECT * FROM system.quotas ORDER BY name"))
|
||||
def check_system_quotas(canonical):
|
||||
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():
|
||||
return TSV(instance.query("SELECT * FROM system.quota_limits ORDER BY quota_name, duration"))
|
||||
def system_quota_limits(canonical):
|
||||
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,"\
|
||||
"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"
|
||||
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, "\
|
||||
"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"
|
||||
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):
|
||||
script_dir = os.path.dirname(os.path.realpath(__file__))
|
||||
@ -40,7 +50,7 @@ def copy_quota_xml(local_file_name, reload_immediately = True):
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
|
||||
instance.query("CREATE TABLE test_table(x UInt32) ENGINE = MergeTree ORDER BY tuple()")
|
||||
instance.query("INSERT INTO test_table SELECT number FROM numbers(50)")
|
||||
|
||||
@ -61,141 +71,141 @@ def reset_quotas_and_usage_info():
|
||||
|
||||
|
||||
def test_quota_from_users_xml():
|
||||
assert 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"]]
|
||||
assert 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"]]
|
||||
check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]])
|
||||
system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]])
|
||||
system_quota_usage([["myQuota", "default", 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")
|
||||
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")
|
||||
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():
|
||||
# Simpliest quota doesn't even track usage.
|
||||
copy_quota_xml('simpliest.xml')
|
||||
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]]
|
||||
assert system_quota_limits() == ""
|
||||
assert system_quota_usage() == [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]
|
||||
check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]])
|
||||
system_quota_limits("")
|
||||
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")
|
||||
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():
|
||||
# Now we're tracking usage.
|
||||
copy_quota_xml('tracking.xml')
|
||||
assert 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"]]
|
||||
assert system_quota_usage() == [["myQuota", "default", 31556952, 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", "\N"]]
|
||||
check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]])
|
||||
system_quota_limits([["myQuota", 31556952, 0, "\N", "\N", "\N", "\N", "\N", "\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")
|
||||
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")
|
||||
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():
|
||||
# Change quota, now the limits are tiny so we will exceed the quota.
|
||||
copy_quota_xml('tiny_limits.xml')
|
||||
assert 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"]]
|
||||
assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1, 0, 1, 0, 1, 0, "\N", 0, 1, 0, "\N", "\N"]]
|
||||
check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]])
|
||||
system_quota_limits([["myQuota", 31556952, 0, 1, 1, 1, "\N", 1, "\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 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.
|
||||
copy_quota_xml('normal_limits.xml')
|
||||
assert 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"]]
|
||||
assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1000, 1, "\N", 0, "\N", 0, "\N", 50, 1000, 0, "\N", "\N"]]
|
||||
|
||||
check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]])
|
||||
system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\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")
|
||||
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():
|
||||
assert 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"]]
|
||||
assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]
|
||||
check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]])
|
||||
system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]])
|
||||
system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]])
|
||||
|
||||
# Add interval.
|
||||
copy_quota_xml('two_intervals.xml')
|
||||
assert 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"],
|
||||
["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"],
|
||||
["myQuota", "default", 63113904, 0, "\N", 0, "\N", 0, "\N", 0, 30000, 0, "\N", 0, 20000, 120]]
|
||||
|
||||
check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952,63113904]", 0, "['default']", "[]"]])
|
||||
system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"],
|
||||
["myQuota", 63113904, 1, "\N", "\N", "\N", 30000, "\N", 20000, 120]])
|
||||
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]])
|
||||
|
||||
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"],
|
||||
["myQuota", "default", 63113904, 1, "\N", 0, "\N", 50, "\N", 200, 30000, 50, "\N", 200, 20000, 120]]
|
||||
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]])
|
||||
|
||||
# Remove interval.
|
||||
copy_quota_xml('normal_limits.xml')
|
||||
assert 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"]]
|
||||
assert system_quota_usage() == [["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"]]
|
||||
|
||||
check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]])
|
||||
system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\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")
|
||||
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.
|
||||
copy_quota_xml('simpliest.xml')
|
||||
assert system_quotas() == [["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]]
|
||||
assert system_quota_limits() == ""
|
||||
assert system_quota_usage() == [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]
|
||||
|
||||
check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[]", 0, "['default']", "[]"]])
|
||||
system_quota_limits("")
|
||||
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")
|
||||
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.
|
||||
copy_quota_xml('normal_limits.xml')
|
||||
assert 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"]]
|
||||
assert system_quota_usage() == [["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]
|
||||
check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]])
|
||||
system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\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():
|
||||
assert 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"]]
|
||||
assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]
|
||||
check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]])
|
||||
system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]])
|
||||
system_quotas_usage([["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]])
|
||||
|
||||
# Add quota.
|
||||
copy_quota_xml('two_quotas.xml')
|
||||
assert 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, "[]", "[]"]]
|
||||
assert system_quota_limits() == [["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"],
|
||||
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, "[]", "[]"]])
|
||||
system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"],
|
||||
["myQuota2", 3600, 1, "\N", "\N", 4000, 400000, 4000, 400000, 60],
|
||||
["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"]]
|
||||
["myQuota2", 2629746, 0, "\N", "\N", "\N", "\N", "\N", "\N", 1800]])
|
||||
system_quotas_usage([["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]])
|
||||
|
||||
# Drop quota.
|
||||
copy_quota_xml('normal_limits.xml')
|
||||
assert 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"]]
|
||||
assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]
|
||||
check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]])
|
||||
system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\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.
|
||||
copy_quota_xml('no_quotas.xml')
|
||||
assert system_quotas() == ""
|
||||
assert system_quota_limits() == ""
|
||||
assert system_quotas_usage() == ""
|
||||
check_system_quotas("")
|
||||
system_quota_limits("")
|
||||
system_quotas_usage("")
|
||||
|
||||
# Add one quota back.
|
||||
copy_quota_xml('normal_limits.xml')
|
||||
assert 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"]]
|
||||
assert system_quotas_usage() == [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]
|
||||
check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]])
|
||||
system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\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():
|
||||
assert 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"]]
|
||||
check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]])
|
||||
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,
|
||||
# because config files are reload by timer only when the modification time is changed.
|
||||
@ -246,7 +256,7 @@ def test_dcl_introspection():
|
||||
def test_dcl_management():
|
||||
copy_quota_xml('no_quotas.xml')
|
||||
assert instance.query("SHOW QUOTA") == ""
|
||||
|
||||
|
||||
instance.query("CREATE QUOTA qA FOR INTERVAL 15 MONTH MAX QUERIES 123 TO CURRENT_USER")
|
||||
assert instance.query("SHOW CREATE QUOTA qA") == "CREATE QUOTA qA FOR INTERVAL 5 quarter MAX queries = 123 TO default\n"
|
||||
assert re.match("qA\\t\\t.*\\t39446190\\t0\\t123\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t0\\t\\\\N\\t.*\\t\\\\N\n",
|
||||
|
@ -14,8 +14,8 @@ from helpers.client import CommandRequest
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
node1 = cluster.add_instance('node1', config_dir='configs', 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})
|
||||
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', 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]
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
|
@ -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>
|
@ -1,3 +1,4 @@
|
||||
|
||||
import time
|
||||
import pytest
|
||||
import os
|
||||
@ -6,10 +7,10 @@ from helpers.cluster import ClickHouseCluster
|
||||
|
||||
|
||||
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__))
|
||||
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")
|
||||
|
@ -12,8 +12,9 @@ from helpers.test_tools import TSV
|
||||
|
||||
node_options = dict(
|
||||
with_zookeeper=True,
|
||||
main_configs=['configs/remote_servers.xml'],
|
||||
config_dir='configs',
|
||||
main_configs=["configs/remote_servers.xml", "configs/config.d/instant_moves.xml",
|
||||
"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'])
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
@ -15,9 +15,9 @@ def cluster():
|
||||
try:
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
cluster.add_instance("node1", config_dir="configs", 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("node3", config_dir="configs", macros={'cluster': 'test1'}, 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", main_configs=["configs/config.d/storage_conf.xml"], 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...")
|
||||
cluster.start()
|
||||
|
@ -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>
|
@ -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>
|
@ -6,8 +6,8 @@ import re
|
||||
import time
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
node = cluster.add_instance('node', config_dir="configs", with_zookeeper=True)
|
||||
node2 = cluster.add_instance('node2', 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', 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]
|
||||
|
||||
|
||||
@ -42,7 +42,7 @@ def started_cluster():
|
||||
|
||||
CREATE TABLE mydb.`.filtered_table4` (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a;
|
||||
INSERT INTO mydb.`.filtered_table4` values (0, 0), (0, 1), (1, 0), (1, 1);
|
||||
|
||||
|
||||
CREATE TABLE mydb.local (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a;
|
||||
''')
|
||||
|
||||
@ -185,7 +185,7 @@ def test_introspection():
|
||||
|
||||
def test_dcl_introspection():
|
||||
assert node.query("SHOW POLICIES") == TSV(["another ON mydb.filtered_table1", "another ON mydb.filtered_table2", "another ON mydb.filtered_table3", "another ON mydb.local", "default ON mydb.filtered_table1", "default ON mydb.filtered_table2", "default ON mydb.filtered_table3", "default ON mydb.local"])
|
||||
|
||||
|
||||
assert node.query("SHOW POLICIES ON mydb.filtered_table1") == TSV([ "another", "default" ])
|
||||
assert node.query("SHOW POLICIES ON mydb.local") == TSV([ "another", "default" ])
|
||||
assert node.query("SHOW POLICIES ON mydb.*") == TSV([ "another ON mydb.filtered_table1", "another ON mydb.filtered_table2", "another ON mydb.filtered_table3", "another ON mydb.local", "default ON mydb.filtered_table1", "default ON mydb.filtered_table2", "default ON mydb.filtered_table3", "default ON mydb.local" ])
|
||||
@ -195,7 +195,7 @@ def test_dcl_introspection():
|
||||
assert node.query("SHOW CREATE POLICY default ON mydb.filtered_table2") == "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default\n"
|
||||
assert node.query("SHOW CREATE POLICY default ON mydb.filtered_table3") == "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default\n"
|
||||
assert node.query("SHOW CREATE POLICY default ON mydb.local") == "CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default\n"
|
||||
|
||||
|
||||
assert node.query("SHOW CREATE POLICY default") == TSV([ "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default", "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default", "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default", "CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default" ])
|
||||
assert node.query("SHOW CREATE POLICIES ON mydb.filtered_table1") == TSV([ "CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another", "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default" ])
|
||||
assert node.query("SHOW CREATE POLICIES ON mydb.*") == TSV([ "CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another", "CREATE ROW POLICY another ON mydb.filtered_table2 FOR SELECT USING 1 TO another", "CREATE ROW POLICY another ON mydb.filtered_table3 FOR SELECT USING 1 TO another", "CREATE ROW POLICY another ON mydb.local FOR SELECT USING a = 1 TO another", "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default", "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default", "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default", "CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default" ])
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user