mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #58297 from azat/qp-alter-part
Support query parameters in ALTER TABLE ... PART
This commit is contained in:
commit
7ded82e51f
@ -281,7 +281,7 @@ ASTPtr DatabaseMaterializedPostgreSQL::createAlterSettingsQuery(const SettingCha
|
||||
|
||||
auto command = std::make_shared<ASTAlterCommand>();
|
||||
command->type = ASTAlterCommand::Type::MODIFY_DATABASE_SETTING;
|
||||
command->settings_changes = std::move(set);
|
||||
command->settings_changes = command->children.emplace_back(std::move(set)).get();
|
||||
|
||||
auto command_list = std::make_shared<ASTExpressionList>();
|
||||
command_list->children.push_back(command);
|
||||
|
@ -30,6 +30,9 @@ void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast)
|
||||
return;
|
||||
}
|
||||
|
||||
/// FIXME: this helper should use updatePointerToChild(), but
|
||||
/// forEachPointerToChild() is not implemented for ASTColumnDeclaration
|
||||
/// (and also some members should be adjusted for this).
|
||||
const auto visit_child_with_shared_ptr = [&](ASTPtr & child)
|
||||
{
|
||||
if (!child)
|
||||
@ -86,22 +89,24 @@ void UserDefinedSQLFunctionVisitor::visit(ASTPtr & ast)
|
||||
|
||||
if (auto * alter = ast->as<ASTAlterCommand>())
|
||||
{
|
||||
visit_child_with_shared_ptr(alter->col_decl);
|
||||
visit_child_with_shared_ptr(alter->column);
|
||||
visit_child_with_shared_ptr(alter->partition);
|
||||
visit_child_with_shared_ptr(alter->order_by);
|
||||
visit_child_with_shared_ptr(alter->sample_by);
|
||||
visit_child_with_shared_ptr(alter->index_decl);
|
||||
visit_child_with_shared_ptr(alter->index);
|
||||
visit_child_with_shared_ptr(alter->constraint_decl);
|
||||
visit_child_with_shared_ptr(alter->constraint);
|
||||
visit_child_with_shared_ptr(alter->projection_decl);
|
||||
visit_child_with_shared_ptr(alter->projection);
|
||||
visit_child_with_shared_ptr(alter->predicate);
|
||||
visit_child_with_shared_ptr(alter->update_assignments);
|
||||
visit_child_with_shared_ptr(alter->values);
|
||||
visit_child_with_shared_ptr(alter->ttl);
|
||||
visit_child_with_shared_ptr(alter->select);
|
||||
/// It is OK to use updatePointerToChild() because ASTAlterCommand implements forEachPointerToChild()
|
||||
const auto visit_child_update_parent = [&](ASTPtr & child)
|
||||
{
|
||||
if (!child)
|
||||
return;
|
||||
|
||||
auto * old_ptr = child.get();
|
||||
visit(child);
|
||||
auto * new_ptr = child.get();
|
||||
|
||||
/// Some AST classes have naked pointers to children elements as members.
|
||||
/// We have to replace them if the child was replaced.
|
||||
if (new_ptr != old_ptr)
|
||||
ast->updatePointerToChild(old_ptr, new_ptr);
|
||||
};
|
||||
|
||||
for (auto & children : alter->children)
|
||||
visit_child_update_parent(children);
|
||||
|
||||
return;
|
||||
}
|
||||
|
@ -262,7 +262,7 @@ MutationCommand createCommandToApplyDeletedMask(const MutationCommand & command)
|
||||
|
||||
auto alter_command = std::make_shared<ASTAlterCommand>();
|
||||
alter_command->type = ASTAlterCommand::DELETE;
|
||||
alter_command->partition = command.partition;
|
||||
alter_command->partition = alter_command->children.emplace_back(command.partition).get();
|
||||
|
||||
auto row_exists_predicate = makeASTFunction("equals",
|
||||
std::make_shared<ASTIdentifier>(LightweightDeleteDescription::FILTER_COLUMN.name),
|
||||
@ -271,7 +271,7 @@ MutationCommand createCommandToApplyDeletedMask(const MutationCommand & command)
|
||||
if (command.predicate)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Mutation command APPLY DELETED MASK does not support WHERE clause");
|
||||
|
||||
alter_command->predicate = row_exists_predicate;
|
||||
alter_command->predicate = alter_command->children.emplace_back(std::move(row_exists_predicate)).get();
|
||||
|
||||
auto mutation_command = MutationCommand::parse(alter_command.get());
|
||||
if (!mutation_command)
|
||||
|
@ -172,6 +172,30 @@ ASTPtr replaceNonDeterministicToScalars(const ASTAlterCommand & alter_command, C
|
||||
auto query = alter_command.clone();
|
||||
auto & new_alter_command = *query->as<ASTAlterCommand>();
|
||||
|
||||
auto remove_child = [](auto & children, IAST *& erase_ptr)
|
||||
{
|
||||
auto it = std::find_if(children.begin(), children.end(), [&](const auto & ptr) { return ptr.get() == erase_ptr; });
|
||||
erase_ptr = nullptr;
|
||||
children.erase(it);
|
||||
};
|
||||
auto visit = [&](auto & visitor)
|
||||
{
|
||||
if (new_alter_command.update_assignments)
|
||||
{
|
||||
ASTPtr update_assignments = new_alter_command.update_assignments->clone();
|
||||
remove_child(new_alter_command.children, new_alter_command.update_assignments);
|
||||
visitor.visit(update_assignments);
|
||||
new_alter_command.update_assignments = new_alter_command.children.emplace_back(std::move(update_assignments)).get();
|
||||
}
|
||||
if (new_alter_command.predicate)
|
||||
{
|
||||
ASTPtr predicate = new_alter_command.predicate->clone();
|
||||
remove_child(new_alter_command.children, new_alter_command.predicate);
|
||||
visitor.visit(predicate);
|
||||
new_alter_command.predicate = new_alter_command.children.emplace_back(std::move(predicate)).get();
|
||||
}
|
||||
};
|
||||
|
||||
if (settings.mutations_execute_subqueries_on_initiator)
|
||||
{
|
||||
Scalars scalars;
|
||||
@ -188,10 +212,7 @@ ASTPtr replaceNonDeterministicToScalars(const ASTAlterCommand & alter_command, C
|
||||
settings.mutations_max_literal_size_to_replace};
|
||||
|
||||
ExecuteScalarSubqueriesVisitor visitor(data);
|
||||
if (new_alter_command.update_assignments)
|
||||
visitor.visit(new_alter_command.update_assignments);
|
||||
if (new_alter_command.predicate)
|
||||
visitor.visit(new_alter_command.predicate);
|
||||
visit(visitor);
|
||||
}
|
||||
|
||||
if (settings.mutations_execute_nondeterministic_on_initiator)
|
||||
@ -200,10 +221,7 @@ ASTPtr replaceNonDeterministicToScalars(const ASTAlterCommand & alter_command, C
|
||||
context, settings.mutations_max_literal_size_to_replace};
|
||||
|
||||
ExecuteNonDeterministicConstFunctionsVisitor visitor(data);
|
||||
if (new_alter_command.update_assignments)
|
||||
visitor.visit(new_alter_command.update_assignments);
|
||||
if (new_alter_command.predicate)
|
||||
visitor.visit(new_alter_command.predicate);
|
||||
visit(visitor);
|
||||
}
|
||||
|
||||
return query;
|
||||
|
@ -638,7 +638,7 @@ ASTs InterpreterAlterImpl::getRewrittenQueries(
|
||||
auto rewritten_command = std::make_shared<ASTAlterCommand>();
|
||||
rewritten_command->type = ASTAlterCommand::ADD_COLUMN;
|
||||
rewritten_command->first = alter_command->first;
|
||||
rewritten_command->col_decl = additional_columns->children[index]->clone();
|
||||
rewritten_command->col_decl = rewritten_command->children.emplace_back(additional_columns->children[index]->clone()).get();
|
||||
|
||||
const auto & column_declare = alter_command->additional_columns->children[index]->as<MySQLParser::ASTDeclareColumn>();
|
||||
if (column_declare && column_declare->column_options)
|
||||
@ -667,8 +667,7 @@ ASTs InterpreterAlterImpl::getRewrittenQueries(
|
||||
|
||||
if (!alter_command->column_name.empty())
|
||||
{
|
||||
rewritten_command->column = std::make_shared<ASTIdentifier>(alter_command->column_name);
|
||||
rewritten_command->children.push_back(rewritten_command->column);
|
||||
rewritten_command->column = rewritten_command->children.emplace_back(std::make_shared<ASTIdentifier>(alter_command->column_name)).get();
|
||||
|
||||
/// For example(when add_column_1 is last column):
|
||||
/// ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT AFTER add_column_1, ADD COLUMN add_column_4 INT
|
||||
@ -679,12 +678,10 @@ ASTs InterpreterAlterImpl::getRewrittenQueries(
|
||||
}
|
||||
else
|
||||
{
|
||||
rewritten_command->column = std::make_shared<ASTIdentifier>(default_after_column);
|
||||
rewritten_command->children.push_back(rewritten_command->column);
|
||||
rewritten_command->column = rewritten_command->children.emplace_back(std::make_shared<ASTIdentifier>(default_after_column)).get();
|
||||
default_after_column = rewritten_command->col_decl->as<ASTColumnDeclaration>()->name;
|
||||
}
|
||||
|
||||
rewritten_command->children.push_back(rewritten_command->col_decl);
|
||||
rewritten_alter_query->command_list->children.push_back(rewritten_command);
|
||||
}
|
||||
}
|
||||
@ -692,7 +689,7 @@ ASTs InterpreterAlterImpl::getRewrittenQueries(
|
||||
{
|
||||
auto rewritten_command = std::make_shared<ASTAlterCommand>();
|
||||
rewritten_command->type = ASTAlterCommand::DROP_COLUMN;
|
||||
rewritten_command->column = std::make_shared<ASTIdentifier>(alter_command->column_name);
|
||||
rewritten_command->column = rewritten_command->children.emplace_back(std::make_shared<ASTIdentifier>(alter_command->column_name)).get();
|
||||
rewritten_alter_query->command_list->children.push_back(rewritten_command);
|
||||
}
|
||||
else if (alter_command->type == MySQLParser::ASTAlterCommand::RENAME_COLUMN)
|
||||
@ -702,8 +699,8 @@ ASTs InterpreterAlterImpl::getRewrittenQueries(
|
||||
/// 'RENAME column_name TO column_name' is not allowed in Clickhouse
|
||||
auto rewritten_command = std::make_shared<ASTAlterCommand>();
|
||||
rewritten_command->type = ASTAlterCommand::RENAME_COLUMN;
|
||||
rewritten_command->column = std::make_shared<ASTIdentifier>(alter_command->old_name);
|
||||
rewritten_command->rename_to = std::make_shared<ASTIdentifier>(alter_command->column_name);
|
||||
rewritten_command->column = rewritten_command->children.emplace_back(std::make_shared<ASTIdentifier>(alter_command->old_name)).get();
|
||||
rewritten_command->rename_to = rewritten_command->children.emplace_back(std::make_shared<ASTIdentifier>(alter_command->column_name)).get();
|
||||
rewritten_alter_query->command_list->children.push_back(rewritten_command);
|
||||
}
|
||||
}
|
||||
@ -726,13 +723,10 @@ ASTs InterpreterAlterImpl::getRewrittenQueries(
|
||||
modify_columns.front().name = alter_command->old_name;
|
||||
|
||||
const auto & modify_columns_description = createColumnsDescription(modify_columns, alter_command->additional_columns);
|
||||
rewritten_command->col_decl = InterpreterCreateQuery::formatColumns(modify_columns_description)->children[0];
|
||||
rewritten_command->col_decl = rewritten_command->children.emplace_back(InterpreterCreateQuery::formatColumns(modify_columns_description)->children[0]).get();
|
||||
|
||||
if (!alter_command->column_name.empty())
|
||||
{
|
||||
rewritten_command->column = std::make_shared<ASTIdentifier>(alter_command->column_name);
|
||||
rewritten_command->children.push_back(rewritten_command->column);
|
||||
}
|
||||
rewritten_command->column = rewritten_command->children.emplace_back(std::make_shared<ASTIdentifier>(alter_command->column_name)).get();
|
||||
|
||||
rewritten_alter_query->command_list->children.push_back(rewritten_command);
|
||||
}
|
||||
@ -741,8 +735,8 @@ ASTs InterpreterAlterImpl::getRewrittenQueries(
|
||||
{
|
||||
auto rewritten_command = std::make_shared<ASTAlterCommand>();
|
||||
rewritten_command->type = ASTAlterCommand::RENAME_COLUMN;
|
||||
rewritten_command->column = std::make_shared<ASTIdentifier>(alter_command->old_name);
|
||||
rewritten_command->rename_to = std::make_shared<ASTIdentifier>(new_column_name);
|
||||
rewritten_command->column = rewritten_command->children.emplace_back(std::make_shared<ASTIdentifier>(alter_command->old_name)).get();
|
||||
rewritten_command->rename_to = rewritten_command->children.emplace_back(std::make_shared<ASTIdentifier>(new_column_name)).get();
|
||||
rewritten_alter_query->command_list->children.push_back(rewritten_command);
|
||||
}
|
||||
}
|
||||
|
@ -22,60 +22,47 @@ ASTPtr ASTAlterCommand::clone() const
|
||||
res->children.clear();
|
||||
|
||||
if (col_decl)
|
||||
{
|
||||
res->col_decl = col_decl->clone();
|
||||
res->children.push_back(res->col_decl);
|
||||
}
|
||||
res->col_decl = res->children.emplace_back(col_decl->clone()).get();
|
||||
if (column)
|
||||
{
|
||||
res->column = column->clone();
|
||||
res->children.push_back(res->column);
|
||||
}
|
||||
res->column = res->children.emplace_back(column->clone()).get();
|
||||
if (order_by)
|
||||
{
|
||||
res->order_by = order_by->clone();
|
||||
res->children.push_back(res->order_by);
|
||||
}
|
||||
res->order_by = res->children.emplace_back(order_by->clone()).get();
|
||||
if (sample_by)
|
||||
res->sample_by = res->children.emplace_back(sample_by->clone()).get();
|
||||
if (index_decl)
|
||||
res->index_decl = res->children.emplace_back(index_decl->clone()).get();
|
||||
if (index)
|
||||
res->index = res->children.emplace_back(index->clone()).get();
|
||||
if (constraint_decl)
|
||||
res->constraint_decl = res->children.emplace_back(constraint_decl->clone()).get();
|
||||
if (constraint)
|
||||
res->constraint = res->children.emplace_back(constraint->clone()).get();
|
||||
if (projection_decl)
|
||||
res->projection_decl = res->children.emplace_back(projection_decl->clone()).get();
|
||||
if (projection)
|
||||
res->projection = res->children.emplace_back(projection->clone()).get();
|
||||
if (statistic_decl)
|
||||
res->statistic_decl = res->children.emplace_back(statistic_decl->clone()).get();
|
||||
if (partition)
|
||||
{
|
||||
res->partition = partition->clone();
|
||||
res->children.push_back(res->partition);
|
||||
}
|
||||
res->partition = res->children.emplace_back(partition->clone()).get();
|
||||
if (predicate)
|
||||
{
|
||||
res->predicate = predicate->clone();
|
||||
res->children.push_back(res->predicate);
|
||||
}
|
||||
if (ttl)
|
||||
{
|
||||
res->ttl = ttl->clone();
|
||||
res->children.push_back(res->ttl);
|
||||
}
|
||||
if (settings_changes)
|
||||
{
|
||||
res->settings_changes = settings_changes->clone();
|
||||
res->children.push_back(res->settings_changes);
|
||||
}
|
||||
if (settings_resets)
|
||||
{
|
||||
res->settings_resets = settings_resets->clone();
|
||||
res->children.push_back(res->settings_resets);
|
||||
}
|
||||
if (values)
|
||||
{
|
||||
res->values = values->clone();
|
||||
res->children.push_back(res->values);
|
||||
}
|
||||
if (rename_to)
|
||||
{
|
||||
res->rename_to = rename_to->clone();
|
||||
res->children.push_back(res->rename_to);
|
||||
}
|
||||
res->predicate = res->children.emplace_back(predicate->clone()).get();
|
||||
if (update_assignments)
|
||||
res->update_assignments = res->children.emplace_back(update_assignments->clone()).get();
|
||||
if (comment)
|
||||
{
|
||||
res->comment = comment->clone();
|
||||
res->children.push_back(res->comment);
|
||||
}
|
||||
res->comment = res->children.emplace_back(comment->clone()).get();
|
||||
if (ttl)
|
||||
res->ttl = res->children.emplace_back(ttl->clone()).get();
|
||||
if (settings_changes)
|
||||
res->settings_changes = res->children.emplace_back(settings_changes->clone()).get();
|
||||
if (settings_resets)
|
||||
res->settings_resets = res->children.emplace_back(settings_resets->clone()).get();
|
||||
if (select)
|
||||
res->select = res->children.emplace_back(select->clone()).get();
|
||||
if (values)
|
||||
res->values = res->children.emplace_back(values->clone()).get();
|
||||
if (rename_to)
|
||||
res->rename_to = res->children.emplace_back(rename_to->clone()).get();
|
||||
|
||||
return res;
|
||||
}
|
||||
@ -486,6 +473,32 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState &
|
||||
throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected type of ALTER");
|
||||
}
|
||||
|
||||
void ASTAlterCommand::forEachPointerToChild(std::function<void(void**)> f)
|
||||
{
|
||||
f(reinterpret_cast<void **>(&col_decl));
|
||||
f(reinterpret_cast<void **>(&column));
|
||||
f(reinterpret_cast<void **>(&order_by));
|
||||
f(reinterpret_cast<void **>(&sample_by));
|
||||
f(reinterpret_cast<void **>(&index_decl));
|
||||
f(reinterpret_cast<void **>(&index));
|
||||
f(reinterpret_cast<void **>(&constraint_decl));
|
||||
f(reinterpret_cast<void **>(&constraint));
|
||||
f(reinterpret_cast<void **>(&projection_decl));
|
||||
f(reinterpret_cast<void **>(&projection));
|
||||
f(reinterpret_cast<void **>(&statistic_decl));
|
||||
f(reinterpret_cast<void **>(&partition));
|
||||
f(reinterpret_cast<void **>(&predicate));
|
||||
f(reinterpret_cast<void **>(&update_assignments));
|
||||
f(reinterpret_cast<void **>(&comment));
|
||||
f(reinterpret_cast<void **>(&ttl));
|
||||
f(reinterpret_cast<void **>(&settings_changes));
|
||||
f(reinterpret_cast<void **>(&settings_resets));
|
||||
f(reinterpret_cast<void **>(&select));
|
||||
f(reinterpret_cast<void **>(&values));
|
||||
f(reinterpret_cast<void **>(&rename_to));
|
||||
}
|
||||
|
||||
|
||||
bool ASTAlterQuery::isOneCommandTypeOnly(const ASTAlterCommand::Type & type) const
|
||||
{
|
||||
if (command_list)
|
||||
@ -624,4 +637,11 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState
|
||||
}
|
||||
}
|
||||
|
||||
void ASTAlterQuery::forEachPointerToChild(std::function<void(void**)> f)
|
||||
{
|
||||
for (const auto & child : command_list->children)
|
||||
child->as<ASTAlterCommand &>().forEachPointerToChild(f);
|
||||
f(reinterpret_cast<void **>(&command_list));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -23,6 +23,8 @@ namespace DB
|
||||
|
||||
class ASTAlterCommand : public IAST
|
||||
{
|
||||
friend class ASTAlterQuery;
|
||||
|
||||
public:
|
||||
enum Type
|
||||
{
|
||||
@ -89,83 +91,85 @@ public:
|
||||
* This field is not used in the DROP query
|
||||
* In MODIFY query, the column name and the new type are stored here
|
||||
*/
|
||||
ASTPtr col_decl;
|
||||
IAST * col_decl = nullptr;
|
||||
|
||||
/** The ADD COLUMN and MODIFY COLUMN query here optionally stores the name of the column following AFTER
|
||||
* The DROP query stores the column name for deletion here
|
||||
* Also used for RENAME COLUMN.
|
||||
*/
|
||||
ASTPtr column;
|
||||
IAST * column = nullptr;
|
||||
|
||||
/** For MODIFY ORDER BY
|
||||
*/
|
||||
ASTPtr order_by;
|
||||
IAST * order_by = nullptr;
|
||||
|
||||
/** For MODIFY SAMPLE BY
|
||||
*/
|
||||
ASTPtr sample_by;
|
||||
IAST * sample_by = nullptr;
|
||||
|
||||
/** The ADD INDEX query stores the IndexDeclaration there.
|
||||
*/
|
||||
ASTPtr index_decl;
|
||||
IAST * index_decl = nullptr;
|
||||
|
||||
/** The ADD INDEX query stores the name of the index following AFTER.
|
||||
* The DROP INDEX query stores the name for deletion.
|
||||
* The MATERIALIZE INDEX query stores the name of the index to materialize.
|
||||
* The CLEAR INDEX query stores the name of the index to clear.
|
||||
*/
|
||||
ASTPtr index;
|
||||
IAST * index = nullptr;
|
||||
|
||||
/** The ADD CONSTRAINT query stores the ConstraintDeclaration there.
|
||||
*/
|
||||
ASTPtr constraint_decl;
|
||||
IAST * constraint_decl = nullptr;
|
||||
|
||||
/** The DROP CONSTRAINT query stores the name for deletion.
|
||||
*/
|
||||
ASTPtr constraint;
|
||||
IAST * constraint = nullptr;
|
||||
|
||||
/** The ADD PROJECTION query stores the ProjectionDeclaration there.
|
||||
*/
|
||||
ASTPtr projection_decl;
|
||||
IAST * projection_decl = nullptr;
|
||||
|
||||
/** The ADD PROJECTION query stores the name of the projection following AFTER.
|
||||
* The DROP PROJECTION query stores the name for deletion.
|
||||
* The MATERIALIZE PROJECTION query stores the name of the projection to materialize.
|
||||
* The CLEAR PROJECTION query stores the name of the projection to clear.
|
||||
*/
|
||||
ASTPtr projection;
|
||||
IAST * projection = nullptr;
|
||||
|
||||
ASTPtr statistic_decl;
|
||||
IAST * statistic_decl = nullptr;
|
||||
|
||||
/** Used in DROP PARTITION, ATTACH PARTITION FROM, UPDATE, DELETE queries.
|
||||
* The value or ID of the partition is stored here.
|
||||
*/
|
||||
ASTPtr partition;
|
||||
IAST * partition = nullptr;
|
||||
|
||||
/// For DELETE/UPDATE WHERE: the predicate that filters the rows to delete/update.
|
||||
ASTPtr predicate;
|
||||
IAST * predicate = nullptr;
|
||||
|
||||
/// A list of expressions of the form `column = expr` for the UPDATE command.
|
||||
ASTPtr update_assignments;
|
||||
IAST * update_assignments = nullptr;
|
||||
|
||||
/// A column comment
|
||||
ASTPtr comment;
|
||||
IAST * comment = nullptr;
|
||||
|
||||
/// For MODIFY TTL query
|
||||
ASTPtr ttl;
|
||||
IAST * ttl = nullptr;
|
||||
|
||||
/// FOR MODIFY_SETTING
|
||||
ASTPtr settings_changes;
|
||||
IAST * settings_changes = nullptr;
|
||||
|
||||
/// FOR RESET_SETTING
|
||||
ASTPtr settings_resets;
|
||||
IAST * settings_resets = nullptr;
|
||||
|
||||
/// For MODIFY_QUERY
|
||||
ASTPtr select;
|
||||
IAST * select = nullptr;
|
||||
|
||||
/** In ALTER CHANNEL, ADD, DROP, SUSPEND, RESUME, REFRESH, MODIFY queries, the list of live views is stored here
|
||||
*/
|
||||
ASTPtr values;
|
||||
/// In ALTER CHANNEL, ADD, DROP, SUSPEND, RESUME, REFRESH, MODIFY queries, the list of live views is stored here
|
||||
IAST * values = nullptr;
|
||||
|
||||
/// Target column name
|
||||
IAST * rename_to = nullptr;
|
||||
|
||||
/// For MODIFY REFRESH
|
||||
ASTPtr refresh;
|
||||
@ -211,9 +215,6 @@ public:
|
||||
String to_database;
|
||||
String to_table;
|
||||
|
||||
/// Target column name
|
||||
ASTPtr rename_to;
|
||||
|
||||
/// Which property user want to remove
|
||||
String remove_property;
|
||||
|
||||
@ -223,6 +224,8 @@ public:
|
||||
|
||||
protected:
|
||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
|
||||
void forEachPointerToChild(std::function<void(void**)> f) override;
|
||||
};
|
||||
|
||||
class ASTAlterQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCluster
|
||||
@ -270,10 +273,7 @@ protected:
|
||||
|
||||
bool isOneCommandTypeOnly(const ASTAlterCommand::Type & type) const;
|
||||
|
||||
void forEachPointerToChild(std::function<void(void**)> f) override
|
||||
{
|
||||
f(reinterpret_cast<void **>(&command_list));
|
||||
}
|
||||
void forEachPointerToChild(std::function<void(void**)> f) override;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -64,11 +64,13 @@ void ASTCreateIndexQuery::formatQueryImpl(const FormatSettings & settings, Forma
|
||||
ASTPtr ASTCreateIndexQuery::convertToASTAlterCommand() const
|
||||
{
|
||||
auto command = std::make_shared<ASTAlterCommand>();
|
||||
|
||||
command->type = ASTAlterCommand::ADD_INDEX;
|
||||
command->index = index_name->clone();
|
||||
command->index_decl = index_decl->clone();
|
||||
command->if_not_exists = if_not_exists;
|
||||
|
||||
command->index = command->children.emplace_back(index_name).get();
|
||||
command->index_decl = command->children.emplace_back(index_decl).get();
|
||||
|
||||
return command;
|
||||
}
|
||||
|
||||
|
@ -55,10 +55,12 @@ void ASTDropIndexQuery::formatQueryImpl(const FormatSettings & settings, FormatS
|
||||
ASTPtr ASTDropIndexQuery::convertToASTAlterCommand() const
|
||||
{
|
||||
auto command = std::make_shared<ASTAlterCommand>();
|
||||
|
||||
command->type = ASTAlterCommand::DROP_INDEX;
|
||||
command->index = index_name->clone();
|
||||
command->if_exists = if_exists;
|
||||
|
||||
command->index = command->children.emplace_back(index_name).get();
|
||||
|
||||
return command;
|
||||
}
|
||||
|
||||
|
@ -15,6 +15,7 @@
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/ParserDatabaseOrNone.h>
|
||||
#include <Parsers/ParserStringAndSubstitution.h>
|
||||
#include <Parsers/parseIdentifierOrStringLiteral.h>
|
||||
#include <base/range.h>
|
||||
#include <boost/algorithm/string/predicate.hpp>
|
||||
@ -43,20 +44,6 @@ namespace
|
||||
});
|
||||
}
|
||||
|
||||
class ParserStringAndSubstitution : public IParserBase
|
||||
{
|
||||
private:
|
||||
const char * getName() const override { return "ParserStringAndSubstitution"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
|
||||
{
|
||||
return ParserStringLiteral{}.parse(pos, node, expected) || ParserSubstitution{}.parse(pos, node, expected);
|
||||
}
|
||||
|
||||
public:
|
||||
explicit ParserStringAndSubstitution() = default;
|
||||
};
|
||||
|
||||
|
||||
bool parseAuthenticationData(IParserBase::Pos & pos, Expected & expected, std::shared_ptr<ASTAuthenticationData> & auth_data)
|
||||
{
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Parsers/ParserStringAndSubstitution.h>
|
||||
#include <Parsers/ParserAlterQuery.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
@ -117,6 +118,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
ParserCompoundIdentifier parser_name;
|
||||
ParserStringLiteral parser_string_literal;
|
||||
ParserStringAndSubstitution parser_string_and_substituion;
|
||||
ParserIdentifier parser_remove_property;
|
||||
ParserCompoundColumnDeclaration parser_col_decl;
|
||||
ParserIndexDeclaration parser_idx_decl;
|
||||
@ -138,6 +140,28 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserRefreshStrategy refresh_p;
|
||||
ParserTTLExpressionList parser_ttl_list;
|
||||
|
||||
ASTPtr command_col_decl;
|
||||
ASTPtr command_column;
|
||||
ASTPtr command_order_by;
|
||||
ASTPtr command_sample_by;
|
||||
ASTPtr command_index_decl;
|
||||
ASTPtr command_index;
|
||||
ASTPtr command_constraint_decl;
|
||||
ASTPtr command_constraint;
|
||||
ASTPtr command_projection_decl;
|
||||
ASTPtr command_projection;
|
||||
ASTPtr command_statistic_decl;
|
||||
ASTPtr command_partition;
|
||||
ASTPtr command_predicate;
|
||||
ASTPtr command_update_assignments;
|
||||
ASTPtr command_comment;
|
||||
ASTPtr command_ttl;
|
||||
ASTPtr command_settings_changes;
|
||||
ASTPtr command_settings_resets;
|
||||
ASTPtr command_select;
|
||||
ASTPtr command_values;
|
||||
ASTPtr command_rename_to;
|
||||
|
||||
switch (alter_object)
|
||||
{
|
||||
case ASTAlterQuery::AlterObjectType::LIVE_VIEW:
|
||||
@ -154,7 +178,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
{
|
||||
if (s_modify_setting.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_settings.parse(pos, command->settings_changes, expected))
|
||||
if (!parser_settings.parse(pos, command_settings_changes, expected))
|
||||
return false;
|
||||
command->type = ASTAlterCommand::MODIFY_DATABASE_SETTING;
|
||||
}
|
||||
@ -171,14 +195,14 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_not_exists.ignore(pos, expected))
|
||||
command->if_not_exists = true;
|
||||
|
||||
if (!parser_col_decl.parse(pos, command->col_decl, expected))
|
||||
if (!parser_col_decl.parse(pos, command_col_decl, expected))
|
||||
return false;
|
||||
|
||||
if (s_first.ignore(pos, expected))
|
||||
command->first = true;
|
||||
else if (s_after.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_name.parse(pos, command->column, expected))
|
||||
if (!parser_name.parse(pos, command_column, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
@ -189,20 +213,20 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->column, expected))
|
||||
if (!parser_name.parse(pos, command_column, expected))
|
||||
return false;
|
||||
|
||||
if (!s_to.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!parser_name.parse(pos, command->rename_to, expected))
|
||||
if (!parser_name.parse(pos, command_rename_to, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::RENAME_COLUMN;
|
||||
}
|
||||
else if (s_materialize_column.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_name.parse(pos, command->column, expected))
|
||||
if (!parser_name.parse(pos, command_column, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::MATERIALIZE_COLUMN;
|
||||
@ -210,20 +234,20 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
if (s_in_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else if (s_drop_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_PARTITION;
|
||||
}
|
||||
else if (s_drop_part.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_string_literal.parse(pos, command->partition, expected))
|
||||
if (!parser_string_and_substituion.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_PARTITION;
|
||||
@ -231,14 +255,14 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_drop_detached_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_DETACHED_PARTITION;
|
||||
}
|
||||
else if (s_drop_detached_part.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_string_literal.parse(pos, command->partition, expected))
|
||||
if (!parser_string_and_substituion.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_DETACHED_PARTITION;
|
||||
@ -249,7 +273,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->column, expected))
|
||||
if (!parser_name.parse(pos, command_column, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_COLUMN;
|
||||
@ -260,7 +284,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->column, expected))
|
||||
if (!parser_name.parse(pos, command_column, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_COLUMN;
|
||||
@ -269,7 +293,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
if (s_in_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@ -278,14 +302,14 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_not_exists.ignore(pos, expected))
|
||||
command->if_not_exists = true;
|
||||
|
||||
if (!parser_idx_decl.parse(pos, command->index_decl, expected))
|
||||
if (!parser_idx_decl.parse(pos, command_index_decl, expected))
|
||||
return false;
|
||||
|
||||
if (s_first.ignore(pos, expected))
|
||||
command->first = true;
|
||||
else if (s_after.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_name.parse(pos, command->index, expected))
|
||||
if (!parser_name.parse(pos, command_index, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
@ -296,7 +320,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->index, expected))
|
||||
if (!parser_name.parse(pos, command_index, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_INDEX;
|
||||
@ -307,7 +331,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->index, expected))
|
||||
if (!parser_name.parse(pos, command_index, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_INDEX;
|
||||
@ -316,7 +340,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
if (s_in_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@ -325,7 +349,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->index, expected))
|
||||
if (!parser_name.parse(pos, command_index, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::MATERIALIZE_INDEX;
|
||||
@ -333,7 +357,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
if (s_in_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@ -342,7 +366,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_not_exists.ignore(pos, expected))
|
||||
command->if_not_exists = true;
|
||||
|
||||
if (!parser_stat_decl.parse(pos, command->statistic_decl, expected))
|
||||
if (!parser_stat_decl.parse(pos, command_statistic_decl, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::ADD_STATISTIC;
|
||||
@ -352,7 +376,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_stat_decl.parse(pos, command->statistic_decl, expected))
|
||||
if (!parser_stat_decl.parse(pos, command_statistic_decl, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_STATISTIC;
|
||||
@ -362,7 +386,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_stat_decl.parse(pos, command->statistic_decl, expected))
|
||||
if (!parser_stat_decl.parse(pos, command_statistic_decl, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_STATISTIC;
|
||||
@ -371,7 +395,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
if (s_in_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@ -380,7 +404,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_stat_decl.parse(pos, command->statistic_decl, expected))
|
||||
if (!parser_stat_decl.parse(pos, command_statistic_decl, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::MATERIALIZE_STATISTIC;
|
||||
@ -388,7 +412,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
if (s_in_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@ -397,14 +421,14 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_not_exists.ignore(pos, expected))
|
||||
command->if_not_exists = true;
|
||||
|
||||
if (!parser_projection_decl.parse(pos, command->projection_decl, expected))
|
||||
if (!parser_projection_decl.parse(pos, command_projection_decl, expected))
|
||||
return false;
|
||||
|
||||
if (s_first.ignore(pos, expected))
|
||||
command->first = true;
|
||||
else if (s_after.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_name.parse(pos, command->projection, expected))
|
||||
if (!parser_name.parse(pos, command_projection, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
@ -415,7 +439,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->projection, expected))
|
||||
if (!parser_name.parse(pos, command_projection, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_PROJECTION;
|
||||
@ -426,7 +450,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->projection, expected))
|
||||
if (!parser_name.parse(pos, command_projection, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_PROJECTION;
|
||||
@ -435,7 +459,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
if (s_in_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@ -444,7 +468,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->projection, expected))
|
||||
if (!parser_name.parse(pos, command_projection, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::MATERIALIZE_PROJECTION;
|
||||
@ -452,13 +476,13 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
if (s_in_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else if (s_move_part.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_string_literal.parse(pos, command->partition, expected))
|
||||
if (!parser_string_and_substituion.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::MOVE_PARTITION;
|
||||
@ -483,7 +507,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_move_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::MOVE_PARTITION;
|
||||
@ -515,7 +539,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_not_exists.ignore(pos, expected))
|
||||
command->if_not_exists = true;
|
||||
|
||||
if (!parser_constraint_decl.parse(pos, command->constraint_decl, expected))
|
||||
if (!parser_constraint_decl.parse(pos, command_constraint_decl, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::ADD_CONSTRAINT;
|
||||
@ -525,7 +549,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->constraint, expected))
|
||||
if (!parser_name.parse(pos, command_constraint, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_CONSTRAINT;
|
||||
@ -533,7 +557,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_detach_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_PARTITION;
|
||||
@ -541,7 +565,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_detach_part.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_string_literal.parse(pos, command->partition, expected))
|
||||
if (!parser_string_and_substituion.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_PARTITION;
|
||||
@ -550,7 +574,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_attach_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
|
||||
if (s_from.ignore(pos))
|
||||
@ -568,7 +592,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_replace_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
|
||||
if (!s_from.ignore(pos, expected))
|
||||
@ -582,7 +606,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_attach_part.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_string_literal.parse(pos, command->partition, expected))
|
||||
if (!parser_string_and_substituion.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
|
||||
command->part = true;
|
||||
@ -590,7 +614,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_fetch_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
|
||||
if (!s_from.ignore(pos, expected))
|
||||
@ -605,7 +629,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_fetch_part.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_string_literal.parse(pos, command->partition, expected))
|
||||
if (!parser_string_and_substituion.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
|
||||
if (!s_from.ignore(pos, expected))
|
||||
@ -622,7 +646,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
{
|
||||
if (s_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::FREEZE_PARTITION;
|
||||
@ -649,7 +673,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
{
|
||||
if (s_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::UNFREEZE_PARTITION;
|
||||
@ -684,7 +708,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (!is_modify)
|
||||
parser_modify_col_decl.enableCheckTypeKeyword();
|
||||
|
||||
if (!parser_modify_col_decl.parse(pos, command->col_decl, expected))
|
||||
if (!parser_modify_col_decl.parse(pos, command_col_decl, expected))
|
||||
return false;
|
||||
|
||||
if (s_remove.ignore(pos, expected))
|
||||
@ -710,7 +734,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
command->first = true;
|
||||
else if (s_after.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_name.parse(pos, command->column, expected))
|
||||
if (!parser_name.parse(pos, command_column, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@ -718,14 +742,14 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_modify_order_by.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_exp_elem.parse(pos, command->order_by, expected))
|
||||
if (!parser_exp_elem.parse(pos, command_order_by, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::MODIFY_ORDER_BY;
|
||||
}
|
||||
else if (s_modify_sample_by.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_exp_elem.parse(pos, command->sample_by, expected))
|
||||
if (!parser_exp_elem.parse(pos, command_sample_by, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::MODIFY_SAMPLE_BY;
|
||||
@ -738,33 +762,33 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
{
|
||||
if (s_in_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!s_where.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!parser_exp_elem.parse(pos, command->predicate, expected))
|
||||
if (!parser_exp_elem.parse(pos, command_predicate, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DELETE;
|
||||
}
|
||||
else if (s_update.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_assignment_list.parse(pos, command->update_assignments, expected))
|
||||
if (!parser_assignment_list.parse(pos, command_update_assignments, expected))
|
||||
return false;
|
||||
|
||||
if (s_in_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!s_where.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!parser_exp_elem.parse(pos, command->predicate, expected))
|
||||
if (!parser_exp_elem.parse(pos, command_predicate, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::UPDATE;
|
||||
@ -774,17 +798,17 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->column, expected))
|
||||
if (!parser_name.parse(pos, command_column, expected))
|
||||
return false;
|
||||
|
||||
if (!parser_string_literal.parse(pos, command->comment, expected))
|
||||
if (!parser_string_literal.parse(pos, command_comment, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::COMMENT_COLUMN;
|
||||
}
|
||||
else if (s_modify_ttl.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_ttl_list.parse(pos, command->ttl, expected))
|
||||
if (!parser_ttl_list.parse(pos, command_ttl, expected))
|
||||
return false;
|
||||
command->type = ASTAlterCommand::MODIFY_TTL;
|
||||
}
|
||||
@ -798,25 +822,25 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
if (s_in_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else if (s_modify_setting.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_settings.parse(pos, command->settings_changes, expected))
|
||||
if (!parser_settings.parse(pos, command_settings_changes, expected))
|
||||
return false;
|
||||
command->type = ASTAlterCommand::MODIFY_SETTING;
|
||||
}
|
||||
else if (s_reset_setting.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_reset_setting.parse(pos, command->settings_resets, expected))
|
||||
if (!parser_reset_setting.parse(pos, command_settings_resets, expected))
|
||||
return false;
|
||||
command->type = ASTAlterCommand::RESET_SETTING;
|
||||
}
|
||||
else if (s_modify_query.ignore(pos, expected))
|
||||
{
|
||||
if (!select_p.parse(pos, command->select, expected))
|
||||
if (!select_p.parse(pos, command_select, expected))
|
||||
return false;
|
||||
command->type = ASTAlterCommand::MODIFY_QUERY;
|
||||
}
|
||||
@ -828,7 +852,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_modify_comment.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_string_literal.parse(pos, command->comment, expected))
|
||||
if (!parser_string_literal.parse(pos, command_comment, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::MODIFY_COMMENT;
|
||||
@ -839,7 +863,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
if (s_in_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
if (!parser_partition.parse(pos, command_partition, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@ -848,44 +872,48 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
}
|
||||
|
||||
if (command->col_decl)
|
||||
command->children.push_back(command->col_decl);
|
||||
if (command->column)
|
||||
command->children.push_back(command->column);
|
||||
if (command->partition)
|
||||
command->children.push_back(command->partition);
|
||||
if (command->order_by)
|
||||
command->children.push_back(command->order_by);
|
||||
if (command->sample_by)
|
||||
command->children.push_back(command->sample_by);
|
||||
if (command->index_decl)
|
||||
command->children.push_back(command->index_decl);
|
||||
if (command->index)
|
||||
command->children.push_back(command->index);
|
||||
if (command->constraint_decl)
|
||||
command->children.push_back(command->constraint_decl);
|
||||
if (command->constraint)
|
||||
command->children.push_back(command->constraint);
|
||||
if (command->projection_decl)
|
||||
command->children.push_back(command->projection_decl);
|
||||
if (command->projection)
|
||||
command->children.push_back(command->projection);
|
||||
if (command->predicate)
|
||||
command->children.push_back(command->predicate);
|
||||
if (command->update_assignments)
|
||||
command->children.push_back(command->update_assignments);
|
||||
if (command->values)
|
||||
command->children.push_back(command->values);
|
||||
if (command->comment)
|
||||
command->children.push_back(command->comment);
|
||||
if (command->ttl)
|
||||
command->children.push_back(command->ttl);
|
||||
if (command->settings_changes)
|
||||
command->children.push_back(command->settings_changes);
|
||||
if (command->select)
|
||||
command->children.push_back(command->select);
|
||||
if (command->rename_to)
|
||||
command->children.push_back(command->rename_to);
|
||||
if (command_col_decl)
|
||||
command->col_decl = command->children.emplace_back(std::move(command_col_decl)).get();
|
||||
if (command_column)
|
||||
command->column = command->children.emplace_back(std::move(command_column)).get();
|
||||
if (command_order_by)
|
||||
command->order_by = command->children.emplace_back(std::move(command_order_by)).get();
|
||||
if (command_sample_by)
|
||||
command->sample_by = command->children.emplace_back(std::move(command_sample_by)).get();
|
||||
if (command_index_decl)
|
||||
command->index_decl = command->children.emplace_back(std::move(command_index_decl)).get();
|
||||
if (command_index)
|
||||
command->index = command->children.emplace_back(std::move(command_index)).get();
|
||||
if (command_constraint_decl)
|
||||
command->constraint_decl = command->children.emplace_back(std::move(command_constraint_decl)).get();
|
||||
if (command_constraint)
|
||||
command->constraint = command->children.emplace_back(std::move(command_constraint)).get();
|
||||
if (command_projection_decl)
|
||||
command->projection_decl = command->children.emplace_back(std::move(command_projection_decl)).get();
|
||||
if (command_projection)
|
||||
command->projection = command->children.emplace_back(std::move(command_projection)).get();
|
||||
if (command_statistic_decl)
|
||||
command->statistic_decl = command->children.emplace_back(std::move(command_statistic_decl)).get();
|
||||
if (command_partition)
|
||||
command->partition = command->children.emplace_back(std::move(command_partition)).get();
|
||||
if (command_predicate)
|
||||
command->predicate = command->children.emplace_back(std::move(command_predicate)).get();
|
||||
if (command_update_assignments)
|
||||
command->update_assignments = command->children.emplace_back(std::move(command_update_assignments)).get();
|
||||
if (command_comment)
|
||||
command->comment = command->children.emplace_back(std::move(command_comment)).get();
|
||||
if (command_ttl)
|
||||
command->ttl = command->children.emplace_back(std::move(command_ttl)).get();
|
||||
if (command_settings_changes)
|
||||
command->settings_changes = command->children.emplace_back(std::move(command_settings_changes)).get();
|
||||
if (command_settings_resets)
|
||||
command->settings_resets = command->children.emplace_back(std::move(command_settings_resets)).get();
|
||||
if (command_select)
|
||||
command->select = command->children.emplace_back(std::move(command_select)).get();
|
||||
if (command_values)
|
||||
command->values = command->children.emplace_back(std::move(command_values)).get();
|
||||
if (command_rename_to)
|
||||
command->rename_to = command->children.emplace_back(std::move(command_rename_to)).get();
|
||||
|
||||
return true;
|
||||
}
|
||||
|
12
src/Parsers/ParserStringAndSubstitution.cpp
Normal file
12
src/Parsers/ParserStringAndSubstitution.cpp
Normal file
@ -0,0 +1,12 @@
|
||||
#include <Parsers/ParserStringAndSubstitution.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserStringAndSubstitution::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
return ParserStringLiteral{}.parse(pos, node, expected) || ParserSubstitution{}.parse(pos, node, expected);
|
||||
}
|
||||
|
||||
}
|
18
src/Parsers/ParserStringAndSubstitution.h
Normal file
18
src/Parsers/ParserStringAndSubstitution.h
Normal file
@ -0,0 +1,18 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ParserStringAndSubstitution : public IParserBase
|
||||
{
|
||||
private:
|
||||
const char * getName() const override { return "ParserStringAndSubstitution"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
|
||||
public:
|
||||
explicit ParserStringAndSubstitution() = default;
|
||||
};
|
||||
|
||||
}
|
@ -137,7 +137,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
command.clear = true;
|
||||
|
||||
if (command_ast->partition)
|
||||
command.partition = command_ast->partition;
|
||||
command.partition = command_ast->partition->clone();
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::MODIFY_COLUMN)
|
||||
@ -206,7 +206,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
AlterCommand command;
|
||||
command.ast = command_ast->clone();
|
||||
command.type = AlterCommand::MODIFY_ORDER_BY;
|
||||
command.order_by = command_ast->order_by;
|
||||
command.order_by = command_ast->order_by->clone();
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::MODIFY_SAMPLE_BY)
|
||||
@ -214,7 +214,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
AlterCommand command;
|
||||
command.ast = command_ast->clone();
|
||||
command.type = AlterCommand::MODIFY_SAMPLE_BY;
|
||||
command.sample_by = command_ast->sample_by;
|
||||
command.sample_by = command_ast->sample_by->clone();
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::REMOVE_SAMPLE_BY)
|
||||
@ -228,7 +228,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
{
|
||||
AlterCommand command;
|
||||
command.ast = command_ast->clone();
|
||||
command.index_decl = command_ast->index_decl;
|
||||
command.index_decl = command_ast->index_decl->clone();
|
||||
command.type = AlterCommand::ADD_INDEX;
|
||||
|
||||
const auto & ast_index_decl = command_ast->index_decl->as<ASTIndexDeclaration &>();
|
||||
@ -247,7 +247,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
{
|
||||
AlterCommand command;
|
||||
command.ast = command_ast->clone();
|
||||
command.statistic_decl = command_ast->statistic_decl;
|
||||
command.statistic_decl = command_ast->statistic_decl->clone();
|
||||
command.type = AlterCommand::ADD_STATISTIC;
|
||||
|
||||
const auto & ast_stat_decl = command_ast->statistic_decl->as<ASTStatisticDeclaration &>();
|
||||
@ -262,7 +262,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
{
|
||||
AlterCommand command;
|
||||
command.ast = command_ast->clone();
|
||||
command.constraint_decl = command_ast->constraint_decl;
|
||||
command.constraint_decl = command_ast->constraint_decl->clone();
|
||||
command.type = AlterCommand::ADD_CONSTRAINT;
|
||||
|
||||
const auto & ast_constraint_decl = command_ast->constraint_decl->as<ASTConstraintDeclaration &>();
|
||||
@ -277,7 +277,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
{
|
||||
AlterCommand command;
|
||||
command.ast = command_ast->clone();
|
||||
command.projection_decl = command_ast->projection_decl;
|
||||
command.projection_decl = command_ast->projection_decl->clone();
|
||||
command.type = AlterCommand::ADD_PROJECTION;
|
||||
|
||||
const auto & ast_projection_decl = command_ast->projection_decl->as<ASTProjectionDeclaration &>();
|
||||
@ -313,7 +313,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
command.clear = true;
|
||||
|
||||
if (command_ast->partition)
|
||||
command.partition = command_ast->partition;
|
||||
command.partition = command_ast->partition->clone();
|
||||
|
||||
return command;
|
||||
}
|
||||
@ -330,7 +330,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
command.clear = command_ast->clear_statistic;
|
||||
|
||||
if (command_ast->partition)
|
||||
command.partition = command_ast->partition;
|
||||
command.partition = command_ast->partition->clone();
|
||||
|
||||
return command;
|
||||
}
|
||||
@ -345,7 +345,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
command.clear = true;
|
||||
|
||||
if (command_ast->partition)
|
||||
command.partition = command_ast->partition;
|
||||
command.partition = command_ast->partition->clone();
|
||||
|
||||
return command;
|
||||
}
|
||||
@ -354,7 +354,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
AlterCommand command;
|
||||
command.ast = command_ast->clone();
|
||||
command.type = AlterCommand::MODIFY_TTL;
|
||||
command.ttl = command_ast->ttl;
|
||||
command.ttl = command_ast->ttl->clone();
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::REMOVE_TTL)
|
||||
@ -399,7 +399,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
AlterCommand command;
|
||||
command.ast = command_ast->clone();
|
||||
command.type = AlterCommand::MODIFY_QUERY;
|
||||
command.select = command_ast->select;
|
||||
command.select = command_ast->select->clone();
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::MODIFY_REFRESH)
|
||||
|
@ -37,8 +37,9 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
|
||||
MutationCommand res;
|
||||
res.ast = command->ptr();
|
||||
res.type = DELETE;
|
||||
res.predicate = command->predicate;
|
||||
res.partition = command->partition;
|
||||
res.predicate = command->predicate->clone();
|
||||
if (command->partition)
|
||||
res.partition = command->partition->clone();
|
||||
return res;
|
||||
}
|
||||
else if (command->type == ASTAlterCommand::UPDATE)
|
||||
@ -46,8 +47,9 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
|
||||
MutationCommand res;
|
||||
res.ast = command->ptr();
|
||||
res.type = UPDATE;
|
||||
res.predicate = command->predicate;
|
||||
res.partition = command->partition;
|
||||
res.predicate = command->predicate->clone();
|
||||
if (command->partition)
|
||||
res.partition = command->partition->clone();
|
||||
for (const ASTPtr & assignment_ast : command->update_assignments->children)
|
||||
{
|
||||
const auto & assignment = assignment_ast->as<ASTAssignment &>();
|
||||
@ -64,8 +66,10 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
|
||||
MutationCommand res;
|
||||
res.ast = command->ptr();
|
||||
res.type = APPLY_DELETED_MASK;
|
||||
res.predicate = command->predicate;
|
||||
res.partition = command->partition;
|
||||
if (command->predicate)
|
||||
res.predicate = command->predicate->clone();
|
||||
if (command->partition)
|
||||
res.partition = command->partition->clone();
|
||||
return res;
|
||||
}
|
||||
else if (command->type == ASTAlterCommand::MATERIALIZE_INDEX)
|
||||
@ -73,7 +77,8 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
|
||||
MutationCommand res;
|
||||
res.ast = command->ptr();
|
||||
res.type = MATERIALIZE_INDEX;
|
||||
res.partition = command->partition;
|
||||
if (command->partition)
|
||||
res.partition = command->partition->clone();
|
||||
res.predicate = nullptr;
|
||||
res.index_name = command->index->as<ASTIdentifier &>().name();
|
||||
return res;
|
||||
@ -83,7 +88,8 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
|
||||
MutationCommand res;
|
||||
res.ast = command->ptr();
|
||||
res.type = MATERIALIZE_STATISTIC;
|
||||
res.partition = command->partition;
|
||||
if (command->partition)
|
||||
res.partition = command->partition->clone();
|
||||
res.predicate = nullptr;
|
||||
res.statistic_columns = command->statistic_decl->as<ASTStatisticDeclaration &>().getColumnNames();
|
||||
return res;
|
||||
@ -93,7 +99,8 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
|
||||
MutationCommand res;
|
||||
res.ast = command->ptr();
|
||||
res.type = MATERIALIZE_PROJECTION;
|
||||
res.partition = command->partition;
|
||||
if (command->partition)
|
||||
res.partition = command->partition->clone();
|
||||
res.predicate = nullptr;
|
||||
res.projection_name = command->projection->as<ASTIdentifier &>().name();
|
||||
return res;
|
||||
@ -103,7 +110,8 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
|
||||
MutationCommand res;
|
||||
res.ast = command->ptr();
|
||||
res.type = MATERIALIZE_COLUMN;
|
||||
res.partition = command->partition;
|
||||
if (command->partition)
|
||||
res.partition = command->partition->clone();
|
||||
res.column_name = getIdentifierName(command->column);
|
||||
return res;
|
||||
}
|
||||
@ -124,7 +132,7 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
|
||||
res.type = MutationCommand::Type::DROP_COLUMN;
|
||||
res.column_name = getIdentifierName(command->column);
|
||||
if (command->partition)
|
||||
res.partition = command->partition;
|
||||
res.partition = command->partition->clone();
|
||||
if (command->clear_column)
|
||||
res.clear = true;
|
||||
|
||||
@ -137,7 +145,7 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
|
||||
res.type = MutationCommand::Type::DROP_INDEX;
|
||||
res.column_name = command->index->as<ASTIdentifier &>().name();
|
||||
if (command->partition)
|
||||
res.partition = command->partition;
|
||||
res.partition = command->partition->clone();
|
||||
if (command->clear_index)
|
||||
res.clear = true;
|
||||
return res;
|
||||
@ -148,7 +156,7 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
|
||||
res.ast = command->ptr();
|
||||
res.type = MutationCommand::Type::DROP_STATISTIC;
|
||||
if (command->partition)
|
||||
res.partition = command->partition;
|
||||
res.partition = command->partition->clone();
|
||||
if (command->clear_index)
|
||||
res.clear = true;
|
||||
res.statistic_columns = command->statistic_decl->as<ASTStatisticDeclaration &>().getColumnNames();
|
||||
@ -161,7 +169,7 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
|
||||
res.type = MutationCommand::Type::DROP_PROJECTION;
|
||||
res.column_name = command->projection->as<ASTIdentifier &>().name();
|
||||
if (command->partition)
|
||||
res.partition = command->partition;
|
||||
res.partition = command->partition->clone();
|
||||
if (command->clear_projection)
|
||||
res.clear = true;
|
||||
return res;
|
||||
@ -180,7 +188,8 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
|
||||
MutationCommand res;
|
||||
res.ast = command->ptr();
|
||||
res.type = MATERIALIZE_TTL;
|
||||
res.partition = command->partition;
|
||||
if (command->partition)
|
||||
res.partition = command->partition->clone();
|
||||
return res;
|
||||
}
|
||||
else
|
||||
|
@ -23,7 +23,7 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
|
||||
{
|
||||
PartitionCommand res;
|
||||
res.type = DROP_PARTITION;
|
||||
res.partition = command_ast->partition;
|
||||
res.partition = command_ast->partition->clone();
|
||||
res.detach = command_ast->detach;
|
||||
res.part = command_ast->part;
|
||||
return res;
|
||||
@ -32,7 +32,7 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
|
||||
{
|
||||
PartitionCommand res;
|
||||
res.type = DROP_DETACHED_PARTITION;
|
||||
res.partition = command_ast->partition;
|
||||
res.partition = command_ast->partition->clone();
|
||||
res.part = command_ast->part;
|
||||
return res;
|
||||
}
|
||||
@ -40,7 +40,7 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
|
||||
{
|
||||
PartitionCommand res;
|
||||
res.type = ATTACH_PARTITION;
|
||||
res.partition = command_ast->partition;
|
||||
res.partition = command_ast->partition->clone();
|
||||
res.part = command_ast->part;
|
||||
return res;
|
||||
}
|
||||
@ -48,7 +48,7 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
|
||||
{
|
||||
PartitionCommand res;
|
||||
res.type = MOVE_PARTITION;
|
||||
res.partition = command_ast->partition;
|
||||
res.partition = command_ast->partition->clone();
|
||||
res.part = command_ast->part;
|
||||
switch (command_ast->move_destination_type)
|
||||
{
|
||||
@ -77,7 +77,7 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
|
||||
{
|
||||
PartitionCommand res;
|
||||
res.type = REPLACE_PARTITION;
|
||||
res.partition = command_ast->partition;
|
||||
res.partition = command_ast->partition->clone();
|
||||
res.replace = command_ast->replace;
|
||||
res.from_database = command_ast->from_database;
|
||||
res.from_table = command_ast->from_table;
|
||||
@ -87,7 +87,7 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
|
||||
{
|
||||
PartitionCommand res;
|
||||
res.type = FETCH_PARTITION;
|
||||
res.partition = command_ast->partition;
|
||||
res.partition = command_ast->partition->clone();
|
||||
res.from_zookeeper_path = command_ast->from;
|
||||
res.part = command_ast->part;
|
||||
return res;
|
||||
@ -96,7 +96,7 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
|
||||
{
|
||||
PartitionCommand res;
|
||||
res.type = FREEZE_PARTITION;
|
||||
res.partition = command_ast->partition;
|
||||
res.partition = command_ast->partition->clone();
|
||||
res.with_name = command_ast->with_name;
|
||||
return res;
|
||||
}
|
||||
@ -111,7 +111,7 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
|
||||
{
|
||||
PartitionCommand res;
|
||||
res.type = PartitionCommand::UNFREEZE_PARTITION;
|
||||
res.partition = command_ast->partition;
|
||||
res.partition = command_ast->partition->clone();
|
||||
res.with_name = command_ast->with_name;
|
||||
return res;
|
||||
}
|
||||
|
@ -418,8 +418,7 @@ ASTPtr StorageWindowView::getCleanupQuery()
|
||||
|
||||
auto alter_command = std::make_shared<ASTAlterCommand>();
|
||||
alter_command->type = ASTAlterCommand::DELETE;
|
||||
alter_command->predicate = function_less;
|
||||
alter_command->children.push_back(alter_command->predicate);
|
||||
alter_command->predicate = alter_command->children.emplace_back(function_less).get();
|
||||
alter_query->command_list->children.push_back(alter_command);
|
||||
return alter_query;
|
||||
}
|
||||
|
@ -9,7 +9,7 @@
|
||||
AlterCommand MODIFY_TTL (children 1)
|
||||
AlterCommand MATERIALIZE_TTL (children 1)
|
||||
AlterCommand MODIFY_SETTING (children 1)
|
||||
AlterCommand RESET_SETTING
|
||||
AlterCommand RESET_SETTING (children 1)
|
||||
AlterCommand MODIFY_QUERY (children 1)
|
||||
AlterCommand REMOVE_TTL
|
||||
AlterCommand REMOVE_SAMPLE_BY
|
||||
|
@ -0,0 +1,15 @@
|
||||
drop table if exists data;
|
||||
create table data (key Int) engine=MergeTree() order by key;
|
||||
|
||||
insert into data values (1);
|
||||
|
||||
set param_part='all_1_1_0';
|
||||
alter table data detach part {part:String};
|
||||
alter table data attach part {part:String};
|
||||
set param_part='all_2_2_0';
|
||||
alter table data detach part {part:String};
|
||||
alter table data drop detached part {part:String} settings allow_drop_detached=1;
|
||||
|
||||
insert into data values (2);
|
||||
set param_part='all_3_3_0';
|
||||
alter table data drop part {part:String};
|
Loading…
Reference in New Issue
Block a user