Support create a new table with CLONE AS

This commit is contained in:
Tuan Pham Anh 2024-08-30 02:21:54 +00:00
parent 589a63a256
commit 14a968b08a
5 changed files with 87 additions and 10 deletions

View File

@ -5,17 +5,19 @@
#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 <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>
@ -826,6 +828,10 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
{
/// Only MergeTree support TTL
properties.columns.resetColumnTTLs();
if (create.is_clone_as)
{
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support 'CLONE AS' with tables of MergeTree family");
}
}
properties.constraints = as_storage_metadata->getConstraints();
@ -1933,6 +1939,57 @@ BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create)
/* async_isnert */ false).execute();
}
/// If the query is a CREATE .. CLONE AS <table>, insert the data into the 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);
StorageID as_table_id = {as_database_name, as_table_saved};
StoragePtr as_table = DatabaseCatalog::instance().tryGetTable(as_table_id, getContext());
if (!as_table)
{
return {};
}
auto merge_tree_table = std::dynamic_pointer_cast<MergeTreeData>(as_table);
if (!merge_tree_table)
{
return {};
}
auto command_list = std::make_shared<ASTExpressionList>();
for (const auto & partition_id : merge_tree_table->getAllPartitionIds())
{
auto partition = std::make_shared<ASTPartition>();
partition->all = false;
partition->setPartitionID(std::make_shared<ASTLiteral>(partition_id));
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();
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>();