mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
parent
e67edb5488
commit
9e1357dd7e
@ -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);
|
||||
}
|
||||
|
@ -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()
|
||||
|
@ -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();
|
||||
|
@ -946,6 +946,8 @@ public:
|
||||
// Reload Zookeeper
|
||||
void reloadZooKeeperIfChanged(const ConfigurationPtr & config) const;
|
||||
|
||||
void reloadQueryMaskingRulesIfChanged(const ConfigurationPtr & config) const;
|
||||
|
||||
void setSystemZooKeeperLogAfterInitializationIfNeeded();
|
||||
|
||||
/// --- Caches ------------------------------------------------------------------------------------------
|
||||
|
@ -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>
|
@ -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>
|
74
tests/integration/test_reload_query_masking_rules/test.py
Normal file
74
tests/integration/test_reload_query_masking_rules/test.py
Normal 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()
|
Loading…
Reference in New Issue
Block a user