未验证 提交 5b72cd23 编写于 作者: A alesapin 提交者: GitHub

Merge pull request #21183 from ClickHouse/fix_mutation_for_storages

Restrict mutations for engines which doesn't support them
......@@ -104,6 +104,7 @@ BlockIO InterpreterAlterQuery::execute()
if (!mutation_commands.empty())
{
table->checkMutationIsPossible(mutation_commands, context.getSettingsRef());
MutationsInterpreter(table, metadata_snapshot, mutation_commands, context, false).validate();
table->mutate(mutation_commands, context);
}
......
......@@ -145,6 +145,11 @@ void IStorage::checkAlterIsPossible(const AlterCommands & commands, const Settin
}
}
void IStorage::checkMutationIsPossible(const MutationCommands & /*commands*/, const Settings & /*settings*/) const
{
throw Exception("Table engine " + getName() + " doesn't support mutations", ErrorCodes::NOT_IMPLEMENTED);
}
void IStorage::checkAlterPartitionIsPossible(
const PartitionCommands & /*commands*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & /*settings*/) const
{
......
......@@ -364,6 +364,11 @@ public:
*/
virtual void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const;
/**
* Checks that mutation commands can be applied to storage.
*/
virtual void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const;
/** ALTER tables with regard to its partitions.
* Should handle locks for each command on its own.
*/
......
......@@ -1670,6 +1670,12 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
}
}
void MergeTreeData::checkMutationIsPossible(const MutationCommands & /*commands*/, const Settings & /*settings*/) const
{
/// Some validation will be added
}
MergeTreeDataPartType MergeTreeData::choosePartType(size_t bytes_uncompressed, size_t rows_count) const
{
const auto settings = getSettings();
......
......@@ -519,6 +519,10 @@ public:
/// If something is wrong, throws an exception.
void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override;
/// Checks if the Mutation can be performed.
/// (currently no additional checks: always ok)
void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override;
/// Checks that partition name in all commands is valid
void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override;
......
......@@ -320,6 +320,12 @@ void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & command
}
}
void StorageMaterializedView::checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const
{
checkStatementCanBeForwarded();
getTargetTable()->checkMutationIsPossible(commands, settings);
}
Pipe StorageMaterializedView::alterPartition(
const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context)
{
......
......@@ -52,6 +52,8 @@ public:
void alter(const AlterCommands & params, const Context & context, TableLockHolder & table_lock_holder) override;
void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override;
void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override;
Pipe alterPartition(const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) override;
......
......@@ -253,6 +253,11 @@ static inline void updateBlockData(Block & old_block, const Block & new_block)
}
}
void StorageMemory::checkMutationIsPossible(const MutationCommands & /*commands*/, const Settings & /*settings*/) const
{
/// Some validation will be added
}
void StorageMemory::mutate(const MutationCommands & commands, const Context & context)
{
std::lock_guard lock(mutex);
......
......@@ -51,6 +51,7 @@ public:
void drop() override;
void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override;
void mutate(const MutationCommands & commands, const Context & context) override;
void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override;
......
......@@ -105,6 +105,9 @@ void StorageView::read(
static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_query)
{
if (!select_query.tables() || select_query.tables()->children.empty())
throw Exception("Logical error: no table expression in view select AST", ErrorCodes::LOGICAL_ERROR);
auto * select_element = select_query.tables()->children[0]->as<ASTTablesInSelectQueryElement>();
if (!select_element->table_expression)
......
DROP VIEW IF EXISTS test_view;
DROP TABLE IF EXISTS test_table;
CREATE TABLE test_table
(
f1 Int32,
f2 Int32,
pk Int32
)
ENGINE = MergeTree()
ORDER BY f1
PARTITION BY pk;
CREATE VIEW test_view AS
SELECT f1, f2
FROM test_table
WHERE pk = 2;
INSERT INTO test_table (f1, f2, pk) VALUES (1,1,1), (1,1,2), (2,1,1), (2,1,2);
SELECT * FROM test_view ORDER BY f1, f2;
ALTER TABLE test_view DELETE WHERE pk = 2; --{serverError 48}
SELECT * FROM test_view ORDER BY f1, f2;
DROP VIEW IF EXISTS test_view;
DROP TABLE IF EXISTS test_table;
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册