mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
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.
This commit is contained in:
parent
2ffc99b893
commit
526afd44ed
@ -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)
|
||||
|
@ -7,4 +7,6 @@
|
||||
6
|
||||
7
|
||||
8
|
||||
9
|
||||
text_log non empty
|
||||
finish
|
||||
|
@ -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
|
||||
|
||||
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'
|
||||
|
||||
wait
|
||||
grep 'TOPSECRET' $tmp_file2 && echo 'fail 5d'
|
||||
|
||||
rm -f $tmp_file2 >/dev/null 2>&1
|
||||
|
||||
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'
|
||||
$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;
|
||||
};
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user