Merge pull request #41562 from vitlibar/refactor-wiping-info-from-logs

Refactor wiping sensitive information from logs.
This commit is contained in:
Vitaly Baranov 2022-10-02 12:58:58 +02:00 committed by GitHub
commit baf48ec400
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 217 additions and 238 deletions

View File

@ -109,12 +109,19 @@ static void checkASTSizeLimits(const IAST & ast, const Settings & settings)
}
static String prepareQueryForLogging(const String & query, ContextPtr context)
/// 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 its hash from CREATE/ALTER USER query because we don't want it to go to logs.
res = wipePasswordFromQuery(res);
// 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);
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.
@ -381,16 +388,16 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
const Settings & settings = context->getSettingsRef();
ASTPtr ast;
const char * query_end;
size_t max_query_size = settings.max_query_size;
/// Don't limit the size of internal queries or distributed subquery.
if (internal || client_info.query_kind == ClientInfo::QueryKind::SECONDARY_QUERY)
max_query_size = 0;
String query_database;
String query_table;
ASTPtr ast;
String query;
String query_for_logging;
/// Parse the query from string.
try
{
if (settings.dialect == Dialect::kusto && !internal)
@ -408,6 +415,50 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth);
}
const char * query_end = end;
if (const auto * insert_query = ast->as<ASTInsertQuery>(); insert_query && insert_query->data)
query_end = insert_query->data;
/// Replace ASTQueryParameter with ASTLiteral for prepared statements.
if (context->hasQueryParameters())
{
ReplaceQueryParameterVisitor visitor(context->getQueryParameters());
visitor.visit(ast);
query = serializeAST(*ast);
}
else
{
/// Copy query into string. It will be written to log and presented in processlist. If an INSERT query, string will not include data to insertion.
query.assign(begin, query_end);
}
/// Wipe any sensitive information (e.g. passwords) from the query.
/// MUST go before any modification (except for prepared statements,
/// since it substitute parameters and without them query does not contain
/// parameters), to keep query as-is in query_log and server log.
query_for_logging = prepareQueryForLogging(query, ast, context);
}
catch (...)
{
/// Anyway log the query.
if (query.empty())
query.assign(begin, std::min(end - begin, static_cast<ptrdiff_t>(max_query_size)));
query_for_logging = prepareQueryForLogging(query, ast, context);
logQuery(query_for_logging, context, internal, stage);
if (!internal)
onExceptionBeforeStart(query_for_logging, context, time_in_microseconds(current_time), ast, query_span);
throw;
}
BlockIO res;
std::shared_ptr<InterpreterTransactionControlQuery> implicit_txn_control{};
String query_database;
String query_table;
try
{
if (auto txn = context->getCurrentTransaction())
{
chassert(txn->getState() != MergeTreeTransaction::COMMITTING);
@ -433,80 +484,34 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
{
if (query_with_output->settings_ast)
InterpreterSetQuery(query_with_output->settings_ast, context).executeForCurrentContext();
}
if (auto * create_query = ast->as<ASTCreateQuery>())
{
if (create_query->select)
if (const auto * create_query = ast->as<ASTCreateQuery>())
{
applySettingsFromSelectWithUnion(create_query->select->as<ASTSelectWithUnionQuery &>(), context);
if (create_query->select)
{
applySettingsFromSelectWithUnion(create_query->select->as<ASTSelectWithUnionQuery &>(), context);
}
}
}
auto * insert_query = ast->as<ASTInsertQuery>();
if (insert_query && insert_query->settings_ast)
InterpreterSetQuery(insert_query->settings_ast, context).executeForCurrentContext();
if (insert_query)
else if (auto * insert_query = ast->as<ASTInsertQuery>())
{
if (insert_query->data)
query_end = insert_query->data;
else
query_end = end;
if (insert_query->settings_ast)
InterpreterSetQuery(insert_query->settings_ast, context).executeForCurrentContext();
insert_query->tail = istr;
}
else
setQuerySpecificSettings(ast, context);
/// There is an option of probabilistic logging of queries.
/// If it is used - do the random sampling and "collapse" the settings.
/// It allows to consistently log queries with all the subqueries in distributed query processing
/// (subqueries on remote nodes will receive these "collapsed" settings)
if (!internal && settings.log_queries && settings.log_queries_probability < 1.0)
{
query_end = end;
}
}
catch (...)
{
/// Anyway log the query.
String query = String(begin, begin + std::min(end - begin, static_cast<ptrdiff_t>(max_query_size)));
std::bernoulli_distribution should_write_log{settings.log_queries_probability};
auto query_for_logging = prepareQueryForLogging(query, context);
logQuery(query_for_logging, context, internal, stage);
if (!internal)
{
onExceptionBeforeStart(query_for_logging, context, time_in_microseconds(current_time), ast, query_span);
}
throw;
}
setQuerySpecificSettings(ast, context);
/// There is an option of probabilistic logging of queries.
/// If it is used - do the random sampling and "collapse" the settings.
/// It allows to consistently log queries with all the subqueries in distributed query processing
/// (subqueries on remote nodes will receive these "collapsed" settings)
if (!internal && settings.log_queries && settings.log_queries_probability < 1.0)
{
std::bernoulli_distribution should_write_log{settings.log_queries_probability};
context->setSetting("log_queries", should_write_log(thread_local_rng));
context->setSetting("log_queries_probability", 1.0);
}
/// Copy query into string. It will be written to log and presented in processlist. If an INSERT query, string will not include data to insertion.
String query(begin, query_end);
BlockIO res;
String query_for_logging;
std::shared_ptr<InterpreterTransactionControlQuery> implicit_txn_control{};
try
{
/// Replace ASTQueryParameter with ASTLiteral for prepared statements.
if (context->hasQueryParameters())
{
ReplaceQueryParameterVisitor visitor(context->getQueryParameters());
visitor.visit(ast);
query = serializeAST(*ast);
context->setSetting("log_queries", should_write_log(thread_local_rng));
context->setSetting("log_queries_probability", 1.0);
}
if (const auto * query_with_table_output = dynamic_cast<const ASTQueryWithTableAndOutput *>(ast.get()))
@ -515,10 +520,6 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
query_table = query_with_table_output->getTable();
}
/// MUST go before any modification (except for prepared statements,
/// since it substitute parameters and without them query does not contain
/// parameters), to keep query as-is in query_log and server log.
query_for_logging = prepareQueryForLogging(query, context);
logQuery(query_for_logging, context, internal, stage);
/// Propagate WITH statement to children ASTSelect.
@ -1083,12 +1084,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
}
if (!internal)
{
if (query_for_logging.empty())
query_for_logging = prepareQueryForLogging(query, context);
onExceptionBeforeStart(query_for_logging, context, time_in_microseconds(current_time), ast, query_span);
}
throw;
}

