mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Restrict mutations for engines which doesn't support them
This commit is contained in:
parent
36c178c023
commit
5c6c318737
@ -104,6 +104,7 @@ BlockIO InterpreterAlterQuery::execute()
|
|||||||
|
|
||||||
if (!mutation_commands.empty())
|
if (!mutation_commands.empty())
|
||||||
{
|
{
|
||||||
|
table->checkMutationIsPossible(mutation_commands, context.getSettingsRef());
|
||||||
MutationsInterpreter(table, metadata_snapshot, mutation_commands, context, false).validate();
|
MutationsInterpreter(table, metadata_snapshot, mutation_commands, context, false).validate();
|
||||||
table->mutate(mutation_commands, context);
|
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(
|
void IStorage::checkAlterPartitionIsPossible(
|
||||||
const PartitionCommands & /*commands*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & /*settings*/) const
|
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;
|
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.
|
/** ALTER tables with regard to its partitions.
|
||||||
* Should handle locks for each command on its own.
|
* 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
|
MergeTreeDataPartType MergeTreeData::choosePartType(size_t bytes_uncompressed, size_t rows_count) const
|
||||||
{
|
{
|
||||||
const auto settings = getSettings();
|
const auto settings = getSettings();
|
||||||
|
@ -519,6 +519,10 @@ public:
|
|||||||
/// If something is wrong, throws an exception.
|
/// If something is wrong, throws an exception.
|
||||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override;
|
void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override;
|
||||||
|
|
||||||
|
/// Checks if the Mutation can be performed.
|
||||||
|
/// (currenly no additional checks: always ok)
|
||||||
|
void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override;
|
||||||
|
|
||||||
/// Checks that partition name in all commands is valid
|
/// Checks that partition name in all commands is valid
|
||||||
void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override;
|
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(
|
Pipe StorageMaterializedView::alterPartition(
|
||||||
const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context)
|
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 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;
|
void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override;
|
||||||
|
|
||||||
Pipe alterPartition(const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) 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)
|
void StorageMemory::mutate(const MutationCommands & commands, const Context & context)
|
||||||
{
|
{
|
||||||
std::lock_guard lock(mutex);
|
std::lock_guard lock(mutex);
|
||||||
|
@ -51,6 +51,7 @@ public:
|
|||||||
|
|
||||||
void drop() override;
|
void drop() override;
|
||||||
|
|
||||||
|
void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override;
|
||||||
void mutate(const MutationCommands & commands, const Context & context) override;
|
void mutate(const MutationCommands & commands, const Context & context) override;
|
||||||
|
|
||||||
void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override;
|
void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override;
|
||||||
|
@ -0,0 +1,4 @@
|
|||||||
|
1 1
|
||||||
|
2 1
|
||||||
|
1 1
|
||||||
|
2 1
|
28
tests/queries/0_stateless/01745_alter_delete_view.sql
Normal file
28
tests/queries/0_stateless/01745_alter_delete_view.sql
Normal file
@ -0,0 +1,28 @@
|
|||||||
|
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;
|
Loading…
Reference in New Issue
Block a user