This commit is contained in:
Yatsishin Ilya 2021-04-13 13:52:22 +03:00
parent db657c03c8
commit e4d1ea846c
6 changed files with 216 additions and 215 deletions

View File

@ -269,14 +269,13 @@ class ClickhouseIntegrationTestsRunner:
return list(sorted(all_tests)) return list(sorted(all_tests))
def _get_parallel_tests(self, repo_path): def _get_parallel_tests(self, repo_path):
parallel_tests_file_path = "{}/tests/integration/parallel.txt".format(repo_path) parallel_tests_file_path = "{}/tests/integration/parallel.json".format(repo_path)
if not os.path.isfile(parallel_tests_file_path) or os.path.getsize(parallel_tests_file_path) == 0: if not os.path.isfile(parallel_tests_file_path) or os.path.getsize(parallel_tests_file_path) == 0:
raise Exception("There is something wrong with getting all tests list: file '{}' is empty or does not exist.".format(parallel_tests_file_path)) raise Exception("There is something wrong with getting all tests list: file '{}' is empty or does not exist.".format(parallel_tests_file_path))
parallel_tests = [] parallel_tests = []
with open(parallel_tests_file_path, "r") as parallel_tests_file: with open(parallel_tests_file_path, "r") as parallel_tests_file:
for line in parallel_tests_file: parallel_tests = json.load(parallel_tests_file)
parallel_tests.append(line.strip())
return list(sorted(parallel_tests)) return list(sorted(parallel_tests))
def group_test_by_file(self, tests): def group_test_by_file(self, tests):

View File

