Skip to content

Commit

Permalink
Allow max_size_to_drop settings in query time (#57452)
Browse files Browse the repository at this point in the history
  • Loading branch information
jrdi committed Dec 19, 2023
1 parent 0a9475a commit 7c281d9
Show file tree
Hide file tree
Showing 10 changed files with 104 additions and 8 deletions.
22 changes: 22 additions & 0 deletions docs/en/operations/settings/settings.md
Expand Up @@ -5134,3 +5134,25 @@ When set to `true` than for all s3 requests first two attempts are made with low
When set to `false` than all attempts are made with identical timeouts.

Default value: `true`.

## max_partition_size_to_drop

Restriction on dropping partitions in query time.

Default value: 50 GB.
The value 0 means that you can drop partitions without any restrictions.

:::note
This query setting overwrites its server setting equivalent, see [max_partition_size_to_drop](/docs/en/operations/server-configuration-parameters/settings.md/#max-partition-size-to-drop)
:::

## max_table_size_to_drop

Restriction on deleting tables in query time.

Default value: 50 GB.
The value 0 means that you can delete all tables without any restrictions.

:::note
This query setting overwrites its server setting equivalent, see [max_table_size_to_drop](/docs/en/operations/server-configuration-parameters/settings.md/#max-table-size-to-drop)
:::
4 changes: 2 additions & 2 deletions src/Core/Settings.h
Expand Up @@ -528,8 +528,8 @@ class IColumn;
M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited.", 0) \
M(Bool, check_query_single_value_result, true, "Return check query result as single 1/0 value", 0) \
M(Bool, allow_drop_detached, false, "Allow ALTER TABLE ... DROP DETACHED PART[ITION] ... queries", 0) \
M(UInt64, max_table_size_to_drop, 0, "Only available in ClickHouse Cloud", 0) \
M(UInt64, max_partition_size_to_drop, 0, "Only available in ClickHouse Cloud", 0) \
M(UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0) \
M(UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0) \
\
M(UInt64, postgresql_connection_pool_size, 16, "Connection pool size for PostgreSQL table engine and database engine.", 0) \
M(UInt64, postgresql_connection_pool_wait_timeout, 5000, "Connection pool push/pop timeout on empty pool for PostgreSQL table engine and database engine. By default it will block on empty pool.", 0) \
Expand Down
11 changes: 10 additions & 1 deletion src/Interpreters/Context.cpp
Expand Up @@ -4053,7 +4053,8 @@ void Context::checkCanBeDropped(const String & database, const String & table, c
"2. File '{}' intended to force DROP {}\n"
"How to fix this:\n"
"1. Either increase (or set to zero) max_[table/partition]_size_to_drop in server config\n"
"2. Either create forcing file {} and make sure that ClickHouse has write permission for it.\n"
"2. Either pass a bigger (or set to zero) max_[table/partition]_size_to_drop through query settings\n"
"3. Either create forcing file {} and make sure that ClickHouse has write permission for it.\n"
"Example:\nsudo touch '{}' && sudo chmod 666 '{}'",
backQuoteIfNeed(database), backQuoteIfNeed(table),
size_str, max_size_to_drop_str,
Expand Down Expand Up @@ -4081,6 +4082,10 @@ void Context::checkTableCanBeDropped(const String & database, const String & tab
checkCanBeDropped(database, table, table_size, max_table_size_to_drop);
}

void Context::checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size, const size_t & max_table_size_to_drop) const
{
checkCanBeDropped(database, table, table_size, max_table_size_to_drop);
}

void Context::setMaxPartitionSizeToDrop(size_t max_size)
{
Expand All @@ -4100,6 +4105,10 @@ void Context::checkPartitionCanBeDropped(const String & database, const String &
checkCanBeDropped(database, table, partition_size, max_partition_size_to_drop);
}

void Context::checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size, const size_t & max_partition_size_to_drop) const
{
checkCanBeDropped(database, table, partition_size, max_partition_size_to_drop);
}

InputFormatPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional<FormatSettings> & format_settings, const std::optional<size_t> max_parsing_threads) const
{
Expand Down
2 changes: 2 additions & 0 deletions src/Interpreters/Context.h
Expand Up @@ -1084,11 +1084,13 @@ class Context: public ContextData, public std::enable_shared_from_this<Context>
void setMaxTableSizeToDrop(size_t max_size);
size_t getMaxTableSizeToDrop() const;
void checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const;
void checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size, const size_t & max_table_size_to_drop) const;

/// Prevents DROP PARTITION if its size is greater than max_size (50GB by default, max_size=0 turn off this check)
void setMaxPartitionSizeToDrop(size_t max_size);
size_t getMaxPartitionSizeToDrop() const;
void checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const;
void checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size, const size_t & max_partition_size_to_drop) const;

/// Lets you select the compression codec according to the conditions described in the configuration file.
std::shared_ptr<ICompressionCodec> chooseCompressionCodec(size_t part_size, double part_size_ratio) const;
Expand Down
20 changes: 18 additions & 2 deletions src/Storages/MergeTree/MergeTreeData.cpp
Expand Up @@ -4835,10 +4835,18 @@ void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition, Context
partition_size += part->getBytesOnDisk();

auto table_id = getStorageID();

const auto & query_settings = local_context->getSettingsRef();
if (query_settings.max_partition_size_to_drop.changed)
{
getContext()->checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, partition_size, query_settings.max_partition_size_to_drop);
return;
}

