Merge pull request #58068 from zhongyuankai/limit_materialized_view_size

A limit on the number of materialized views attached to a table.
This commit is contained in:
Nikita Taranov 2024-01-22 22:50:56 +01:00 committed by GitHub
commit 09e24ed6c5
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
7 changed files with 73 additions and 0 deletions

View File

@ -2866,3 +2866,10 @@ This also allows a mix of resolver types can be used.
### disable_tunneling_for_https_requests_over_http_proxy {#disable_tunneling_for_https_requests_over_http_proxy}
By default, tunneling (i.e, `HTTP CONNECT`) is used to make `HTTPS` requests over `HTTP` proxy. This setting can be used to disable it.
## max_materialized_views_count_for_table {#max_materialized_views_count_for_table}
A limit on the number of materialized views attached to a table.
Note that only directly dependent views are considered here, and the creation of one view on top of another view is not considered.
Default value: `0`.

View File

@ -591,6 +591,7 @@
M(709, CANNOT_GET_REPLICATED_DATABASE_SNAPSHOT) \
M(710, FAULT_INJECTED) \
M(711, FILECACHE_ACCESS_DENIED) \
M(712, TOO_MANY_MATERIALIZED_VIEWS) \
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \

View File

@ -113,6 +113,7 @@ namespace DB
M(UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \
M(Bool, validate_tcp_client_information, false, "Validate client_information in the query packet over the native TCP protocol.", 0) \
M(Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0) \
M(UInt64, max_materialized_views_count_for_table, 0, "A limit on the number of materialized views attached to a table.", 0) \
/// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp

View File

@ -21,6 +21,7 @@
#include <Common/typeid_cast.h>
#include <Common/checkStackSize.h>
#include <Core/ServerSettings.h>
#include <QueryPipeline/Pipe.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/ExpressionStep.h>
@ -38,6 +39,7 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
extern const int NOT_IMPLEMENTED;
extern const int INCORRECT_QUERY;
extern const int TOO_MANY_MATERIALIZED_VIEWS;
}
namespace ActionLocks
@ -87,6 +89,16 @@ StorageMaterializedView::StorageMaterializedView(
"either ENGINE or an existing table in a TO clause");
auto select = SelectQueryDescription::getSelectQueryFromASTForMatView(query.select->clone(), query.refresh_strategy != nullptr, local_context);
if (select.select_table_id)
{
auto select_table_dependent_views = DatabaseCatalog::instance().getDependentViews(select.select_table_id);
auto max_materialized_views_count_for_table = getContext()->getServerSettings().max_materialized_views_count_for_table;
if (max_materialized_views_count_for_table && select_table_dependent_views.size() >= max_materialized_views_count_for_table)
throw Exception(ErrorCodes::TOO_MANY_MATERIALIZED_VIEWS,
"Too many materialized views, maximum: {}", max_materialized_views_count_for_table);
}
storage_metadata.setSelectQuery(select);
if (!comment.empty())
storage_metadata.setComment(comment);

View File

@ -0,0 +1,3 @@
<clickhouse>
<max_materialized_views_count_for_table>1</max_materialized_views_count_for_table>
</clickhouse>

View File

@ -0,0 +1,49 @@
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance(
"node",
main_configs=["configs/max_num_limit.xml"],
stay_alive=True,
)
config = """<clickhouse>
<max_materialized_views_count_for_table>2</max_materialized_views_count_for_table>
</clickhouse>
"""
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
for _, node in cluster.instances.items():
node.query(
f"""
CREATE TABLE test_tb (a String) ENGINE = MergeTree ORDER BY a;
"""
)
yield cluster
finally:
cluster.shutdown()
def test_limit_materialized_view_count(started_cluster):
node.query(
"CREATE MATERIALIZED VIEW test_view1 ENGINE = MergeTree ORDER BY a AS SELECT * FROM test_tb;"
)
assert "Too many materialized views" in node.query_and_get_error(
"CREATE MATERIALIZED VIEW test_view2 ENGINE = MergeTree ORDER BY a AS SELECT * FROM test_tb;"
)
node.replace_config("/etc/clickhouse-server/config.d/max_num_limit.xml", config)
node.restart_clickhouse()
node.query(
"CREATE MATERIALIZED VIEW test_view2 ENGINE = MergeTree ORDER BY a AS SELECT * FROM test_tb;"
)
assert "Too many materialized views" in node.query_and_get_error(
"CREATE MATERIALIZED VIEW test_view3 ENGINE = MergeTree ORDER BY a AS SELECT * FROM test_tb;"
)