未验证 提交 1142c142 编写于 作者: T tavplubix 提交者: GitHub

Merge pull request #7779 from GrigoryPervakov/master

Update max_table_size_to_drop and max_partition_size_to_drop with config reload
......@@ -438,6 +438,13 @@ int Server::main(const std::vector<std::string> & /*args*/)
buildLoggers(*config, logger());
global_context->setClustersConfig(config);
global_context->setMacros(std::make_unique<Macros>(*config, "macros"));
/// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default)
if (config->has("max_table_size_to_drop"))
global_context->setMaxTableSizeToDrop(config->getUInt64("max_table_size_to_drop"));
if (config->has("max_partition_size_to_drop"))
global_context->setMaxPartitionSizeToDrop(config->getUInt64("max_partition_size_to_drop"));
},
/* already_loaded = */ true);
......@@ -469,13 +476,6 @@ int Server::main(const std::vector<std::string> & /*args*/)
/// Limit on total number of concurrently executed queries.
global_context->getProcessList().setMaxSize(config().getInt("max_concurrent_queries", 0));
/// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default)
if (config().has("max_table_size_to_drop"))
global_context->setMaxTableSizeToDrop(config().getUInt64("max_table_size_to_drop"));
if (config().has("max_partition_size_to_drop"))
global_context->setMaxPartitionSizeToDrop(config().getUInt64("max_partition_size_to_drop"));
/// Set up caches.
/// Lower cache size on low-memory systems.
......
......@@ -411,7 +411,7 @@
<!-- Protection from accidental DROP.
If size of a MergeTree table is greater than max_table_size_to_drop (in bytes) than table could not be dropped with any DROP query.
If you want do delete one table and don't want to restart clickhouse-server, you could create special file <clickhouse-path>/flags/force_drop_table and make DROP once.
If you want do delete one table and don't want to change clickhouse-server config, you could create special file <clickhouse-path>/flags/force_drop_table and make DROP once.
By default max_table_size_to_drop is 50GB; max_table_size_to_drop=0 allows to DROP any tables.
The same for max_partition_size_to_drop.
Uncomment to disable protection.
......
......@@ -150,12 +150,12 @@ struct ContextShared
/// Storage policy chooser
mutable std::unique_ptr<DiskSpace::StoragePolicySelector> merge_tree_storage_policy_selector;
std::optional<MergeTreeSettings> merge_tree_settings; /// Settings of MergeTree* engines.
size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default)
size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default)
std::optional<MergeTreeSettings> merge_tree_settings; /// Settings of MergeTree* engines.
std::atomic_size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default)
std::atomic_size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default)
String format_schema_path; /// Path to a directory that contains schema files used by input formats.
ActionLocksManagerPtr action_locks_manager; /// Set of storages' action lockers
std::optional<SystemLogs> system_logs; /// Used to log queries and operations on parts
std::optional<SystemLogs> system_logs; /// Used to log queries and operations on parts
std::unique_ptr<TraceCollector> trace_collector; /// Thread collecting traces from threads executing queries
......@@ -1891,14 +1891,14 @@ void Context::checkCanBeDropped(const String & database, const String & table, c
void Context::setMaxTableSizeToDrop(size_t max_size)
{
// Is initialized at server startup
shared->max_table_size_to_drop = max_size;
// Is initialized at server startup and updated at config reload
shared->max_table_size_to_drop.store(max_size, std::memory_order_relaxed);
}
void Context::checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const
{
size_t max_table_size_to_drop = shared->max_table_size_to_drop;
size_t max_table_size_to_drop = shared->max_table_size_to_drop.load(std::memory_order_relaxed);
checkCanBeDropped(database, table, table_size, max_table_size_to_drop);
}
......@@ -1906,14 +1906,14 @@ void Context::checkTableCanBeDropped(const String & database, const String & tab
void Context::setMaxPartitionSizeToDrop(size_t max_size)
{
// Is initialized at server startup
shared->max_partition_size_to_drop = max_size;
// Is initialized at server startup and updated at config reload
shared->max_partition_size_to_drop.store(max_size, std::memory_order_relaxed);
}
void Context::checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const
{
size_t max_partition_size_to_drop = shared->max_partition_size_to_drop;
size_t max_partition_size_to_drop = shared->max_partition_size_to_drop.load(std::memory_order_relaxed);
checkCanBeDropped(database, table, partition_size, max_partition_size_to_drop);
}
......
<?xml version="1.0"?>
<yandex>
<logger>
<level>trace</level>
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
<size>1000M</size>
<count>10</count>
</logger>
<tcp_port>9000</tcp_port>
<listen_host>127.0.0.1</listen_host>
<openSSL>
<client>
<cacheSessions>true</cacheSessions>
<verificationMode>none</verificationMode>
<invalidCertificateHandler>
<name>AcceptCertificateHandler</name>
</invalidCertificateHandler>
</client>
</openSSL>
<max_concurrent_queries>500</max_concurrent_queries>
<mark_cache_size>5368709120</mark_cache_size>
<path>./clickhouse/</path>
<users_config>users.xml</users_config>
<max_table_size_to_drop>1</max_table_size_to_drop>
<max_partition_size_to_drop>1</max_partition_size_to_drop>
</yandex>
<?xml version="1.0"?>
<yandex>
<profiles>
<default>
</default>
</profiles>
<users>
<default>
<password></password>
<networks incl="networks" replace="replace">
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
</default>
</users>
<quotas>
<default>
</default>
</quotas>
</yandex>
import time
import pytest
import os
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', config_dir="configs")
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
CONFIG_PATH = os.path.join(SCRIPT_DIR, './_instances/node/configs/config.xml')
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
node.query("CREATE TABLE test(date Date, id UInt32) ENGINE = MergeTree() PARTITION BY date ORDER BY id")
yield cluster
finally:
cluster.shutdown()
def test_reload_max_table_size_to_drop(start_cluster):
node.query("INSERT INTO test VALUES (now(), 0)")
time.sleep(5) # wait for data part commit
drop = node.get_query_request("DROP TABLE test")
out, err = drop.get_answer_and_error()
assert out == ""
assert err != ""
config = open(CONFIG_PATH, 'r')
config_lines = config.readlines()
config.close()
config_lines = map(lambda line: line.replace("<max_table_size_to_drop>1", "<max_table_size_to_drop>1000000"),
config_lines)
config = open(CONFIG_PATH, 'w')
config.writelines(config_lines)
config.close()
node.query("SYSTEM RELOAD CONFIG")
drop = node.get_query_request("DROP TABLE test")
out, err = drop.get_answer_and_error()
assert out == ""
assert err == ""
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册