View File

@ -34,96 +34,109 @@ namespace
}
String auth_type_name = AuthenticationTypeInfo::get(auth_type).name;
String value_prefix;
std::optional<String> value;
String prefix; /// "BY" or "SERVER" or "REALM"
std::optional<String> password; /// either a password or hash
std::optional<String> salt;
const boost::container::flat_set<String> * values = nullptr;
std::optional<String> parameter;
const boost::container::flat_set<String> * parameters = nullptr;
if (show_password ||
auth_type == AuthenticationType::LDAP ||
auth_type == AuthenticationType::KERBEROS ||
auth_type == AuthenticationType::SSL_CERTIFICATE)
switch (auth_type)
{
switch (auth_type)
case AuthenticationType::PLAINTEXT_PASSWORD:
{
case AuthenticationType::PLAINTEXT_PASSWORD:
{
value_prefix = "BY";
value = auth_data.getPassword();
break;
}
case AuthenticationType::SHA256_PASSWORD:
{
auth_type_name = "sha256_hash";
value_prefix = "BY";
value = auth_data.getPasswordHashHex();
if (!auth_data.getSalt().empty())
{
salt = auth_data.getSalt();
}
break;
}
case AuthenticationType::DOUBLE_SHA1_PASSWORD:
{
auth_type_name = "double_sha1_hash";
value_prefix = "BY";
value = auth_data.getPasswordHashHex();
break;
}
case AuthenticationType::LDAP:
{
value_prefix = "SERVER";
value = auth_data.getLDAPServerName();
break;
}
case AuthenticationType::KERBEROS:
{
const auto & realm = auth_data.getKerberosRealm();
if (!realm.empty())
{
value_prefix = "REALM";
value = realm;
}
break;
}
case AuthenticationType::SSL_CERTIFICATE:
{
value_prefix = "CN";
values = &auth_data.getSSLCertificateCommonNames();
break;
}
case AuthenticationType::NO_PASSWORD: [[fallthrough]];
case AuthenticationType::MAX:
throw Exception("AST: Unexpected authentication type " + toString(auth_type), ErrorCodes::LOGICAL_ERROR);
prefix = "BY";
password = auth_data.getPassword();
break;
}
case AuthenticationType::SHA256_PASSWORD:
{
auth_type_name = "sha256_hash";
prefix = "BY";
password = auth_data.getPasswordHashHex();
if (!auth_data.getSalt().empty())
salt = auth_data.getSalt();
break;
}
case AuthenticationType::DOUBLE_SHA1_PASSWORD:
{
auth_type_name = "double_sha1_hash";
prefix = "BY";
password = auth_data.getPasswordHashHex();
break;
}
case AuthenticationType::LDAP:
{
prefix = "SERVER";
parameter = auth_data.getLDAPServerName();
break;
}
case AuthenticationType::KERBEROS:
{
const auto & realm = auth_data.getKerberosRealm();
if (!realm.empty())
{
prefix = "REALM";
parameter = realm;
}
break;
}
case AuthenticationType::SSL_CERTIFICATE:
{
prefix = "CN";
parameters = &auth_data.getSSLCertificateCommonNames();
break;
}
case AuthenticationType::NO_PASSWORD: [[fallthrough]];
case AuthenticationType::MAX:
throw Exception("AST: Unexpected authentication type " + toString(auth_type), ErrorCodes::LOGICAL_ERROR);
}
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " IDENTIFIED WITH " << auth_type_name
<< (settings.hilite ? IAST::hilite_none : "");
if (!value_prefix.empty())
if (password && !show_password)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " " << value_prefix
prefix = "";
password.reset();
salt.reset();
auth_type_name = AuthenticationTypeInfo::get(auth_type).name;
}
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " IDENTIFIED" << (settings.hilite ? IAST::hilite_none : "");
if (!auth_type_name.empty())
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " WITH " << auth_type_name
<< (settings.hilite ? IAST::hilite_none : "");
}
if (value)
if (!prefix.empty())
{
settings.ostr << " " << quoteString(*value);
if (salt)
settings.ostr << " SALT " << quoteString(*salt);
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " " << prefix << (settings.hilite ? IAST::hilite_none : "");
}
else if (values)
if (password)
{
settings.ostr << " " << quoteString(*password);
}
if (salt)
{
settings.ostr << " SALT " << quoteString(*salt);
}
if (parameter)
{
settings.ostr << " " << quoteString(*parameter);
}
else if (parameters)
{
settings.ostr << " ";
bool need_comma = false;
for (const auto & item : *values)
for (const auto & param : *parameters)
{
if (std::exchange(need_comma, true))
settings.ostr << ", ";
settings.ostr << quoteString(item);
settings.ostr << quoteString(param);
}
}
}

