Skip to content

Commit

Permalink
Backport #56948 to 23.3: Prevent incompatible ALTER of projection col…
Browse files Browse the repository at this point in the history
…umns
  • Loading branch information
robot-clickhouse committed Dec 4, 2023
1 parent 9f49693 commit 5b27800
Show file tree
Hide file tree
Showing 7 changed files with 128 additions and 9 deletions.
15 changes: 14 additions & 1 deletion src/Storages/AlterCommands.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,9 @@
#include <Interpreters/TreeRewriter.h>
#include <Interpreters/RenameColumnVisitor.h>
#include <Interpreters/GinFilter.h>
#include <Interpreters/inplaceBlockConversions.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTColumnDeclaration.h>
#include <Parsers/ASTConstraintDeclaration.h>
Expand Down Expand Up @@ -42,6 +45,8 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
extern const int DUPLICATE_COLUMN;
extern const int NOT_IMPLEMENTED;
extern const int SUPPORT_IS_DISABLED;
extern const int ALTER_OF_COLUMN_IS_FORBIDDEN;
}

namespace
Expand Down Expand Up @@ -962,7 +967,15 @@ void AlterCommands::apply(StorageInMemoryMetadata & metadata, ContextPtr context
{
try
{
new_projections.add(ProjectionDescription::getProjectionFromAST(projection.definition_ast, metadata_copy.columns, context));
/// Check if we can still build projection from new metadata.
auto new_projection = ProjectionDescription::getProjectionFromAST(projection.definition_ast, metadata_copy.columns, context);
/// Check if new metadata has the same keys as the old one.
if (!blocksHaveEqualStructure(projection.sample_block_for_keys, new_projection.sample_block_for_keys))
throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, "Cannot ALTER column");
/// Check if new metadata is convertible from old metadata for projection.
Block old_projection_block = projection.sample_block;
performRequiredConversions(old_projection_block, new_projection.sample_block.getNamesAndTypesList(), context);
new_projections.add(std::move(new_projection));
}
catch (Exception & exception)
{
Expand Down
48 changes: 42 additions & 6 deletions src/Storages/MergeTree/MergeTreeData.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -2931,12 +2931,6 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
columns_alter_type_check_safe_for_partition.insert(col);
}

for (const auto & index : old_metadata.getSecondaryIndices())
{
for (const String & col : index.expression->getRequiredColumns())
columns_alter_type_forbidden.insert(col);
}

if (old_metadata.hasSortingKey())
{
auto sorting_key_expr = old_metadata.getSortingKey().expression;
Expand All @@ -2960,6 +2954,20 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
columns_in_keys.insert(columns_alter_type_metadata_only.begin(), columns_alter_type_metadata_only.end());
columns_in_keys.insert(columns_alter_type_check_safe_for_partition.begin(), columns_alter_type_check_safe_for_partition.end());

std::unordered_map<String, String> columns_in_indices;
for (const auto & index : old_metadata.getSecondaryIndices())
{
for (const String & col : index.expression->getRequiredColumns())
columns_in_indices.emplace(col, index.name);
}

std::unordered_map<String, String> columns_in_projections;
for (const auto & projection : old_metadata.getProjections())
{
for (const String & col : projection.getRequiredColumns())
columns_in_projections.emplace(col, projection.name);
}

NameSet dropped_columns;

std::map<String, const IDataType *> old_types;
Expand Down Expand Up @@ -3038,6 +3046,17 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
"Trying to ALTER RENAME key {} column which is a part of key expression",
backQuoteIfNeed(command.column_name));
}

/// Don't check columns in indices here. RENAME works fine with index columns.

if (auto it = columns_in_projections.find(command.column_name); it != columns_in_projections.end())
{
throw Exception(
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
"Trying to ALTER RENAME {} column which is a part of projection {}",
backQuoteIfNeed(command.column_name),
it->second);
}
}
else if (command.type == AlterCommand::DROP_COLUMN)
{
Expand All @@ -3047,6 +3066,11 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
"Trying to ALTER DROP key {} column which is a part of key expression", backQuoteIfNeed(command.column_name));
}

