提交 526afd44 编写于 作者: F filimonov 提交者: akuzm

Make SensitiveDataMasker a singleton. (#6810)

Among other things, it is used to filter logs, which are being written even after the global server context is deinitialized, so we can't keep masker there.
上级 2ffc99b8
......@@ -15,6 +15,7 @@
#include <common/logger_useful.h>
#include <ext/scope_guard.h>
#include <ext/range.h>
#include <Common/SensitiveDataMasker.h>
namespace DB
{
......@@ -165,8 +166,7 @@ int ODBCBridge::main(const std::vector<std::string> & /*args*/)
if (config().has("query_masking_rules"))
{
context->setSensitiveDataMasker(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
setLoggerSensitiveDataMasker(logger(), context->getSensitiveDataMasker());
SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
}
auto server = Poco::Net::HTTPServer(
......
......@@ -55,6 +55,7 @@
#include "TCPHandlerFactory.h"
#include "Common/config_version.h"
#include "MySQLHandlerFactory.h"
#include <Common/SensitiveDataMasker.h>
#if defined(__linux__)
......@@ -279,8 +280,6 @@ int Server::main(const std::vector<std::string> & /*args*/)
*/
LOG_INFO(log, "Shutting down storages.");
// global_context is the owner of sensitive_data_masker, which will be destoyed after global_context->shutdown() call
setLoggerSensitiveDataMasker(logger(), nullptr);
global_context->shutdown();
LOG_DEBUG(log, "Shutted down storages.");
......@@ -414,7 +413,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
if (config().has("query_masking_rules"))
{
global_context->setSensitiveDataMasker(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
}
auto main_config_reloader = std::make_unique<ConfigReloader>(config_path,
......@@ -426,10 +425,6 @@ int Server::main(const std::vector<std::string> & /*args*/)
{
setTextLog(global_context->getTextLog());
buildLoggers(*config, logger());
if (auto masker = global_context->getSensitiveDataMasker())
{
setLoggerSensitiveDataMasker(logger(), masker);
}
global_context->setClustersConfig(config);
global_context->setMacros(std::make_unique<Macros>(*config, "macros"));
},
......
......@@ -447,7 +447,7 @@
<query_masking_rules>
<rule>
<name>hide SSN</name>
<regexp>(^|\D)\d{3}-\d{2}-\d{4}($|\D)</regexp>
<regexp>\b\d{3}-\d{2}-\d{4}\b</regexp>
<replace>000-00-0000</replace>
</rule>
</query_masking_rules>
......
......@@ -24,6 +24,7 @@ namespace DB
namespace ErrorCodes
{
extern const int CANNOT_COMPILE_REGEXP;
extern const int LOGICAL_ERROR;
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int INVALID_CONFIG_PARAMETER;
}
......@@ -38,7 +39,9 @@ private:
const RE2 regexp;
const re2::StringPiece replacement;
#ifndef NDEBUG
mutable std::atomic<std::uint64_t> matches_count = 0;
#endif
public:
//* TODO: option with hyperscan? https://software.intel.com/en-us/articles/why-and-how-to-replace-pcre-with-hyperscan
......@@ -61,15 +64,37 @@ public:
uint64_t apply(std::string & data) const
{
auto m = RE2::GlobalReplace(&data, regexp, replacement);
#ifndef NDEBUG
matches_count += m;
#endif
return m;
}
const std::string & getName() const { return name; }
const std::string & getReplacementString() const { return replacement_string; }
#ifndef NDEBUG
uint64_t getMatchesCount() const { return matches_count; }
#endif
};
std::unique_ptr<SensitiveDataMasker> SensitiveDataMasker::sensitive_data_masker = nullptr;
void SensitiveDataMasker::setInstance(std::unique_ptr<SensitiveDataMasker> sensitive_data_masker_)
{
if (!sensitive_data_masker_)
throw Exception("Logical error: the 'sensitive_data_masker' is not set", ErrorCodes::LOGICAL_ERROR);
if (sensitive_data_masker_->rulesCount() > 0)
{
sensitive_data_masker = std::move(sensitive_data_masker_);
}
}
SensitiveDataMasker * SensitiveDataMasker::getInstance()
{
return sensitive_data_masker.get();
}
SensitiveDataMasker::SensitiveDataMasker(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
......
......@@ -4,7 +4,6 @@
#include <vector>
#include <cstdint>
namespace Poco
{
namespace Util
......@@ -13,6 +12,32 @@ namespace Util
}
}
/// SensitiveDataMasker allows to remove sensitive data from queries using set of regexp-based rules
/// It's used as a singelton via getInstance method
/// Initially it's empty (nullptr) and after manual initialization
/// (one-time, done by setInstance call) it takes the proper value which
/// is stored in unique_ptr.
/// It looks like the singelton is the best option here, as
/// two users of that object (OwnSplitChannel & Interpreters/executeQuery)
/// can't own/share that Masker properly without syncronization & locks,
/// and we can't afford setting global locks for each logged line.
/// I've considered singleton alternatives, but it's unclear who should own the object,
/// and it introduce unnecessary complexity in implementation (passing references back and forward):
///
/// context can't own, as Context is destroyed before logger,
/// and logger lives longer and logging can still happen after Context destruction.
/// resetting masker in the logger at the moment of
/// context destruction can't be done w/o synchronization / locks in a safe manner.
///
/// logger is Poco derived and i didn't want to brake it's interface,
/// also logger can be dynamically reconfigured without server restart,
/// and it actually recreates OwnSplitChannel when reconfiguration happen,
/// so that makes it's quite tricky. So it a bad candidate for owning masker too.
namespace DB
{
class SensitiveDataMasker
......@@ -20,6 +45,7 @@ class SensitiveDataMasker
private:
class MaskingRule;
std::vector<std::unique_ptr<MaskingRule>> all_masking_rules;
static std::unique_ptr<SensitiveDataMasker> sensitive_data_masker;
public:
SensitiveDataMasker(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
......@@ -28,6 +54,11 @@ public:
/// Returns the number of matched rules.
size_t wipeSensitiveData(std::string & data) const;
/// setInstance is not thread-safe and should be called once in single-thread mode.
/// https://github.com/yandex/ClickHouse/pull/6810#discussion_r321183367
static void setInstance(std::unique_ptr<SensitiveDataMasker> sensitive_data_masker_);
static SensitiveDataMasker * getInstance();
/// Used in tests.
void addMaskingRule(const std::string & name, const std::string & regexp_string, const std::string & replacement_string);
......
......@@ -143,8 +143,6 @@ struct ContextShared
std::unique_ptr<DDLWorker> ddl_worker; /// Process ddl commands from zk.
/// Rules for selecting the compression settings, depending on the size of the part.
mutable std::unique_ptr<CompressionCodecSelector> compression_codec_selector;
/// Allows to remove sensitive data from queries using set of regexp-based rules
std::unique_ptr<SensitiveDataMasker> sensitive_data_masker;
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)
......@@ -287,8 +285,6 @@ struct ContextShared
/// Stop trace collector if any
trace_collector.reset();
sensitive_data_masker.reset();
}
bool hasTraceCollector()
......@@ -538,23 +534,6 @@ String Context::getUserFilesPath() const
return shared->user_files_path;
}
void Context::setSensitiveDataMasker(std::unique_ptr<SensitiveDataMasker> sensitive_data_masker)
{
if (!sensitive_data_masker)
throw Exception("Logical error: the 'sensitive_data_masker' is not set", ErrorCodes::LOGICAL_ERROR);
if (sensitive_data_masker->rulesCount() > 0)
{
auto lock = getLock();
shared->sensitive_data_masker = std::move(sensitive_data_masker);
}
}
SensitiveDataMasker * Context::getSensitiveDataMasker() const
{
return shared->sensitive_data_masker.get();
}
void Context::setPath(const String & path)
{
auto lock = getLock();
......
......@@ -12,7 +12,6 @@
#include <Common/MultiVersion.h>
#include <Common/ThreadPool.h>
#include "config_core.h"
#include <Common/SensitiveDataMasker.h>
#include <Storages/IStorage_fwd.h>
#include <atomic>
#include <chrono>
......@@ -178,9 +177,6 @@ public:
String getFlagsPath() const;
String getUserFilesPath() const;
void setSensitiveDataMasker(std::unique_ptr<SensitiveDataMasker> sensitive_data_masker);
SensitiveDataMasker * getSensitiveDataMasker() const;
void setPath(const String & path);
void setTemporaryPath(const String & path);
void setFlagsPath(const String & path);
......
......@@ -31,6 +31,7 @@
#include <Common/ProfileEvents.h>
#include <Interpreters/DNSCacheUpdater.h>
#include <Common/SensitiveDataMasker.h>
#include <Processors/Transforms/LimitsCheckingTransform.h>
#include <Processors/Transforms/MaterializingTransform.h>
......@@ -76,7 +77,7 @@ static String prepareQueryForLogging(const String & query, Context & context)
// wiping sensitive data before cropping query by log_queries_cut_to_length,
// otherwise something like credit card without last digit can go to log
if (auto masker = context.getSensitiveDataMasker())
if (auto masker = SensitiveDataMasker::getInstance())
{
auto matches = masker->wipeSensitiveData(res);
if (matches > 0)
......
......@@ -11,6 +11,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
cur_name=$(basename "${BASH_SOURCE[0]}")
tmp_file=${CLICKHOUSE_TMP}/$cur_name"_server.logs"
tmp_file2=${CLICKHOUSE_TMP}/$cur_name"_server.2.logs"
rm -f $tmp_file >/dev/null 2>&1
echo 1
......@@ -55,9 +56,10 @@ grep 'TOPSECRET' $tmp_file && echo 'fail 4b'
echo 5
# run in background
rm -f $tmp_file2 >/dev/null 2>&1
bash -c "$CLICKHOUSE_CLIENT \
--query=\"select sleepEachRow(0.5) from numbers(4) where ignore('find_me_TOPSECRET=TOPSECRET')=0 and ignore('fwerkh_that_magic_string_make_me_unique') = 0 FORMAT Null\" \
--log_queries=1 --ignore-error --multiquery 2>&1 | grep TOPSECRET" &
--log_queries=1 --ignore-error --multiquery >$tmp_file2 2>&1" &
sleep 0.1
......@@ -67,12 +69,14 @@ rm -f $tmp_file >/dev/null 2>&1
echo '5.1'
# check that executing query doesn't expose secrets in processlist
$CLICKHOUSE_CLIENT --query="SHOW PROCESSLIST" --log_queries=0 >$tmp_file 2>&1
wait
grep 'TOPSECRET' $tmp_file2 && echo 'fail 5d'
grep 'fwerkh_that_magic_string_make_me_unique' $tmp_file >/dev/null || echo 'fail 5a'
( grep 'fwerkh_that_magic_string_make_me_unique' $tmp_file | grep 'find_me_\[hidden\]' $tmp_file >/dev/null ) || echo 'fail 5b'
grep 'TOPSECRET' $tmp_file && echo 'fail 5c'
rm -f $tmp_file2 >/dev/null 2>&1
wait
grep 'fwerkh_that_magic_string_make_me_unique' $tmp_file >$tmp_file2 || echo 'fail 5a'
grep 'find_me_\[hidden\]' $tmp_file2 >/dev/null || echo 'fail 5b'
grep 'TOPSECRET' $tmp_file && echo 'fail 5c'
# instead of disabling send_logs_level=trace (enabled globally for that test) - redir it's output to /dev/null
......@@ -107,4 +111,15 @@ drop table sensitive;" --log_queries=1 --ignore-error --multiquery >$tmp_file 2>
grep 'find_me_\[hidden\]' $tmp_file >/dev/null || echo 'fail 8a'
grep 'TOPSECRET' $tmp_file && echo 'fail 8b'
echo 'finish'
\ No newline at end of file
$CLICKHOUSE_CLIENT --query="SYSTEM FLUSH LOGS" --server_logs_file=/dev/null
sleep 0.1;
echo 9
$CLICKHOUSE_CLIENT \
--server_logs_file=/dev/null \
--query="SELECT if( count() > 0, 'text_log non empty', 'text_log empty') FROM system.text_log WHERE event_time>now() - 60 and message like '%find_me%';
select * from system.text_log where event_time>now() - 60 and message like '%TOPSECRET=TOPSECRET%';" --ignore-error --multiquery
echo 'finish'
rm -f $tmp_file >/dev/null 2>&1
rm -f $tmp_file2 >/dev/null 2>&1
......@@ -167,14 +167,6 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
logger.root().get(level).setLevel(config.getString("logger.levels." + level, "trace"));
}
void Loggers::setLoggerSensitiveDataMasker(Poco::Logger & logger, DB::SensitiveDataMasker * sensitive_data_masker)
{
if (auto split = dynamic_cast<DB::OwnSplitChannel *>(logger.getChannel()))
{
split->setMasker(sensitive_data_masker);
}
}
void Loggers::closeLogs(Poco::Logger & logger)
{
if (log_file)
......
......@@ -11,18 +11,10 @@ namespace Poco::Util
class AbstractConfiguration;
}
namespace DB
{
class SensitiveDataMasker;
}
class Loggers
{
public:
void buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger, const std::string & cmd_name = "");
void setLoggerSensitiveDataMasker(Poco::Logger & logger, DB::SensitiveDataMasker * sensitive_data_masker);
/// Close log files. On next log write files will be reopened.
void closeLogs(Poco::Logger & logger);
......
......@@ -9,7 +9,7 @@
#include <Common/CurrentThread.h>
#include <Common/DNSResolver.h>
#include <common/getThreadNumber.h>
#include <Common/SensitiveDataMasker.h>
namespace DB
{
......@@ -20,7 +20,7 @@ void OwnSplitChannel::log(const Poco::Message & msg)
if (channels.empty() && (logs_queue == nullptr || msg.getPriority() > logs_queue->max_priority))
return;
if (auto masker = sensitive_data_masker.load())
if (auto masker = SensitiveDataMasker::getInstance())
{
auto message_text = msg.getText();
auto matches = masker->wipeSensitiveData(message_text);
......@@ -31,6 +31,7 @@ void OwnSplitChannel::log(const Poco::Message & msg)
}
}
logSplit(msg);
}
......@@ -100,11 +101,6 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg)
}
void OwnSplitChannel::setMasker(DB::SensitiveDataMasker * _sensitive_data_masker)
{
sensitive_data_masker.store(_sensitive_data_masker);
}
void OwnSplitChannel::addChannel(Poco::AutoPtr<Poco::Channel> channel)
{
channels.emplace_back(std::move(channel), dynamic_cast<ExtendedLogChannel *>(channel.get()));
......
......@@ -4,7 +4,6 @@
#include <Poco/AutoPtr.h>
#include <Poco/Channel.h>
#include "ExtendedLogChannel.h"
#include <Common/SensitiveDataMasker.h>
#include <Interpreters/TextLog.h>
......@@ -18,9 +17,6 @@ class OwnSplitChannel : public Poco::Channel
public:
/// Makes an extended message from msg and passes it to the client logs queue and child (if possible)
void log(const Poco::Message & msg) override;
void setMasker(DB::SensitiveDataMasker * _sensitive_data_masker);
/// Adds a child channel
void addChannel(Poco::AutoPtr<Poco::Channel> channel);
......@@ -33,9 +29,9 @@ private:
/// Handler and its pointer casted to extended interface
using ExtendedChannelPtrPair = std::pair<ChannelPtr, ExtendedLogChannel *>;
std::vector<ExtendedChannelPtrPair> channels;
std::atomic<DB::SensitiveDataMasker *> sensitive_data_masker = nullptr; // global context owns that object, pointer should be reset before context destroying.
std::mutex text_log_mutex;
std::weak_ptr<DB::TextLog> text_log;
};
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册