View File

@ -45,7 +45,7 @@ public:
std::optional<String> new_name;
std::optional<AuthenticationData> auth_data;
bool show_password = true; /// formatImpl() will show the password or hash.
bool show_password = true; /// formatImpl() shows a password or hash by default
std::optional<AllowedClientHosts> hosts;
std::optional<AllowedClientHosts> add_hosts;

View File

@ -1,7 +1,4 @@
#include <Parsers/Access/ASTCreateUserQuery.h>
#include <Parsers/Access/ParserCreateUserQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/wipePasswordFromQuery.h>
#include <Common/typeid_cast.h>
@ -9,24 +6,17 @@
namespace DB
{
String wipePasswordFromQuery(const String & query)
bool canContainPassword(const IAST & ast)
{
String error_message;
const char * begin = query.data();
const char * end = begin + query.size();
return ast.as<ASTCreateUserQuery>();
}
void wipePasswordFromQuery(ASTPtr ast)
{
if (auto * create_query = ast->as<ASTCreateUserQuery>())
{
ParserCreateUserQuery parser;
const char * pos = begin;
if (auto ast = tryParseQuery(parser, pos, end, error_message, false, "", false, 0, 0))
{
auto create_query = typeid_cast<std::shared_ptr<ASTCreateUserQuery>>(ast);
create_query->show_password = false;
return serializeAST(*create_query);
}
create_query->show_password = false;
}
return query;
}
}

