mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Added ALTER TABLE support to constraints (ADD CONSTRAINT, DROP CONSTRAINT)
This commit is contained in:
parent
2b5cf46842
commit
e7293486bd
@ -131,6 +131,7 @@ void DatabaseDictionary::alterTable(
|
||||
const String &,
|
||||
const ColumnsDescription &,
|
||||
const IndicesDescription &,
|
||||
const ConstraintsDescription &,
|
||||
const ASTModifier &)
|
||||
{
|
||||
throw Exception("DatabaseDictionary: alterTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
@ -72,6 +72,7 @@ public:
|
||||
const String & name,
|
||||
const ColumnsDescription & columns,
|
||||
const IndicesDescription & indices,
|
||||
const ConstraintsDescription & constraints,
|
||||
const ASTModifier & engine_modifier) override;
|
||||
|
||||
time_t getTableMetadataModificationTime(
|
||||
|
@ -54,6 +54,7 @@ void DatabaseMemory::alterTable(
|
||||
const String &,
|
||||
const ColumnsDescription &,
|
||||
const IndicesDescription &,
|
||||
const ConstraintsDescription &,
|
||||
const ASTModifier &)
|
||||
{
|
||||
throw Exception("DatabaseMemory: alterTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
@ -49,6 +49,7 @@ public:
|
||||
const String & name,
|
||||
const ColumnsDescription & columns,
|
||||
const IndicesDescription & indices,
|
||||
const ConstraintsDescription & constraints,
|
||||
const ASTModifier & engine_modifier) override;
|
||||
|
||||
time_t getTableMetadataModificationTime(
|
||||
|
@ -516,6 +516,7 @@ void DatabaseOrdinary::alterTable(
|
||||
const String & table_name,
|
||||
const ColumnsDescription & columns,
|
||||
const IndicesDescription & indices,
|
||||
const ConstraintsDescription & constraints,
|
||||
const ASTModifier & storage_modifier)
|
||||
{
|
||||
/// Read the definition of the table and replace the necessary parts with new ones.
|
||||
@ -538,6 +539,7 @@ void DatabaseOrdinary::alterTable(
|
||||
|
||||
ASTPtr new_columns = InterpreterCreateQuery::formatColumns(columns);
|
||||
ASTPtr new_indices = InterpreterCreateQuery::formatIndices(indices);
|
||||
ASTPtr new_constraints = InterpreterCreateQuery::formatConstraints(constraints);
|
||||
|
||||
ast_create_query.columns_list->replace(ast_create_query.columns_list->columns, new_columns);
|
||||
|
||||
@ -546,6 +548,11 @@ void DatabaseOrdinary::alterTable(
|
||||
else
|
||||
ast_create_query.columns_list->set(ast_create_query.columns_list->indices, new_indices);
|
||||
|
||||
if (ast_create_query.columns_list->constraints)
|
||||
ast_create_query.columns_list->replace(ast_create_query.columns_list->constraints, new_constraints);
|
||||
else
|
||||
ast_create_query.columns_list->set(ast_create_query.columns_list->constraints, new_constraints);
|
||||
|
||||
if (storage_modifier)
|
||||
storage_modifier(*ast_create_query.storage);
|
||||
|
||||
|
@ -43,6 +43,7 @@ public:
|
||||
const String & name,
|
||||
const ColumnsDescription & columns,
|
||||
const IndicesDescription & indices,
|
||||
const ConstraintsDescription & constraints,
|
||||
const ASTModifier & engine_modifier) override;
|
||||
|
||||
time_t getTableMetadataModificationTime(
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/IndicesDescription.h>
|
||||
#include <Storages/ConstraintsDescription.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Poco/File.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
@ -114,6 +115,7 @@ public:
|
||||
const String & name,
|
||||
const ColumnsDescription & columns,
|
||||
const IndicesDescription & indices,
|
||||
const ConstraintsDescription & constraints,
|
||||
const ASTModifier & engine_modifier) = 0;
|
||||
|
||||
/// Returns time of table's metadata change, 0 if there is no corresponding metadata file.
|
||||
|
@ -105,6 +105,17 @@ void ASTAlterCommand::formatImpl(
|
||||
<< "DROP INDEX " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
|
||||
index->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::ADD_CONSTRAINT)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD CONSTRAINT" << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "");
|
||||
constraint_decl->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::DROP_CONSTRAINT)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str
|
||||
<< "DROP CONSTRAINT " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
|
||||
constraint->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::DROP_PARTITION)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << (detach ? "DETACH" : "DROP") << " PARTITION "
|
||||
|
@ -32,6 +32,9 @@ public:
|
||||
ADD_INDEX,
|
||||
DROP_INDEX,
|
||||
|
||||
ADD_CONSTRAINT,
|
||||
DROP_CONSTRAINT,
|
||||
|
||||
DROP_PARTITION,
|
||||
ATTACH_PARTITION,
|
||||
REPLACE_PARTITION,
|
||||
@ -69,7 +72,15 @@ public:
|
||||
/** The ADD INDEX query stores the name of the index following AFTER.
|
||||
* The DROP INDEX query stores the name for deletion.
|
||||
*/
|
||||
ASTPtr index;
|
||||
ASTPtr index;
|
||||
|
||||
/** The ADD CONSTRAINT query stores the ConstraintDeclaration there.
|
||||
*/
|
||||
ASTPtr constraint_decl;
|
||||
|
||||
/** The DROP CONSTRAINT query stores the name for deletion.
|
||||
*/
|
||||
ASTPtr constraint;
|
||||
|
||||
/** Used in DROP PARTITION and ATTACH PARTITION FROM queries.
|
||||
* The value or ID of the partition is stored here.
|
||||
|
@ -32,6 +32,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserKeyword s_add_index("ADD INDEX");
|
||||
ParserKeyword s_drop_index("DROP INDEX");
|
||||
|
||||
ParserKeyword s_add_constraint("ADD CONSTRAINT");
|
||||
ParserKeyword s_drop_constraint("DROP CONSTRAINT");
|
||||
|
||||
ParserKeyword s_attach_partition("ATTACH PARTITION");
|
||||
ParserKeyword s_detach_partition("DETACH PARTITION");
|
||||
ParserKeyword s_drop_partition("DROP PARTITION");
|
||||
@ -57,6 +60,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserStringLiteral parser_string_literal;
|
||||
ParserCompoundColumnDeclaration parser_col_decl;
|
||||
ParserIndexDeclaration parser_idx_decl;
|
||||
ParserConstraintDeclaration parser_constraint_decl;
|
||||
ParserCompoundColumnDeclaration parser_modify_col_decl(false);
|
||||
ParserPartition parser_partition;
|
||||
ParserExpression parser_exp_elem;
|
||||
@ -125,6 +129,27 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
command->type = ASTAlterCommand::DROP_INDEX;
|
||||
command->detach = false;
|
||||
}
|
||||
else if (s_add_constraint.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_not_exists.ignore(pos, expected))
|
||||
command->if_not_exists = true;
|
||||
|
||||
if (!parser_constraint_decl.parse(pos, command->constraint_decl, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::ADD_CONSTRAINT;
|
||||
}
|
||||
else if (s_drop_constraint.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->constraint, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_CONSTRAINT;
|
||||
command->detach = false;
|
||||
}
|
||||
else if (s_clear_column.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTIndexDeclaration.h>
|
||||
#include <Parsers/ASTConstraintDeclaration.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
@ -165,6 +166,32 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::ADD_CONSTRAINT)
|
||||
{
|
||||
AlterCommand command;
|
||||
command.constraint_decl = command_ast->constraint_decl;
|
||||
command.type = AlterCommand::ADD_CONSTRAINT;
|
||||
|
||||
const auto & ast_constraint_decl = command_ast->constraint_decl->as<ASTConstraintDeclaration &>();
|
||||
|
||||
command.constraint_name = ast_constraint_decl.name;
|
||||
|
||||
command.if_not_exists = command_ast->if_not_exists;
|
||||
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::DROP_CONSTRAINT)
|
||||
{
|
||||
if (command_ast->clear_column)
|
||||
throw Exception("\"ALTER TABLE table CLEAR COLUMN column\" queries are not supported yet. Use \"CLEAR COLUMN column IN PARTITION\".", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
AlterCommand command;
|
||||
command.type = AlterCommand::DROP_CONSTRAINT;
|
||||
command.constraint_name = command_ast->constraint->as<ASTIdentifier &>().name;
|
||||
command.if_exists = command_ast->if_exists;
|
||||
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::MODIFY_TTL)
|
||||
{
|
||||
AlterCommand command;
|
||||
@ -178,7 +205,8 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
|
||||
|
||||
void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescription & indices_description,
|
||||
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const
|
||||
ConstraintsDescription & constraints_description,
|
||||
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const
|
||||
{
|
||||
if (type == ADD_COLUMN)
|
||||
{
|
||||
@ -298,6 +326,44 @@ void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescri
|
||||
|
||||
indices_description.indices.erase(erase_it);
|
||||
}
|
||||
else if (type == ADD_CONSTRAINT)
|
||||
{
|
||||
if (std::any_of(
|
||||
constraints_description.constraints.cbegin(),
|
||||
constraints_description.constraints.cend(),
|
||||
[this](const ASTPtr & constraint_ast)
|
||||
{
|
||||
return constraint_ast->as<ASTConstraintDeclaration &>().name == constraint_name;
|
||||
}))
|
||||
{
|
||||
if (if_not_exists)
|
||||
return;
|
||||
throw Exception("Cannot add constraint " + constraint_name + ": constraint with this name already exists",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
auto insert_it = constraints_description.constraints.end();
|
||||
|
||||
constraints_description.constraints.emplace(insert_it, std::dynamic_pointer_cast<ASTConstraintDeclaration>(constraint_decl));
|
||||
}
|
||||
else if (type == DROP_CONSTRAINT)
|
||||
{
|
||||
auto erase_it = std::find_if(
|
||||
constraints_description.constraints.begin(),
|
||||
constraints_description.constraints.end(),
|
||||
[this](const ASTPtr & constraint_ast)
|
||||
{
|
||||
return constraint_ast->as<ASTConstraintDeclaration &>().name == constraint_name;
|
||||
});
|
||||
|
||||
if (erase_it == constraints_description.constraints.end()) {
|
||||
if (if_exists)
|
||||
return;
|
||||
throw Exception("Wrong constraint name. Cannot find constraint `" + constraint_name + "` to drop.",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
constraints_description.constraints.erase(erase_it);
|
||||
}
|
||||
else if (type == MODIFY_TTL)
|
||||
{
|
||||
ttl_table_ast = ttl;
|
||||
@ -317,20 +383,23 @@ bool AlterCommand::isMutable() const
|
||||
}
|
||||
|
||||
void AlterCommands::apply(ColumnsDescription & columns_description, IndicesDescription & indices_description,
|
||||
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const
|
||||
ConstraintsDescription & constraints_description,
|
||||
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const
|
||||
{
|
||||
auto new_columns_description = columns_description;
|
||||
auto new_indices_description = indices_description;
|
||||
auto new_constraints_description = constraints_description;
|
||||
auto new_order_by_ast = order_by_ast;
|
||||
auto new_primary_key_ast = primary_key_ast;
|
||||
auto new_ttl_table_ast = ttl_table_ast;
|
||||
|
||||
for (const AlterCommand & command : *this)
|
||||
if (!command.ignore)
|
||||
command.apply(new_columns_description, new_indices_description, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
|
||||
command.apply(new_columns_description, new_indices_description, new_constraints_description, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
|
||||
|
||||
columns_description = std::move(new_columns_description);
|
||||
indices_description = std::move(new_indices_description);
|
||||
constraints_description = std::move(new_constraints_description);
|
||||
order_by_ast = std::move(new_order_by_ast);
|
||||
primary_key_ast = std::move(new_primary_key_ast);
|
||||
ttl_table_ast = std::move(new_ttl_table_ast);
|
||||
@ -518,10 +587,11 @@ void AlterCommands::apply(ColumnsDescription & columns_description) const
|
||||
{
|
||||
auto out_columns_description = columns_description;
|
||||
IndicesDescription indices_description;
|
||||
ConstraintsDescription constraints_description;
|
||||
ASTPtr out_order_by;
|
||||
ASTPtr out_primary_key;
|
||||
ASTPtr out_ttl_table;
|
||||
apply(out_columns_description, indices_description, out_order_by, out_primary_key, out_ttl_table);
|
||||
apply(out_columns_description, indices_description, constraints_description, out_order_by, out_primary_key, out_ttl_table);
|
||||
|
||||
if (out_order_by)
|
||||
throw Exception("Storage doesn't support modifying ORDER BY expression", ErrorCodes::NOT_IMPLEMENTED);
|
||||
@ -529,6 +599,8 @@ void AlterCommands::apply(ColumnsDescription & columns_description) const
|
||||
throw Exception("Storage doesn't support modifying PRIMARY KEY expression", ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (!indices_description.indices.empty())
|
||||
throw Exception("Storage doesn't support modifying indices", ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (!constraints_description.constraints.empty())
|
||||
throw Exception("Storage doesn't support modifying constraints", ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (out_ttl_table)
|
||||
throw Exception("Storage doesn't support modifying TTL expression", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Storages/IndicesDescription.h>
|
||||
#include <Storages/ConstraintsDescription.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -25,6 +26,8 @@ struct AlterCommand
|
||||
MODIFY_ORDER_BY,
|
||||
ADD_INDEX,
|
||||
DROP_INDEX,
|
||||
ADD_CONSTRAINT,
|
||||
DROP_CONSTRAINT,
|
||||
MODIFY_TTL,
|
||||
UKNOWN_TYPE,
|
||||
};
|
||||
@ -62,6 +65,12 @@ struct AlterCommand
|
||||
/// For ADD/DROP INDEX
|
||||
String index_name;
|
||||
|
||||
// For ADD CONSTRAINT
|
||||
ASTPtr constraint_decl;
|
||||
|
||||
// For ADD/DROP CONSTRAINT
|
||||
String constraint_name;
|
||||
|
||||
/// For MODIFY TTL
|
||||
ASTPtr ttl;
|
||||
|
||||
@ -84,7 +93,8 @@ struct AlterCommand
|
||||
static std::optional<AlterCommand> parse(const ASTAlterCommand * command);
|
||||
|
||||
void apply(ColumnsDescription & columns_description, IndicesDescription & indices_description,
|
||||
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const;
|
||||
ConstraintsDescription & constraints_description,
|
||||
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const;
|
||||
|
||||
/// Checks that not only metadata touched by that command
|
||||
bool isMutable() const;
|
||||
@ -95,8 +105,9 @@ class Context;
|
||||
class AlterCommands : public std::vector<AlterCommand>
|
||||
{
|
||||
public:
|
||||
void apply(ColumnsDescription & columns_description, IndicesDescription & indices_description, ASTPtr & order_by_ast,
|
||||
ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const;
|
||||
void apply(ColumnsDescription & columns_description, IndicesDescription & indices_description,
|
||||
ConstraintsDescription & constraints_description,
|
||||
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast) const;
|
||||
|
||||
/// For storages that don't support MODIFY_ORDER_BY.
|
||||
void apply(ColumnsDescription & columns_description) const;
|
||||
|
@ -346,8 +346,9 @@ void IStorage::alter(
|
||||
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
params.apply(new_columns);
|
||||
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, {});
|
||||
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, new_constraints, {});
|
||||
setColumns(std::move(new_columns));
|
||||
}
|
||||
|
||||
|
@ -121,7 +121,7 @@ MergeTreeData::MergeTreeData(
|
||||
data_parts_by_info(data_parts_indexes.get<TagByInfo>()),
|
||||
data_parts_by_state_and_info(data_parts_indexes.get<TagByStateAndInfo>())
|
||||
{
|
||||
setPrimaryKeyIndicesAndColumns(order_by_ast_, primary_key_ast_, columns_, indices_);
|
||||
setPrimaryKeyIndicesAndColumns(order_by_ast_, primary_key_ast_, columns_, indices_, constraints_);
|
||||
setConstraints(constraints_);
|
||||
|
||||
/// NOTE: using the same columns list as is read when performing actual merges.
|
||||
@ -255,7 +255,8 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam
|
||||
|
||||
void MergeTreeData::setPrimaryKeyIndicesAndColumns(
|
||||
const ASTPtr & new_order_by_ast, const ASTPtr & new_primary_key_ast,
|
||||
const ColumnsDescription & new_columns, const IndicesDescription & indices_description, bool only_check)
|
||||
const ColumnsDescription & new_columns, const IndicesDescription & indices_description,
|
||||
const ConstraintsDescription & constraints_description, bool only_check)
|
||||
{
|
||||
if (!new_order_by_ast)
|
||||
throw Exception("ORDER BY cannot be empty", ErrorCodes::BAD_ARGUMENTS);
|
||||
@ -425,6 +426,8 @@ void MergeTreeData::setPrimaryKeyIndicesAndColumns(
|
||||
setIndices(indices_description);
|
||||
skip_indices = std::move(new_indices);
|
||||
|
||||
setConstraints(constraints_description);
|
||||
|
||||
primary_key_and_skip_indices_expr = new_indices_with_primary_key_expr;
|
||||
sorting_key_and_skip_indices_expr = new_indices_with_sorting_key_expr;
|
||||
}
|
||||
@ -1180,11 +1183,11 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c
|
||||
/// Check that needed transformations can be applied to the list of columns without considering type conversions.
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
ASTPtr new_order_by_ast = order_by_ast;
|
||||
ASTPtr new_primary_key_ast = primary_key_ast;
|
||||
ASTPtr new_ttl_table_ast = ttl_table_ast;
|
||||
commands.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
|
||||
|
||||
commands.apply(new_columns, new_indices, new_constraints, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
|
||||
if (getIndices().empty() && !new_indices.empty() &&
|
||||
!context.getSettingsRef().allow_experimental_data_skipping_indices)
|
||||
throw Exception("You must set the setting `allow_experimental_data_skipping_indices` to 1 " \
|
||||
@ -1267,7 +1270,7 @@ void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & c
|
||||
}
|
||||
|
||||
setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast,
|
||||
new_columns, new_indices, /* only_check = */ true);
|
||||
new_columns, new_indices, new_constraints, /* only_check = */ true);
|
||||
|
||||
setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast, /* only_check = */ true);
|
||||
|
||||
|
@ -778,7 +778,8 @@ protected:
|
||||
|
||||
void setPrimaryKeyIndicesAndColumns(const ASTPtr & new_order_by_ast, const ASTPtr & new_primary_key_ast,
|
||||
const ColumnsDescription & new_columns,
|
||||
const IndicesDescription & indices_description, bool only_check = false);
|
||||
const IndicesDescription & indices_description,
|
||||
const ConstraintsDescription & constraints_description, bool only_check = false);
|
||||
|
||||
void initPartitionKey();
|
||||
|
||||
|
@ -46,6 +46,7 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr
|
||||
partition_key = formattedAST(MergeTreeData::extractKeyExpressionList(data.partition_by_ast));
|
||||
|
||||
skip_indices = data.getIndices().toString();
|
||||
constraints = data.getConstraints().toString();
|
||||
index_granularity_bytes = data.index_granularity_info.index_granularity_bytes;
|
||||
ttl_table = formattedAST(data.ttl_table_ast);
|
||||
}
|
||||
@ -229,6 +230,21 @@ ReplicatedMergeTreeTableMetadata::checkAndFindDiff(const ReplicatedMergeTreeTabl
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
}
|
||||
|
||||
if (constraints != from_zk.constraints)
|
||||
{
|
||||
if (allow_alter)
|
||||
{
|
||||
diff.constraints_changed = true;
|
||||
diff.new_constraints = from_zk.constraints;
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Existing table metadata in ZooKeeper differs in constraints."
|
||||
" Stored in ZooKeeper: " + from_zk.constraints +
|
||||
", local: " + constraints,
|
||||
ErrorCodes::METADATA_MISMATCH);
|
||||
}
|
||||
|
||||
if (index_granularity_bytes != from_zk.index_granularity_bytes)
|
||||
throw Exception("Existing table metadata in ZooKeeper differs in index granularity bytes."
|
||||
" Stored in ZooKeeper: " + DB::toString(from_zk.index_granularity_bytes) +
|
||||
|
@ -26,6 +26,7 @@ struct ReplicatedMergeTreeTableMetadata
|
||||
String partition_key;
|
||||
String sorting_key;
|
||||
String skip_indices;
|
||||
String constraints;
|
||||
UInt64 index_granularity_bytes;
|
||||
String ttl_table;
|
||||
|
||||
@ -46,10 +47,13 @@ struct ReplicatedMergeTreeTableMetadata
|
||||
bool skip_indices_changed = false;
|
||||
String new_skip_indices;
|
||||
|
||||
bool constraints_changed = false;
|
||||
String new_constraints;
|
||||
|
||||
bool ttl_table_changed = false;
|
||||
String new_ttl_table;
|
||||
|
||||
bool empty() const { return !sorting_key_changed && !skip_indices_changed; }
|
||||
bool empty() const { return !sorting_key_changed && !skip_indices_changed && !constraints_changed; }
|
||||
};
|
||||
|
||||
Diff checkAndFindDiff(const ReplicatedMergeTreeTableMetadata & from_zk, bool allow_alter) const;
|
||||
|
@ -701,8 +701,9 @@ void StorageBuffer::alter(const AlterCommands & params, const String & database_
|
||||
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
params.apply(new_columns);
|
||||
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, {});
|
||||
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, new_constraints, {});
|
||||
setColumns(std::move(new_columns));
|
||||
}
|
||||
|
||||
|
@ -346,8 +346,9 @@ void StorageDistributed::alter(
|
||||
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
params.apply(new_columns);
|
||||
context.getDatabase(database_name)->alterTable(context, current_table_name, new_columns, new_indices, {});
|
||||
context.getDatabase(database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, {});
|
||||
setColumns(std::move(new_columns));
|
||||
}
|
||||
|
||||
|
@ -417,8 +417,9 @@ void StorageMerge::alter(
|
||||
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
params.apply(new_columns);
|
||||
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, {});
|
||||
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, new_constraints, {});
|
||||
setColumns(new_columns);
|
||||
}
|
||||
|
||||
|
@ -242,8 +242,9 @@ void StorageMergeTree::alter(
|
||||
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
params.apply(new_columns);
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, {});
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, {});
|
||||
setColumns(std::move(new_columns));
|
||||
return;
|
||||
}
|
||||
@ -252,15 +253,14 @@ void StorageMergeTree::alter(
|
||||
auto merge_blocker = merger_mutator.actions_blocker.cancel();
|
||||
|
||||
lockNewDataStructureExclusively(table_lock_holder, context.getCurrentQueryId());
|
||||
|
||||
checkAlter(params, context);
|
||||
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
ASTPtr new_order_by_ast = order_by_ast;
|
||||
ASTPtr new_primary_key_ast = primary_key_ast;
|
||||
ASTPtr new_ttl_table_ast = ttl_table_ast;
|
||||
params.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
|
||||
params.apply(new_columns, new_indices, new_constraints, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
|
||||
|
||||
auto transactions = prepareAlterTransactions(new_columns, new_indices, context);
|
||||
|
||||
@ -279,11 +279,10 @@ void StorageMergeTree::alter(
|
||||
if (new_ttl_table_ast.get() != ttl_table_ast.get())
|
||||
storage_ast.set(storage_ast.ttl_table, new_ttl_table_ast);
|
||||
};
|
||||
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, storage_modifier);
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, storage_modifier);
|
||||
|
||||
/// Reinitialize primary key because primary key column types might have changed.
|
||||
setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast, new_columns, new_indices);
|
||||
setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast, new_columns, new_indices, new_constraints);
|
||||
|
||||
setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast);
|
||||
|
||||
@ -834,10 +833,11 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi
|
||||
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
ASTPtr ignored_order_by_ast;
|
||||
ASTPtr ignored_primary_key_ast;
|
||||
ASTPtr ignored_ttl_table_ast;
|
||||
alter_command.apply(new_columns, new_indices, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast);
|
||||
alter_command.apply(new_columns, new_indices, new_constraints, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast);
|
||||
|
||||
auto columns_for_parts = new_columns.getAllPhysical();
|
||||
for (const auto & part : parts)
|
||||
|
@ -38,8 +38,9 @@ void StorageNull::alter(
|
||||
|
||||
ColumnsDescription new_columns = getColumns();
|
||||
IndicesDescription new_indices = getIndices();
|
||||
ConstraintsDescription new_constraints = getConstraints();
|
||||
params.apply(new_columns);
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, {});
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, {});
|
||||
setColumns(std::move(new_columns));
|
||||
}
|
||||
|
||||
|
@ -422,6 +422,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
|
||||
ASTPtr new_primary_key_ast = primary_key_ast;
|
||||
ASTPtr new_order_by_ast = order_by_ast;
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
ASTPtr new_ttl_table_ast = ttl_table_ast;
|
||||
IDatabase::ASTModifier storage_modifier;
|
||||
if (!metadata_diff.empty())
|
||||
@ -451,6 +452,9 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
|
||||
if (metadata_diff.skip_indices_changed)
|
||||
new_indices = IndicesDescription::parse(metadata_diff.new_skip_indices);
|
||||
|
||||
if (metadata_diff.constraints_changed)
|
||||
new_constraints = ConstraintsDescription::parse(metadata_diff.new_constraints);
|
||||
|
||||
if (metadata_diff.ttl_table_changed)
|
||||
{
|
||||
ParserExpression parser;
|
||||
@ -476,11 +480,11 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
|
||||
};
|
||||
}
|
||||
|
||||
global_context.getDatabase(database_name)->alterTable(global_context, table_name, new_columns, new_indices, storage_modifier);
|
||||
global_context.getDatabase(database_name)->alterTable(global_context, table_name, new_columns, new_indices, new_constraints, storage_modifier);
|
||||
|
||||
/// Even if the primary/sorting keys didn't change we must reinitialize it
|
||||
/// because primary key column types might have changed.
|
||||
setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast, new_columns, new_indices);
|
||||
setPrimaryKeyIndicesAndColumns(new_order_by_ast, new_primary_key_ast, new_columns, new_indices, new_constraints);
|
||||
setTTLExpressions(new_columns.getColumnTTLs(), new_ttl_table_ast);
|
||||
}
|
||||
|
||||
@ -1507,10 +1511,11 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry &
|
||||
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
ASTPtr ignored_order_by_ast;
|
||||
ASTPtr ignored_primary_key_ast;
|
||||
ASTPtr ignored_ttl_table_ast;
|
||||
alter_command.apply(new_columns, new_indices, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast);
|
||||
alter_command.apply(new_columns, new_indices, new_constraints, ignored_order_by_ast, ignored_primary_key_ast, ignored_ttl_table_ast);
|
||||
|
||||
size_t modified_parts = 0;
|
||||
auto parts = getDataParts();
|
||||
@ -3114,10 +3119,11 @@ void StorageReplicatedMergeTree::alter(
|
||||
|
||||
ColumnsDescription new_columns = getColumns();
|
||||
IndicesDescription new_indices = getIndices();
|
||||
ConstraintsDescription new_constraints = getConstraints();
|
||||
ASTPtr new_order_by_ast = order_by_ast;
|
||||
ASTPtr new_primary_key_ast = primary_key_ast;
|
||||
ASTPtr new_ttl_table_ast = ttl_table_ast;
|
||||
params.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
|
||||
params.apply(new_columns, new_indices, new_constraints, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast);
|
||||
|
||||
String new_columns_str = new_columns.toString();
|
||||
if (new_columns_str != getColumns().toString())
|
||||
|
@ -0,0 +1,4 @@
|
||||
1 2
|
||||
ok
|
||||
1 2
|
||||
ok
|
40
dbms/tests/queries/0_stateless/00953_constraints_operations.sh
Executable file
40
dbms/tests/queries/0_stateless/00953_constraints_operations.sh
Executable file
@ -0,0 +1,40 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
EXCEPTION_SUCCESS_TEXT=ok
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_constraints;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE test_constraints
|
||||
(
|
||||
a UInt32,
|
||||
b UInt32,
|
||||
CONSTRAINT b_constraint CHECK b > 0
|
||||
)
|
||||
ENGINE = MergeTree ORDER BY (a);"
|
||||
|
||||
# This one must succeed
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 2);"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;"
|
||||
|
||||
# This one must throw and exception
|
||||
EXCEPTION_TEXT="Some constraints are not satisfied"
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 0);" 2>&1 \
|
||||
| grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT * FROM test_constraints;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE test_constraints DROP CONSTRAINT b_constraint;"
|
||||
|
||||
# This one must suceed now
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 0);"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE test_constraints ADD CONSTRAINT b_constraint CHECK b > 10;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 10);" 2>&1 \
|
||||
| grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not thrown an exception"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="INSERT INTO test_constraints VALUES (1, 11);"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE test_constraints;"
|
Loading…
Reference in New Issue
Block a user