Update query_masking_rules when reloading the config

Fixes #56449
This commit is contained in:
Mikhail Koviazin 2023-11-07 13:44:52 +02:00
parent e67edb5488
commit 9e1357dd7e
No known key found for this signature in database
GPG Key ID: 0EEAA1BF0787792F
8 changed files with 129 additions and 0 deletions

View File

@ -1372,6 +1372,8 @@ try
global_context->reloadAuxiliaryZooKeepersConfigIfChanged(config);
global_context->reloadQueryMaskingRulesIfChanged(config);
std::lock_guard lock(servers_lock);
updateServers(*config, server_pool, async_metrics, servers, servers_to_start_before_tables);
}

View File

@ -104,6 +104,10 @@ void SensitiveDataMasker::setInstance(std::unique_ptr<SensitiveDataMasker> sensi
{
sensitive_data_masker = std::move(sensitive_data_masker_);
}
else
{
sensitive_data_masker.reset();
}
}
SensitiveDataMasker * SensitiveDataMasker::getInstance()

View File

@ -4,6 +4,7 @@
#include <memory>
#include <Poco/UUID.h>
#include <Poco/Util/Application.h>
#include <Common/SensitiveDataMasker.h>
#include <Common/Macros.h>
#include <Common/EventNotifier.h>
#include <Common/Stopwatch.h>
@ -196,6 +197,9 @@ struct ContextSharedPart : boost::noncopyable
mutable zkutil::ZooKeeperPtr zookeeper TSA_GUARDED_BY(zookeeper_mutex); /// Client for ZooKeeper.
ConfigurationPtr zookeeper_config TSA_GUARDED_BY(zookeeper_mutex); /// Stores zookeeper configs
mutable std::mutex sensitive_data_masker_mutex;
ConfigurationPtr sensitive_data_masker_config;
#if USE_NURAFT
mutable std::mutex keeper_dispatcher_mutex;
mutable std::shared_ptr<KeeperDispatcher> keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex);
@ -3198,6 +3202,18 @@ bool Context::hasAuxiliaryZooKeeper(const String & name) const
return getConfigRef().has("auxiliary_zookeepers." + name);
}
void Context::reloadQueryMaskingRulesIfChanged(const ConfigurationPtr & config) const
{
std::lock_guard lock(shared->sensitive_data_masker_mutex);
const auto old_config = shared->sensitive_data_masker_config;
if (old_config && isSameConfiguration(*config, *old_config, "query_masking_rules"))
return;
SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(*config, "query_masking_rules"));
shared->sensitive_data_masker_config = config;
}
InterserverCredentialsPtr Context::getInterserverCredentials() const
{
return shared->interserver_io_credentials.get();

View File

@ -946,6 +946,8 @@ public:
// Reload Zookeeper
void reloadZooKeeperIfChanged(const ConfigurationPtr & config) const;
void reloadQueryMaskingRulesIfChanged(const ConfigurationPtr & config) const;
void setSystemZooKeeperLogAfterInitializationIfNeeded();
/// --- Caches ------------------------------------------------------------------------------------------

View File

@ -0,0 +1,19 @@
<clickhouse>
<query_log>
<database>system</database>
<table>query_log</table>
<partition_by>toYYYYMM(event_date)</partition_by>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
<max_size_rows>1048576</max_size_rows>
<reserved_size_rows>8192</reserved_size_rows>
<buffer_size_rows_flush_threshold>524288</buffer_size_rows_flush_threshold>
<flush_on_crash>false</flush_on_crash>
</query_log>
<query_masking_rules>
<rule>
<regexp>TOPSECRET.TOPSECRET</regexp>
<replace>[hidden]</replace>
</rule>
</query_masking_rules>
</clickhouse>

View File

@ -0,0 +1,12 @@
<clickhouse>
<query_log>
<database>system</database>
<table>query_log</table>
<partition_by>toYYYYMM(event_date)</partition_by>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
<max_size_rows>1048576</max_size_rows>
<reserved_size_rows>8192</reserved_size_rows>
<buffer_size_rows_flush_threshold>524288</buffer_size_rows_flush_threshold>
<flush_on_crash>false</flush_on_crash>
</query_log>
</clickhouse>

View File

@ -0,0 +1,74 @@
import pytest
import os
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry, assert_logs_contain_with_retry
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance("node", user_configs=["configs/empty_settings.xml"])
@pytest.fixture(scope="module", autouse=True)
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
@pytest.fixture(autouse=True)
def reset_to_normal_settings_after_test():
try:
node.copy_file_to_container(
os.path.join(SCRIPT_DIR, "configs/empty_settings.xml"),
"/etc/clickhouse-server/config.d/z.xml",
)
node.query("SYSTEM RELOAD CONFIG")
yield
finally:
pass
# @pytest.mark.parametrize("reload_strategy", ["force", "timeout"])
def test_reload_query_masking_rules():
# At first, empty configuration is fed to ClickHouse. The query
# "SELECT 'TOPSECRET.TOPSECRET'" will not be redacted, and the new masking
# event will not be registered
node.query("SELECT 'TOPSECRET.TOPSECRET'")
assert_logs_contain_with_retry(node, "SELECT 'TOPSECRET.TOPSECRET'")
# If there were no 'QueryMaskingRulesMatch' events, the query below returns
# 0 rows
assert (
node.query(
"SELECT count(value) FROM system.events WHERE name = 'QueryMaskingRulesMatch'"
)
== "0\n"
)
node.copy_file_to_container(
os.path.join(SCRIPT_DIR, "configs/changed_settings.xml"),
"/etc/clickhouse-server/config.d/z.xml",
)
node.query("SYSTEM RELOAD CONFIG")
# Now the same query will be redacted in the logs and the counter of events
# will be incremented
node.query("SELECT 'TOPSECRET.TOPSECRET'")
assert_eq_with_retry(
node,
"SELECT count(value) FROM system.events WHERE name = 'QueryMaskingRulesMatch'",
"1",
)
assert_logs_contain_with_retry(node, r"SELECT '\[hidden\]'")
assert (
node.query(
"SELECT value FROM system.events WHERE name = 'QueryMaskingRulesMatch'"
)
== "1\n"
)
node.rotate_logs()