Merge pull request #5710 from filimonov/query_masking

Query masking rules
This commit is contained in:
alexey-milovidov 2019-09-03 22:58:29 +03:00 committed by GitHub
commit cdf98cc604
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
22 changed files with 758 additions and 21 deletions

View File

@ -410,6 +410,6 @@ if (ENABLE_TESTS AND USE_GTEST)
# gtest framework has substandard code
target_compile_options(unit_tests_dbms PRIVATE -Wno-zero-as-null-pointer-constant -Wno-undef -Wno-sign-compare -Wno-used-but-marked-unused -Wno-missing-noreturn)
target_link_libraries(unit_tests_dbms PRIVATE ${GTEST_BOTH_LIBRARIES} clickhouse_functions clickhouse_parsers dbms clickhouse_common_zookeeper)
target_link_libraries(unit_tests_dbms PRIVATE ${GTEST_BOTH_LIBRARIES} clickhouse_functions clickhouse_parsers dbms clickhouse_common_zookeeper string_utils)
add_check(unit_tests_dbms)
endif ()

View File

@ -608,6 +608,7 @@ private:
if (!ends_with_backslash && (ends_with_semicolon || has_vertical_output_suffix || (!config().has("multiline") && !hasDataInSTDIN())))
{
// TODO: should we do sensitive data masking on client too? History file can be source of secret leaks.
if (input != prev_input)
{
/// Replace line breaks with spaces to prevent the following problem.

View File

@ -74,6 +74,7 @@ void LocalServer::initialize(Poco::Util::Application & self)
if (config().has("logger") || config().has("logger.level") || config().has("logger.log"))
{
// sensitive data rules are not used here
buildLoggers(config(), logger());
}
else

View File

@ -124,6 +124,7 @@ void ODBCBridge::initialize(Application & self)
config().setString("logger", "ODBCBridge");
buildLoggers(config(), logger());
log = &logger();
hostname = config().getString("listen-host", "localhost");
port = config().getUInt("http-port");
@ -162,6 +163,12 @@ int ODBCBridge::main(const std::vector<std::string> & /*args*/)
context = std::make_shared<Context>(Context::createGlobal());
context->makeGlobalContext();
if (config().has("query_masking_rules"))
{
context->setSensitiveDataMasker(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
setLoggerSensitiveDataMasker(logger(), context->getSensitiveDataMasker());
}
auto server = Poco::Net::HTTPServer(
new HandlerFactory("ODBCRequestHandlerFactory-factory", keep_alive_timeout, context), server_pool, socket, http_params);
server.start();

View File

@ -278,7 +278,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
* table engines could use Context on destroy.
*/
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.");
/** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available.
@ -407,6 +411,12 @@ int Server::main(const std::vector<std::string> & /*args*/)
/// Initialize main config reloader.
std::string include_from_path = config().getString("include_from", "/etc/metrika.xml");
if (config().has("query_masking_rules"))
{
global_context->setSensitiveDataMasker(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
}
auto main_config_reloader = std::make_unique<ConfigReloader>(config_path,
include_from_path,
config().getString("path", ""),
@ -416,6 +426,10 @@ 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"));
},

View File

@ -439,6 +439,20 @@
-->
<format_schema_path>/var/lib/clickhouse/format_schemas/</format_schema_path>
<!-- Uncomment to use query masking rules.
name - name for the rule (optional)
regexp - RE2 compatible regular expression (mandatory)
replace - substitution string for sensitive data (optional, by default - six asterisks)
<query_masking_rules>
<rule>
<name>hide SSN</name>
<regexp>(^|\D)\d{3}-\d{2}-\d{4}($|\D)</regexp>
<replace>000-00-0000</replace>
</rule>
</query_masking_rules>
-->
<!-- Uncomment to disable ClickHouse internal DNS caching. -->
<!-- <disable_internal_dns_cache>1</disable_internal_dns_cache> -->
</yandex>

View File

@ -46,6 +46,8 @@
M(NetworkSendElapsedMicroseconds, "") \
M(ThrottlerSleepMicroseconds, "Total time a query was sleeping to conform the 'max_network_bandwidth' setting.") \
\
M(QueryMaskingRulesMatch, "Number of times query masking rules was successfully matched.") \
\
M(ReplicatedPartFetches, "Number of times a data part was downloaded from replica of a ReplicatedMergeTree table.") \
M(ReplicatedPartFailedFetches, "") \
M(ObsoleteReplicatedParts, "") \

View File

@ -0,0 +1,166 @@
#include "SensitiveDataMasker.h"
#include <set>
#include <string>
#include <atomic>
#include <re2/re2.h>
#include <re2/stringpiece.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <common/logger_useful.h>
#include <Common/Exception.h>
#include <Common/StringUtils/StringUtils.h>
#ifndef NDEBUG
# include <iostream>
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_COMPILE_REGEXP;
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int INVALID_CONFIG_PARAMETER;
}
class SensitiveDataMasker::MaskingRule
{
private:
const std::string name;
const std::string replacement_string;
const std::string regexp_string;
const RE2 regexp;
const re2::StringPiece replacement;
mutable std::atomic<std::uint64_t> matches_count = 0;
public:
//* TODO: option with hyperscan? https://software.intel.com/en-us/articles/why-and-how-to-replace-pcre-with-hyperscan
// re2::set should also work quite fast, but it doesn't return the match position, only which regexp was matched
MaskingRule(const std::string & name, const std::string & _regexp_string, const std::string & _replacement_string)
: name(name)
, replacement_string(_replacement_string)
, regexp_string(_regexp_string)
, regexp(regexp_string, RE2::Quiet)
, replacement(replacement_string)
{
if (!regexp.ok())
throw DB::Exception(
"SensitiveDataMasker: cannot compile re2: " + _regexp_string + ", error: " + regexp.error()
+ ". Look at https://github.com/google/re2/wiki/Syntax for reference.",
DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
}
int apply(std::string & data) const
{
auto m = RE2::GlobalReplace(&data, regexp, replacement);
matches_count += m;
return m;
}
const std::string & getName() const { return name; }
const std::string & getReplacementString() const { return replacement_string; }
uint64_t getMatchesCount() const { return matches_count; }
};
SensitiveDataMasker::SensitiveDataMasker(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix, keys);
Logger * logger = &Logger::get("SensitiveDataMaskerConfigRead");
std::set<std::string> used_names;
for (const auto & rule : keys)
{
if (startsWith(rule, "rule"))
{
auto rule_config_prefix = config_prefix + "." + rule;
auto rule_name = config.getString(rule_config_prefix + ".name", rule_config_prefix);
if (used_names.count(rule_name) == 0)
{
used_names.insert(rule_name);
}
else
{
throw Exception(
"query_masking_rules configuration contains more than one rule named '" + rule_name + "'.",
ErrorCodes::INVALID_CONFIG_PARAMETER);
}
auto regexp = config.getString(rule_config_prefix + ".regexp", "");
if (regexp == "")
{
throw Exception(
"query_masking_rules configuration, rule '" + rule_name + "' has no <regexp> node or <regexp> is empty.",
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
}
auto replace = config.getString(rule_config_prefix + ".replace", "******");
try
{
addMaskingRule(rule_name, regexp, replace);
}
catch (DB::Exception & e)
{
e.addMessage("while adding query masking rule '" + rule_name + "'.");
throw;
}
}
else
{
LOG_WARNING(logger, "Unused param " << config_prefix << '.' << rule);
}
}
auto rules_count = this->rulesCount();
if (rules_count > 0)
{
LOG_INFO(logger, rules_count << " query masking rules loaded.");
}
}
SensitiveDataMasker::~SensitiveDataMasker() {}
void SensitiveDataMasker::addMaskingRule(
const std::string & name, const std::string & regexp_string, const std::string & replacement_string)
{
all_masking_rules.push_back(std::make_unique<MaskingRule>(name, regexp_string, replacement_string));
}
int SensitiveDataMasker::wipeSensitiveData(std::string & data) const
{
int matches = 0;
for (auto & rule : all_masking_rules)
{
matches += rule->apply(data);
}
return matches;
}
#ifndef NDEBUG
void SensitiveDataMasker::printStats()
{
for (auto & rule : all_masking_rules)
{
std::cout << rule->getName() << " (replacement to " << rule->getReplacementString() << ") matched " << rule->getMatchesCount()
<< " times" << std::endl;
}
}
#endif
unsigned long SensitiveDataMasker::rulesCount() const
{
return all_masking_rules.size();
}
}

View File

@ -0,0 +1,35 @@
#pragma once
#include <memory>
#include <vector>
namespace Poco
{
namespace Util
{
class AbstractConfiguration;
}
}
namespace DB
{
class SensitiveDataMasker
{
private:
class MaskingRule;
std::vector<std::unique_ptr<MaskingRule>> all_masking_rules;
public:
SensitiveDataMasker(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
~SensitiveDataMasker();
void addMaskingRule(const std::string & name, const std::string & regexp_string, const std::string & replacement_string);
int wipeSensitiveData(std::string & data) const;
#ifndef NDEBUG
void printStats();
#endif
unsigned long rulesCount() const;
};
};

View File

@ -0,0 +1,228 @@
#include <Common/Exception.h>
#include <Common/SensitiveDataMasker.h>
#include <Poco/AutoPtr.h>
#include <Poco/Util/XMLConfiguration.h>
#include <Poco/XML/XMLException.h>
#pragma GCC diagnostic ignored "-Wsign-compare"
#ifdef __clang__
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
# pragma clang diagnostic ignored "-Wundef"
#endif
#include <gtest/gtest.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_COMPILE_REGEXP;
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int INVALID_CONFIG_PARAMETER;
}
};
TEST(Common, SensitiveDataMasker)
{
Poco::AutoPtr<Poco::Util::XMLConfiguration> empty_xml_config = new Poco::Util::XMLConfiguration();
DB::SensitiveDataMasker masker(*empty_xml_config , "");
masker.addMaskingRule("all a letters", "a+", "--a--");
masker.addMaskingRule("all b letters", "b+", "--b--");
masker.addMaskingRule("all d letters", "d+", "--d--");
masker.addMaskingRule("all x letters", "x+", "--x--");
masker.addMaskingRule("rule \"d\" result", "--d--", "*****"); // RE2 regexps are applied one-by-one in order
std::string x = "aaaaaaaaaaaaa bbbbbbbbbb cccc aaaaaaaaaaaa d ";
EXPECT_EQ(masker.wipeSensitiveData(x), 5);
EXPECT_EQ(x, "--a-- --b-- cccc --a-- ***** ");
#ifndef NDEBUG
masker.printStats();
#endif
EXPECT_EQ(masker.wipeSensitiveData(x), 3);
EXPECT_EQ(x, "----a---- ----b---- cccc ----a---- ***** ");
#ifndef NDEBUG
masker.printStats();
#endif
DB::SensitiveDataMasker masker2(*empty_xml_config , "");
masker2.addMaskingRule("hide root password", "qwerty123", "******");
masker2.addMaskingRule("hide SSN", "[0-9]{3}-[0-9]{2}-[0-9]{4}", "000-00-0000");
masker2.addMaskingRule("hide email", "[A-Za-z0-9._%+-]+@[A-Za-z0-9.-]+\\.[A-Za-z]{2,4}", "hidden@hidden.test");
std::string query = "SELECT id FROM mysql('localhost:3308', 'database', 'table', 'root', 'qwerty123') WHERE ssn='123-45-6789' or "
"email='JonhSmith@secret.domain.test'";
EXPECT_EQ(masker2.wipeSensitiveData(query), 3);
EXPECT_EQ(
query,
"SELECT id FROM mysql('localhost:3308', 'database', 'table', 'root', '******') WHERE "
"ssn='000-00-0000' or email='hidden@hidden.test'");
#ifndef NDEBUG
// simple benchmark
auto start = std::chrono::high_resolution_clock::now();
constexpr unsigned long int iterations = 200000;
for (int i = 0; i < iterations; ++i)
{
std::string query2 = "SELECT id FROM mysql('localhost:3308', 'database', 'table', 'root', 'qwerty123') WHERE ssn='123-45-6789' or "
"email='JonhSmith@secret.domain.test'";
masker2.wipeSensitiveData(query2);
}
auto finish = std::chrono::high_resolution_clock::now();
std::chrono::duration<double> elapsed = finish - start;
std::cout << "Elapsed time: " << elapsed.count() << "s per " << iterations <<" calls (" << elapsed.count() * 1000000 / iterations << "µs per call)"
<< std::endl;
// I have: "Elapsed time: 3.44022s per 200000 calls (17.2011µs per call)"
masker2.printStats();
#endif
DB::SensitiveDataMasker maskerbad(*empty_xml_config , "");
// gtest has not good way to check exception content, so just do it manually (see https://github.com/google/googletest/issues/952 )
try
{
maskerbad.addMaskingRule("bad regexp", "**", "");
ADD_FAILURE() << "addMaskingRule() should throw an error" << std::endl;
}
catch (DB::Exception & e)
{
EXPECT_EQ(
std::string(e.what()),
"SensitiveDataMasker: cannot compile re2: **, error: no argument for repetition operator: *. Look at "
"https://github.com/google/re2/wiki/Syntax for reference.");
EXPECT_EQ(e.code(), DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
}
/* catch (...) { // not needed, gtest will react unhandled exception
FAIL() << "ERROR: Unexpected exception thrown: " << std::current_exception << std::endl; // std::current_exception is part of C++11x
} */
EXPECT_EQ(maskerbad.rulesCount(), 0);
EXPECT_EQ(maskerbad.wipeSensitiveData(x), 0);
{
std::istringstream xml_isteam(R"END(<?xml version="1.0"?>
<clickhouse>
<query_masking_rules>
<rule>
<name>hide SSN</name><!-- by default: it will use xml path, like query_masking_rules.rule[1] -->
<regexp>[0-9]{3}-[0-9]{2}-[0-9]{4}</regexp><!-- mandatory -->
<replace>000-00-0000</replace><!-- by default - six asterisks (******) -->
</rule>
<rule>
<name>hide root password</name>
<regexp>qwerty123</regexp>
</rule>
<rule>
<regexp>(?i)Ivan</regexp>
<replace>John</replace>
</rule>
<rule>
<regexp>(?i)Petrov</regexp>
<replace>Doe</replace>
</rule>
<rule>
<name>hide email</name>
<regexp>(?i)[A-Z0-9._%+-]+@[A-Z0-9.-]+\.[A-Z]{2,4}</regexp>
<replace>hidden@hidden.test</replace>
</rule>
<rule>
<name>remove selects to bad_words table</name>
<regexp>^.*bad_words.*$</regexp>
<replace>[QUERY IS CENSORED]</replace>
</rule>
</query_masking_rules>
</clickhouse>)END");
Poco::AutoPtr<Poco::Util::XMLConfiguration> xml_config = new Poco::Util::XMLConfiguration(xml_isteam);
DB::SensitiveDataMasker masker_xml_based(*xml_config, "query_masking_rules");
std::string top_secret = "The e-mail of IVAN PETROV is kotik1902@sdsdf.test, and the password is qwerty123";
EXPECT_EQ(masker_xml_based.wipeSensitiveData(top_secret), 4);
EXPECT_EQ(top_secret, "The e-mail of John Doe is hidden@hidden.test, and the password is ******");
top_secret = "SELECT * FROM bad_words";
EXPECT_EQ(masker_xml_based.wipeSensitiveData(top_secret), 1);
EXPECT_EQ(top_secret, "[QUERY IS CENSORED]");
#ifndef NDEBUG
masker_xml_based.printStats();
#endif
}
try
{
std::istringstream xml_isteam_bad(R"END(<?xml version="1.0"?>
<clickhouse>
<query_masking_rules>
<rule>
<name>test</name>
<regexp>abc</regexp>
</rule>
<rule>
<name>test</name>
<regexp>abc</regexp>
</rule>
</query_masking_rules>
</clickhouse>)END");
Poco::AutoPtr<Poco::Util::XMLConfiguration> xml_config = new Poco::Util::XMLConfiguration(xml_isteam_bad);
DB::SensitiveDataMasker masker_xml_based_exception_check(*xml_config, "query_masking_rules");
ADD_FAILURE() << "XML should throw an error on bad XML" << std::endl;
}
catch (DB::Exception & e)
{
EXPECT_EQ(
std::string(e.what()),
"query_masking_rules configuration contains more than one rule named 'test'.");
EXPECT_EQ(e.code(), DB::ErrorCodes::INVALID_CONFIG_PARAMETER);
}
try
{
std::istringstream xml_isteam_bad(R"END(<?xml version="1.0"?>
<clickhouse>
<query_masking_rules>
<rule><name>test</name></rule>
</query_masking_rules>
</clickhouse>)END");
Poco::AutoPtr<Poco::Util::XMLConfiguration> xml_config = new Poco::Util::XMLConfiguration(xml_isteam_bad);
DB::SensitiveDataMasker masker_xml_based_exception_check(*xml_config, "query_masking_rules");
ADD_FAILURE() << "XML should throw an error on bad XML" << std::endl;
}
catch (DB::Exception & e)
{
EXPECT_EQ(
std::string(e.what()),
"query_masking_rules configuration, rule 'test' has no <regexp> node or <regexp> is empty.");
EXPECT_EQ(e.code(), DB::ErrorCodes::NO_ELEMENTS_IN_CONFIG);
}
try
{
std::istringstream xml_isteam_bad(R"END(<?xml version="1.0"?>
<clickhouse>
<query_masking_rules>
<rule><name>test</name><regexp>())(</regexp></rule>
</query_masking_rules>
</clickhouse>)END");
Poco::AutoPtr<Poco::Util::XMLConfiguration> xml_config = new Poco::Util::XMLConfiguration(xml_isteam_bad);
DB::SensitiveDataMasker masker_xml_based_exception_check(*xml_config, "query_masking_rules");
ADD_FAILURE() << "XML should throw an error on bad XML" << std::endl;
}
catch (DB::Exception & e)
{
EXPECT_EQ(
std::string(e.message()),
"SensitiveDataMasker: cannot compile re2: ())(, error: missing ): ())(. Look at https://github.com/google/re2/wiki/Syntax for reference.: while adding query masking rule 'test'."
);
EXPECT_EQ(e.code(), DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
}
}

View File

@ -86,6 +86,7 @@ namespace ErrorCodes
extern const int SESSION_NOT_FOUND;
extern const int SESSION_IS_LOCKED;
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
extern const int LOGICAL_ERROR;
}
@ -142,6 +143,8 @@ 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)
@ -284,6 +287,8 @@ struct ContextShared
/// Stop trace collector if any
trace_collector.reset();
sensitive_data_masker.reset();
}
bool hasTraceCollector()
@ -533,6 +538,23 @@ 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();