@ -706,7 +706,7 @@ class ClickHouseCluster:
["bash", "-c", "echo {} | base64 --decode > {}".format(encodedStr, dest_path)], ["bash", "-c", "echo {} | base64 --decode > {}".format(encodedStr, dest_path)],
user='root') user='root')
def wait_mysql_to_start(self, timeout=60): def wait_mysql_to_start(self, timeout=180):
self.mysql_ip = self.get_instance_ip('mysql57') self.mysql_ip = self.get_instance_ip('mysql57')
start = time.time() start = time.time()
errors = [] errors = []
@ -740,7 +740,7 @@ class ClickHouseCluster:
subprocess_call(['docker-compose', 'ps', '--services', '--all']) subprocess_call(['docker-compose', 'ps', '--services', '--all'])
raise Exception("Cannot wait MySQL 8 container") raise Exception("Cannot wait MySQL 8 container")
def wait_mysql_cluster_to_start(self, timeout=60): def wait_mysql_cluster_to_start(self, timeout=180):
self.mysql2_ip = self.get_instance_ip(self.mysql2_host) self.mysql2_ip = self.get_instance_ip(self.mysql2_host)
self.mysql3_ip = self.get_instance_ip(self.mysql3_host) self.mysql3_ip = self.get_instance_ip(self.mysql3_host)
self.mysql4_ip = self.get_instance_ip(self.mysql4_host) self.mysql4_ip = self.get_instance_ip(self.mysql4_host)
@ -761,7 +761,7 @@ class ClickHouseCluster:
logging.error("Can't connect to MySQL:{}".format(errors)) logging.error("Can't connect to MySQL:{}".format(errors))
raise Exception("Cannot wait MySQL container") raise Exception("Cannot wait MySQL container")
def wait_postgres_to_start(self, timeout=60): def wait_postgres_to_start(self, timeout=180):
self.postgres_ip = self.get_instance_ip(self.postgres_host) self.postgres_ip = self.get_instance_ip(self.postgres_host)
start = time.time() start = time.time()
while time.time() - start < timeout: while time.time() - start < timeout:
@ -776,7 +776,7 @@ class ClickHouseCluster:
raise Exception("Cannot wait Postgres container") raise Exception("Cannot wait Postgres container")
def wait_postgres_cluster_to_start(self, timeout=60): def wait_postgres_cluster_to_start(self, timeout=180):
self.postgres2_ip = self.get_instance_ip(self.postgres2_host) self.postgres2_ip = self.get_instance_ip(self.postgres2_host)
self.postgres3_ip = self.get_instance_ip(self.postgres3_host) self.postgres3_ip = self.get_instance_ip(self.postgres3_host)
self.postgres4_ip = self.get_instance_ip(self.postgres4_host) self.postgres4_ip = self.get_instance_ip(self.postgres4_host)
@ -794,7 +794,7 @@ class ClickHouseCluster:
raise Exception("Cannot wait Postgres container") raise Exception("Cannot wait Postgres container")
def wait_zookeeper_to_start(self, timeout=60): def wait_zookeeper_to_start(self, timeout=180):
start = time.time() start = time.time()
while time.time() - start < timeout: while time.time() - start < timeout:
try: try:
@ -848,7 +848,7 @@ class ClickHouseCluster:
time.sleep(1) time.sleep(1)
def wait_hdfs_to_start(self, hdfs_api, timeout=60): def wait_hdfs_to_start(self, hdfs_api, timeout=300):
start = time.time() start = time.time()
while time.time() - start < timeout: while time.time() - start < timeout:
try: try:
@ -861,7 +861,7 @@ class ClickHouseCluster:
raise Exception("Can't wait HDFS to start") raise Exception("Can't wait HDFS to start")
def wait_mongo_to_start(self, timeout=30): def wait_mongo_to_start(self, timeout=180):
connection_str = 'mongodb://{user}:{password}@{host}:{port}'.format( connection_str = 'mongodb://{user}:{password}@{host}:{port}'.format(
host='localhost', port=self.mongo_port, user='root', password='clickhouse') host='localhost', port=self.mongo_port, user='root', password='clickhouse')
connection = pymongo.MongoClient(connection_str) connection = pymongo.MongoClient(connection_str)
@ -875,7 +875,7 @@ class ClickHouseCluster:
logging.debug("Can't connect to Mongo " + str(ex)) logging.debug("Can't connect to Mongo " + str(ex))
time.sleep(1) time.sleep(1)
def wait_minio_to_start(self, timeout=120, secure=False): def wait_minio_to_start(self, timeout=180, secure=False):
os.environ['SSL_CERT_FILE'] = p.join(self.base_dir, self.minio_dir, 'certs', 'public.crt') os.environ['SSL_CERT_FILE'] = p.join(self.base_dir, self.minio_dir, 'certs', 'public.crt')
minio_client = Minio('localhost:{}'.format(self.minio_port), minio_client = Minio('localhost:{}'.format(self.minio_port),
access_key='minio', access_key='minio',
@ -904,7 +904,7 @@ class ClickHouseCluster:
raise Exception("Can't wait Minio to start") raise Exception("Can't wait Minio to start")
def wait_schema_registry_to_start(self, timeout=10): def wait_schema_registry_to_start(self, timeout=180):
sr_client = CachedSchemaRegistryClient({"url":'http://localhost:{}'.format(self.schema_registry_port)}) sr_client = CachedSchemaRegistryClient({"url":'http://localhost:{}'.format(self.schema_registry_port)})
start = time.time() start = time.time()
while time.time() - start < timeout: while time.time() - start < timeout:
@ -919,7 +919,7 @@ class ClickHouseCluster:
raise Exception("Can't wait Schema Registry to start") raise Exception("Can't wait Schema Registry to start")
def wait_cassandra_to_start(self, timeout=120): def wait_cassandra_to_start(self, timeout=180):
self.cassandra_ip = self.get_instance_ip(self.cassandra_host) self.cassandra_ip = self.get_instance_ip(self.cassandra_host)
cass_client = cassandra.cluster.Cluster([self.cassandra_ip], port=self.cassandra_port, load_balancing_policy=RoundRobinPolicy()) cass_client = cassandra.cluster.Cluster([self.cassandra_ip], port=self.cassandra_port, load_balancing_policy=RoundRobinPolicy())
start = time.time() start = time.time()
@ -994,7 +994,7 @@ class ClickHouseCluster:
run_and_check(self.base_zookeeper_cmd + common_opts, env=env) run_and_check(self.base_zookeeper_cmd + common_opts, env=env)
for command in self.pre_zookeeper_commands: for command in self.pre_zookeeper_commands:
self.run_kazoo_commands_with_retries(command, repeats=5) self.run_kazoo_commands_with_retries(command, repeats=5)
self.wait_zookeeper_to_start(120) self.wait_zookeeper_to_start()
if self.with_mysql and self.base_mysql_cmd: if self.with_mysql and self.base_mysql_cmd:
logging.debug('Setup MySQL') logging.debug('Setup MySQL')
@ -1003,7 +1003,7 @@ class ClickHouseCluster:
os.makedirs(self.mysql_logs_dir) os.makedirs(self.mysql_logs_dir)
os.chmod(self.mysql_logs_dir, stat.S_IRWXO) os.chmod(self.mysql_logs_dir, stat.S_IRWXO)
subprocess_check_call(self.base_mysql_cmd + common_opts) subprocess_check_call(self.base_mysql_cmd + common_opts)
self.wait_mysql_to_start(180) self.wait_mysql_to_start()
if self.with_mysql8 and self.base_mysql8_cmd: if self.with_mysql8 and self.base_mysql8_cmd:
logging.debug('Setup MySQL 8') logging.debug('Setup MySQL 8')
@ -1012,7 +1012,7 @@ class ClickHouseCluster:
os.makedirs(self.mysql8_logs_dir) os.makedirs(self.mysql8_logs_dir)
os.chmod(self.mysql8_logs_dir, stat.S_IRWXO) os.chmod(self.mysql8_logs_dir, stat.S_IRWXO)
subprocess_check_call(self.base_mysql8_cmd + common_opts) subprocess_check_call(self.base_mysql8_cmd + common_opts)
self.wait_mysql8_to_start(180) self.wait_mysql8_to_start()
if self.with_mysql_cluster and self.base_mysql_cluster_cmd: if self.with_mysql_cluster and self.base_mysql_cluster_cmd:
print('Setup MySQL') print('Setup MySQL')
@ -1022,7 +1022,7 @@ class ClickHouseCluster:
os.chmod(self.mysql_cluster_logs_dir, stat.S_IRWXO) os.chmod(self.mysql_cluster_logs_dir, stat.S_IRWXO)
subprocess_check_call(self.base_mysql_cluster_cmd + common_opts) subprocess_check_call(self.base_mysql_cluster_cmd + common_opts)
self.wait_mysql_cluster_to_start(120) self.wait_mysql_cluster_to_start()
if self.with_postgres and self.base_postgres_cmd: if self.with_postgres and self.base_postgres_cmd:
logging.debug('Setup Postgres') logging.debug('Setup Postgres')
@ -1032,7 +1032,7 @@ class ClickHouseCluster:
os.chmod(self.postgres_logs_dir, stat.S_IRWXO) os.chmod(self.postgres_logs_dir, stat.S_IRWXO)
subprocess_check_call(self.base_postgres_cmd + common_opts) subprocess_check_call(self.base_postgres_cmd + common_opts)
self.wait_postgres_to_start(120) self.wait_postgres_to_start()
if self.with_postgres_cluster and self.base_postgres_cluster_cmd: if self.with_postgres_cluster and self.base_postgres_cluster_cmd:
print('Setup Postgres') print('Setup Postgres')
@ -1043,13 +1043,13 @@ class ClickHouseCluster:
os.makedirs(self.postgres4_logs_dir) os.makedirs(self.postgres4_logs_dir)
os.chmod(self.postgres4_logs_dir, stat.S_IRWXO) os.chmod(self.postgres4_logs_dir, stat.S_IRWXO)
subprocess_check_call(self.base_postgres_cluster_cmd + common_opts) subprocess_check_call(self.base_postgres_cluster_cmd + common_opts)
self.wait_postgres_cluster_to_start(120) self.wait_postgres_cluster_to_start()
if self.with_kafka and self.base_kafka_cmd: if self.with_kafka and self.base_kafka_cmd:
logging.debug('Setup Kafka') logging.debug('Setup Kafka')
subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes']) subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes'])
self.wait_kafka_is_available(self.kafka_docker_id, self.kafka_port) self.wait_kafka_is_available(self.kafka_docker_id, self.kafka_port)
self.wait_schema_registry_to_start(30) self.wait_schema_registry_to_start()
if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd:
logging.debug('Setup kerberized kafka') logging.debug('Setup kerberized kafka')
@ -1066,7 +1066,7 @@ class ClickHouseCluster:
os.chmod(self.hdfs_logs_dir, stat.S_IRWXO) os.chmod(self.hdfs_logs_dir, stat.S_IRWXO)
subprocess_check_call(self.base_hdfs_cmd + common_opts) subprocess_check_call(self.base_hdfs_cmd + common_opts)
hdfs_api = self.make_hdfs_api() hdfs_api = self.make_hdfs_api()
self.wait_hdfs_to_start(hdfs_api, 300) self.wait_hdfs_to_start(hdfs_api)
if self.with_kerberized_hdfs and self.base_kerberized_hdfs_cmd: if self.with_kerberized_hdfs and self.base_kerberized_hdfs_cmd:
logging.debug('Setup kerberized HDFS') logging.debug('Setup kerberized HDFS')
@ -1074,7 +1074,7 @@ class ClickHouseCluster:
os.chmod(self.hdfs_kerberized_logs_dir, stat.S_IRWXO) os.chmod(self.hdfs_kerberized_logs_dir, stat.S_IRWXO)
run_and_check(self.base_kerberized_hdfs_cmd + common_opts) run_and_check(self.base_kerberized_hdfs_cmd + common_opts)
hdfs_api = self.make_hdfs_api(kerberized=True) hdfs_api = self.make_hdfs_api(kerberized=True)
self.wait_hdfs_to_start(hdfs_api, timeout=300) self.wait_hdfs_to_start(hdfs_api)
if self.with_mongo and self.base_mongo_cmd: if self.with_mongo and self.base_mongo_cmd:
logging.debug('Setup Mongo') logging.debug('Setup Mongo')
@ -1656,8 +1656,8 @@ class ClickHouseInstance:
"Database": "postgres", "Database": "postgres",
"UserName": "postgres", "UserName": "postgres",
"Password": "mysecretpassword", "Password": "mysecretpassword",
"Port": "5432", "Port": str(self.cluster.postgres_port),
"Servername": self.cluster.postgres_host, "Servername": self.cluster.postgres_ip,
"Protocol": "9.3", "Protocol": "9.3",
"ReadOnly": "No", "ReadOnly": "No",
"RowVersioning": "No", "RowVersioning": "No",

