mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-01 03:52:15 +00:00
Fix Alter ttl modification unsupported table engine
This commit is contained in:
parent
0e37e8b4a0
commit
a6254516e0
@ -170,7 +170,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
|
||||
{
|
||||
auto alter_lock = table->lockForAlter(getContext()->getSettingsRef().lock_acquire_timeout);
|
||||
StorageInMemoryMetadata metadata = table->getInMemoryMetadata();
|
||||
alter_commands.validate(metadata, getContext());
|
||||
alter_commands.validate(table, getContext());
|
||||
alter_commands.prepare(metadata);
|
||||
table->checkAlterIsPossible(alter_commands, getContext());
|
||||
table->alter(alter_commands, getContext(), alter_lock);
|
||||
|
@ -998,8 +998,9 @@ void AlterCommands::prepare(const StorageInMemoryMetadata & metadata)
|
||||
}
|
||||
|
||||
|
||||
void AlterCommands::validate(const StorageInMemoryMetadata & metadata, ContextPtr context) const
|
||||
void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
|
||||
{
|
||||
const StorageInMemoryMetadata & metadata = table->getInMemoryMetadata();
|
||||
auto all_columns = metadata.columns;
|
||||
/// Default expression for all added/modified columns
|
||||
ASTPtr default_expr_list = std::make_shared<ASTExpressionList>();
|
||||
@ -1008,6 +1009,9 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, ContextPt
|
||||
{
|
||||
const auto & command = (*this)[i];
|
||||
|
||||
if (command.ttl && !table->supportsTTL())
|
||||
throw Exception("Engine " + table->getName() + " doesn't support TTL clause", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
const auto & column_name = command.column_name;
|
||||
if (command.type == AlterCommand::ADD_COLUMN)
|
||||
{
|
||||
|
@ -186,7 +186,7 @@ public:
|
||||
/// Checks that all columns exist and dependencies between them.
|
||||
/// This check is lightweight and base only on metadata.
|
||||
/// More accurate check have to be performed with storage->checkAlterIsPossible.
|
||||
void validate(const StorageInMemoryMetadata & metadata, ContextPtr context) const;
|
||||
void validate(const StoragePtr & table, ContextPtr context) const;
|
||||
|
||||
/// Prepare alter commands. Set ignore flag to some of them and set some
|
||||
/// parts to commands from storage's metadata (for example, absent default)
|
||||
|
@ -132,6 +132,9 @@ public:
|
||||
/// Returns true if the storage supports insert queries with the PARTITION BY section.
|
||||
virtual bool supportsPartitionBy() const { return false; }
|
||||
|
||||
/// Returns true if the storage supports queries with the TTL section.
|
||||
virtual bool supportsTTL() const { return false; }
|
||||
|
||||
/// Returns true if the storage supports queries with the PREWHERE section.
|
||||
virtual bool supportsPrewhere() const { return false; }
|
||||
|
||||
|
@ -414,6 +414,8 @@ public:
|
||||
|
||||
bool supportsSubcolumns() const override { return true; }
|
||||
|
||||
bool supportsTTL() const override { return true; }
|
||||
|
||||
NamesAndTypesList getVirtuals() const override;
|
||||
|
||||
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, ContextPtr, const StorageMetadataPtr & metadata_snapshot) const override;
|
||||
|
@ -59,6 +59,7 @@ public:
|
||||
bool supports_skipping_indices = false;
|
||||
bool supports_projections = false;
|
||||
bool supports_sort_order = false;
|
||||
/// See also IStorage::supportsTTL()
|
||||
bool supports_ttl = false;
|
||||
/// See also IStorage::supportsReplication()
|
||||
bool supports_replication = false;
|
||||
|
59
tests/queries/0_stateless/02184_storage_add_support_ttl.sql
Normal file
59
tests/queries/0_stateless/02184_storage_add_support_ttl.sql
Normal file
@ -0,0 +1,59 @@
|
||||
DROP TABLE IF EXISTS mergeTree_02184;
|
||||
CREATE TABLE mergeTree_02184 (id UInt64, name String, dt Date) Engine=MergeTree ORDER BY id;
|
||||
ALTER TABLE mergeTree_02184 MODIFY COLUMN name String TTL dt + INTERVAL 1 MONTH;
|
||||
DETACH TABLE mergeTree_02184;
|
||||
ATTACH TABLE mergeTree_02184;
|
||||
|
||||
DROP TABLE IF EXISTS distributed_02184;
|
||||
CREATE TABLE distributed_02184 (id UInt64, name String, dt Date) Engine=Distributed('test_cluster_two_shards', 'default', 'mergeTree_02184', rand());
|
||||
ALTER TABLE distributed_02184 MODIFY COLUMN name String TTL dt + INTERVAL 1 MONTH; -- { serverError BAD_ARGUMENTS }
|
||||
DETACH TABLE distributed_02184;
|
||||
ATTACH TABLE distributed_02184;
|
||||
|
||||
DROP TABLE IF EXISTS buffer_02184;
|
||||
CREATE TABLE buffer_02184 (id UInt64, name String, dt Date) ENGINE = Buffer(default, mergeTree_02184, 16, 10, 100, 10000, 1000000, 10000000, 100000000);
|
||||
ALTER TABLE buffer_02184 MODIFY COLUMN name String TTL dt + INTERVAL 1 MONTH; -- { serverError BAD_ARGUMENTS }
|
||||
DETACH TABLE buffer_02184;
|
||||
ATTACH TABLE buffer_02184;
|
||||
|
||||
DROP TABLE IF EXISTS merge_02184;
|
||||
CREATE TABLE merge_02184 (id UInt64, name String, dt Date) ENGINE = Merge('default', 'distributed_02184');
|
||||
ALTER TABLE merge_02184 MODIFY COLUMN name String TTL dt + INTERVAL 1 MONTH; -- { serverError BAD_ARGUMENTS }
|
||||
DETACH TABLE merge_02184;
|
||||
ATTACH TABLE merge_02184;
|
||||
|
||||
DROP TABLE IF EXISTS null_02184;
|
||||
CREATE TABLE null_02184 AS system.one Engine=Null();
|
||||
ALTER TABLE null_02184 MODIFY COLUMN dummy Int TTL now() + INTERVAL 1 MONTH; -- { serverError BAD_ARGUMENTS }
|
||||
DETACH TABLE null_02184;
|
||||
ATTACH TABLE null_02184;
|
||||
|
||||
DROP TABLE IF EXISTS file_02184;
|
||||
CREATE TABLE file_02184 (id UInt64, name String, dt Date) ENGINE = File(TabSeparated);
|
||||
ALTER TABLE file_02184 MODIFY COLUMN name String TTL dt + INTERVAL 1 MONTH; -- { serverError BAD_ARGUMENTS }
|
||||
DETACH TABLE file_02184;
|
||||
ATTACH TABLE file_02184;
|
||||
|
||||
DROP TABLE IF EXISTS memory_02184;
|
||||
CREATE TABLE memory_02184 (id UInt64, name String, dt Date) ENGINE = Memory();
|
||||
ALTER TABLE memory_02184 MODIFY COLUMN name String TTL dt + INTERVAL 1 MONTH; -- { serverError BAD_ARGUMENTS }
|
||||
DETACH TABLE memory_02184;
|
||||
ATTACH TABLE memory_02184;
|
||||
|
||||
DROP TABLE IF EXISTS log_02184;
|
||||
CREATE TABLE log_02184 (id UInt64, name String, dt Date) ENGINE = Log();
|
||||
ALTER TABLE log_02184 MODIFY COLUMN name String TTL dt + INTERVAL 1 MONTH; -- { serverError BAD_ARGUMENTS }
|
||||
DETACH TABLE log_02184;
|
||||
ATTACH TABLE log_02184;
|
||||
|
||||
DROP TABLE IF EXISTS ting_log_02184;
|
||||
CREATE TABLE ting_log_02184 (id UInt64, name String, dt Date) ENGINE = TinyLog();
|
||||
ALTER TABLE ting_log_02184 MODIFY COLUMN name String TTL dt + INTERVAL 1 MONTH; -- { serverError BAD_ARGUMENTS }
|
||||
DETACH TABLE ting_log_02184;
|
||||
ATTACH TABLE ting_log_02184;
|
||||
|
||||
DROP TABLE IF EXISTS stripe_log_02184;
|
||||
CREATE TABLE stripe_log_02184 (id UInt64, name String, dt Date) ENGINE = StripeLog;
|
||||
ALTER TABLE stripe_log_02184 MODIFY COLUMN name String TTL dt + INTERVAL 1 MONTH; -- { serverError BAD_ARGUMENTS }
|
||||
DETACH TABLE stripe_log_02184;
|
||||
ATTACH TABLE stripe_log_02184;
|
Loading…
Reference in New Issue
Block a user