diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 7428e6cd6ca..250cddb09dc 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -43,6 +43,19 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name AS [db2.]name2 [ENGINE = engine] Creates a table with the same structure as another table. You can specify a different engine for the table. If the engine is not specified, the same engine will be used as for the `db2.name2` table. +### With a Schema and Data Cloned from Another Table + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name CLONE AS [db2.]name2 [ENGINE = engine] +``` + +Creates a table with the same structure as another table. You can specify a different engine for the table. If the engine is not specified, the same engine will be used as for the `db2.name2` table. After the new table is created, all partitions from `db2.name2` are attached to it. In other words, the data of `db2.name2` is cloned into `db.table_name` upon creation. This query is equivalent to the following: + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name AS [db2.]name2 [ENGINE = engine]; +ALTER TABLE [db.]table_name ATTACH PARTITION ALL FROM [db2].name2; +``` + ### From a Table Function ``` sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 60d4abd0ef8..a120cfca214 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -5,17 +5,20 @@ #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include +#include +#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include @@ -812,6 +815,8 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti /// as_storage->getColumns() and setEngine(...) must be called under structure lock of other_table for CREATE ... AS other_table. as_storage_lock = as_storage->lockForShare(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); + + auto as_storage_metadata = as_storage->getInMemoryMetadataPtr(); properties.columns = as_storage_metadata->getColumns(); @@ -832,6 +837,43 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti } properties.constraints = as_storage_metadata->getConstraints(); + + if (create.is_clone_as) + { + if (!endsWith(as_storage->getName(), "MergeTree")) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support CLONE AS from tables of the MergeTree family"); + + if (create.storage) + { + if (!endsWith(create.storage->engine->name, "MergeTree")) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support CLONE AS with tables of the MergeTree family"); + + /// Ensure that as_storage and the new storage has the same primary key, sorting key and partition key + auto query_to_string = [](const IAST * ast) { return ast ? queryToString(*ast) : ""; }; + + const String as_storage_sorting_key_str = query_to_string(as_storage_metadata->getSortingKeyAST().get()); + const String as_storage_primary_key_str = query_to_string(as_storage_metadata->getPrimaryKeyAST().get()); + const String as_storage_partition_key_str = query_to_string(as_storage_metadata->getPartitionKeyAST().get()); + + const String storage_sorting_key_str = query_to_string(create.storage->order_by); + const String storage_primary_key_str = query_to_string(create.storage->primary_key); + const String storage_partition_key_str = query_to_string(create.storage->partition_by); + + if (as_storage_sorting_key_str != storage_sorting_key_str) + { + /// It is possible that the storage only has primary key and an empty sorting key, and as_storage has both primary key and sorting key with the same value. + if (as_storage_sorting_key_str != as_storage_primary_key_str || as_storage_sorting_key_str != storage_primary_key_str) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different ordering"); + } + } + if (as_storage_partition_key_str != storage_partition_key_str) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); + + if (as_storage_primary_key_str != storage_primary_key_str) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different primary key"); + } + } } else if (create.select) { @@ -1498,37 +1540,48 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (create.select && create.is_materialized_view && mode <= LoadingStrictnessLevel::CREATE) validateMaterializedViewColumnsAndEngine(create, properties, database); - bool allow_heavy_populate = getContext()->getSettingsRef().database_replicated_allow_heavy_create && create.is_populate; - if (!allow_heavy_populate && database && database->getEngineName() == "Replicated" && (create.select || create.is_populate)) + bool is_storage_replicated = false; + + if (create.storage && isReplicated(*create.storage)) + is_storage_replicated = true; + + if (create.targets) { - bool is_storage_replicated = false; - - if (create.storage && isReplicated(*create.storage)) - is_storage_replicated = true; - - if (create.targets) + for (const auto & inner_table_engine : create.targets->getInnerEngines()) { - for (const auto & inner_table_engine : create.targets->getInnerEngines()) + if (isReplicated(*inner_table_engine)) + is_storage_replicated = true; + } + } + + bool allow_heavy_populate = getContext()->getSettingsRef().database_replicated_allow_heavy_create && create.is_populate; + if (!allow_heavy_populate && database && database->getEngineName() == "Replicated" && (create.select || create.is_populate)) + { + const bool allow_create_select_for_replicated + = (create.isView() && !create.is_populate) || create.is_create_empty || !is_storage_replicated; + if (!allow_create_select_for_replicated) { - if (isReplicated(*inner_table_engine)) - is_storage_replicated = true; + /// POPULATE can be enabled with setting, provide hint in error message + if (create.is_populate) + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "CREATE with POPULATE is not supported with Replicated databases. Consider using separate CREATE and INSERT " + "queries. " + "Alternatively, you can enable 'database_replicated_allow_heavy_create' setting to allow this operation, use with " + "caution"); + + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "CREATE AS SELECT is not supported with Replicated databases. Consider using separate CREATE and INSERT queries."); } } - const bool allow_create_select_for_replicated = (create.isView() && !create.is_populate) || create.is_create_empty || !is_storage_replicated; - if (!allow_create_select_for_replicated) - { - /// POPULATE can be enabled with setting, provide hint in error message - if (create.is_populate) - throw Exception( - ErrorCodes::SUPPORT_IS_DISABLED, - "CREATE with POPULATE is not supported with Replicated databases. Consider using separate CREATE and INSERT queries. " - "Alternatively, you can enable 'database_replicated_allow_heavy_create' setting to allow this operation, use with caution"); - + if (create.is_clone_as) + { + if (database && database->getEngineName() == "Replicated") throw Exception( ErrorCodes::SUPPORT_IS_DISABLED, - "CREATE AS SELECT is not supported with Replicated databases. Consider using separate CREATE and INSERT queries."); - } + "CREATE CLONE AS is not supported with Replicated databases. Consider using separate CREATE and INSERT queries."); } if (database && database->shouldReplicateQuery(getContext(), query_ptr)) @@ -1969,6 +2022,38 @@ BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create) /* async_isnert */ false).execute(); } + /// If the query is a CREATE TABLE .. CLONE AS ..., attach all partitions of the source table to the newly created table. + if (create.is_clone_as && !as_table_saved.empty() && !create.is_create_empty && !create.is_ordinary_view && !create.is_live_view + && (!(create.is_materialized_view || create.is_window_view) || create.is_populate)) + { + String as_database_name = getContext()->resolveDatabase(create.as_database); + + auto partition = std::make_shared(); + partition->all = true; + + auto command = std::make_shared(); + command->replace = false; + command->type = ASTAlterCommand::REPLACE_PARTITION; + command->partition = command->children.emplace_back(std::move(partition)).get(); + command->from_database = as_database_name; + command->from_table = as_table_saved; + command->to_database = create.getDatabase(); + command->to_table = create.getTable(); + + auto command_list = std::make_shared(); + command_list->children.push_back(command); + + auto query = std::make_shared(); + query->database = create.database; + query->table = create.table; + query->uuid = create.uuid; + auto * alter = query->as(); + + alter->alter_object = ASTAlterQuery::AlterObjectType::TABLE; + alter->set(alter->command_list, command_list); + return InterpreterAlterQuery(query, getContext()).execute(); + } + return {}; } diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index d7f5b8f9702..66422efe660 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -425,9 +425,19 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat settings.ostr << (settings.hilite ? hilite_keyword : "") << " EMPTY" << (settings.hilite ? hilite_none : ""); }; + bool should_add_clone = is_clone_as; + auto add_clone_if_needed = [&] + { + if (!should_add_clone) + return; + should_add_clone = false; + settings.ostr << (settings.hilite ? hilite_keyword : "") << " CLONE" << (settings.hilite ? hilite_none : ""); + }; + if (!as_table.empty()) { add_empty_if_needed(); + add_clone_if_needed(); settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "") << (!as_database.empty() ? backQuoteIfNeed(as_database) + "." : "") << backQuoteIfNeed(as_table); @@ -446,6 +456,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat } add_empty_if_needed(); + add_clone_if_needed(); settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : ""); as_table_function->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index 6be0fa78903..813e56eaf02 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -100,6 +100,7 @@ public: bool is_time_series_table{false}; /// CREATE TABLE ... ENGINE=TimeSeries() ... bool is_populate{false}; bool is_create_empty{false}; /// CREATE TABLE ... EMPTY AS SELECT ... + bool is_clone_as{false}; /// CREATE TABLE ... CLONE AS ... bool replace_view{false}; /// CREATE OR REPLACE VIEW bool has_uuid{false}; // CREATE TABLE x UUID '...' diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 46e08cf3f7e..6fa9e028633 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -84,6 +84,7 @@ namespace DB MR_MACROS(CLEAR_INDEX, "CLEAR INDEX") \ MR_MACROS(CLEAR_PROJECTION, "CLEAR PROJECTION") \ MR_MACROS(CLEAR_STATISTICS, "CLEAR STATISTICS") \ + MR_MACROS(CLONE_AS, "CLONE AS") \ MR_MACROS(CLUSTER, "CLUSTER") \ MR_MACROS(CLUSTERS, "CLUSTERS") \ MR_MACROS(CN, "CN") \ diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 31dc2075db4..8c5b926ef71 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -675,6 +675,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe bool if_not_exists = false; bool is_temporary = false; bool is_create_empty = false; + bool is_clone_as = false; if (s_create.ignore(pos, expected)) { @@ -759,13 +760,18 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe return true; }; - auto need_parse_as_select = [&is_create_empty, &pos, &expected]() + auto need_parse_as_select = [&is_create_empty, &is_clone_as, &pos, &expected]() { if (ParserKeyword{Keyword::EMPTY_AS}.ignore(pos, expected)) { is_create_empty = true; return true; } + if (ParserKeyword{Keyword::CLONE_AS}.ignore(pos, expected)) + { + is_clone_as = true; + return true; + } return ParserKeyword{Keyword::AS}.ignore(pos, expected); }; @@ -893,6 +899,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe query->set(query->select, select); query->set(query->targets, targets); query->is_create_empty = is_create_empty; + query->is_clone_as = is_clone_as; if (from_path) query->attach_from_path = from_path->as().value.safeGet(); diff --git a/tests/queries/0_stateless/03231_create_with_clone_as.reference b/tests/queries/0_stateless/03231_create_with_clone_as.reference new file mode 100644 index 00000000000..d2046e19a23 --- /dev/null +++ b/tests/queries/0_stateless/03231_create_with_clone_as.reference @@ -0,0 +1,42 @@ +CREATE TABLE default.foo_memory\n(\n `x` Int8,\n `y` String\n)\nENGINE = Memory +CREATE TABLE default.foo_file\n(\n `x` Int8,\n `y` String\n)\nENGINE = File(\'TabSeparated\') +CREATE TABLE default.foo_merge_tree\n(\n `x` Int8,\n `y` String\n)\nENGINE = MergeTree\nPRIMARY KEY x\nORDER BY x\nSETTINGS index_granularity = 8192 +1 a +2 b +CREATE TABLE default.clone_as_foo_merge_tree\n(\n `x` Int8,\n `y` String\n)\nENGINE = MergeTree\nPRIMARY KEY x\nORDER BY x\nSETTINGS index_granularity = 8192 +from foo_merge_tree +1 a +2 b +from clone_as_foo_merge_tree +1 a +2 b +from clone_as_foo_merge_tree_p_x +1 a +2 b +CREATE TABLE default.foo_replacing_merge_tree\n(\n `x` Int8,\n `y` String\n)\nENGINE = ReplacingMergeTree\nPRIMARY KEY x\nORDER BY x\nSETTINGS index_granularity = 8192 +1 a +2 b +CREATE TABLE default.clone_as_foo_replacing_merge_tree\n(\n `x` Int8,\n `y` String\n)\nENGINE = ReplacingMergeTree\nPRIMARY KEY x\nORDER BY x\nSETTINGS index_granularity = 8192 +from foo_replacing_merge_tree +1 a +2 b +from clone_as_foo_replacing_merge_tree +1 a +2 b +from clone_as_foo_replacing_merge_tree_p_x +1 a +2 b +CREATE TABLE default.foo_replicated_merge_tree\n(\n `x` Int8,\n `y` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/test_foo_replicated_merge_tree\', \'r1\')\nPRIMARY KEY x\nORDER BY x\nSETTINGS index_granularity = 8192 +from foo_replicated_merge_tree +1 a +2 b +CREATE TABLE default.clone_as_foo_replicated_merge_tree_p_x\n(\n `x` Int8,\n `y` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/clone_as_foo_replicated_merge_tree_p_x\', \'r1\')\nPRIMARY KEY x\nORDER BY x\nSETTINGS index_granularity = 8192 +from clone_as_foo_replicated_merge_tree_p_x +1 a +2 b +s1 r1 OK 0 0 +CREATE TABLE default_1.foo_merge_tree\n(\n `x` Int8,\n `y` String\n)\nENGINE = MergeTree\nPRIMARY KEY x\nORDER BY x\nSETTINGS index_granularity = 8192 +from foo_merge_tree +1 a +2 b +s1 r1 OK 0 0 diff --git a/tests/queries/0_stateless/03231_create_with_clone_as.sql b/tests/queries/0_stateless/03231_create_with_clone_as.sql new file mode 100644 index 00000000000..c5793206e88 --- /dev/null +++ b/tests/queries/0_stateless/03231_create_with_clone_as.sql @@ -0,0 +1,121 @@ +-- Tags: no-replicated-database +-- Tag no-replicated-database: Unsupported type of CREATE TABLE ... CLONE AS ... query + +DROP TABLE IF EXISTS foo_memory; +DROP TABLE IF EXISTS clone_as_foo_memory; +DROP TABLE IF EXISTS foo_file; +DROP TABLE IF EXISTS clone_as_foo_file; +DROP TABLE IF EXISTS foo_merge_tree; +DROP TABLE IF EXISTS clone_as_foo_merge_tree; +DROP TABLE IF EXISTS clone_as_foo_merge_tree_p_x; +DROP TABLE IF EXISTS clone_as_foo_merge_tree_p_y; +DROP TABLE IF EXISTS foo_replacing_merge_tree; +DROP TABLE IF EXISTS clone_as_foo_replacing_merge_tree; +DROP TABLE IF EXISTS clone_as_foo_replacing_merge_tree_p_x; +DROP TABLE IF EXISTS clone_as_foo_replacing_merge_tree_p_y; +DROP TABLE IF EXISTS foo_replicated_merge_tree; +DROP TABLE IF EXISTS clone_as_foo_replicated_merge_tree; +DROP TABLE IF EXISTS clone_as_foo_replicated_merge_tree_p_x; +DROP TABLE IF EXISTS clone_as_foo_replicated_merge_tree_p_y; + +-- CLONE AS with a table of Memory engine +CREATE TABLE foo_memory (x Int8, y String) ENGINE=Memory; +SHOW CREATE TABLE foo_memory; +INSERT INTO foo_memory VALUES (1, 'a'), (2, 'b'); + +CREATE TABLE clone_as_foo_memory CLONE AS foo_memory; -- { serverError SUPPORT_IS_DISABLED } + +-- CLONE AS with a table of File engine +CREATE TABLE foo_file (x Int8, y String) ENGINE=File(TabSeparated); +SHOW CREATE TABLE foo_file; +INSERT INTO foo_file VALUES (1, 'a'), (2, 'b'); + +CREATE TABLE clone_as_foo_file CLONE AS foo_file; -- { serverError SUPPORT_IS_DISABLED } + +-- CLONE AS with a table of MergeTree engine +CREATE TABLE foo_merge_tree (x Int8, y String) ENGINE=MergeTree PRIMARY KEY x; +SHOW CREATE TABLE foo_merge_tree; +INSERT INTO foo_merge_tree VALUES (1, 'a'), (2, 'b'); +SELECT * FROM foo_merge_tree; + +CREATE TABLE clone_as_foo_merge_tree CLONE AS foo_merge_tree; +SHOW CREATE TABLE clone_as_foo_merge_tree; +SELECT 'from foo_merge_tree'; +SELECT * FROM foo_merge_tree; +SELECT 'from clone_as_foo_merge_tree'; +SELECT * FROM clone_as_foo_merge_tree; + +-- Specify ENGINE +CREATE TABLE clone_as_foo_merge_tree_p_x CLONE AS foo_merge_tree ENGINE=MergeTree PRIMARY KEY x; +SELECT 'from clone_as_foo_merge_tree_p_x'; +SELECT * FROM clone_as_foo_merge_tree_p_x; +CREATE TABLE clone_as_foo_merge_tree_p_y CLONE AS foo_merge_tree ENGINE=MergeTree PRIMARY KEY y; -- { serverError BAD_ARGUMENTS } + +-- CLONE AS with a table of ReplacingMergeTree engine +CREATE TABLE foo_replacing_merge_tree (x Int8, y String) ENGINE=ReplacingMergeTree PRIMARY KEY x; +SHOW CREATE TABLE foo_replacing_merge_tree; +INSERT INTO foo_replacing_merge_tree VALUES (1, 'a'), (2, 'b'); +SELECT * FROM foo_replacing_merge_tree; + +CREATE TABLE clone_as_foo_replacing_merge_tree CLONE AS foo_replacing_merge_tree; +SHOW CREATE TABLE clone_as_foo_replacing_merge_tree; +SELECT 'from foo_replacing_merge_tree'; +SELECT * FROM foo_replacing_merge_tree; +SELECT 'from clone_as_foo_replacing_merge_tree'; +SELECT * FROM clone_as_foo_replacing_merge_tree; + +-- Specify ENGINE +CREATE TABLE clone_as_foo_replacing_merge_tree_p_x CLONE AS foo_replacing_merge_tree ENGINE=ReplacingMergeTree PRIMARY KEY x; +SELECT 'from clone_as_foo_replacing_merge_tree_p_x'; +SELECT * FROM clone_as_foo_replacing_merge_tree_p_x; +CREATE TABLE clone_as_foo_replacing_merge_tree_p_y CLONE AS foo_replacing_merge_tree ENGINE=ReplacingMergeTree PRIMARY KEY y; -- { serverError BAD_ARGUMENTS } + +-- CLONE AS with a table of ReplicatedMergeTree engine +CREATE TABLE foo_replicated_merge_tree (x Int8, y String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/test_foo_replicated_merge_tree', 'r1') PRIMARY KEY x; +SHOW CREATE TABLE foo_replicated_merge_tree; +INSERT INTO foo_replicated_merge_tree VALUES (1, 'a'), (2, 'b'); +SELECT 'from foo_replicated_merge_tree'; +SELECT * FROM foo_replicated_merge_tree; + +CREATE TABLE clone_as_foo_replicated_merge_tree CLONE AS foo_replicated_merge_tree; -- { serverError REPLICA_ALREADY_EXISTS } + +-- Specify ENGINE +CREATE TABLE clone_as_foo_replicated_merge_tree_p_x CLONE AS foo_replicated_merge_tree ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/clone_as_foo_replicated_merge_tree_p_x', 'r1') PRIMARY KEY x; +SHOW CREATE TABLE clone_as_foo_replicated_merge_tree_p_x; +SELECT 'from clone_as_foo_replicated_merge_tree_p_x'; +SELECT * FROM foo_replicated_merge_tree; +CREATE TABLE clone_as_foo_replicated_merge_tree_p_y CLONE AS foo_replicated_merge_tree ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/clone_as_foo_replicated_merge_tree_p_y', 'r1') PRIMARY KEY y; -- { serverError BAD_ARGUMENTS } + +DROP TABLE IF EXISTS foo_memory; +DROP TABLE IF EXISTS clone_as_foo_memory; +DROP TABLE IF EXISTS foo_file; +DROP TABLE IF EXISTS clone_as_foo_file; +DROP TABLE IF EXISTS foo_merge_tree; +DROP TABLE IF EXISTS clone_as_foo_merge_tree; +DROP TABLE IF EXISTS clone_as_foo_merge_tree_p_x; +DROP TABLE IF EXISTS clone_as_foo_merge_tree_p_y; +DROP TABLE IF EXISTS foo_replacing_merge_tree; +DROP TABLE IF EXISTS clone_as_foo_replacing_merge_tree; +DROP TABLE IF EXISTS clone_as_foo_replacing_merge_tree_p_x; +DROP TABLE IF EXISTS clone_as_foo_replacing_merge_tree_p_y; +DROP TABLE IF EXISTS foo_replicated_merge_tree; +DROP TABLE IF EXISTS clone_as_foo_replicated_merge_tree; +DROP TABLE IF EXISTS clone_as_foo_replicated_merge_tree_p_x; +DROP TABLE IF EXISTS clone_as_foo_replicated_merge_tree_p_y; + +-- CLONE AS with a Replicated database +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; + +CREATE DATABASE {CLICKHOUSE_DATABASE_1:Identifier} ENGINE = Replicated('/test/databases/{database}/test_03231', 's1', 'r1'); +USE {CLICKHOUSE_DATABASE_1:Identifier}; + +CREATE TABLE foo_merge_tree (x Int8, y String) ENGINE=MergeTree PRIMARY KEY x; +SHOW CREATE TABLE foo_merge_tree; +INSERT INTO foo_merge_tree VALUES (1, 'a'), (2, 'b'); +SELECT 'from foo_merge_tree'; +SELECT * FROM foo_merge_tree; +CREATE TABLE clone_as_foo_merge_tree CLONE AS foo_merge_tree; -- { serverError SUPPORT_IS_DISABLED } + +DROP TABLE IF EXISTS clone_as_foo_merge_tree; +DROP TABLE IF EXISTS foo_merge_tree; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; \ No newline at end of file