View File

@ -1,189 +0,0 @@
test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[cache]
test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[direct]
test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[flat]
test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[hashed]
test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_cache]
test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_direct]
test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_hashed]
test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_ranged[range_hashed]
test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_hashed]
test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_cache]
test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_hashed]
test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_cache]
test_dictionaries_mysql/test.py::test_load_mysql_dictionaries
test_dictionaries_postgresql/test.py::test_load_dictionaries
test_dictionaries_postgresql/test.py::test_invalidate_query
test_dictionaries_postgresql/test.py::test_dictionary_with_replicas
test_disabled_mysql_server/test.py::test_disabled_mysql_server
test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary]
test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic]
test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary]
test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic]
test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary]
test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic]
test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary]
test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic]
test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary]
test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic]
test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary]
test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic]
test_materialize_mysql_database/test.py::test_select_without_columns_5_7[ordinary]
test_materialize_mysql_database/test.py::test_select_without_columns_5_7[atomic]
test_materialize_mysql_database/test.py::test_select_without_columns_8_0[ordinary]
test_materialize_mysql_database/test.py::test_select_without_columns_8_0[atomic]
test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary]
test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic]
test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary]
test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic]
test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary]
test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic]
test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary]
test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic]
test_materialize_mysql_database/test.py::test_network_partition_5_7[ordinary]
test_materialize_mysql_database/test.py::test_network_partition_5_7[atomic]
test_materialize_mysql_database/test.py::test_network_partition_8_0[ordinary]
test_materialize_mysql_database/test.py::test_network_partition_8_0[atomic]
test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary]
test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic]
test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary]
test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic]
test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary]
test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic]
test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary]
test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic]
test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary]
test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[atomic]
test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary]
test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[atomic]
test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node0]
test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node1]
test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node0]
test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node1]
test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node0]
test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node1]
test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node0]
test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node1]
test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database
test_mysql_database_engine/test.py::test_clickhouse_ddl_for_mysql_database
test_mysql_database_engine/test.py::test_clickhouse_dml_for_mysql_database
test_mysql_database_engine/test.py::test_clickhouse_join_for_mysql_database
test_mysql_database_engine/test.py::test_bad_arguments_for_mysql_database_engine
test_mysql_database_engine/test.py::test_data_types_support_level_for_mysql_database_engine
test_mysql_database_engine/test.py::test_mysql_types[common_types_1]
test_mysql_database_engine/test.py::test_mysql_types[common_types_2]
test_mysql_database_engine/test.py::test_mysql_types[common_types_3]
test_mysql_database_engine/test.py::test_mysql_types[common_types_4]
test_mysql_database_engine/test.py::test_mysql_types[common_types_5]
test_mysql_database_engine/test.py::test_mysql_types[common_types_6]
test_mysql_database_engine/test.py::test_mysql_types[common_types_7]
test_mysql_database_engine/test.py::test_mysql_types[common_types_80]
test_mysql_database_engine/test.py::test_mysql_types[common_types_81]
test_mysql_database_engine/test.py::test_mysql_types[common_types_9]
test_mysql_database_engine/test.py::test_mysql_types[common_types_10]
test_mysql_database_engine/test.py::test_mysql_types[common_types_11]
test_mysql_database_engine/test.py::test_mysql_types[common_types_12]
test_mysql_database_engine/test.py::test_mysql_types[common_types_13]
test_mysql_database_engine/test.py::test_mysql_types[common_types_14]
test_mysql_database_engine/test.py::test_mysql_types[common_types_15]
test_mysql_database_engine/test.py::test_mysql_types[common_types_16]
test_mysql_database_engine/test.py::test_mysql_types[common_types_17]
test_mysql_database_engine/test.py::test_mysql_types[common_types_18]
test_mysql_database_engine/test.py::test_mysql_types[common_types_19]
test_mysql_database_engine/test.py::test_mysql_types[common_types_20]
test_mysql_database_engine/test.py::test_mysql_types[decimal_1]
test_mysql_database_engine/test.py::test_mysql_types[decimal_2]
test_mysql_database_engine/test.py::test_mysql_types[decimal_3]
test_mysql_database_engine/test.py::test_mysql_types[decimal_4]
test_mysql_database_engine/test.py::test_mysql_types[timestamp_default]
test_mysql_database_engine/test.py::test_mysql_types[timestamp_6]
test_mysql_database_engine/test.py::test_mysql_types[datetime_default]
test_mysql_database_engine/test.py::test_mysql_types[datetime_6_1]
test_mysql_database_engine/test.py::test_mysql_types[decimal_40_6]
test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_1]
test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_2]
test_mysql_database_engine/test.py::test_mysql_types[datetime_6_2]
test_mysql_database_engine/test.py::test_mysql_types[datetime_6_3]
test_mysql_protocol/test.py::test_mysql_client
test_mysql_protocol/test.py::test_mysql_client_exception
test_mysql_protocol/test.py::test_mysql_affected_rows
test_mysql_protocol/test.py::test_mysql_replacement_query
test_mysql_protocol/test.py::test_mysql_explain
test_mysql_protocol/test.py::test_mysql_federated
test_mysql_protocol/test.py::test_mysql_set_variables
test_mysql_protocol/test.py::test_python_client
test_mysql_protocol/test.py::test_golang_client
test_mysql_protocol/test.py::test_php_client
test_mysql_protocol/test.py::test_mysqljs_client
test_mysql_protocol/test.py::test_java_client
test_mysql_protocol/test.py::test_types
test_odbc_interaction/test.py::test_mysql_simple_select_works
test_odbc_interaction/test.py::test_mysql_insert
test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_with_schema
test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow
test_odbc_interaction/test.py::test_postgres_insert
test_odbc_interaction/test.py::test_odbc_postgres_date_data_type
test_postgresql_database_engine/test.py::test_postgres_database_engine_with_postgres_ddl
test_postgresql_database_engine/test.py::test_postgresql_database_engine_with_clickhouse_ddl
test_postgresql_database_engine/test.py::test_postgresql_database_engine_queries
test_postgresql_database_engine/test.py::test_get_create_table_query_with_multidim_arrays
test_postgresql_database_engine/test.py::test_postgresql_database_engine_table_cache
test_postgresql_protocol/test.py::test_psql_is_ready
test_postgresql_protocol/test.py::test_psql_client
test_postgresql_protocol/test.py::test_python_client
test_postgresql_protocol/test.py::test_java_client
test_storage_kafka/test.py::test_kafka_json_as_string
test_storage_kafka/test.py::test_kafka_formats
test_storage_kafka/test.py::test_kafka_settings_old_syntax
test_storage_kafka/test.py::test_kafka_settings_new_syntax
test_storage_kafka/test.py::test_kafka_issue11308
test_storage_kafka/test.py::test_kafka_issue4116
test_storage_kafka/test.py::test_kafka_consumer_hang
test_storage_kafka/test.py::test_kafka_consumer_hang2
test_storage_kafka/test.py::test_kafka_csv_with_delimiter
test_storage_kafka/test.py::test_kafka_tsv_with_delimiter
test_storage_kafka/test.py::test_kafka_select_empty
test_storage_kafka/test.py::test_kafka_json_without_delimiter
test_storage_kafka/test.py::test_kafka_protobuf
test_storage_kafka/test.py::test_kafka_string_field_on_first_position_in_protobuf
test_storage_kafka/test.py::test_kafka_protobuf_no_delimiter
test_storage_kafka/test.py::test_kafka_materialized_view
test_storage_kafka/test.py::test_librdkafka_compression
test_storage_kafka/test.py::test_kafka_materialized_view_with_subquery
test_storage_kafka/test.py::test_kafka_many_materialized_views
test_storage_kafka/test.py::test_kafka_flush_on_big_message
test_storage_kafka/test.py::test_kafka_virtual_columns
test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view
test_storage_kafka/test.py::test_kafka_insert
test_storage_kafka/test.py::test_kafka_produce_consume
test_storage_kafka/test.py::test_kafka_commit_on_block_write
test_storage_kafka/test.py::test_kafka_virtual_columns2
test_storage_kafka/test.py::test_kafka_produce_key_timestamp
test_storage_kafka/test.py::test_kafka_flush_by_time
test_storage_kafka/test.py::test_kafka_flush_by_block_size
test_storage_kafka/test.py::test_kafka_lot_of_partitions_partial_commit_of_bulk
test_storage_kafka/test.py::test_kafka_rebalance
test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed
test_storage_kafka/test.py::test_exception_from_destructor
test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop
test_storage_kafka/test.py::test_bad_reschedule
test_storage_kafka/test.py::test_kafka_duplicates_when_commit_failed
test_storage_kafka/test.py::test_premature_flush_on_eof
test_storage_kafka/test.py::test_kafka_unavailable
test_storage_kafka/test.py::test_kafka_issue14202
test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer
test_storage_kerberized_kafka/test.py::test_kafka_json_as_string
test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc
test_storage_mysql/test.py::test_many_connections
test_storage_mysql/test.py::test_insert_select
test_storage_mysql/test.py::test_replace_select
test_storage_mysql/test.py::test_insert_on_duplicate_select
test_storage_mysql/test.py::test_where
test_storage_mysql/test.py::test_table_function
test_storage_mysql/test.py::test_binary_type
test_storage_mysql/test.py::test_enum_type
test_storage_mysql/test.py::test_mysql_distributed
test_storage_postgresql/test.py::test_postgres_select_insert
test_storage_postgresql/test.py::test_postgres_conversions
test_storage_postgresql/test.py::test_non_default_scema
test_storage_postgresql/test.py::test_concurrent_queries
test_storage_postgresql/test.py::test_postgres_distributed

