mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-17 20:02:05 +00:00
fix ALTER MODIFY COLUMN of columns, that participates in TTL expressions
This commit is contained in:
parent
8ba6a5393f
commit
662d5b8495
@ -388,7 +388,6 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
if (commands.empty())
|
||||
throw Exception("Empty mutation commands list", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
|
||||
const ColumnsDescription & columns_desc = metadata_snapshot->getColumns();
|
||||
const IndicesDescription & indices_desc = metadata_snapshot->getSecondaryIndices();
|
||||
const ProjectionsDescription & projections_desc = metadata_snapshot->getProjections();
|
||||
@ -426,7 +425,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
|
||||
}
|
||||
|
||||
/// Columns, that we need to read for calculation of skip indices, projections or TTL expressions.
|
||||
auto dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns);
|
||||
dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns);
|
||||
|
||||
/// First, break a sequence of commands into stages.
|
||||
for (auto & command : commands)
|
||||
@ -921,6 +920,10 @@ const Block & MutationsInterpreter::getUpdatedHeader() const
|
||||
return *updated_header;
|
||||
}
|
||||
|
||||
const ColumnDependencies & MutationsInterpreter::getColumnDependencies() const
|
||||
{
|
||||
return dependencies;
|
||||
}
|
||||
|
||||
size_t MutationsInterpreter::evaluateCommandsSize()
|
||||
{
|
||||
|
@ -56,6 +56,8 @@ public:
|
||||
/// Only changed columns.
|
||||
const Block & getUpdatedHeader() const;
|
||||
|
||||
const ColumnDependencies & getColumnDependencies() const;
|
||||
|
||||
/// Latest mutation stage affects all columns in storage
|
||||
bool isAffectingAllColumns() const;
|
||||
|
||||
@ -148,6 +150,8 @@ private:
|
||||
NameSet materialized_projections;
|
||||
|
||||
MutationKind mutation_kind; /// Do we meet any index or projection mutation.
|
||||
|
||||
ColumnDependencies dependencies;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1267,7 +1267,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
|
||||
bool need_sync = needSyncPart(source_part->rows_count, source_part->getBytesOnDisk(), *data_settings);
|
||||
bool need_remove_expired_values = false;
|
||||
|
||||
if (in && shouldExecuteTTL(metadata_snapshot, in->getHeader().getNamesAndTypesList().getNames(), commands_for_part))
|
||||
if (in && shouldExecuteTTL(metadata_snapshot, interpreter->getColumnDependencies(), commands_for_part))
|
||||
need_remove_expired_values = true;
|
||||
|
||||
/// All columns from part are changed and may be some more that were missing before in part
|
||||
@ -1956,7 +1956,8 @@ std::set<MergeTreeProjectionPtr> MergeTreeDataMergerMutator::getProjectionsToRec
|
||||
return projections_to_recalc;
|
||||
}
|
||||
|
||||
bool MergeTreeDataMergerMutator::shouldExecuteTTL(const StorageMetadataPtr & metadata_snapshot, const Names & columns, const MutationCommands & commands)
|
||||
bool MergeTreeDataMergerMutator::shouldExecuteTTL(
|
||||
const StorageMetadataPtr & metadata_snapshot, const ColumnDependencies & dependencies, const MutationCommands & commands)
|
||||
{
|
||||
if (!metadata_snapshot->hasAnyTTL())
|
||||
return false;
|
||||
@ -1965,7 +1966,6 @@ bool MergeTreeDataMergerMutator::shouldExecuteTTL(const StorageMetadataPtr & met
|
||||
if (command.type == MutationCommand::MATERIALIZE_TTL)
|
||||
return true;
|
||||
|
||||
auto dependencies = metadata_snapshot->getColumnDependencies(NameSet(columns.begin(), columns.end()));
|
||||
for (const auto & dependency : dependencies)
|
||||
if (dependency.kind == ColumnDependency::TTL_EXPRESSION || dependency.kind == ColumnDependency::TTL_TARGET)
|
||||
return true;
|
||||
|
@ -200,7 +200,8 @@ private:
|
||||
const ProjectionsDescription & all_projections,
|
||||
const MutationCommands & commands_for_removes);
|
||||
|
||||
static bool shouldExecuteTTL(const StorageMetadataPtr & metadata_snapshot, const Names & columns, const MutationCommands & commands);
|
||||
static bool shouldExecuteTTL(
|
||||
const StorageMetadataPtr & metadata_snapshot, const ColumnDependencies & dependencies, const MutationCommands & commands);
|
||||
|
||||
/// Return set of indices which should be recalculated during mutation also
|
||||
/// wraps input stream into additional expression stream
|
||||
|
@ -0,0 +1,2 @@
|
||||
2 ['Int16']
|
||||
2 ['Date']
|
43
tests/queries/0_stateless/01923_ttl_with_modify_column.sql
Normal file
43
tests/queries/0_stateless/01923_ttl_with_modify_column.sql
Normal file
@ -0,0 +1,43 @@
|
||||
DROP TABLE IF EXISTS t_ttl_modify_column;
|
||||
|
||||
CREATE TABLE t_ttl_modify_column
|
||||
(
|
||||
InsertionDateTime DateTime,
|
||||
TTLDays Int32 DEFAULT CAST(365, 'Int32')
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY tuple()
|
||||
TTL InsertionDateTime + toIntervalDay(TTLDays)
|
||||
SETTINGS min_bytes_for_wide_part = 0;
|
||||
|
||||
INSERT INTO t_ttl_modify_column VALUES (now(), 23);
|
||||
|
||||
SET mutations_sync = 2;
|
||||
|
||||
ALTER TABLE t_ttl_modify_column modify column TTLDays Int16 DEFAULT CAST(365, 'Int16');
|
||||
|
||||
INSERT INTO t_ttl_modify_column VALUES (now(), 23);
|
||||
|
||||
SELECT sum(rows), groupUniqArray(type) FROM system.parts_columns
|
||||
WHERE database = currentDatabase() AND table = 't_ttl_modify_column' AND column = 'TTLDays' AND active;
|
||||
|
||||
DROP TABLE IF EXISTS t_ttl_modify_column;
|
||||
|
||||
CREATE TABLE t_ttl_modify_column (InsertionDateTime DateTime)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY tuple()
|
||||
TTL InsertionDateTime + INTERVAL 3 DAY
|
||||
SETTINGS min_bytes_for_wide_part = 0;
|
||||
|
||||
INSERT INTO t_ttl_modify_column VALUES (now());
|
||||
|
||||
ALTER TABLE t_ttl_modify_column MODIFY COLUMN InsertionDateTime Date;
|
||||
|
||||
INSERT INTO t_ttl_modify_column VALUES (now());
|
||||
|
||||
SELECT sum(rows), groupUniqArray(type) FROM system.parts_columns
|
||||
WHERE database = currentDatabase() AND table = 't_ttl_modify_column' AND column = 'InsertionDateTime' AND active;
|
||||
|
||||
ALTER TABLE t_ttl_modify_column MODIFY COLUMN InsertionDateTime String; -- { serverError 43 }
|
||||
|
||||
DROP TABLE IF EXISTS t_ttl_modify_column;
|
Loading…
Reference in New Issue
Block a user