/// Don't check columns in indices or projections here. If required columns of indices
/// or projections get dropped, it will be checked later in AlterCommands::apply. This
/// allows projections with * to drop columns. One example can be found in
/// 02691_drop_column_with_projections_replicated.sql.

if (!command.clear)
{
const auto & deps_mv = name_deps[command.column_name];
Expand Down Expand Up @@ -3088,6 +3112,18 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, "ALTER of key column {} is forbidden",
backQuoteIfNeed(command.column_name));

if (auto it = columns_in_indices.find(command.column_name); it != columns_in_indices.end())
{
throw Exception(
ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN,
"Trying to ALTER {} column which is a part of index {}",
backQuoteIfNeed(command.column_name),
it->second);
}

/// Don't check columns in projections here. If required columns of projections get
/// modified, it will be checked later in AlterCommands::apply.

if (command.type == AlterCommand::MODIFY_COLUMN)
{
if (columns_alter_type_check_safe_for_partition.contains(command.column_name))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,4 @@ ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key2 TO renamed_key2

ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN key3 TO renamed_key3; --{serverError 524}

ALTER TABLE table_for_rename_with_primary_key RENAME COLUMN value1 TO renamed_value1; --{serverError 524}

DROP TABLE IF EXISTS table_for_rename_with_primary_key;
Original file line number Diff line number Diff line change
@@ -0,0 +1,8 @@
1
1
1
1
1
1
1
1
33 changes: 33 additions & 0 deletions tests/queries/0_stateless/02920_alter_column_of_projections.sql
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
DROP TABLE IF EXISTS t;

CREATE TABLE t (uid Int16, name String, age Nullable(Int8), i Int16, j Int16, projection p1 (select name, age, uniq(i), count(j) group by name, age)) ENGINE=MergeTree order by uid settings index_granularity = 1;

INSERT INTO t VALUES (1231, 'John', 11, 1, 1), (6666, 'Ksenia', 1, 2, 2), (8888, 'Alice', 1, 3, 3), (6667, 'Ksenia', null, 4, 4);

-- Cannot ALTER, which breaks key column of projection.
ALTER TABLE t MODIFY COLUMN age Nullable(Int32); -- { serverError ALTER_OF_COLUMN_IS_FORBIDDEN }

-- Cannot ALTER, uniq(Int16) is not compatible with uniq(Int32).
ALTER TABLE t MODIFY COLUMN i Int32; -- { serverError CANNOT_CONVERT_TYPE }

SYSTEM STOP MERGES t;

SET alter_sync = 0;

-- Can ALTER, count(Int16) is compatible with count(Int32).
ALTER TABLE t MODIFY COLUMN j Int32;

-- Projection query works without mutation applied.
SELECT count(j) FROM t GROUP BY name, age;

SYSTEM START MERGES t;

SET alter_sync = 1;

-- Another ALTER to wait for.
ALTER TABLE t MODIFY COLUMN j Int64 SETTINGS mutations_sync = 2;

-- Projection query works with mutation applied.
SELECT count(j) FROM t GROUP BY name, age;

DROP TABLE t;
Empty file.
31 changes: 31 additions & 0 deletions tests/queries/0_stateless/02920_rename_column_of_skip_indices.sql
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
DROP TABLE IF EXISTS t;

CREATE TABLE t
(
key1 UInt64,
value1 String,
value2 String,
INDEX idx (value1) TYPE set(10) GRANULARITY 1
)
ENGINE MergeTree ORDER BY key1 SETTINGS index_granularity = 1;

INSERT INTO t SELECT toDate('2019-10-01') + number % 3, toString(number), toString(number) from numbers(9);

SYSTEM STOP MERGES t;

SET alter_sync = 0;

ALTER TABLE t RENAME COLUMN value1 TO value11;

-- Index works without mutation applied.
SELECT * FROM t WHERE value11 = '000' SETTINGS max_rows_to_read = 0;

SYSTEM START MERGES t;

-- Another ALTER to wait for.
ALTER TABLE t RENAME COLUMN value11 TO value12 SETTINGS mutations_sync = 2;

-- Index works with mutation applied.
SELECT * FROM t WHERE value12 = '000' SETTINGS max_rows_to_read = 0;

DROP TABLE t;

0 comments on commit 5b27800

Please sign in to comment.