diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 849784e41bd..38127891e4d 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -412,6 +412,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 () diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 091a1ac063f..88e485a66ab 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -607,6 +607,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. diff --git a/dbms/programs/local/LocalServer.cpp b/dbms/programs/local/LocalServer.cpp index bed55a0fc5f..6b07fc93ee0 100644 --- a/dbms/programs/local/LocalServer.cpp +++ b/dbms/programs/local/LocalServer.cpp @@ -73,6 +73,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 diff --git a/dbms/programs/odbc-bridge/ODBCBridge.cpp b/dbms/programs/odbc-bridge/ODBCBridge.cpp index cf265eb6abb..214d9f75328 100644 --- a/dbms/programs/odbc-bridge/ODBCBridge.cpp +++ b/dbms/programs/odbc-bridge/ODBCBridge.cpp @@ -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 & /*args*/) context = std::make_shared(Context::createGlobal()); context->makeGlobalContext(); + if (config().has("query_masking_rules")) + { + context->setSensitiveDataMasker(std::make_unique(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(); diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index 3d50466df84..1ca29c40745 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -269,7 +269,11 @@ int Server::main(const std::vector & /*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. @@ -399,6 +403,12 @@ int Server::main(const std::vector & /*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(config(), "query_masking_rules")); + } + auto main_config_reloader = std::make_unique(config_path, include_from_path, config().getString("path", ""), @@ -407,6 +417,10 @@ int Server::main(const std::vector & /*args*/) [&](ConfigurationPtr config) { buildLoggers(*config, logger()); + if (auto masker = global_context->getSensitiveDataMasker()) + { + setLoggerSensitiveDataMasker(logger(), masker); + } global_context->setClustersConfig(config); global_context->setMacros(std::make_unique(*config, "macros")); }, diff --git a/dbms/programs/server/config.xml b/dbms/programs/server/config.xml index 154ebf6c35e..699c18638c5 100644 --- a/dbms/programs/server/config.xml +++ b/dbms/programs/server/config.xml @@ -410,6 +410,20 @@ --> /var/lib/clickhouse/format_schemas/ + + + diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index 7059e02d76c..3489b17178f 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -44,6 +44,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, "") \ diff --git a/dbms/src/Common/SensitiveDataMasker.cpp b/dbms/src/Common/SensitiveDataMasker.cpp new file mode 100644 index 00000000000..488c0be4245 --- /dev/null +++ b/dbms/src/Common/SensitiveDataMasker.cpp @@ -0,0 +1,166 @@ +#include "SensitiveDataMasker.h" + +#include +#include +#include + +#include +#include + +#include + +#include + +#include +#include + +#ifndef NDEBUG +# include +#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 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 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 node or 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(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(); +} + +} diff --git a/dbms/src/Common/SensitiveDataMasker.h b/dbms/src/Common/SensitiveDataMasker.h new file mode 100644 index 00000000000..02e86976d02 --- /dev/null +++ b/dbms/src/Common/SensitiveDataMasker.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include + +namespace Poco +{ +namespace Util +{ + class AbstractConfiguration; +} +} + +namespace DB +{ +class SensitiveDataMasker +{ +private: + class MaskingRule; + std::vector> 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; +}; + +}; diff --git a/dbms/src/Common/tests/gtest_sensitive_data_masker.cpp b/dbms/src/Common/tests/gtest_sensitive_data_masker.cpp new file mode 100644 index 00000000000..d79b7b9932c --- /dev/null +++ b/dbms/src/Common/tests/gtest_sensitive_data_masker.cpp @@ -0,0 +1,228 @@ +#include +#include +#include +#include +#include + +#pragma GCC diagnostic ignored "-Wsign-compare" +#ifdef __clang__ +# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant" +# pragma clang diagnostic ignored "-Wundef" +#endif + +#include + + +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 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 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( + + + + hide SSN + [0-9]{3}-[0-9]{2}-[0-9]{4} + 000-00-0000 + + + hide root password + qwerty123 + + + (?i)Ivan + John + + + (?i)Petrov + Doe + + + hide email + (?i)[A-Z0-9._%+-]+@[A-Z0-9.-]+\.[A-Z]{2,4} + hidden@hidden.test + + + remove selects to bad_words table + ^.*bad_words.*$ + [QUERY IS CENSORED] + + +)END"); + + Poco::AutoPtr 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( + + + + test + abc + + + test + abc + + +)END"); + Poco::AutoPtr 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( + + + test + +)END"); + + Poco::AutoPtr 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 node or is empty."); + EXPECT_EQ(e.code(), DB::ErrorCodes::NO_ELEMENTS_IN_CONFIG); + } + + + try + { + std::istringstream xml_isteam_bad(R"END( + + + test())( + +)END"); + + Poco::AutoPtr 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); + } + +} diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index bf779ada6b4..13f213392e3 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -146,6 +146,8 @@ struct ContextShared std::unique_ptr ddl_worker; /// Process ddl commands from zk. /// Rules for selecting the compression settings, depending on the size of the part. mutable std::unique_ptr compression_codec_selector; + /// Allows to remove sensitive data from queries using set of regexp-based rules + std::unique_ptr sensitive_data_masker; std::optional 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) @@ -285,6 +287,7 @@ struct ContextShared background_pool.reset(); schedule_pool.reset(); ddl_worker.reset(); + sensitive_data_masker.reset(); } private: @@ -522,6 +525,19 @@ String Context::getUserFilesPath() const return shared->user_files_path; } +void Context::setSensitiveDataMasker(std::unique_ptr sensitive_data_masker) +{ + 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(); diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 44547f97ef2..ce9661334a3 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -11,6 +11,7 @@ #include #include #include "config_core.h" +#include #include #include #include @@ -172,6 +173,9 @@ public: String getFlagsPath() const; String getUserFilesPath() const; + void setSensitiveDataMasker(std::unique_ptr sensitive_data_masker); + SensitiveDataMasker * getSensitiveDataMasker() const; + void setPath(const String & path); void setTemporaryPath(const String & path); void setFlagsPath(const String & path); diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index b001c9bcb0a..66e67321777 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -34,6 +34,13 @@ #include #include +#include + +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,26 @@ static String joinLines(const String & query) } +static String prepareQueryForLogging(const String & query, Context & context) +{ + String res = query; + // cropping is cheap and can reduce the amount of work for wipeSensitiveData + // side effect: if the query had long string with sensitive data which were removed, + // we will store less than log_queries_cut_to_length in logs + res = res.substr(0, context.getSettingsRef().log_queries_cut_to_length); + + if (auto masker = context.getSensitiveDataMasker()) + { + auto matches = masker->wipeSensitiveData(res); + if (matches > 0) + { + ProfileEvents::increment(ProfileEvents::QueryMaskingRulesMatch, matches); + } + } + return res; +} + + /// Log query into text log (not into system table). static void logQuery(const String & query, const Context & context, bool internal) { @@ -111,7 +137,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 +152,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 +218,12 @@ static std::tuple executeQueryImpl( { /// Anyway log the query. String query = String(begin, begin + std::min(end - begin, static_cast(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 +233,8 @@ static std::tuple executeQueryImpl( BlockIO res; QueryPipeline & pipeline = res.pipeline; + String query_for_logging = ""; + try { /// Replace ASTQueryParameter with ASTLiteral for prepared statements. @@ -217,7 +247,9 @@ static std::tuple 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 +263,8 @@ static std::tuple executeQueryImpl( ProcessList::EntryPtr process_list_entry; if (!internal && !ast->as()) { - 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()); } @@ -318,7 +351,7 @@ static std::tuple 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(); @@ -462,7 +495,12 @@ static std::tuple 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; } diff --git a/dbms/tests/queries/0_stateless/00956_sensitive_data_masking.reference b/dbms/tests/queries/0_stateless/00956_sensitive_data_masking.reference new file mode 100644 index 00000000000..10856627eac --- /dev/null +++ b/dbms/tests/queries/0_stateless/00956_sensitive_data_masking.reference @@ -0,0 +1,10 @@ +1 +2 +3 +4 +5 +5.1 +6 +7 +8 +finish diff --git a/dbms/tests/queries/0_stateless/00956_sensitive_data_masking.sh b/dbms/tests/queries/0_stateless/00956_sensitive_data_masking.sh new file mode 100755 index 00000000000..ef66d30581a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00956_sensitive_data_masking.sh @@ -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 sensetive;" --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' \ No newline at end of file diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md index 2f87233e6a3..5d3e0d93ab6 100644 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -554,6 +554,36 @@ If the table doesn't exist, ClickHouse will create it. If the structure of the q ``` +## query_masking_rules + +Regexp-based rules, which will be applied to queries before storing them in server logs, +`system.query_log` table, `system.processes` table. That allows preventing +sensitive data leakage from SQL queries (like names / emails / personal +identifiers / credit card numbers etc) to logs. + +**Example** + +```xml + + + hide SSN + (^|\D)\d{3}-\d{2}-\d{4}($|\D) + 000-00-0000 + + +``` + +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 diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index b40f00a700f..f3508b3156b 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -652,6 +652,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) diff --git a/libs/libloggers/loggers/Loggers.cpp b/libs/libloggers/loggers/Loggers.cpp index bc53cff27aa..0269fe0787f 100644 --- a/libs/libloggers/loggers/Loggers.cpp +++ b/libs/libloggers/loggers/Loggers.cpp @@ -1,6 +1,7 @@ #include "Loggers.h" #include +#include #include #include #include "OwnFormattingChannel.h" @@ -152,12 +153,20 @@ 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(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)."); diff --git a/libs/libloggers/loggers/Loggers.h b/libs/libloggers/loggers/Loggers.h index 7b3c0860273..848b4eff10b 100644 --- a/libs/libloggers/loggers/Loggers.h +++ b/libs/libloggers/loggers/Loggers.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include "OwnFormattingChannel.h" namespace Poco::Util { @@ -14,6 +16,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); @@ -27,10 +31,10 @@ protected: std::optional layer; private: - /// Файлы с логами. Poco::AutoPtr log_file; Poco::AutoPtr error_log_file; Poco::AutoPtr syslog_channel; + /// Previous value of logger element in config. It is used to reinitialize loggers whenever the value changed. std::string config_logger; }; diff --git a/libs/libloggers/loggers/OwnPatternFormatter.h b/libs/libloggers/loggers/OwnPatternFormatter.h index 0e100aa6f33..dc1254cba29 100644 --- a/libs/libloggers/loggers/OwnPatternFormatter.h +++ b/libs/libloggers/loggers/OwnPatternFormatter.h @@ -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; diff --git a/libs/libloggers/loggers/OwnSplitChannel.cpp b/libs/libloggers/loggers/OwnSplitChannel.cpp index 0779f6477c3..5069494c507 100644 --- a/libs/libloggers/loggers/OwnSplitChannel.cpp +++ b/libs/libloggers/loggers/OwnSplitChannel.cpp @@ -19,6 +19,23 @@ void OwnSplitChannel::log(const Poco::Message & msg) if (channels.empty() && (logs_queue == nullptr || msg.getPriority() > logs_queue->max_priority)) return; + if (sensitive_data_masker) + { + auto message_text = msg.getText(); + auto matches = sensitive_data_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 @@ -30,6 +47,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) { @@ -51,6 +70,13 @@ void OwnSplitChannel::log(const Poco::Message & msg) /// TODO: Also log to system.internal_text_log table } + +void OwnSplitChannel::setMasker(DB::SensitiveDataMasker * _sensitive_data_masker) +{ + std::lock_guard lock(mutex); + sensitive_data_masker = _sensitive_data_masker; +} + void OwnSplitChannel::addChannel(Poco::AutoPtr channel) { channels.emplace_back(std::move(channel), dynamic_cast(channel.get())); diff --git a/libs/libloggers/loggers/OwnSplitChannel.h b/libs/libloggers/loggers/OwnSplitChannel.h index 3579218f75c..768bdef9dca 100644 --- a/libs/libloggers/loggers/OwnSplitChannel.h +++ b/libs/libloggers/loggers/OwnSplitChannel.h @@ -1,8 +1,10 @@ #pragma once #include +#include #include #include #include "ExtendedLogChannel.h" +#include namespace DB @@ -16,14 +18,20 @@ 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 channel); private: + void logSplit(const Poco::Message & msg); + using ChannelPtr = Poco::AutoPtr; /// Handler and its pointer casted to extended interface using ExtendedChannelPtrPair = std::pair; std::vector channels; + DB::SensitiveDataMasker * sensitive_data_masker = nullptr; // global context owns that object, pointer should be reset before context destroying. + std::mutex mutex; }; }