From 13e41cdd20c8204efedcf9c7219443a16915c1e0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 21 Sep 2020 13:24:10 +0300 Subject: [PATCH] try enable atomic database by default --- docker/test/fasttest/run.sh | 1 + docker/test/stateful/run.sh | 7 +++--- docker/test/stateless/run.sh | 7 +++--- docker/test/stateless_unbundled/run.sh | 7 +++--- docker/test/stress/stress | 2 +- programs/copier/Internals.cpp | 17 +++----------- programs/copier/Internals.h | 2 +- programs/copier/TaskTableAndShard.h | 22 +++++++++++-------- src/Core/Settings.h | 2 +- src/Storages/StorageDictionary.cpp | 8 +++++-- src/Storages/StorageDictionary.h | 3 ++- tests/clickhouse-test | 8 +++---- tests/config/database_atomic_usersd.xml | 1 - tests/config/database_ordinary_usersd.xml | 7 ++++++ .../test_cluster_copier/task0_description.xml | 4 ++-- .../task_month_to_week_description.xml | 4 ++-- .../task_test_block_size.xml | 4 ++-- tests/integration/test_cluster_copier/test.py | 21 ++++++++++-------- .../test_cluster_copier/trivial_test.py | 2 +- .../test_dictionaries_dependency/test.py | 2 +- ...lized_view_and_deduplication_zookeeper.sql | 16 +++++++------- .../00604_show_create_database.reference | 2 +- .../0_stateless/00609_mv_index_in_in.sql | 4 ++-- .../0_stateless/00738_lock_for_inner_table.sh | 4 ++-- .../01190_full_attach_syntax.reference | 18 +++++++-------- .../0_stateless/01190_full_attach_syntax.sql | 16 +++----------- .../01224_no_superfluous_dict_reload.sql | 2 +- ...1225_show_create_table_from_dictionary.sql | 2 +- ...9_bad_arguments_for_bloom_filter.reference | 6 ++--- .../01249_bad_arguments_for_bloom_filter.sql | 6 ++++- ...20_create_sync_race_condition_zookeeper.sh | 9 ++++---- tests/queries/skip_list.json | 16 +++++++------- 32 files changed, 119 insertions(+), 113 deletions(-) create mode 100644 tests/config/database_ordinary_usersd.xml diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index ccbadb84f27..560fa9c77af 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -136,6 +136,7 @@ ln -s /usr/share/clickhouse-test/config/graphite.xml /etc/clickhouse-server/conf ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml # Keep original query_masking_rules.xml diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index c3576acc0e4..34980508488 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -23,10 +23,11 @@ ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-serv ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ -if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then - ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ - ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ +if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then #FIXME USE_DATABASE_ORDINARY + ln -s /usr/share/clickhouse-test/config/database_ordinary_configd.xml /etc/clickhouse-server/config.d/ fi function start() diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 4a9ad891883..5546f0eef18 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -33,6 +33,8 @@ ln -s /usr/share/clickhouse-test/config/graphite.xml /etc/clickhouse-server/conf ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ # Retain any pre-existing config and allow ClickHouse to load it if required ln -s --backup=simple --suffix=_original.xml \ @@ -41,9 +43,8 @@ ln -s --backup=simple --suffix=_original.xml \ if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/polymorphic_parts.xml /etc/clickhouse-server/config.d/ fi -if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then - ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ - ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ +if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then #FIXME USE_DATABASE_ORDINARY + ln -s /usr/share/clickhouse-test/config/database_ordinary_usersd.xml /etc/clickhouse-server/users.d/ fi ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml diff --git a/docker/test/stateless_unbundled/run.sh b/docker/test/stateless_unbundled/run.sh index 4a9ad891883..13ded2290fb 100755 --- a/docker/test/stateless_unbundled/run.sh +++ b/docker/test/stateless_unbundled/run.sh @@ -33,6 +33,8 @@ ln -s /usr/share/clickhouse-test/config/graphite.xml /etc/clickhouse-server/conf ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ # Retain any pre-existing config and allow ClickHouse to load it if required ln -s --backup=simple --suffix=_original.xml \ @@ -41,9 +43,8 @@ ln -s --backup=simple --suffix=_original.xml \ if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/polymorphic_parts.xml /etc/clickhouse-server/config.d/ fi -if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then - ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ - ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ +if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then #FIXME USE_DATABASE_ORDINARY + ln -s /usr/share/clickhouse-test/config/database_ordinary_configd.xml /etc/clickhouse-server/config.d/ fi ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 60db5ec465c..a36adda3aad 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -29,7 +29,7 @@ def get_options(i): if 0 < i: options += " --order=random" if i % 2 == 1: - options += " --atomic-db-engine" + options += " --db-engine=Ordinary" return options diff --git a/programs/copier/Internals.cpp b/programs/copier/Internals.cpp index 12da07a772a..ca26f0d1831 100644 --- a/programs/copier/Internals.cpp +++ b/programs/copier/Internals.cpp @@ -215,31 +215,20 @@ Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast) return primary_key_columns; } -String extractReplicatedTableZookeeperPath(const ASTPtr & storage_ast) +bool isReplicatedTableEngine(const ASTPtr & storage_ast) { - String storage_str = queryToString(storage_ast); - const auto & storage = storage_ast->as(); const auto & engine = storage.engine->as(); if (!endsWith(engine.name, "MergeTree")) { + String storage_str = queryToString(storage_ast); throw Exception( "Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported", ErrorCodes::BAD_ARGUMENTS); } - if (!startsWith(engine.name, "Replicated")) - { - return ""; - } - - auto replicated_table_arguments = engine.arguments->children; - - auto zk_table_path_ast = replicated_table_arguments[0]->as(); - auto zk_table_path_string = zk_table_path_ast.value.safeGet(); - - return zk_table_path_string; + return startsWith(engine.name, "Replicated"); } ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random) diff --git a/programs/copier/Internals.h b/programs/copier/Internals.h index b61b6d59629..7e45c0ea2ee 100644 --- a/programs/copier/Internals.h +++ b/programs/copier/Internals.h @@ -200,7 +200,7 @@ ASTPtr extractOrderBy(const ASTPtr & storage_ast); Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast); -String extractReplicatedTableZookeeperPath(const ASTPtr & storage_ast); +bool isReplicatedTableEngine(const ASTPtr & storage_ast); ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random); diff --git a/programs/copier/TaskTableAndShard.h b/programs/copier/TaskTableAndShard.h index 11ceffd12cd..27c4b89377d 100644 --- a/programs/copier/TaskTableAndShard.h +++ b/programs/copier/TaskTableAndShard.h @@ -48,7 +48,7 @@ struct TaskTable String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const; - bool isReplicatedTable() const { return engine_push_zk_path != ""; } + bool isReplicatedTable() const { return is_replicated_table; } /// Partitions will be split into number-of-splits pieces. /// Each piece will be copied independently. (10 by default) @@ -78,6 +78,7 @@ struct TaskTable /// First argument of Replicated...MergeTree() String engine_push_zk_path; + bool is_replicated_table; ASTPtr rewriteReplicatedCreateQueryToPlain(); @@ -269,7 +270,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf engine_push_ast = parseQuery(parser_storage, engine_push_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); primary_key_comma_separated = Nested::createCommaSeparatedStringFrom(extractPrimaryKeyColumnNames(engine_push_ast)); - engine_push_zk_path = extractReplicatedTableZookeeperPath(engine_push_ast); + is_replicated_table = isReplicatedTableEngine(engine_push_ast); } sharding_key_str = config.getString(table_prefix + "sharding_key"); @@ -372,15 +373,18 @@ inline ASTPtr TaskTable::rewriteReplicatedCreateQueryToPlain() auto & new_storage_ast = prev_engine_push_ast->as(); auto & new_engine_ast = new_storage_ast.engine->as(); - auto & replicated_table_arguments = new_engine_ast.arguments->children; - - /// Delete first two arguments of Replicated...MergeTree() table. - replicated_table_arguments.erase(replicated_table_arguments.begin()); - replicated_table_arguments.erase(replicated_table_arguments.begin()); - - /// Remove replicated from name + /// Remove "Replicated" from name new_engine_ast.name = new_engine_ast.name.substr(10); + if (new_engine_ast.arguments) + { + auto & replicated_table_arguments = new_engine_ast.arguments->children; + + /// Delete first two arguments of Replicated...MergeTree() table. + replicated_table_arguments.erase(replicated_table_arguments.begin()); + replicated_table_arguments.erase(replicated_table_arguments.begin()); + } + return new_storage_ast.clone(); } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b96b1b12c24..9449cd571a1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -350,7 +350,7 @@ class IColumn; M(UInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.", 0) \ M(UInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \ \ - M(DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Ordinary, "Default database engine.", 0) \ + M(DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Atomic, "Default database engine.", 0) \ M(Bool, show_table_uuid_in_table_create_query_if_not_nil, false, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 0) \ M(Bool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 5d92b9cec55..99645d09d00 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -100,6 +100,9 @@ StorageDictionary::StorageDictionary( Location location_) : IStorage(table_id_) , dictionary_name(dictionary_name_) + , resolved_dictionary_name(location_ == Location::SameDatabaseAndNameAsDictionary + ? dictionary_name + : DatabaseCatalog::instance().resolveDictionaryName(dictionary_name)) , location(location_) { StorageInMemoryMetadata storage_metadata; @@ -132,7 +135,7 @@ Pipe StorageDictionary::read( const size_t max_block_size, const unsigned /*threads*/) { - auto dictionary = context.getExternalDictionariesLoader().getDictionary(dictionary_name); + auto dictionary = context.getExternalDictionariesLoader().getDictionary(resolved_dictionary_name); auto stream = dictionary->getBlockInputStream(column_names, max_block_size); /// TODO: update dictionary interface for processors. return Pipe(std::make_shared(stream)); @@ -152,7 +155,8 @@ void registerStorageDictionary(StorageFactory & factory) if (!args.attach) { - const auto & dictionary = args.context.getExternalDictionariesLoader().getDictionary(dictionary_name); + auto resolved = DatabaseCatalog::instance().resolveDictionaryName(dictionary_name); + const auto & dictionary = args.context.getExternalDictionariesLoader().getDictionary(resolved); const DictionaryStructure & dictionary_structure = dictionary->getStructure(); checkNamesAndTypesCompatibleWithDictionary(dictionary_name, args.columns, dictionary_structure); } diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index d822552124d..528ee6533b7 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -28,7 +28,7 @@ public: static NamesAndTypesList getNamesAndTypes(const DictionaryStructure & dictionary_structure); static String generateNamesAndTypesDescription(const NamesAndTypesList & list); - const String & dictionaryName() const { return dictionary_name; } + const String & dictionaryName() const { return resolved_dictionary_name; } /// Specifies where the table is located relative to the dictionary. enum class Location @@ -50,6 +50,7 @@ public: private: const String dictionary_name; + const String resolved_dictionary_name; const Location location; protected: diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a3bed189d55..de296cabb7a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -107,9 +107,9 @@ def remove_control_characters(s): return s def get_db_engine(args): - if args.atomic_db_engine: - return " ENGINE=Atomic" - return "" + if args.db_engine: + return " ENGINE=" + args.db_engine + return "" # Will use default engine def run_single_test(args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file): @@ -792,7 +792,7 @@ if __name__ == '__main__': parser.add_argument('-r', '--server-check-retries', default=30, type=int, help='Num of tries to execute SELECT 1 before tests started') parser.add_argument('--skip-list-path', help="Path to skip-list file") parser.add_argument('--use-skip-list', action='store_true', default=False, help="Use skip list to skip tests if found") - parser.add_argument('--atomic-db-engine', action='store_true', help='Create databases with Atomic engine by default') + parser.add_argument('--db-engine', help='Database engine name') parser.add_argument('--no-stateless', action='store_true', help='Disable all stateless tests') parser.add_argument('--no-stateful', action='store_true', help='Disable all stateful tests') diff --git a/tests/config/database_atomic_usersd.xml b/tests/config/database_atomic_usersd.xml index 201d476da24..58e36cf3884 100644 --- a/tests/config/database_atomic_usersd.xml +++ b/tests/config/database_atomic_usersd.xml @@ -1,7 +1,6 @@ - Atomic 0 diff --git a/tests/config/database_ordinary_usersd.xml b/tests/config/database_ordinary_usersd.xml new file mode 100644 index 00000000000..68f3b044f75 --- /dev/null +++ b/tests/config/database_ordinary_usersd.xml @@ -0,0 +1,7 @@ + + + + Ordinary + + + diff --git a/tests/integration/test_cluster_copier/task0_description.xml b/tests/integration/test_cluster_copier/task0_description.xml index 72eff8d464d..d56053ffd39 100644 --- a/tests/integration/test_cluster_copier/task0_description.xml +++ b/tests/integration/test_cluster_copier/task0_description.xml @@ -33,7 +33,7 @@ 3 4 5 6 1 2 0 - ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/hits', '{replica}') PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16 + ENGINE=ReplicatedMergeTree PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16 d + 1 @@ -93,4 +93,4 @@ - \ No newline at end of file + diff --git a/tests/integration/test_cluster_copier/task_month_to_week_description.xml b/tests/integration/test_cluster_copier/task_month_to_week_description.xml index ee134603310..26dfc7d3e00 100644 --- a/tests/integration/test_cluster_copier/task_month_to_week_description.xml +++ b/tests/integration/test_cluster_copier/task_month_to_week_description.xml @@ -34,7 +34,7 @@ ENGINE= - ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/b', '{replica}') + ReplicatedMergeTree PARTITION BY toMonday(date) ORDER BY d @@ -97,4 +97,4 @@ - \ No newline at end of file + diff --git a/tests/integration/test_cluster_copier/task_test_block_size.xml b/tests/integration/test_cluster_copier/task_test_block_size.xml index ea63d580c1c..c9c99a083ea 100644 --- a/tests/integration/test_cluster_copier/task_test_block_size.xml +++ b/tests/integration/test_cluster_copier/task_test_block_size.xml @@ -28,7 +28,7 @@ ENGINE= - ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/test_block_size', '{replica}') + ReplicatedMergeTree ORDER BY d PARTITION BY partition @@ -99,4 +99,4 @@ - \ No newline at end of file + diff --git a/tests/integration/test_cluster_copier/test.py b/tests/integration/test_cluster_copier/test.py index 2a9e696ca46..88dac06f158 100644 --- a/tests/integration/test_cluster_copier/test.py +++ b/tests/integration/test_cluster_copier/test.py @@ -81,11 +81,11 @@ class Task1: for cluster_num in ["0", "1"]: ddl_check_query(instance, "DROP DATABASE IF EXISTS default ON CLUSTER cluster{}".format(cluster_num)) ddl_check_query(instance, - "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{} ENGINE=Ordinary".format( + "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{}".format( cluster_num)) ddl_check_query(instance, "CREATE TABLE hits ON CLUSTER cluster0 (d UInt64, d1 UInt64 MATERIALIZED d+1) " + - "ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/hits', '{replica}') " + + "ENGINE=ReplicatedMergeTree " + "PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16") ddl_check_query(instance, "CREATE TABLE hits_all ON CLUSTER cluster0 (d UInt64) ENGINE=Distributed(cluster0, default, hits, d)") @@ -110,10 +110,11 @@ class Task1: class Task2: - def __init__(self, cluster): + def __init__(self, cluster, unique_zk_path): self.cluster = cluster self.zk_task_path = "/clickhouse-copier/task_month_to_week_partition" self.copier_task_config = open(os.path.join(CURRENT_TEST_DIR, 'task_month_to_week_description.xml'), 'r').read() + self.unique_zk_path = unique_zk_path def start(self): instance = cluster.instances['s0_0_0'] @@ -121,11 +122,13 @@ class Task2: for cluster_num in ["0", "1"]: ddl_check_query(instance, "DROP DATABASE IF EXISTS default ON CLUSTER cluster{}".format(cluster_num)) ddl_check_query(instance, - "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{} ENGINE=Ordinary".format( + "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{}".format( cluster_num)) ddl_check_query(instance, - "CREATE TABLE a ON CLUSTER cluster0 (date Date, d UInt64, d1 UInt64 ALIAS d+1) ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/a', '{replica}', date, intHash64(d), (date, intHash64(d)), 8192)") + "CREATE TABLE a ON CLUSTER cluster0 (date Date, d UInt64, d1 UInt64 ALIAS d+1) " + "ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/" + self.unique_zk_path + "', " + "'{replica}', date, intHash64(d), (date, intHash64(d)), 8192)") ddl_check_query(instance, "CREATE TABLE a_all ON CLUSTER cluster0 (date Date, d UInt64) ENGINE=Distributed(cluster0, default, a, d)") @@ -169,7 +172,7 @@ class Task_test_block_size: ddl_check_query(instance, """ CREATE TABLE test_block_size ON CLUSTER shard_0_0 (partition Date, d UInt64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/test_block_size', '{replica}') + ENGINE=ReplicatedMergeTree ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d)""", 2) instance.query( @@ -332,17 +335,17 @@ def test_copy_with_recovering_after_move_faults(started_cluster, use_sample_offs @pytest.mark.timeout(600) def test_copy_month_to_week_partition(started_cluster): - execute_task(Task2(started_cluster), []) + execute_task(Task2(started_cluster, "test1"), []) @pytest.mark.timeout(600) def test_copy_month_to_week_partition_with_recovering(started_cluster): - execute_task(Task2(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) + execute_task(Task2(started_cluster, "test2"), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) @pytest.mark.timeout(600) def test_copy_month_to_week_partition_with_recovering_after_move_faults(started_cluster): - execute_task(Task2(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) + execute_task(Task2(started_cluster, "test3"), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) def test_block_size(started_cluster): diff --git a/tests/integration/test_cluster_copier/trivial_test.py b/tests/integration/test_cluster_copier/trivial_test.py index 3d0c5d0f5b0..035faf0bb9f 100644 --- a/tests/integration/test_cluster_copier/trivial_test.py +++ b/tests/integration/test_cluster_copier/trivial_test.py @@ -59,7 +59,7 @@ class TaskTrivial: for node in [source, destination]: node.query("DROP DATABASE IF EXISTS default") - node.query("CREATE DATABASE IF NOT EXISTS default ENGINE=Ordinary") + node.query("CREATE DATABASE IF NOT EXISTS default") source.query("CREATE TABLE trivial (d UInt64, d1 UInt64 MATERIALIZED d+1) " "ENGINE=ReplicatedMergeTree('/clickhouse/tables/source_trivial_cluster/1/trivial', '1') " diff --git a/tests/integration/test_dictionaries_dependency/test.py b/tests/integration/test_dictionaries_dependency/test.py index 119bd7c6863..9c36da229e1 100644 --- a/tests/integration/test_dictionaries_dependency/test.py +++ b/tests/integration/test_dictionaries_dependency/test.py @@ -12,7 +12,7 @@ def start_cluster(): try: cluster.start() for node in nodes: - node.query("CREATE DATABASE IF NOT EXISTS test") + node.query("CREATE DATABASE IF NOT EXISTS test ENGINE=Ordinary") node.query("CREATE DATABASE IF NOT EXISTS atest") node.query("CREATE DATABASE IF NOT EXISTS ztest") node.query("CREATE TABLE test.source(x UInt64, y UInt64) ENGINE=Log") diff --git a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql index 48e1cd65c49..8df012a8588 100644 --- a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql +++ b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql @@ -8,10 +8,10 @@ CREATE TABLE with_deduplication(x UInt32) CREATE TABLE without_deduplication(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00510/without_deduplication', 'r1') ORDER BY x SETTINGS replicated_deduplication_window = 0; -CREATE MATERIALIZED VIEW with_deduplication_mv +CREATE MATERIALIZED VIEW with_deduplication_mv UUID '00000510-1000-4000-8000-000000000001' ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/test_00510/with_deduplication_mv', 'r1') ORDER BY dummy AS SELECT 0 AS dummy, countState(x) AS cnt FROM with_deduplication; -CREATE MATERIALIZED VIEW without_deduplication_mv +CREATE MATERIALIZED VIEW without_deduplication_mv UUID '00000510-1000-4000-8000-000000000002' ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/test_00510/without_deduplication_mv', 'r1') ORDER BY dummy AS SELECT 0 AS dummy, countState(x) AS cnt FROM without_deduplication; @@ -32,12 +32,12 @@ SELECT countMerge(cnt) FROM with_deduplication_mv; SELECT countMerge(cnt) FROM without_deduplication_mv; -- Explicit insert is deduplicated -ALTER TABLE `.inner.with_deduplication_mv` DROP PARTITION ID 'all'; -ALTER TABLE `.inner.without_deduplication_mv` DROP PARTITION ID 'all'; -INSERT INTO `.inner.with_deduplication_mv` SELECT 0 AS dummy, arrayReduce('countState', [toUInt32(42)]) AS cnt; -INSERT INTO `.inner.with_deduplication_mv` SELECT 0 AS dummy, arrayReduce('countState', [toUInt32(42)]) AS cnt; -INSERT INTO `.inner.without_deduplication_mv` SELECT 0 AS dummy, arrayReduce('countState', [toUInt32(42)]) AS cnt; -INSERT INTO `.inner.without_deduplication_mv` SELECT 0 AS dummy, arrayReduce('countState', [toUInt32(42)]) AS cnt; +ALTER TABLE `.inner_id.00000510-1000-4000-8000-000000000001` DROP PARTITION ID 'all'; +ALTER TABLE `.inner_id.00000510-1000-4000-8000-000000000002` DROP PARTITION ID 'all'; +INSERT INTO `.inner_id.00000510-1000-4000-8000-000000000001` SELECT 0 AS dummy, arrayReduce('countState', [toUInt32(42)]) AS cnt; +INSERT INTO `.inner_id.00000510-1000-4000-8000-000000000001` SELECT 0 AS dummy, arrayReduce('countState', [toUInt32(42)]) AS cnt; +INSERT INTO `.inner_id.00000510-1000-4000-8000-000000000002` SELECT 0 AS dummy, arrayReduce('countState', [toUInt32(42)]) AS cnt; +INSERT INTO `.inner_id.00000510-1000-4000-8000-000000000002` SELECT 0 AS dummy, arrayReduce('countState', [toUInt32(42)]) AS cnt; SELECT ''; SELECT countMerge(cnt) FROM with_deduplication_mv; diff --git a/tests/queries/0_stateless/00604_show_create_database.reference b/tests/queries/0_stateless/00604_show_create_database.reference index a9ad6abea25..c05b088280e 100644 --- a/tests/queries/0_stateless/00604_show_create_database.reference +++ b/tests/queries/0_stateless/00604_show_create_database.reference @@ -1 +1 @@ -CREATE DATABASE test_00604\nENGINE = Ordinary +CREATE DATABASE test_00604\nENGINE = Atomic diff --git a/tests/queries/0_stateless/00609_mv_index_in_in.sql b/tests/queries/0_stateless/00609_mv_index_in_in.sql index 7064d8e36cd..28085194327 100644 --- a/tests/queries/0_stateless/00609_mv_index_in_in.sql +++ b/tests/queries/0_stateless/00609_mv_index_in_in.sql @@ -4,11 +4,11 @@ DROP TABLE IF EXISTS test_mv_00609; create table test_00609 (a Int8) engine=Memory; insert into test_00609 values (1); -create materialized view test_mv_00609 Engine=MergeTree(date, (a), 8192) populate as select a, toDate('2000-01-01') date from test_00609; +create materialized view test_mv_00609 uuid '00000609-1000-4000-8000-000000000001' Engine=MergeTree(date, (a), 8192) populate as select a, toDate('2000-01-01') date from test_00609; select * from test_mv_00609; -- OK select * from test_mv_00609 where a in (select a from test_mv_00609); -- EMPTY (bug) -select * from ".inner.test_mv_00609" where a in (select a from test_mv_00609); -- OK +select * from ".inner_id.00000609-1000-4000-8000-000000000001" where a in (select a from test_mv_00609); -- OK DROP TABLE test_00609; DROP TABLE test_mv_00609; diff --git a/tests/queries/0_stateless/00738_lock_for_inner_table.sh b/tests/queries/0_stateless/00738_lock_for_inner_table.sh index 2f7035b6759..4570c853f31 100755 --- a/tests/queries/0_stateless/00738_lock_for_inner_table.sh +++ b/tests/queries/0_stateless/00738_lock_for_inner_table.sh @@ -7,13 +7,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo "DROP TABLE IF EXISTS tab_00738; DROP TABLE IF EXISTS mv; CREATE TABLE tab_00738(a Int) ENGINE = Log; -CREATE MATERIALIZED VIEW mv ENGINE = Log AS SELECT a FROM tab_00738;" | ${CLICKHOUSE_CLIENT} -n +CREATE MATERIALIZED VIEW mv UUID '00000738-1000-4000-8000-000000000001' ENGINE = Log AS SELECT a FROM tab_00738;" | ${CLICKHOUSE_CLIENT} -n ${CLICKHOUSE_CLIENT} --query_id test_00738 --query "INSERT INTO tab_00738 SELECT number FROM numbers(10000000)" & function drop() { - ${CLICKHOUSE_CLIENT} --query "DROP TABLE \`.inner.mv\`" -n + ${CLICKHOUSE_CLIENT} --query "DROP TABLE \`.inner_id.00000738-1000-4000-8000-000000000001\`" -n } function wait_for_query_to_start() diff --git a/tests/queries/0_stateless/01190_full_attach_syntax.reference b/tests/queries/0_stateless/01190_full_attach_syntax.reference index 619861849c8..4e6eabcd6f0 100644 --- a/tests/queries/0_stateless/01190_full_attach_syntax.reference +++ b/tests/queries/0_stateless/01190_full_attach_syntax.reference @@ -1,13 +1,13 @@ CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) -CREATE TABLE default.log\n(\n `s` String\n)\nENGINE = Log -CREATE TABLE default.log\n(\n `s` String\n)\nENGINE = Log() +CREATE TABLE test_01190.log\n(\n `s` String\n)\nENGINE = Log +CREATE TABLE test_01190.log\n(\n `s` String\n)\nENGINE = Log() test -CREATE TABLE default.mt\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date MATERIALIZED \'2000-01-01\'\n)\nENGINE = MergeTree(d, (key, s, n), 1) +CREATE TABLE test_01190.mt\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date MATERIALIZED \'2000-01-01\'\n)\nENGINE = MergeTree(d, (key, s, n), 1) [1,2] Hello 2 -CREATE TABLE default.mt\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date\n)\nENGINE = MergeTree(d, (key, s, n), 1) -CREATE MATERIALIZED VIEW default.mv\n(\n `s` String\n)\nENGINE = Null AS\nSELECT *\nFROM default.log -CREATE MATERIALIZED VIEW default.mv\n(\n `s` String\n)\nENGINE = Null AS\nSELECT *\nFROM default.log -CREATE MATERIALIZED VIEW default.mv\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date\n)\nENGINE = Null AS\nSELECT *\nFROM default.mt -CREATE LIVE VIEW default.lv\n(\n `1` UInt8\n) AS\nSELECT 1 -CREATE LIVE VIEW default.lv\n(\n `1` UInt8\n) AS\nSELECT 1 +CREATE TABLE test_01190.mt\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date\n)\nENGINE = MergeTree(d, (key, s, n), 1) +CREATE MATERIALIZED VIEW test_01190.mv\n(\n `s` String\n)\nENGINE = Null AS\nSELECT *\nFROM test_01190.log +CREATE MATERIALIZED VIEW test_01190.mv\n(\n `s` String\n)\nENGINE = Null AS\nSELECT *\nFROM test_01190.log +CREATE MATERIALIZED VIEW test_01190.mv\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date\n)\nENGINE = Null AS\nSELECT *\nFROM test_01190.mt +CREATE LIVE VIEW test_01190.lv\n(\n `1` UInt8\n) AS\nSELECT 1 +CREATE LIVE VIEW test_01190.lv\n(\n `1` UInt8\n) AS\nSELECT 1 diff --git a/tests/queries/0_stateless/01190_full_attach_syntax.sql b/tests/queries/0_stateless/01190_full_attach_syntax.sql index 3a91eccc8cd..225e66bed5c 100644 --- a/tests/queries/0_stateless/01190_full_attach_syntax.sql +++ b/tests/queries/0_stateless/01190_full_attach_syntax.sql @@ -1,5 +1,6 @@ DROP DATABASE IF EXISTS test_01190; -CREATE DATABASE test_01190; +CREATE DATABASE test_01190 ENGINE=Ordinary; +USE test_01190; CREATE TABLE test_01190.table_for_dict (key UInt64, col UInt8) ENGINE = Memory; @@ -14,14 +15,6 @@ ATTACH DICTIONARY test_01190.dict (key UInt64 DEFAULT 0, col UInt8 DEFAULT 42) P ATTACH DICTIONARY test_01190.dict; SHOW CREATE DICTIONARY test_01190.dict; -DROP DATABASE test_01190; - - -DROP TABLE IF EXISTS log; -DROP TABLE IF EXISTS mt; -DROP TABLE IF EXISTS mv; -DROP TABLE IF EXISTS lv; - CREATE TABLE log ENGINE = Log AS SELECT 'test' AS s; SHOW CREATE log; DETACH TABLE log; @@ -58,9 +51,6 @@ DETACH VIEW lv; ATTACH LIVE VIEW lv AS SELECT 1; SHOW CREATE lv; -DROP TABLE log; -DROP TABLE mt; -DROP TABLE mv; -DROP TABLE lv; +DROP DATABASE test_01190; diff --git a/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql b/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql index cf8b2a471c4..55689411ad6 100644 --- a/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql +++ b/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql @@ -1,6 +1,6 @@ DROP DATABASE IF EXISTS dict_db_01224; DROP DATABASE IF EXISTS dict_db_01224_dictionary; -CREATE DATABASE dict_db_01224; +CREATE DATABASE dict_db_01224 ENGINE=Ordinary; CREATE DATABASE dict_db_01224_dictionary Engine=Dictionary; CREATE TABLE dict_db_01224.dict_data (key UInt64, val UInt64) Engine=Memory(); diff --git a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql index a494511ebd8..8980a9fd70b 100644 --- a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql +++ b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql @@ -1,6 +1,6 @@ DROP DATABASE IF EXISTS dict_db_01225; DROP DATABASE IF EXISTS dict_db_01225_dictionary; -CREATE DATABASE dict_db_01225; +CREATE DATABASE dict_db_01225 ENGINE=Ordinary; CREATE DATABASE dict_db_01225_dictionary Engine=Dictionary; CREATE TABLE dict_db_01225.dict_data (key UInt64, val UInt64) Engine=Memory(); diff --git a/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.reference b/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.reference index e3f4955d4cf..fb993e8d572 100644 --- a/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.reference +++ b/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.reference @@ -1,3 +1,3 @@ -CREATE TABLE default.bloom_filter_idx_good\n(\n `u64` UInt64,\n `i32` Int32,\n `f64` Float64,\n `d` Decimal(10, 2),\n `s` String,\n `e` Enum8(\'a\' = 1, \'b\' = 2, \'c\' = 3),\n `dt` Date,\n INDEX bloom_filter_a i32 TYPE bloom_filter(0., 1.) GRANULARITY 1\n)\nENGINE = MergeTree()\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.bloom_filter_idx_good\n(\n `u64` UInt64,\n `i32` Int32,\n `f64` Float64,\n `d` Decimal(10, 2),\n `s` String,\n `e` Enum8(\'a\' = 1, \'b\' = 2, \'c\' = 3),\n `dt` Date,\n INDEX bloom_filter_a i32 TYPE bloom_filter(-0.1) GRANULARITY 1\n)\nENGINE = MergeTree()\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.bloom_filter_idx_good\n(\n `u64` UInt64,\n `i32` Int32,\n `f64` Float64,\n `d` Decimal(10, 2),\n `s` String,\n `e` Enum8(\'a\' = 1, \'b\' = 2, \'c\' = 3),\n `dt` Date,\n INDEX bloom_filter_a i32 TYPE bloom_filter(1.01) GRANULARITY 1\n)\nENGINE = MergeTree()\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01249.bloom_filter_idx_good\n(\n `u64` UInt64,\n `i32` Int32,\n `f64` Float64,\n `d` Decimal(10, 2),\n `s` String,\n `e` Enum8(\'a\' = 1, \'b\' = 2, \'c\' = 3),\n `dt` Date,\n INDEX bloom_filter_a i32 TYPE bloom_filter(0., 1.) GRANULARITY 1\n)\nENGINE = MergeTree()\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01249.bloom_filter_idx_good\n(\n `u64` UInt64,\n `i32` Int32,\n `f64` Float64,\n `d` Decimal(10, 2),\n `s` String,\n `e` Enum8(\'a\' = 1, \'b\' = 2, \'c\' = 3),\n `dt` Date,\n INDEX bloom_filter_a i32 TYPE bloom_filter(-0.1) GRANULARITY 1\n)\nENGINE = MergeTree()\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01249.bloom_filter_idx_good\n(\n `u64` UInt64,\n `i32` Int32,\n `f64` Float64,\n `d` Decimal(10, 2),\n `s` String,\n `e` Enum8(\'a\' = 1, \'b\' = 2, \'c\' = 3),\n `dt` Date,\n INDEX bloom_filter_a i32 TYPE bloom_filter(1.01) GRANULARITY 1\n)\nENGINE = MergeTree()\nORDER BY u64\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.sql b/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.sql index b60fbc05457..7bf334582a2 100644 --- a/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.sql +++ b/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.sql @@ -1,3 +1,7 @@ +DROP DATABASE IF EXISTS test_01249; +CREATE DATABASE test_01249 ENGINE=Ordinary; +USE test_01249; + CREATE TABLE bloom_filter_idx_good(`u64` UInt64, `i32` Int32, `f64` Float64, `d` Decimal(10, 2), `s` String, `e` Enum8('a' = 1, 'b' = 2, 'c' = 3), `dt` Date, INDEX bloom_filter_a i32 TYPE bloom_filter(0, 1) GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192; -- { serverError 42 } CREATE TABLE bloom_filter_idx_good(`u64` UInt64, `i32` Int32, `f64` Float64, `d` Decimal(10, 2), `s` String, `e` Enum8('a' = 1, 'b' = 2, 'c' = 3), `dt` Date, INDEX bloom_filter_a i32 TYPE bloom_filter(-0.1) GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192; -- { serverError 36 } CREATE TABLE bloom_filter_idx_good(`u64` UInt64, `i32` Int32, `f64` Float64, `d` Decimal(10, 2), `s` String, `e` Enum8('a' = 1, 'b' = 2, 'c' = 3), `dt` Date, INDEX bloom_filter_a i32 TYPE bloom_filter(1.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192; -- { serverError 36 } @@ -14,4 +18,4 @@ DROP TABLE IF EXISTS bloom_filter_idx_good; ATTACH TABLE bloom_filter_idx_good(`u64` UInt64, `i32` Int32, `f64` Float64, `d` Decimal(10, 2), `s` String, `e` Enum8('a' = 1, 'b' = 2, 'c' = 3), `dt` Date, INDEX bloom_filter_a i32 TYPE bloom_filter(1.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192; SHOW CREATE TABLE bloom_filter_idx_good; -DROP TABLE IF EXISTS bloom_filter_idx_good; +DROP DATABASE test_01249; diff --git a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh index 5bbec57a236..ce1f08b4c51 100755 --- a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh +++ b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh @@ -5,16 +5,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS r;" +$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS test_01320" +$CLICKHOUSE_CLIENT --query "CREATE DATABASE test_01320 ENGINE=Ordinary" function thread1() { - while true; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE r (x UInt64) ENGINE = ReplicatedMergeTree('/test_01320/table', 'r') ORDER BY x; DROP TABLE r;"; done + while true; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE test_01320.r (x UInt64) ENGINE = ReplicatedMergeTree('/test_01320/table', 'r') ORDER BY x; DROP TABLE test_01320.r;"; done } function thread2() { - while true; do $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA r" 2>/dev/null; done + while true; do $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA test_01320.r" 2>/dev/null; done } export -f thread1 @@ -25,4 +26,4 @@ timeout 10 bash -c thread2 & wait -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS r;" +$CLICKHOUSE_CLIENT --query "DROP DATABASE test_01320" diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index e4713b2d960..4ffa2bd74cb 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -90,17 +90,17 @@ ], "database-atomic": [ /// Inner tables of materialized views have different names - "00738_lock_for_inner_table", - "00609_mv_index_in_in", - "00510_materizlized_view_and_deduplication_zookeeper", + ///"00738_lock_for_inner_table", + ///"00609_mv_index_in_in", + ///"00510_materizlized_view_and_deduplication_zookeeper", /// Different database engine - "00604_show_create_database", + ///"00604_show_create_database", /// UUID must be specified in ATTACH TABLE - "01190_full_attach_syntax", + ///"01190_full_attach_syntax", /// Assumes blocking DROP - "01320_create_sync_race_condition", + ///"01320_create_sync_race_condition", /// Internal distionary name is different - "01225_show_create_table_from_dictionary", - "01224_no_superfluous_dict_reload" + ///"01225_show_create_table_from_dictionary", + ///"01224_no_superfluous_dict_reload" ] }