mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 09:02:00 +00:00
explicitly disallow ALTERs and mutations for plain
This commit is contained in:
parent
01ee500b06
commit
89f28f3c18
@ -363,6 +363,9 @@ public:
|
||||
|
||||
virtual bool isWriteOnce() const { return false; }
|
||||
|
||||
/// Whether this disk support mutations.
|
||||
virtual bool isMutable() const { return true; }
|
||||
|
||||
/// Check if disk is broken. Broken disks will have 0 space and cannot be used.
|
||||
virtual bool isBroken() const { return false; }
|
||||
|
||||
|
@ -435,6 +435,11 @@ bool DiskObjectStorage::isWriteOnce() const
|
||||
return object_storage->isWriteOnce();
|
||||
}
|
||||
|
||||
bool DiskObjectStorage::isMutable() const
|
||||
{
|
||||
return !isWriteOnce() && !object_storage->isPlain();
|
||||
}
|
||||
|
||||
DiskObjectStoragePtr DiskObjectStorage::createDiskObjectStorage()
|
||||
{
|
||||
const auto config_prefix = "storage_configuration.disks." + name;
|
||||
|
@ -183,6 +183,8 @@ public:
|
||||
/// MergeTree table on this disk.
|
||||
bool isWriteOnce() const override;
|
||||
|
||||
bool isMutable() const override;
|
||||
|
||||
/// Get structure of object storage this disk works with. Examples:
|
||||
/// DiskObjectStorage(S3ObjectStorage)
|
||||
/// DiskObjectStorage(CachedObjectStorage(S3ObjectStorage))
|
||||
|
@ -2964,6 +2964,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
|
||||
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
|
||||
"Experimental Inverted Index feature is not enabled (turn on setting 'allow_experimental_inverted_index')");
|
||||
|
||||
for (const auto & disk : getDisks())
|
||||
if (!disk->isMutable())
|
||||
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "ALTER TABLE is not supported for immutable disk '{}'", disk->getName());
|
||||
|
||||
/// Set of columns that shouldn't be altered.
|
||||
NameSet columns_alter_type_forbidden;
|
||||
|
||||
@ -3334,7 +3338,9 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
|
||||
|
||||
void MergeTreeData::checkMutationIsPossible(const MutationCommands & /*commands*/, const Settings & /*settings*/) const
|
||||
{
|
||||
/// Some validation will be added
|
||||
for (const auto & disk : getDisks())
|
||||
if (!disk->isMutable())
|
||||
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Mutations are not supported for immutable disk '{}'", disk->getName());
|
||||
}
|
||||
|
||||
MergeTreeDataPartFormat MergeTreeData::choosePartFormat(size_t bytes_uncompressed, size_t rows_count) const
|
||||
@ -4824,6 +4830,11 @@ void MergeTreeData::removePartContributionToColumnAndSecondaryIndexSizes(const D
|
||||
void MergeTreeData::checkAlterPartitionIsPossible(
|
||||
const PartitionCommands & commands, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & settings, ContextPtr local_context) const
|
||||
{
|
||||
for (const auto & disk : getDisks())
|
||||
if (!disk->isMutable())
|
||||
throw Exception(
|
||||
ErrorCodes::SUPPORT_IS_DISABLED, "ALTER TABLE PARTITION is not supported for immutable disk '{}'", disk->getName());
|
||||
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
if (command.type == PartitionCommand::DROP_DETACHED_PARTITION
|
||||
|
@ -1,11 +1,11 @@
|
||||
1000006
|
||||
0 0
|
||||
1 1
|
||||
1 2
|
||||
2 2
|
||||
2 2
|
||||
3 1
|
||||
3 3
|
||||
4 4
|
||||
4 7
|
||||
5 5
|
||||
10006
|
||||
0 0 0
|
||||
1 1 1
|
||||
1 2 0
|
||||
2 2 2
|
||||
2 2 2
|
||||
3 1 9
|
||||
3 3 3
|
||||
4 4 4
|
||||
4 7 7
|
||||
5 5 5
|
||||
|
@ -2,21 +2,37 @@
|
||||
-- Tag: no-fasttest -- requires S3
|
||||
|
||||
drop table if exists test_mt;
|
||||
create table test_mt (a Int32, b Int64) engine = MergeTree() order by a
|
||||
create table test_mt (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 500) order by tuple(a, b)
|
||||
settings disk = disk(
|
||||
type = s3_plain_rewritable,
|
||||
endpoint = 'http://localhost:11111/test/test_mt/',
|
||||
access_key_id = clickhouse,
|
||||
secret_access_key = clickhouse);
|
||||
|
||||
insert into test_mt (*) values (1, 2), (2, 2), (3, 1), (4, 7), (5, 10), (6, 12);
|
||||
insert into test_mt (*) select number, number from numbers_mt(1000000);
|
||||
insert into test_mt (*) values (1, 2, 0), (2, 2, 2), (3, 1, 9), (4, 7, 7), (5, 10, 2), (6, 12, 5);
|
||||
insert into test_mt (*) select number, number, number from numbers_mt(10000);
|
||||
|
||||
select count(*) from test_mt;
|
||||
select (*) from test_mt order by tuple(a, b) limit 10;
|
||||
|
||||
-- File moving is not supported.
|
||||
alter table test_mt update b = 0 where a % 2 = 1; --{ serverError NOT_IMPLEMENTED }
|
||||
optimize table test_mt final;
|
||||
|
||||
alter table test_mt add column c Int64 after b; --{ serverError BAD_GET }
|
||||
alter table test_mt drop column b; --{ serverError BAD_GET }
|
||||
alter table test_mt add projection test_mt_projection (
|
||||
select * order by b); -- { serverError SUPPORT_IS_DISABLED }
|
||||
|
||||
alter table test_mt update c = 0 where a % 2 = 1; -- { serverError SUPPORT_IS_DISABLED }
|
||||
alter table test_mt add column d Int64 after c; -- { serverError SUPPORT_IS_DISABLED }
|
||||
alter table test_mt drop column c; -- { serverError SUPPORT_IS_DISABLED }
|
||||
|
||||
detach table test_mt;
|
||||
attach table test_mt;
|
||||
|
||||
drop table if exists test_mt_dst;
|
||||
|
||||
create table test_mt_dst (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 500) order by tuple(a, b)
|
||||
settings disk = disk(
|
||||
type = s3_plain_rewritable,
|
||||
endpoint = 'http://localhost:11111/test/test_mt/',
|
||||
access_key_id = clickhouse,
|
||||
secret_access_key = clickhouse);
|
||||
alter table test_mt move partition 0 to table test_mt_dst; -- { serverError SUPPORT_IS_DISABLED }
|
||||
|
Loading…
Reference in New Issue
Block a user