View File

@ -1,14 +1,18 @@
#pragma once
#include <base/types.h>
#include <Parsers/IAST_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.
String wipePasswordFromQuery(const String & query);
void wipePasswordFromQuery(ASTPtr ast);
}

View File

@ -15,6 +15,32 @@ def started_cluster():
cluster.shutdown()
def check_logs(must_contain, must_not_contain):
node.query("SYSTEM FLUSH LOGS")
for str in must_contain:
assert node.contains_in_log(str)
assert (
int(
node.query(
f"SELECT COUNT() FROM system.query_log WHERE query LIKE '%{str}%'"
).strip()
)
>= 1
)
for str in must_not_contain:
assert not node.contains_in_log(str)
assert (
int(
node.query(
f"SELECT COUNT() FROM system.query_log WHERE query LIKE '%{str}%'"
).strip()
)
== 0
)
# Passwords in CREATE/ALTER queries must be hidden in logs.
def test_create_alter_user():
node.query("CREATE USER u1 IDENTIFIED BY 'qwe123' SETTINGS custom_a = 'a'")
@ -32,68 +58,18 @@ def test_create_alter_user():
== "CREATE USER u2 IDENTIFIED WITH plaintext_password SETTINGS custom_c = \\'c\\'\n"
)
node.query("SYSTEM FLUSH LOGS")
assert node.contains_in_log("CREATE USER u1")
assert node.contains_in_log("ALTER USER u1")
assert node.contains_in_log("CREATE USER u2")
assert not node.contains_in_log("qwe123")
assert not node.contains_in_log("123qwe")
assert not node.contains_in_log("plainpasswd")
assert not node.contains_in_log("IDENTIFIED WITH sha256_password BY")
assert not node.contains_in_log("IDENTIFIED WITH sha256_hash BY")
assert not node.contains_in_log("IDENTIFIED WITH plaintext_password BY")
assert (
int(
node.query(
"SELECT COUNT() FROM system.query_log WHERE query LIKE 'CREATE USER u1%IDENTIFIED WITH sha256_password%'"
).strip()
)
>= 1
)
assert (
int(
node.query(
"SELECT COUNT() FROM system.query_log WHERE query LIKE 'CREATE USER u1%IDENTIFIED WITH sha256_password BY%'"
).strip()
)
== 0
)
assert (
int(
node.query(
"SELECT COUNT() FROM system.query_log WHERE query LIKE 'ALTER USER u1%IDENTIFIED WITH sha256_password%'"
).strip()
)
>= 1
)
assert (
int(
node.query(
"SELECT COUNT() FROM system.query_log WHERE query LIKE 'ALTER USER u1%IDENTIFIED WITH sha256_password BY%'"
).strip()
)
== 0
)
assert (
int(
node.query(
"SELECT COUNT() FROM system.query_log WHERE query LIKE 'CREATE USER u2%IDENTIFIED WITH plaintext_password%'"
).strip()
)
>= 1
)
assert (
int(
node.query(
"SELECT COUNT() FROM system.query_log WHERE query LIKE 'CREATE USER u2%IDENTIFIED WITH plaintext_password BY%'"
).strip()
)
== 0
check_logs(
must_contain=[
"CREATE USER u1 IDENTIFIED WITH sha256_password",
"ALTER USER u1 IDENTIFIED WITH sha256_password",
"CREATE USER u2 IDENTIFIED WITH plaintext_password",
],
must_not_contain=[
"qwe123",
"123qwe",
"plainpasswd",
"IDENTIFIED WITH sha256_password BY",
"IDENTIFIED WITH sha256_hash BY",
"IDENTIFIED WITH plaintext_password BY",
],
)

View File

@ -1 +1 @@
CREATE USER user IDENTIFIED WITH plaintext_password BY 'hello'
CREATE USER user IDENTIFIED WITH plaintext_password BY 'hello'