From 620f9d64b5dd83e0c83aaa28459f6ff26788ee9f Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Wed, 24 Apr 2024 14:17:00 -0700 Subject: [PATCH 1/5] [Docs] Remove experimental tag from Replicated database engine --- docs/en/engines/database-engines/replicated.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/replicated.md b/docs/en/engines/database-engines/replicated.md index 27bdc20ec57..4fcd9272f5f 100644 --- a/docs/en/engines/database-engines/replicated.md +++ b/docs/en/engines/database-engines/replicated.md @@ -4,7 +4,7 @@ sidebar_position: 30 sidebar_label: Replicated --- -# [experimental] Replicated +# Replicated The engine is based on the [Atomic](../../engines/database-engines/atomic.md) engine. It supports replication of metadata via DDL log being written to ZooKeeper and executed on all of the replicas for a given database. From f43a155aa9500d9aa2a1c45dfee8b6b9fbedc15c Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 25 Apr 2024 13:05:14 +0000 Subject: [PATCH 2/5] Make allow_experimental_database_replicated obsolete. --- src/Interpreters/InterpreterCreateQuery.cpp | 9 --------- 1 file changed, 9 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 1f38449fb32..7cda0267fdf 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -258,15 +258,6 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) "Enable allow_experimental_database_materialized_mysql to use it"); } - if (create.storage->engine->name == "Replicated" - && !getContext()->getSettingsRef().allow_experimental_database_replicated - && !internal && !create.attach) - { - throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, - "Replicated is an experimental database engine. " - "Enable allow_experimental_database_replicated to use it"); - } - if (create.storage->engine->name == "MaterializedPostgreSQL" && !getContext()->getSettingsRef().allow_experimental_database_materialized_postgresql && !internal && !create.attach) From 2724a08cbc6c7f8ba982b90a061a46679e3cf3b8 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 25 Apr 2024 13:06:09 +0000 Subject: [PATCH 3/5] Better --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 375bdb1c516..c1333b3a204 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -706,7 +706,6 @@ class IColumn; M(Bool, engine_file_skip_empty_files, false, "Allows to skip empty files in file table engine", 0) \ M(Bool, engine_url_skip_empty_files, false, "Allows to skip empty files in url table engine", 0) \ M(Bool, enable_url_encoding, true, " Allows to enable/disable decoding/encoding path in uri in URL table engine", 0) \ - M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \ M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \ M(Bool, database_replicated_enforce_synchronous_settings, false, "Enforces synchronous waiting for some queries (see also database_atomic_wait_for_drop_and_detach_synchronously, mutation_sync, alter_sync). Not recommended to enable these settings.", 0) \ M(UInt64, max_distributed_depth, 5, "Maximum distributed query depth", 0) \ @@ -933,6 +932,7 @@ class IColumn; MAKE_OBSOLETE(M, Bool, allow_experimental_query_cache, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_alter_materialized_view_structure, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_shared_merge_tree, true) \ + MAKE_OBSOLETE(M, Bool, allow_experimental_database_replicated, true) \ \ MAKE_OBSOLETE(M, Milliseconds, async_insert_stale_timeout_ms, 0) \ MAKE_OBSOLETE(M, StreamingHandleErrorMode, handle_kafka_error_mode, StreamingHandleErrorMode::DEFAULT) \ From 9c3ebc5a99c2988bcfcc739d72bcc7d844aa7880 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 25 Apr 2024 15:39:42 +0200 Subject: [PATCH 4/5] Added setting change to the history --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index d3b5de06e70..d999f919112 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -92,6 +92,7 @@ static std::map sett {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, {"first_day_of_week", "Monday", "Monday", "Added a setting for the first day of the week for date/time functions"}, + {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, }}, {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, From a8683a8bd5b880b5c4d04a40e5446477a7b01436 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 25 Apr 2024 22:13:14 +0200 Subject: [PATCH 5/5] Remove the setting from tests --- docs/en/operations/settings/settings.md | 13 ------------- docs/ru/operations/settings/settings.md | 11 ----------- tests/ci/stress.py | 1 - tests/config/users.d/database_replicated.xml | 1 - .../configs/settings.xml | 1 - .../configs/allow_database_types.xml | 1 - .../configs/allow_database_types.xml | 1 - tests/integration/test_disk_over_web_server/test.py | 2 -- .../configs/drop_if_empty_check.xml | 1 - tests/integration/test_drop_if_empty/test.py | 5 ----- tests/integration/test_external_cluster/test.py | 1 - .../test_replicated_database/configs/settings.xml | 1 - .../configs/settings.xml | 1 - .../01111_create_drop_replicated_db_stress.sh | 2 +- .../01148_zookeeper_path_macros_unfolding.sql | 1 - .../02021_create_database_with_comment.sh | 1 - .../02232_allow_only_replicated_engine.sh | 2 +- .../02400_create_table_on_cluster_normalization.sql | 1 - .../02445_replicated_db_alter_partition.sh | 6 +++--- .../0_stateless/02447_drop_database_replica.sh | 8 ++++---- ...3_add_engine_full_column_to_system_databases.sql | 4 +--- ...2514_database_replicated_no_arguments_for_rmt.sh | 2 +- .../02710_default_replicated_parameters.sql | 2 -- .../02762_replicated_database_no_args.sql | 1 - ...3_replicated_database_forbid_create_as_select.sh | 2 +- 25 files changed, 12 insertions(+), 60 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 869a0be2574..ad997e1f57b 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3931,19 +3931,6 @@ For example, `avg(if(cond, col, null))` can be rewritten to `avgOrNullIf(cond, c Supported only with experimental analyzer (`allow_experimental_analyzer = 1`). ::: -## allow_experimental_database_replicated {#allow_experimental_database_replicated} - -Enables to create databases with [Replicated](../../engines/database-engines/replicated.md) engine. - -Possible values: - -- 0 — Disabled. -- 1 — Enabled. - -Default value: `0`. - -Cloud default value: `1`. - ## database_replicated_initial_query_timeout_sec {#database_replicated_initial_query_timeout_sec} Sets how long initial DDL query should wait for Replicated database to process previous DDL queue entries in seconds. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index f9456e34a56..2b3607dcf08 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3447,17 +3447,6 @@ SELECT FROM fuse_tbl ``` -## allow_experimental_database_replicated {#allow_experimental_database_replicated} - -Позволяет создавать базы данных с движком [Replicated](../../engines/database-engines/replicated.md). - -Возможные значения: - -- 0 — Disabled. -- 1 — Enabled. - -Значение по умолчанию: `0`. - ## database_replicated_initial_query_timeout_sec {#database_replicated_initial_query_timeout_sec} Устанавливает, как долго начальный DDL-запрос должен ждать, пока реплицированная база данных прецессирует предыдущие записи очереди DDL в секундах. diff --git a/tests/ci/stress.py b/tests/ci/stress.py index b1f5a28ec9e..a9f5916464d 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -19,7 +19,6 @@ def get_options(i: int, upgrade_check: bool) -> str: if i % 3 == 2 and not upgrade_check: options.append(f'''--db-engine="Replicated('/test/db/test_{i}', 's1', 'r1')"''') - client_options.append("allow_experimental_database_replicated=1") client_options.append("enable_deflate_qpl_codec=1") client_options.append("enable_zstd_qat_codec=1") diff --git a/tests/config/users.d/database_replicated.xml b/tests/config/users.d/database_replicated.xml index 2b96e7418b6..c049c3559fc 100644 --- a/tests/config/users.d/database_replicated.xml +++ b/tests/config/users.d/database_replicated.xml @@ -1,7 +1,6 @@ - 1 none 120 120 diff --git a/tests/integration/test_attach_table_from_s3_plain_readonly/configs/settings.xml b/tests/integration/test_attach_table_from_s3_plain_readonly/configs/settings.xml index 3e6d615557d..16caee9ba20 100644 --- a/tests/integration/test_attach_table_from_s3_plain_readonly/configs/settings.xml +++ b/tests/integration/test_attach_table_from_s3_plain_readonly/configs/settings.xml @@ -1,7 +1,6 @@ - 1 diff --git a/tests/integration/test_backup_restore_on_cluster/configs/allow_database_types.xml b/tests/integration/test_backup_restore_on_cluster/configs/allow_database_types.xml index e0e026210b1..8ffd2f27a62 100644 --- a/tests/integration/test_backup_restore_on_cluster/configs/allow_database_types.xml +++ b/tests/integration/test_backup_restore_on_cluster/configs/allow_database_types.xml @@ -1,7 +1,6 @@ - 1 1 diff --git a/tests/integration/test_dictionaries_replace/configs/allow_database_types.xml b/tests/integration/test_dictionaries_replace/configs/allow_database_types.xml index 0434df06457..db0dd71de56 100644 --- a/tests/integration/test_dictionaries_replace/configs/allow_database_types.xml +++ b/tests/integration/test_dictionaries_replace/configs/allow_database_types.xml @@ -1,7 +1,6 @@ - 1 diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index fba51949ef0..8ddc1ff3c31 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -295,7 +295,6 @@ def test_replicated_database(cluster): node1 = cluster.instances["node3"] node1.query( "CREATE DATABASE rdb ENGINE=Replicated('/test/rdb', 's1', 'r1')", - settings={"allow_experimental_database_replicated": 1}, ) global uuids @@ -312,7 +311,6 @@ def test_replicated_database(cluster): node2 = cluster.instances["node2"] node2.query( "CREATE DATABASE rdb ENGINE=Replicated('/test/rdb', 's1', 'r2')", - settings={"allow_experimental_database_replicated": 1}, ) node2.query("SYSTEM SYNC DATABASE REPLICA rdb") diff --git a/tests/integration/test_drop_if_empty/configs/drop_if_empty_check.xml b/tests/integration/test_drop_if_empty/configs/drop_if_empty_check.xml index 34a78ac9919..dbad8899646 100644 --- a/tests/integration/test_drop_if_empty/configs/drop_if_empty_check.xml +++ b/tests/integration/test_drop_if_empty/configs/drop_if_empty_check.xml @@ -3,7 +3,6 @@ 9000 - 1 diff --git a/tests/integration/test_drop_if_empty/test.py b/tests/integration/test_drop_if_empty/test.py index d96936eb826..251ed302b38 100644 --- a/tests/integration/test_drop_if_empty/test.py +++ b/tests/integration/test_drop_if_empty/test.py @@ -37,18 +37,13 @@ def start_cluster(): def test_drop_if_empty(start_cluster): - settings = { - "allow_experimental_database_replicated": 1, - } node1.query( "CREATE DATABASE replicateddb " "ENGINE = Replicated('/clickhouse/databases/replicateddb', 'shard1', 'node1')", - settings=settings, ) node2.query( "CREATE DATABASE replicateddb " "ENGINE = Replicated('/clickhouse/databases/replicateddb', 'shard1', 'node2')", - settings=settings, ) node1.query( "CREATE TABLE default.tbl ON CLUSTER 'cluster' (" diff --git a/tests/integration/test_external_cluster/test.py b/tests/integration/test_external_cluster/test.py index 6fa9cd16dbc..306ecf66bc7 100644 --- a/tests/integration/test_external_cluster/test.py +++ b/tests/integration/test_external_cluster/test.py @@ -66,7 +66,6 @@ def test_ddl(started_cluster): def test_ddl_replicated(started_cluster): control_node.query( "CREATE DATABASE test_db ON CLUSTER 'external' ENGINE=Replicated('/replicated')", - settings={"allow_experimental_database_replicated": 1}, ) # Exception is expected assert "It's not initial query" in control_node.query_and_get_error( diff --git a/tests/integration/test_replicated_database/configs/settings.xml b/tests/integration/test_replicated_database/configs/settings.xml index 7c0e60a044e..c637fe8eead 100644 --- a/tests/integration/test_replicated_database/configs/settings.xml +++ b/tests/integration/test_replicated_database/configs/settings.xml @@ -2,7 +2,6 @@ 1 - 1 1 0 0 diff --git a/tests/integration/test_replicated_database_cluster_groups/configs/settings.xml b/tests/integration/test_replicated_database_cluster_groups/configs/settings.xml index 5666ffeace8..872a6cd0b6c 100644 --- a/tests/integration/test_replicated_database_cluster_groups/configs/settings.xml +++ b/tests/integration/test_replicated_database_cluster_groups/configs/settings.xml @@ -2,7 +2,6 @@ 1 - 1 1 diff --git a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh index 8ebe1807a1b..91f7a276ea3 100755 --- a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh +++ b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh @@ -15,7 +15,7 @@ function create_db() SUFFIX=$(($RANDOM % 16)) # Multiple database replicas on one server are actually not supported (until we have namespaces). # So CREATE TABLE queries will fail on all replicas except one. But it's still makes sense for a stress test. - $CLICKHOUSE_CLIENT --allow_experimental_database_replicated=1 --query \ + $CLICKHOUSE_CLIENT --query \ "create database if not exists ${CLICKHOUSE_DATABASE}_repl_01111_$SUFFIX engine=Replicated('/test/01111/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '$SHARD', '$REPLICA')" \ 2>&1| grep -Fa "Exception: " | grep -Fv "REPLICA_ALREADY_EXISTS" | grep -Fiv "Will not try to start it up" | \ grep -Fv "Coordination::Exception" | grep -Fv "already contains some data and it does not look like Replicated database path" diff --git a/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql b/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql index fc3fcb34fc0..de244e64999 100644 --- a/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql +++ b/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql @@ -43,7 +43,6 @@ DROP DATABASE test_01148_atomic; DROP TABLE rmt; DROP TABLE rmt1; -SET allow_experimental_database_replicated=1; DROP DATABASE IF EXISTS imdb_01148; CREATE DATABASE imdb_01148 ENGINE = Replicated('/test/databases/imdb_01148', '{shard}', '{replica}'); CREATE TABLE imdb_01148.movie_directors (`director_id` UInt64, `movie_id` UInt64) ENGINE = ReplicatedMergeTree ORDER BY (director_id, movie_id) SETTINGS index_granularity = 8192; diff --git a/tests/queries/0_stateless/02021_create_database_with_comment.sh b/tests/queries/0_stateless/02021_create_database_with_comment.sh index 8432963e059..f77397dc482 100755 --- a/tests/queries/0_stateless/02021_create_database_with_comment.sh +++ b/tests/queries/0_stateless/02021_create_database_with_comment.sh @@ -34,5 +34,4 @@ test_db_comments "Ordinary" test_db_comments "Lazy(1)" # test_db_comments "MySQL('127.0.0.1:9004', 'default', 'default', '')" # fails due to CH internal reasons # test_db_comments "SQLite('dummy_sqlitedb')" -## needs to be explicitly enabled with `SET allow_experimental_database_replicated=1` # test_db_comments "Replicated('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') ORDER BY k" diff --git a/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh b/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh index 791102b9cbd..d1a3825d286 100755 --- a/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh +++ b/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh @@ -10,7 +10,7 @@ ${CLICKHOUSE_CLIENT} -q "create table mute_stylecheck (x UInt32) engine = Replic ${CLICKHOUSE_CLIENT} -q "CREATE USER user_${CLICKHOUSE_DATABASE} settings database_replicated_allow_only_replicated_engine=1" ${CLICKHOUSE_CLIENT} -q "GRANT CREATE TABLE ON ${CLICKHOUSE_DATABASE}_db.* TO user_${CLICKHOUSE_DATABASE}" ${CLICKHOUSE_CLIENT} -q "GRANT TABLE ENGINE ON Memory, TABLE ENGINE ON MergeTree, TABLE ENGINE ON ReplicatedMergeTree TO user_${CLICKHOUSE_DATABASE}" -${CLICKHOUSE_CLIENT} --allow_experimental_database_replicated=1 --query "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')" +${CLICKHOUSE_CLIENT} -q "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')" ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_memory (x UInt32) engine = Memory;" ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;" 2>&1 | grep -o "Only tables with a Replicated engine" ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;" diff --git a/tests/queries/0_stateless/02400_create_table_on_cluster_normalization.sql b/tests/queries/0_stateless/02400_create_table_on_cluster_normalization.sql index 54e4ccf6762..85831a21b47 100644 --- a/tests/queries/0_stateless/02400_create_table_on_cluster_normalization.sql +++ b/tests/queries/0_stateless/02400_create_table_on_cluster_normalization.sql @@ -21,7 +21,6 @@ select * from t_l5ydey order by c_qv5rv; show create t_l5ydey; -- Correct error code if creating database with the same path as table has -set allow_experimental_database_replicated=1; create database local_t_l5ydey engine=Replicated('/clickhouse/tables/test_' || currentDatabase() || '/{shard}/local_t_l5ydey', '1', '1'); -- { serverError BAD_ARGUMENTS } drop table local_t_l5ydey; diff --git a/tests/queries/0_stateless/02445_replicated_db_alter_partition.sh b/tests/queries/0_stateless/02445_replicated_db_alter_partition.sh index e07737225db..4d9048354a1 100755 --- a/tests/queries/0_stateless/02445_replicated_db_alter_partition.sh +++ b/tests/queries/0_stateless/02445_replicated_db_alter_partition.sh @@ -8,7 +8,7 @@ db="rdb_$CLICKHOUSE_DATABASE" db2="${db}_2" db3="${db}_3" -$CLICKHOUSE_CLIENT --allow_experimental_database_replicated=1 -q "create database $db engine=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', 's1', 'r1')" +$CLICKHOUSE_CLIENT -q "create database $db engine=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', 's1', 'r1')" $CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none -q "create table $db.mt (n int) engine=MergeTree order by tuple()" $CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none -q "create table $db.rmt (n int) engine=ReplicatedMergeTree order by tuple()" @@ -16,8 +16,8 @@ $CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none -q "create table $db.rmt ( $CLICKHOUSE_CLIENT -q "insert into $db.rmt values (0), (1)" $CLICKHOUSE_CLIENT -q "insert into $db.mt values (0), (1)" -$CLICKHOUSE_CLIENT --allow_experimental_database_replicated=1 -q "create database $db2 engine=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', 's1', 'r2')" -$CLICKHOUSE_CLIENT --allow_experimental_database_replicated=1 -q "create database $db3 engine=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', 's2', 'r1')" +$CLICKHOUSE_CLIENT -q "create database $db2 engine=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', 's1', 'r2')" +$CLICKHOUSE_CLIENT -q "create database $db3 engine=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', 's2', 'r1')" $CLICKHOUSE_CLIENT -q "alter table $db.mt drop partition id 'all', add column m int" 2>&1| grep -Eo "not allowed to execute ALTERs of different types" | head -1 $CLICKHOUSE_CLIENT -q "alter table $db.rmt drop partition id 'all', add column m int" 2>&1| grep -Eo "not allowed to execute ALTERs of different types" | head -1 diff --git a/tests/queries/0_stateless/02447_drop_database_replica.sh b/tests/queries/0_stateless/02447_drop_database_replica.sh index 5c4604bc8cd..1604d527f2b 100755 --- a/tests/queries/0_stateless/02447_drop_database_replica.sh +++ b/tests/queries/0_stateless/02447_drop_database_replica.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) db="rdb_$CLICKHOUSE_DATABASE" $CLICKHOUSE_CLIENT -q "system flush logs" -$CLICKHOUSE_CLIENT --allow_experimental_database_replicated=1 -q "create database $db engine=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', 's1', 'r1')" +$CLICKHOUSE_CLIENT -q "create database $db engine=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', 's1', 'r1')" $CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none -q "create table $db.t as system.query_log" # Suppress style check: current_database=$CLICKHOUSE_DATABASE $CLICKHOUSE_CLIENT -q "show tables from $db" @@ -26,8 +26,8 @@ $CLICKHOUSE_CLIENT -q "system drop database replica 's2/r1' from zkpath '/test/$ db2="${db}_2" db3="${db}_3" -$CLICKHOUSE_CLIENT --allow_experimental_database_replicated=1 -q "create database $db2 engine=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', 's1', 'r2')" -$CLICKHOUSE_CLIENT --allow_experimental_database_replicated=1 -q "create database $db3 engine=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', 's2', 'r1')" +$CLICKHOUSE_CLIENT -q "create database $db2 engine=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', 's1', 'r2')" +$CLICKHOUSE_CLIENT -q "create database $db3 engine=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', 's2', 'r1')" $CLICKHOUSE_CLIENT -q "system sync database replica $db" $CLICKHOUSE_CLIENT -q "select cluster, shard_num, replica_num, database_shard_name, database_replica_name, is_active from system.clusters where cluster='$db' and shard_num=1 and replica_num=1" $CLICKHOUSE_CLIENT -q "system drop database replica 's1|r1' from database $db2" 2>&1| grep -Fac "is active, cannot drop it" @@ -56,7 +56,7 @@ $CLICKHOUSE_CLIENT --distributed_ddl_output_mode=none -q "create table $db.t2 as $CLICKHOUSE_CLIENT -q "show tables from $db" db4="${db}_4" -$CLICKHOUSE_CLIENT --allow_experimental_database_replicated=1 -q "create database $db4 engine=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', 's1', 'r1')" +$CLICKHOUSE_CLIENT -q "create database $db4 engine=Replicated('/test/$CLICKHOUSE_DATABASE/rdb', 's1', 'r1')" $CLICKHOUSE_CLIENT -q "system sync database replica $db4" $CLICKHOUSE_CLIENT -q "select cluster, shard_num, replica_num, database_shard_name, database_replica_name, is_active from system.clusters where cluster='$db4'" diff --git a/tests/queries/0_stateless/02483_add_engine_full_column_to_system_databases.sql b/tests/queries/0_stateless/02483_add_engine_full_column_to_system_databases.sql index 1f35a0f6273..c4e54e3cc45 100644 --- a/tests/queries/0_stateless/02483_add_engine_full_column_to_system_databases.sql +++ b/tests/queries/0_stateless/02483_add_engine_full_column_to_system_databases.sql @@ -1,8 +1,6 @@ -- Tags: no-parallel DROP DATABASE IF EXISTS replicated_database_test; -SET allow_experimental_database_replicated=1; CREATE DATABASE IF NOT EXISTS replicated_database_test ENGINE = Replicated('some/path/' || currentDatabase() || '/replicated_database_test', 'shard_1', 'replica_1') SETTINGS max_broken_tables_ratio=1; SELECT engine_full FROM system.databases WHERE name = 'replicated_database_test'; -DROP DATABASE IF EXISTS replicated_database_test; - +DROP DATABASE IF EXISTS replicated_database_test; diff --git a/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh b/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh index a050f7b00d7..a0f228e6af4 100755 --- a/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh +++ b/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh @@ -13,7 +13,7 @@ ${CLICKHOUSE_CLIENT} -q "create table mute_stylecheck (x UInt32) engine = Replic ${CLICKHOUSE_CLIENT} -q "CREATE USER user_${CLICKHOUSE_DATABASE} settings database_replicated_allow_replicated_engine_arguments=0" ${CLICKHOUSE_CLIENT} -q "GRANT CREATE TABLE ON ${CLICKHOUSE_DATABASE}_db.* TO user_${CLICKHOUSE_DATABASE}" ${CLICKHOUSE_CLIENT} -q "GRANT TABLE ENGINE ON ReplicatedMergeTree TO user_${CLICKHOUSE_DATABASE}" -${CLICKHOUSE_CLIENT} --allow_experimental_database_replicated=1 --query "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')" +${CLICKHOUSE_CLIENT} -q "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')" ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_ok (x UInt32) engine = ReplicatedMergeTree order by x;" ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_fail (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/root/{shard}', '{replica}') order by x; -- { serverError 80 }" ${CLICKHOUSE_CLIENT} --query "DROP DATABASE ${CLICKHOUSE_DATABASE}_db" diff --git a/tests/queries/0_stateless/02710_default_replicated_parameters.sql b/tests/queries/0_stateless/02710_default_replicated_parameters.sql index 279b7e81bdd..faeea6cdf02 100644 --- a/tests/queries/0_stateless/02710_default_replicated_parameters.sql +++ b/tests/queries/0_stateless/02710_default_replicated_parameters.sql @@ -1,7 +1,5 @@ -- Tags: no-parallel -SET allow_experimental_database_replicated=1; - DROP DATABASE IF EXISTS replicated_database_params; CREATE DATABASE replicated_database_params ENGINE = Replicated('some/path/' || currentDatabase() || '/replicated_database_params'); diff --git a/tests/queries/0_stateless/02762_replicated_database_no_args.sql b/tests/queries/0_stateless/02762_replicated_database_no_args.sql index 1409a059b02..b811bb7d428 100644 --- a/tests/queries/0_stateless/02762_replicated_database_no_args.sql +++ b/tests/queries/0_stateless/02762_replicated_database_no_args.sql @@ -1,4 +1,3 @@ -- Tags: no-parallel -set allow_experimental_database_replicated=1; create database replicated_db_no_args engine=Replicated; -- { serverError BAD_ARGUMENTS } diff --git a/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.sh b/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.sh index c295f5be43b..8a6904b6bd7 100755 --- a/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.sh +++ b/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --allow_experimental_database_replicated=1 --query "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')" +${CLICKHOUSE_CLIENT} --query "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')" # Non-replicated engines are allowed ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.test (id UInt64) ENGINE = MergeTree() ORDER BY id AS SELECT 1" # Replicated storafes are forbidden