View File

@ -12,6 +12,7 @@
#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>
@ -177,6 +178,9 @@ 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);

View File

@ -28,12 +28,19 @@
#include <Interpreters/InterpreterSetQuery.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Interpreters/executeQuery.h>
#include <Common/ProfileEvents.h>
#include <Interpreters/DNSCacheUpdater.h>
#include <Processors/Transforms/LimitsCheckingTransform.h>
#include <Processors/Transforms/MaterializingTransform.h>
#include <Processors/Formats/IOutputFormat.h>
namespace ProfileEvents
{
extern const Event QueryMaskingRulesMatch;
}
namespace DB
{
@ -54,7 +61,6 @@ static void checkASTSizeLimits(const IAST & ast, const Settings & settings)
ast.checkSize(settings.max_ast_elements);
}
/// NOTE This is wrong in case of single-line comments and in case of multiline string literals.
static String joinLines(const String & query)
{
@ -64,6 +70,27 @@ static String joinLines(const String & query)
}
static String prepareQueryForLogging(const String & query, Context & context)
{
String res = query;
// 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())
{
auto matches = masker->wipeSensitiveData(res);
if (matches > 0)
{
ProfileEvents::increment(ProfileEvents::QueryMaskingRulesMatch, matches);
}
}
res = res.substr(0, context.getSettingsRef().log_queries_cut_to_length);
return res;
}
/// Log query into text log (not into system table).
static void logQuery(const String & query, const Context & context, bool internal)
{
@ -111,7 +138,7 @@ static void logException(Context & context, QueryLogElement & elem)
}
static void onExceptionBeforeStart(const String & query, Context & context, time_t current_time)
static void onExceptionBeforeStart(const String & query_for_logging, Context & context, time_t current_time)
{
/// Exception before the query execution.
context.getQuota().addError();
@ -126,7 +153,7 @@ static void onExceptionBeforeStart(const String & query, Context & context, time
elem.event_time = current_time;
elem.query_start_time = current_time;
elem.query = query.substr(0, settings.log_queries_cut_to_length);
elem.query = query_for_logging;
elem.exception = getCurrentExceptionMessage(false);
elem.client_info = context.getClientInfo();
@ -192,10 +219,12 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
{
/// Anyway log the query.
String query = String(begin, begin + std::min(end - begin, static_cast<ptrdiff_t>(max_query_size)));
logQuery(query.substr(0, settings.log_queries_cut_to_length), context, internal);
auto query_for_logging = prepareQueryForLogging(query, context);
logQuery(query_for_logging, context, internal);
if (!internal)
onExceptionBeforeStart(query, context, current_time);
onExceptionBeforeStart(query_for_logging, context, current_time);
throw;
}
@ -205,6 +234,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
BlockIO res;
QueryPipeline & pipeline = res.pipeline;
String query_for_logging = "";
try
{
/// Replace ASTQueryParameter with ASTLiteral for prepared statements.
@ -217,7 +248,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
query = serializeAST(*ast);
}
logQuery(query.substr(0, settings.log_queries_cut_to_length), context, internal);
query_for_logging = prepareQueryForLogging(query, context);
logQuery(query_for_logging, context, internal);
/// Check the limits.
checkASTSizeLimits(*ast, settings);
@ -231,7 +264,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
ProcessList::EntryPtr process_list_entry;
if (!internal && !ast->as<ASTShowProcesslistQuery>())
{
process_list_entry = context.getProcessList().insert(query, ast.get(), context);
/// processlist also has query masked now, to avoid secrets leaks though SHOW PROCESSLIST by other users.
process_list_entry = context.getProcessList().insert(query_for_logging, ast.get(), context);
context.setProcessListElement(&process_list_entry->get());
}
@ -323,7 +357,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
elem.event_time = current_time;
elem.query_start_time = current_time;
elem.query = query.substr(0, settings.log_queries_cut_to_length);
elem.query = query_for_logging;
elem.client_info = context.getClientInfo();
@ -469,7 +503,12 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
catch (...)
{
if (!internal)
onExceptionBeforeStart(query, context, current_time);
{
if (query_for_logging.empty())
query_for_logging = prepareQueryForLogging(query, context);
onExceptionBeforeStart(query_for_logging, context, current_time);
}
throw;
}

View File

@ -0,0 +1,10 @@
1
2
3
4
5
5.1
6
7
8
finish

View File

@ -0,0 +1,110 @@
#!/usr/bin/env bash
# Get all server logs
export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL="trace"
#export CLICKHOUSE_BINARY='../../../../build-vscode/Debug/dbms/programs/clickhouse'
#export CLICKHOUSE_PORT_TCP=59000
#export CLICKHOUSE_CLIENT_BINARY='../../../../cmake-build-debug/dbms/programs/clickhouse client'
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
cur_name=$(basename "${BASH_SOURCE[0]}")
tmp_file=${CLICKHOUSE_TMP}/$cur_name"_server.logs"
rm -f $tmp_file >/dev/null 2>&1
echo 1
# normal execution
$CLICKHOUSE_CLIENT \
--query="SELECT 'find_me_TOPSECRET=TOPSECRET' FROM numbers(1) FORMAT Null" \
--log_queries=1 --ignore-error --multiquery >$tmp_file 2>&1
grep 'find_me_\[hidden\]' $tmp_file >/dev/null || echo 'fail 1a'
grep 'TOPSECRET' $tmp_file && echo 'fail 1b'
rm -f $tmp_file >/dev/null 2>&1
echo 2
# failure at parsing stage
echo "SELECT 'find_me_TOPSECRET=TOPSECRET' FRRRROM numbers" | ${CLICKHOUSE_CURL} -sSg ${CLICKHOUSE_URL} -d @- >$tmp_file 2>&1
#cat $tmp_file
## can't be checked on client side!
# grep 'find_me_\[hidden\]' $tmp_file >/dev/null || echo 'fail 2a'
grep 'TOPSECRET' $tmp_file && echo 'fail 2b'
rm -f $tmp_file >/dev/null 2>&1
echo 3
# failure at before query start
$CLICKHOUSE_CLIENT \
--query="SELECT 'find_me_TOPSECRET=TOPSECRET' FROM non_existing_table FORMAT Null" \
--log_queries=1 --ignore-error --multiquery >$tmp_file 2>&1
grep 'find_me_\[hidden\]' $tmp_file >/dev/null || echo 'fail 3a'
grep 'TOPSECRET' $tmp_file && echo 'fail 3b'
rm -f $tmp_file >/dev/null 2>&1
echo 4
# failure at the end of query
$CLICKHOUSE_CLIENT \
--query="SELECT 'find_me_TOPSECRET=TOPSECRET', intDiv( 100, number - 10) FROM numbers(11) FORMAT Null" \
--log_queries=1 --ignore-error --max_block_size=2 --multiquery >$tmp_file 2>&1
grep 'find_me_\[hidden\]' $tmp_file >/dev/null || echo 'fail 4a'
grep 'TOPSECRET' $tmp_file && echo 'fail 4b'
echo 5
# run in background
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" &
sleep 0.1
# $CLICKHOUSE_CLIENT --query='SHOW PROCESSLIST'
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
# instead of disabling send_logs_level=trace (enabled globally for that test) - redir it's output to /dev/null
$CLICKHOUSE_CLIENT \
--server_logs_file=/dev/null \
--query="system flush logs"
echo 6
# check events count properly increments
$CLICKHOUSE_CLIENT \
--server_logs_file=/dev/null \
--query="select * from (select sum(value) as matches from system.events where event='QueryMaskingRulesMatch') where matches < 5"
echo 7
# and finally querylog
$CLICKHOUSE_CLIENT \
--server_logs_file=/dev/null \
--query="select * from system.query_log where event_time>now() - 10 and query like '%TOPSECRET%';"
rm -f $tmp_file >/dev/null 2>&1
echo 8
$CLICKHOUSE_CLIENT \
--query="drop table if exists sensetive; create table sensitive ( id UInt64, date Date, value1 String, value2 UInt64) Engine=MergeTree ORDER BY id PARTITION BY date;
insert into sensitive select number as id, toDate('2019-01-01') as date, 'abcd' as value1, rand() as valuer from numbers(10000);
insert into sensitive select number as id, toDate('2019-01-01') as date, 'find_me_TOPSECRET=TOPSECRET' as value1, rand() as valuer from numbers(10);
insert into sensitive select number as id, toDate('2019-01-01') as date, 'abcd' as value1, rand() as valuer from numbers(10000);
select * from sensitive WHERE value1 = 'find_me_TOPSECRET=TOPSECRET' FORMAT Null;
drop table sensitive;" --log_queries=1 --ignore-error --multiquery >$tmp_file 2>&1
grep 'find_me_\[hidden\]' $tmp_file >/dev/null || echo 'fail 8a'
grep 'TOPSECRET' $tmp_file && echo 'fail 8b'
echo 'finish'

View File

@ -554,6 +554,36 @@ If the table doesn't exist, ClickHouse will create it. If the structure of the q
</query_log>
```
## query_masking_rules
Regexp-based rules, which will be applied to queries as well as all log messages before storing them in server logs,
`system.query_log`, `system.text_log`, `system.processes` table, and in logs sent to client. That allows preventing
sensitive data leakage from SQL queries (like names / emails / personal
identifiers / credit card numbers etc) to logs.
**Example**
```xml
<query_masking_rules>
<rule>
<name>hide SSN</name>
<regexp>(^|\D)\d{3}-\d{2}-\d{4}($|\D)</regexp>
<replace>000-00-0000</replace>
</rule>
</query_masking_rules>
```
Config fields:
- `name` - name for the rule (optional)
- `regexp` - RE2 compatible regular expression (mandatory)
- `replace` - substitution string for sensitive data (optional, by default - six asterisks)
The masking rules are applied on whole query (to prevent leaks of sensitive data from malformed / non parsable queries).
`system.events` table have counter `QueryMaskingRulesMatch` which have overall number of query masking rules matches.
For distributed queries each server have to be configured separately, otherwise subquries passed to other
nodes will be stored without masking.
## remote_servers {#server_settings_remote_servers}

View File

@ -648,6 +648,7 @@ void BaseDaemon::initialize(Application & self)
throw Poco::Exception("Cannot change directory to /tmp");
}
// sensitive data masking rules are not used here
buildLoggers(config(), logger());
if (is_daemon)

View File

@ -11,6 +11,11 @@
#include <Poco/Net/RemoteSyslogChannel.h>
#include <Poco/Path.h>
namespace DB
{
class SensitiveDataMasker;
}
// TODO: move to libcommon
static std::string createDirectory(const std::string & file)
@ -162,12 +167,21 @@ 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)
log_file->close();
if (error_log_file)
error_log_file->close();
// Shouldn't syslog_channel be closed here too?
if (!log_file)
logger.warning("Logging to console but received signal to close log file (ignoring).");

View File

@ -8,7 +8,12 @@
namespace Poco::Util
{
class AbstractConfiguration;
class AbstractConfiguration;
}
namespace DB
{
class SensitiveDataMasker;
}
@ -16,6 +21,8 @@ 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);
@ -31,10 +38,10 @@ protected:
std::optional<size_t> layer;
private:
/// Файлы с логами.
Poco::AutoPtr<Poco::FileChannel> log_file;
Poco::AutoPtr<Poco::FileChannel> error_log_file;
Poco::AutoPtr<Poco::Channel> syslog_channel;
/// Previous value of logger element in config. It is used to reinitialize loggers whenever the value changed.
std::string config_logger;

View File

@ -5,18 +5,18 @@
#include "ExtendedLogChannel.h"
/** Форматирует по своему.
* Некоторые детали невозможно получить, используя только Poco::PatternFormatter.
/** Format log messages own way.
* We can't obtain some details using Poco::PatternFormatter.
*
* Во-первых, используется номер потока не среди потоков Poco::Thread,
* а среди всех потоков, для которых был получен номер (см. ThreadNumber.h)
* Firstly, the thread number here is peaked not from Poco::Thread
* threads only, but from all threads with number assigned (see ThreadNumber.h)
*
* Во-вторых, корректно выводится локальная дата и время.
* Poco::PatternFormatter плохо работает с локальным временем,
* в ситуациях, когда в ближайшем будущем намечается отмена или введение daylight saving time.
* - см. исходники Poco и http://thread.gmane.org/gmane.comp.time.tz/8883
* Secondly, the local date and time are correctly displayed.
* Poco::PatternFormatter does not work well with local time,
* when timestamps are close to DST timeshift moments.
* - see Poco sources and http://thread.gmane.org/gmane.comp.time.tz/8883
*
* Также сделан чуть более эффективным (что имеет мало значения).
* Also it's made a bit more efficient (unimportant).
*/
class Loggers;

View File

@ -20,6 +20,23 @@ 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())
{
auto message_text = msg.getText();
auto matches = masker->wipeSensitiveData(message_text);
if (matches > 0)
{
logSplit({msg, message_text}); // we will continue with the copy of original message with text modified
return;
}
}
logSplit(msg);
}
void OwnSplitChannel::logSplit(const Poco::Message & msg)
{
ExtendedLogMessage msg_ext = ExtendedLogMessage::getFrom(msg);
/// Log data to child channels
@ -31,6 +48,8 @@ void OwnSplitChannel::log(const Poco::Message & msg)
channel.first->log(msg); // ordinary child
}
auto logs_queue = CurrentThread::getInternalTextLogsQueue();
/// Log to "TCP queue" if message is not too noisy
if (logs_queue && msg.getPriority() <= logs_queue->max_priority)
{
@ -80,6 +99,12 @@ void OwnSplitChannel::log(const Poco::Message & msg)
log->add(elem);
}
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()));

View File

@ -1,8 +1,10 @@
#pragma once
#include <atomic>
#include <vector>
#include <Poco/AutoPtr.h>
#include <Poco/Channel.h>
#include "ExtendedLogChannel.h"
#include <Common/SensitiveDataMasker.h>
#include <Interpreters/TextLog.h>
@ -17,16 +19,21 @@ 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);
void addTextLog(std::shared_ptr<DB::TextLog> log);
private:
void logSplit(const Poco::Message & msg);
using ChannelPtr = Poco::AutoPtr<Poco::Channel>;
/// 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;