mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 17:41:59 +00:00
Move prepareQueryForLogging() to a separate header.
This commit is contained in:
parent
f2bd560866
commit
dcf8724adf
@ -55,10 +55,9 @@
|
|||||||
#include <Interpreters/SelectQueryOptions.h>
|
#include <Interpreters/SelectQueryOptions.h>
|
||||||
#include <Interpreters/TransactionLog.h>
|
#include <Interpreters/TransactionLog.h>
|
||||||
#include <Interpreters/executeQuery.h>
|
#include <Interpreters/executeQuery.h>
|
||||||
#include <Interpreters/wipePasswordFromQuery.h>
|
#include <Interpreters/maskSensitiveInfoInQueryForLogging.h>
|
||||||
#include <Common/ProfileEvents.h>
|
#include <Common/ProfileEvents.h>
|
||||||
|
|
||||||
#include <Common/SensitiveDataMasker.h>
|
|
||||||
#include <IO/CompressionMethod.h>
|
#include <IO/CompressionMethod.h>
|
||||||
|
|
||||||
#include <Processors/Transforms/LimitsCheckingTransform.h>
|
#include <Processors/Transforms/LimitsCheckingTransform.h>
|
||||||
@ -77,7 +76,6 @@
|
|||||||
|
|
||||||
namespace ProfileEvents
|
namespace ProfileEvents
|
||||||
{
|
{
|
||||||
extern const Event QueryMaskingRulesMatch;
|
|
||||||
extern const Event FailedQuery;
|
extern const Event FailedQuery;
|
||||||
extern const Event FailedInsertQuery;
|
extern const Event FailedInsertQuery;
|
||||||
extern const Event FailedSelectQuery;
|
extern const Event FailedSelectQuery;
|
||||||
@ -109,37 +107,6 @@ static void checkASTSizeLimits(const IAST & ast, const Settings & settings)
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/// Makes a version of a query without sensitive information (e.g. passwords) for logging.
|
|
||||||
/// The parameter `parsed query` can be nullptr if the query cannot be parsed.
|
|
||||||
static String prepareQueryForLogging(const String & query, const ASTPtr & parsed_query, ContextPtr context)
|
|
||||||
{
|
|
||||||
String res = query;
|
|
||||||
|
|
||||||
// Wiping a password or hash from CREATE/ALTER USER query because we don't want it to go to logs.
|
|
||||||
if (parsed_query && canContainPassword(*parsed_query))
|
|
||||||
{
|
|
||||||
ASTPtr ast_for_logging = parsed_query->clone();
|
|
||||||
wipePasswordFromQuery(ast_for_logging, context);
|
|
||||||
res = serializeAST(*ast_for_logging);
|
|
||||||
}
|
|
||||||
|
|
||||||
// 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 = SensitiveDataMasker::getInstance())
|
|
||||||
{
|
|
||||||
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).
|
/// Log query into text log (not into system table).
|
||||||
static void logQuery(const String & query, ContextPtr context, bool internal, QueryProcessingStage::Enum stage)
|
static void logQuery(const String & query, ContextPtr context, bool internal, QueryProcessingStage::Enum stage)
|
||||||
{
|
{
|
||||||
@ -425,14 +392,14 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
/// MUST go before any modification (except for prepared statements,
|
/// MUST go before any modification (except for prepared statements,
|
||||||
/// since it substitute parameters and without them query does not contain
|
/// since it substitute parameters and without them query does not contain
|
||||||
/// parameters), to keep query as-is in query_log and server log.
|
/// parameters), to keep query as-is in query_log and server log.
|
||||||
query_for_logging = prepareQueryForLogging(query, ast, context);
|
query_for_logging = maskSensitiveInfoInQueryForLogging(query, ast, context);
|
||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
/// Anyway log the query.
|
/// Anyway log the query.
|
||||||
if (query.empty())
|
if (query.empty())
|
||||||
query.assign(begin, std::min(end - begin, static_cast<ptrdiff_t>(max_query_size)));
|
query.assign(begin, std::min(end - begin, static_cast<ptrdiff_t>(max_query_size)));
|
||||||
query_for_logging = prepareQueryForLogging(query, ast, context);
|
query_for_logging = maskSensitiveInfoInQueryForLogging(query, ast, context);
|
||||||
|
|
||||||
logQuery(query_for_logging, context, internal, stage);
|
logQuery(query_for_logging, context, internal, stage);
|
||||||
|
|
||||||
|
@ -1,6 +1,7 @@
|
|||||||
#include <Interpreters/wipePasswordFromQuery.h>
|
#include <Interpreters/maskSensitiveInfoInQueryForLogging.h>
|
||||||
|
|
||||||
#include <Formats/FormatFactory.h>
|
#include <Formats/FormatFactory.h>
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
#include <Interpreters/InDepthNodeVisitor.h>
|
#include <Interpreters/InDepthNodeVisitor.h>
|
||||||
#include <Interpreters/evaluateConstantExpression.h>
|
#include <Interpreters/evaluateConstantExpression.h>
|
||||||
#include <Parsers/ASTBackupQuery.h>
|
#include <Parsers/ASTBackupQuery.h>
|
||||||
@ -8,10 +9,19 @@
|
|||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/Access/ASTCreateUserQuery.h>
|
#include <Parsers/Access/ASTCreateUserQuery.h>
|
||||||
|
#include <Parsers/formatAST.h>
|
||||||
#include <TableFunctions/TableFunctionFactory.h>
|
#include <TableFunctions/TableFunctionFactory.h>
|
||||||
|
#include <Common/ProfileEvents.h>
|
||||||
|
#include <Common/SensitiveDataMasker.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace ProfileEvents
|
||||||
|
{
|
||||||
|
extern const Event QueryMaskingRulesMatch;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -513,26 +523,57 @@ namespace
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
/// Checks the type of a specified AST and returns true if it can contain a password.
|
||||||
|
bool canContainPassword(const IAST & ast)
|
||||||
|
{
|
||||||
|
using WipingVisitor = PasswordWipingVisitor</*check_only= */ true>;
|
||||||
|
WipingVisitor::Data data;
|
||||||
|
WipingVisitor::Visitor visitor{data};
|
||||||
|
ASTPtr ast_ptr = std::const_pointer_cast<IAST>(ast.shared_from_this());
|
||||||
|
visitor.visit(ast_ptr);
|
||||||
|
return data.can_contain_password;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Removes a password or its hash from a query if it's specified there or replaces it with some placeholder.
|
||||||
|
/// This function is used to prepare a query for storing in logs (we don't want logs to contain sensitive information).
|
||||||
|
void wipePasswordFromQuery(ASTPtr ast, const ContextPtr & context)
|
||||||
|
{
|
||||||
|
using WipingVisitor = PasswordWipingVisitor</*check_only= */ false>;
|
||||||
|
WipingVisitor::Data data;
|
||||||
|
data.context = context;
|
||||||
|
WipingVisitor::Visitor visitor{data};
|
||||||
|
visitor.visit(ast);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
bool canContainPassword(const IAST & ast)
|
String maskSensitiveInfoInQueryForLogging(const String & query, const ASTPtr & parsed_query, const ContextPtr & context)
|
||||||
{
|
{
|
||||||
using WipingVisitor = PasswordWipingVisitor</*check_only= */ true>;
|
String res = query;
|
||||||
WipingVisitor::Data data;
|
|
||||||
WipingVisitor::Visitor visitor{data};
|
|
||||||
ASTPtr ast_ptr = std::const_pointer_cast<IAST>(ast.shared_from_this());
|
|
||||||
visitor.visit(ast_ptr);
|
|
||||||
return data.can_contain_password;
|
|
||||||
}
|
|
||||||
|
|
||||||
void wipePasswordFromQuery(ASTPtr ast, const ContextPtr & context)
|
// Wiping a password or hash from CREATE/ALTER USER query because we don't want it to go to logs.
|
||||||
{
|
if (parsed_query && canContainPassword(*parsed_query))
|
||||||
using WipingVisitor = PasswordWipingVisitor</*check_only= */ false>;
|
{
|
||||||
WipingVisitor::Data data;
|
ASTPtr ast_without_password = parsed_query->clone();
|
||||||
data.context = context;
|
wipePasswordFromQuery(ast_without_password, context);
|
||||||
WipingVisitor::Visitor visitor{data};
|
res = serializeAST(*ast_without_password);
|
||||||
visitor.visit(ast);
|
}
|
||||||
|
|
||||||
|
// 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 = SensitiveDataMasker::getInstance())
|
||||||
|
{
|
||||||
|
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;
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
14
src/Interpreters/maskSensitiveInfoInQueryForLogging.h
Normal file
14
src/Interpreters/maskSensitiveInfoInQueryForLogging.h
Normal file
@ -0,0 +1,14 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
#include <Interpreters/Context_fwd.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/// Makes a version of a query without sensitive information (e.g. passwords) for logging.
|
||||||
|
/// The parameter `parsed query` is allowed to be nullptr if the query cannot be parsed.
|
||||||
|
String maskSensitiveInfoInQueryForLogging(const String & query, const ASTPtr & parsed_query, const ContextPtr & context);
|
||||||
|
|
||||||
|
}
|
@ -1,19 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include <Parsers/IAST_fwd.h>
|
|
||||||
#include <Interpreters/Context_fwd.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
/// Checks the type of a specified AST and returns true if it can contain a password.
|
|
||||||
bool canContainPassword(const IAST & ast);
|
|
||||||
|
|
||||||
/// Removes a password or its hash from a query if it's specified there or replaces it with some placeholder.
|
|
||||||
/// This function is used to prepare a query for storing in logs (we don't want logs to contain sensitive information).
|
|
||||||
/// The function changes only following types of queries:
|
|
||||||
/// CREATE/ALTER USER.
|
|
||||||
void wipePasswordFromQuery(ASTPtr ast, const ContextPtr & context);
|
|
||||||
|
|
||||||
}
|
|
Loading…
Reference in New Issue
Block a user