Compare commits

...

17 Commits

Author SHA1 Message Date
tuanpach
9393f77815
Merge 561daee66f into 7fd2207626 2024-09-18 23:30:43 +08:00
Tuan Pham Anh
561daee66f Update document 2024-09-10 10:24:31 +00:00
Tuan Pham Anh
0b7f28f75b 1) Support CLONE AS with ReplicatedMergeTree
2) Support CLONE AS with specifying ENGINE
2024-09-10 07:55:21 +00:00
Tuan Pham Anh
841dc275bd 1) Throw an error when creating a table with CLONE AS and specifying ENGINE. 2) Add document for creating a table with Schema and Data CLoned from another table. 2024-09-09 08:05:29 +00:00
Tuan Pham Anh
c01e63fd2b 1) Update checking MergeTree storage. 2) Remove unused code segment. 3) Update the test 2024-09-06 15:10:35 +00:00
Tuan Pham Anh
4aa9459c63 Change the test file from .sh to .sql 2024-09-06 09:48:27 +00:00
Tuan Pham Anh
5b4a9bc62b Throw an error if CLONE ASfrom a Replicated storage 2024-09-06 06:39:57 +00:00
Alexander Tokmakov
d0c6d8f118 Merge branch 'master' into create-table-with-clone-as 2024-09-05 17:38:07 +02:00
Tuan Pham Anh
23f0701fc8 Throw an error if CREATE ... CLONE AS ... with a Replicated database 2024-09-05 08:45:11 +00:00
Tuan Pham Anh
782353d831 Use ATTACH PARTITION ALL to copy data from the source table 2024-09-03 08:04:07 +00:00
Tuan Pham Anh
4b3da04f6c Merge remote-tracking branch 'allmazz/feat/59376' into create-table-with-clone-as 2024-09-03 07:54:12 +00:00
Tuan Pham Anh
4455d354c7 Show muatations of clone_as_foo_replacing_merge_tree 2024-09-02 08:56:07 +00:00
Tuan Pham Anh
28a77f2ef0 Separate select result in test case 2024-09-02 03:40:19 +00:00
Tuan Pham Anh
63b57c0987 Show mutation of stable in the test 2024-09-02 01:03:38 +00:00
Tuan Pham Anh
8e1c823b35 Update test case: show create table and values of the original tables. 2024-08-31 01:42:02 +00:00
Tuan Pham Anh
809a739636 Add stateless test for CREATE with CLONE AS 2024-08-30 08:32:08 +00:00
Tuan Pham Anh
14a968b08a Support create a new table with CLONE AS 2024-08-30 07:24:24 +00:00
8 changed files with 314 additions and 33 deletions

View File

@ -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

View File

@ -5,17 +5,20 @@
#include <Access/AccessControl.h>
#include <Access/User.h>
#include <Common/Exception.h>
#include <Common/StringUtils.h>
#include <Common/escapeForFileName.h>
#include <Common/typeid_cast.h>
#include <Common/Macros.h>
#include <Common/randomSeed.h>
#include <Common/atomicRename.h>
#include <Common/PoolId.h>
#include <Common/logger_useful.h>
#include <Core/Settings.h>
#include <Interpreters/InterpreterAlterQuery.h>
#include <Parsers/ASTPartition.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/queryToString.h>
#include <Common/Exception.h>
#include <Common/Macros.h>
#include <Common/PoolId.h>
#include <Common/StringUtils.h>
#include <Common/atomicRename.h>
#include <Common/escapeForFileName.h>
#include <Common/logger_useful.h>
#include <Common/randomSeed.h>
#include <Common/typeid_cast.h>
#include <Core/Defines.h>
#include <Core/SettingsEnums.h>
@ -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,9 +1540,6 @@ 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))
@ -1515,15 +1554,21 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
}
}
const bool allow_create_select_for_replicated = (create.isView() && !create.is_populate) || create.is_create_empty || !is_storage_replicated;
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)
{
/// 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");
"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,
@ -1531,6 +1576,14 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
}
}
if (create.is_clone_as)
{
if (database && database->getEngineName() == "Replicated")
throw Exception(
ErrorCodes::SUPPORT_IS_DISABLED,
"CREATE CLONE AS is not supported with Replicated databases. Consider using separate CREATE and INSERT queries.");
}
if (database && database->shouldReplicateQuery(getContext(), query_ptr))
{
chassert(!ddl_guard);
@ -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<ASTPartition>();
partition->all = true;
auto command = std::make_shared<ASTAlterCommand>();
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<ASTExpressionList>();
command_list->children.push_back(command);
auto query = std::make_shared<ASTAlterQuery>();
query->database = create.database;
query->table = create.table;
query->uuid = create.uuid;
auto * alter = query->as<ASTAlterQuery>();
alter->alter_object = ASTAlterQuery::AlterObjectType::TABLE;
alter->set(alter->command_list, command_list);
return InterpreterAlterQuery(query, getContext()).execute();
}
return {};
}

View File

@ -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);
}

View File

@ -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 '...'

View File

@ -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") \

View File

@ -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<ASTLiteral &>().value.safeGet<String>();

View File

@ -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

View File

@ -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};