Added ALTER TABLE support to constraints (ADD CONSTRAINT, DROP CONSTRAINT)

This commit is contained in:
Gleb Novikov 2019-06-02 17:41:12 +03:00
parent 2b5cf46842
commit e7293486bd
25 changed files with 255 additions and 32 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View 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 "

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -0,0 +1,4 @@
1 2
ok
1 2
ok

View 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;"