View File

@ -0,0 +1,191 @@
[
"test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[cache]",
"test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[direct]",
"test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[flat]",
"test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[hashed]",
"test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_cache]",
"test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_direct]",
"test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_hashed]",
"test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_ranged[range_hashed]",
"test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_hashed]",
"test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_cache]",
"test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_hashed]",
"test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_cache]",
"test_dictionaries_mysql/test.py::test_load_mysql_dictionaries",
"test_dictionaries_postgresql/test.py::test_load_dictionaries",
"test_dictionaries_postgresql/test.py::test_invalidate_query",
"test_dictionaries_postgresql/test.py::test_dictionary_with_replicas",
"test_disabled_mysql_server/test.py::test_disabled_mysql_server",
"test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_select_without_columns_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_select_without_columns_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_select_without_columns_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_select_without_columns_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_network_partition_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_network_partition_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_network_partition_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_network_partition_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary]",
"test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[atomic]",
"test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary]",
"test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[atomic]",
"test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node0]",
"test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node1]",
"test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node0]",
"test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node1]",
"test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node0]",
"test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node1]",
"test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node0]",
"test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node1]",
"test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database",
"test_mysql_database_engine/test.py::test_clickhouse_ddl_for_mysql_database",
"test_mysql_database_engine/test.py::test_clickhouse_dml_for_mysql_database",
"test_mysql_database_engine/test.py::test_clickhouse_join_for_mysql_database",
"test_mysql_database_engine/test.py::test_bad_arguments_for_mysql_database_engine",
"test_mysql_database_engine/test.py::test_data_types_support_level_for_mysql_database_engine",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_1]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_2]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_3]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_4]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_5]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_6]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_7]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_80]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_81]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_9]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_10]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_11]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_12]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_13]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_14]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_15]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_16]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_17]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_18]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_19]",
"test_mysql_database_engine/test.py::test_mysql_types[common_types_20]",
"test_mysql_database_engine/test.py::test_mysql_types[decimal_1]",
"test_mysql_database_engine/test.py::test_mysql_types[decimal_2]",
"test_mysql_database_engine/test.py::test_mysql_types[decimal_3]",
"test_mysql_database_engine/test.py::test_mysql_types[decimal_4]",
"test_mysql_database_engine/test.py::test_mysql_types[timestamp_default]",
"test_mysql_database_engine/test.py::test_mysql_types[timestamp_6]",
"test_mysql_database_engine/test.py::test_mysql_types[datetime_default]",
"test_mysql_database_engine/test.py::test_mysql_types[datetime_6_1]",
"test_mysql_database_engine/test.py::test_mysql_types[decimal_40_6]",
"test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_1]",
"test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_2]",
"test_mysql_database_engine/test.py::test_mysql_types[datetime_6_2]",
"test_mysql_database_engine/test.py::test_mysql_types[datetime_6_3]",
"test_mysql_protocol/test.py::test_mysql_client",
"test_mysql_protocol/test.py::test_mysql_client_exception",
"test_mysql_protocol/test.py::test_mysql_affected_rows",
"test_mysql_protocol/test.py::test_mysql_replacement_query",
"test_mysql_protocol/test.py::test_mysql_explain",
"test_mysql_protocol/test.py::test_mysql_federated",
"test_mysql_protocol/test.py::test_mysql_set_variables",
"test_mysql_protocol/test.py::test_python_client",
"test_mysql_protocol/test.py::test_golang_client",
"test_mysql_protocol/test.py::test_php_client",
"test_mysql_protocol/test.py::test_mysqljs_client",
"test_mysql_protocol/test.py::test_java_client",
"test_mysql_protocol/test.py::test_types",
"test_odbc_interaction/test.py::test_mysql_simple_select_works",
"test_odbc_interaction/test.py::test_mysql_insert",
"test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_with_schema",
"test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow",
"test_odbc_interaction/test.py::test_postgres_insert",
"test_odbc_interaction/test.py::test_odbc_postgres_date_data_type",
"test_postgresql_database_engine/test.py::test_postgres_database_engine_with_postgres_ddl",
"test_postgresql_database_engine/test.py::test_postgresql_database_engine_with_clickhouse_ddl",
"test_postgresql_database_engine/test.py::test_postgresql_database_engine_queries",
"test_postgresql_database_engine/test.py::test_get_create_table_query_with_multidim_arrays",
"test_postgresql_database_engine/test.py::test_postgresql_database_engine_table_cache",
"test_postgresql_protocol/test.py::test_psql_is_ready",
"test_postgresql_protocol/test.py::test_psql_client",
"test_postgresql_protocol/test.py::test_python_client",
"test_postgresql_protocol/test.py::test_java_client",
"test_storage_kafka/test.py::test_kafka_json_as_string",
"test_storage_kafka/test.py::test_kafka_formats",
"test_storage_kafka/test.py::test_kafka_settings_old_syntax",
"test_storage_kafka/test.py::test_kafka_settings_new_syntax",
"test_storage_kafka/test.py::test_kafka_issue11308",
"test_storage_kafka/test.py::test_kafka_issue4116",
"test_storage_kafka/test.py::test_kafka_consumer_hang",
"test_storage_kafka/test.py::test_kafka_consumer_hang2",
"test_storage_kafka/test.py::test_kafka_csv_with_delimiter",
"test_storage_kafka/test.py::test_kafka_tsv_with_delimiter",
"test_storage_kafka/test.py::test_kafka_select_empty",
"test_storage_kafka/test.py::test_kafka_json_without_delimiter",
"test_storage_kafka/test.py::test_kafka_protobuf",
"test_storage_kafka/test.py::test_kafka_string_field_on_first_position_in_protobuf",
"test_storage_kafka/test.py::test_kafka_protobuf_no_delimiter",
"test_storage_kafka/test.py::test_kafka_materialized_view",
"test_storage_kafka/test.py::test_librdkafka_compression",
"test_storage_kafka/test.py::test_kafka_materialized_view_with_subquery",
"test_storage_kafka/test.py::test_kafka_many_materialized_views",
"test_storage_kafka/test.py::test_kafka_flush_on_big_message",
"test_storage_kafka/test.py::test_kafka_virtual_columns",
"test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view",
"test_storage_kafka/test.py::test_kafka_insert",
"test_storage_kafka/test.py::test_kafka_produce_consume",
"test_storage_kafka/test.py::test_kafka_commit_on_block_write",
"test_storage_kafka/test.py::test_kafka_virtual_columns2",
"test_storage_kafka/test.py::test_kafka_produce_key_timestamp",
"test_storage_kafka/test.py::test_kafka_flush_by_time",
"test_storage_kafka/test.py::test_kafka_flush_by_block_size",
"test_storage_kafka/test.py::test_kafka_lot_of_partitions_partial_commit_of_bulk",
"test_storage_kafka/test.py::test_kafka_rebalance",
"test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed",
"test_storage_kafka/test.py::test_exception_from_destructor",
"test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop",
"test_storage_kafka/test.py::test_bad_reschedule",
"test_storage_kafka/test.py::test_kafka_duplicates_when_commit_failed",
"test_storage_kafka/test.py::test_premature_flush_on_eof",
"test_storage_kafka/test.py::test_kafka_unavailable",
"test_storage_kafka/test.py::test_kafka_issue14202",
"test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer",
"test_storage_kerberized_kafka/test.py::test_kafka_json_as_string",
"test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc",
"test_storage_mysql/test.py::test_many_connections",
"test_storage_mysql/test.py::test_insert_select",
"test_storage_mysql/test.py::test_replace_select",
"test_storage_mysql/test.py::test_insert_on_duplicate_select",
"test_storage_mysql/test.py::test_where",
"test_storage_mysql/test.py::test_table_function",
"test_storage_mysql/test.py::test_binary_type",
"test_storage_mysql/test.py::test_enum_type",
"test_storage_mysql/test.py::test_mysql_distributed",
"test_storage_postgresql/test.py::test_postgres_select_insert",
"test_storage_postgresql/test.py::test_postgres_conversions",
"test_storage_postgresql/test.py::test_non_default_scema",
"test_storage_postgresql/test.py::test_concurrent_queries",
"test_storage_postgresql/test.py::test_postgres_distributed"
]

View File

@ -328,7 +328,7 @@ def test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow(started_cluster):
cursor.execute("insert into clickhouse.test_table values(3, 'xxx')") cursor.execute("insert into clickhouse.test_table values(3, 'xxx')")
for i in range(100): for i in range(100):
try: try:
node1.query("system reload dictionary postgres_odbc_hashed", timeout=5) node1.query("system reload dictionary postgres_odbc_hashed", timeout=15)
except Exception as ex: except Exception as ex:
assert False, "Exception occured -- odbc-bridge hangs: " + str(ex) assert False, "Exception occured -- odbc-bridge hangs: " + str(ex)

View File

@ -147,7 +147,7 @@ def test_put(started_cluster, maybe_auth, positive, compression):
# Test put no data to S3. # Test put no data to S3.
@pytest.mark.parametrize("auth", [ @pytest.mark.parametrize("auth", [
pytest.param("'minio','minio123'", id="minio") pytest.param("'minio','minio123',", id="minio")
]) ])
def test_empty_put(started_cluster, auth): def test_empty_put(started_cluster, auth):
# type: (ClickHouseCluster) -> None # type: (ClickHouseCluster) -> None