getContext()->checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, partition_size);
}

void MergeTreeData::checkPartCanBeDropped(const String & part_name)
void MergeTreeData::checkPartCanBeDropped(const String & part_name, ContextPtr local_context)
{
if (!supportsReplication() && isStaticStorage())
return;
Expand All @@ -4848,6 +4856,14 @@ void MergeTreeData::checkPartCanBeDropped(const String & part_name)
throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in committed state", part_name);

auto table_id = getStorageID();

const auto & query_settings = local_context->getSettingsRef();
if (query_settings.max_partition_size_to_drop.changed)
{
getContext()->checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, part->getBytesOnDisk(), query_settings.max_partition_size_to_drop);
return;
}

getContext()->checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, part->getBytesOnDisk());
}

Expand Down Expand Up @@ -5035,7 +5051,7 @@ Pipe MergeTreeData::alterPartition(
if (command.part)
{
auto part_name = command.partition->as<ASTLiteral &>().value.safeGet<String>();
checkPartCanBeDropped(part_name);
checkPartCanBeDropped(part_name, query_context);
dropPart(part_name, command.detach, query_context);
}
else
Expand Down
2 changes: 1 addition & 1 deletion src/Storages/MergeTree/MergeTreeData.h
Expand Up @@ -795,7 +795,7 @@ class MergeTreeData : public IStorage, public WithMutableContext
/// We do not use mutex because it is not very important that the size could change during the operation.
void checkPartitionCanBeDropped(const ASTPtr & partition, ContextPtr local_context);

void checkPartCanBeDropped(const String & part_name);
void checkPartCanBeDropped(const String & part_name, ContextPtr local_context);

Pipe alterPartition(
const StorageMetadataPtr & metadata_snapshot,
Expand Down
10 changes: 9 additions & 1 deletion src/Storages/StorageMergeTree.cpp
Expand Up @@ -280,12 +280,20 @@ StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & met
*this, metadata_snapshot, settings.max_partitions_per_insert_block, local_context);
}

void StorageMergeTree::checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const
void StorageMergeTree::checkTableCanBeDropped(ContextPtr query_context) const
{
if (!supportsReplication() && isStaticStorage())
return;

auto table_id = getStorageID();

const auto & query_settings = query_context->getSettingsRef();
if (query_settings.max_table_size_to_drop.changed)
{
getContext()->checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes(), query_settings.max_table_size_to_drop);
return;
}

getContext()->checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes());
}

Expand Down
10 changes: 9 additions & 1 deletion src/Storages/StorageReplicatedMergeTree.cpp
Expand Up @@ -6423,9 +6423,17 @@ PartitionCommandsResultInfo StorageReplicatedMergeTree::attachPartition(
}


void StorageReplicatedMergeTree::checkTableCanBeDropped([[ maybe_unused ]] ContextPtr query_context) const
void StorageReplicatedMergeTree::checkTableCanBeDropped(ContextPtr query_context) const
{
auto table_id = getStorageID();

const auto & query_settings = query_context->getSettingsRef();
if (query_settings.max_table_size_to_drop.changed)
{
getContext()->checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes(), query_settings.max_table_size_to_drop);
return;
}

getContext()->checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes());
}

Expand Down
Empty file.
31 changes: 31 additions & 0 deletions tests/queries/0_stateless/02932_query_settings_max_size_drop.sql
@@ -0,0 +1,31 @@
CREATE TABLE test_max_size_drop
Engine = MergeTree()
ORDER BY number
AS SELECT number
FROM numbers(1000)
;

DROP TABLE test_max_size_drop SETTINGS max_table_size_to_drop = 1; -- { serverError 359 }
DROP TABLE test_max_size_drop;

CREATE TABLE test_max_size_drop
Engine = MergeTree()
ORDER BY number
AS SELECT number
FROM numbers(1000)
;

ALTER TABLE test_max_size_drop DROP PARTITION tuple() SETTINGS max_partition_size_to_drop = 1; -- { serverError 359 }
ALTER TABLE test_max_size_drop DROP PARTITION tuple();
DROP TABLE test_max_size_drop;

CREATE TABLE test_max_size_drop
Engine = MergeTree()
ORDER BY number
AS SELECT number
FROM numbers(1000)
;

ALTER TABLE test_max_size_drop DROP PART 'all_1_1_0' SETTINGS max_partition_size_to_drop = 1; -- { serverError 359 }
ALTER TABLE test_max_size_drop DROP PART 'all_1_1_0';
DROP TABLE test_max_size_drop;

0 comments on commit 7c281d9

Please sign in to comment.