Merge pull request #43227 from vitlibar/improve-masking-sensitive-info

Improve masking sensitive info
This commit is contained in:
Vitaly Baranov 2022-11-18 15:37:50 +01:00 committed by GitHub
commit a348332eab
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
43 changed files with 733 additions and 746 deletions

View File

@ -45,7 +45,7 @@ TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfigurati
engine_push_str = config.getString(table_prefix + "engine", "rand()");
{
ParserStorage parser_storage;
ParserStorage parser_storage{ParserStorage::TABLE_ENGINE};
engine_push_ast = parseQuery(parser_storage, engine_push_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
engine_push_partition_key_ast = extractPartitionKey(engine_push_ast);
primary_key_comma_separated = boost::algorithm::join(extractPrimaryKeyColumnNames(engine_push_ast), ", ");

View File

@ -166,7 +166,12 @@ ASTPtr FunctionNode::toASTImpl() const
auto function_ast = std::make_shared<ASTFunction>();
function_ast->name = function_name;
function_ast->is_window_function = isWindowFunction();
if (isWindowFunction())
{
function_ast->is_window_function = true;
function_ast->kind = ASTFunction::Kind::WINDOW_FUNCTION;
}
const auto & parameters = getParameters();
if (!parameters.getNodes().empty())

View File

@ -6,7 +6,6 @@
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Interpreters/maskSensitiveInfoInQueryForLogging.h>
namespace DB
@ -36,6 +35,7 @@ ASTPtr BackupInfo::toAST() const
auto func = std::make_shared<ASTFunction>();
func->name = backup_engine_name;
func->no_empty_args = true;
func->kind = ASTFunction::Kind::BACKUP_NAME;
auto list = std::make_shared<ASTExpressionList>();
func->arguments = list;
@ -93,10 +93,9 @@ BackupInfo BackupInfo::fromAST(const IAST & ast)
}
String BackupInfo::toStringForLogging(const ContextPtr & context) const
String BackupInfo::toStringForLogging() const
{
ASTPtr ast = toAST();
return maskSensitiveInfoInBackupNameForLogging(serializeAST(*ast), ast, context);
return toAST()->formatForLogging();
}
}

View File

@ -22,7 +22,7 @@ struct BackupInfo
ASTPtr toAST() const;
static BackupInfo fromAST(const IAST & ast);
String toStringForLogging(const ContextPtr & context) const;
String toStringForLogging() const;
};
}

View File

@ -2,8 +2,8 @@
#include <Backups/BackupSettings.h>
#include <Core/SettingsFields.h>
#include <Parsers/ASTBackupQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTLiteral.h>
#include <IO/ReadHelpers.h>
@ -126,7 +126,12 @@ void BackupSettings::copySettingsToQuery(ASTBackupQuery & query) const
query.settings = query_settings;
query.base_backup_name = base_backup_info ? base_backup_info->toAST() : nullptr;
auto base_backup_name = base_backup_info ? base_backup_info->toAST() : nullptr;
if (base_backup_name)
query.setOrReplace(query.base_backup_name, base_backup_name);
else
query.reset(query.base_backup_name);
query.cluster_host_ids = !cluster_host_ids.empty() ? Util::clusterHostIDsToAST(cluster_host_ids) : nullptr;
}

View File

@ -16,6 +16,7 @@
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Parsers/ASTBackupQuery.h>
#include <Parsers/ASTFunction.h>
#include <Common/Exception.h>
#include <Common/Macros.h>
#include <Common/logger_useful.h>
@ -166,7 +167,7 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context
}
auto backup_info = BackupInfo::fromAST(*backup_query->backup_name);
String backup_name_for_logging = backup_info.toStringForLogging(context);
String backup_name_for_logging = backup_info.toStringForLogging();
try
{
addInfo(backup_id, backup_name_for_logging, backup_settings.internal, BackupStatus::CREATING_BACKUP);
@ -388,7 +389,7 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
try
{
auto backup_info = BackupInfo::fromAST(*restore_query->backup_name);
String backup_name_for_logging = backup_info.toStringForLogging(context);
String backup_name_for_logging = backup_info.toStringForLogging();
addInfo(restore_id, backup_name_for_logging, restore_settings.internal, BackupStatus::RESTORING);
/// Prepare context to use.

View File

@ -3,6 +3,7 @@
#include <Backups/RestoreSettings.h>
#include <Core/SettingsFields.h>
#include <Parsers/ASTBackupQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSetQuery.h>
#include <boost/algorithm/string/predicate.hpp>
#include <Common/FieldVisitorConvertToNumber.h>
@ -213,7 +214,12 @@ void RestoreSettings::copySettingsToQuery(ASTBackupQuery & query) const
query.settings = query_settings;
query.base_backup_name = base_backup_info ? base_backup_info->toAST() : nullptr;
auto base_backup_name = base_backup_info ? base_backup_info->toAST() : nullptr;
if (base_backup_name)
query.setOrReplace(query.base_backup_name, base_backup_name);
else
query.reset(query.base_backup_name);
query.cluster_host_ids = !cluster_host_ids.empty() ? BackupSettings::Util::clusterHostIDsToAST(cluster_host_ids) : nullptr;
}

View File

@ -47,7 +47,7 @@ void registerBackupEngineS3(BackupFactory & factory)
auto creator_fn = []([[maybe_unused]] const BackupFactory::CreateParams & params) -> std::unique_ptr<IBackup>
{
#if USE_AWS_S3
String backup_name_for_logging = params.backup_info.toStringForLogging(params.context);
String backup_name_for_logging = params.backup_info.toStringForLogging();
const String & id_arg = params.backup_info.id_arg;
const auto & args = params.backup_info.args;

View File

@ -99,7 +99,7 @@ void registerBackupEnginesFileAndDisk(BackupFactory & factory)
{
auto creator_fn = [](const BackupFactory::CreateParams & params) -> std::unique_ptr<IBackup>
{
String backup_name_for_logging = params.backup_info.toStringForLogging(params.context);
String backup_name_for_logging = params.backup_info.toStringForLogging();
const String & engine_name = params.backup_info.backup_engine_name;
if (!params.backup_info.id_arg.empty())

View File

@ -0,0 +1,48 @@
#include <Common/KnownObjectNames.h>
#include <Poco/String.h>
namespace DB
{
bool KnownObjectNames::exists(const String & name) const
{
std::lock_guard lock{mutex};
if (names.contains(name))
return true;
if (!case_insensitive_names.empty())
{
String lower_name = Poco::toLower(name);
if (case_insensitive_names.contains(lower_name))
return true;
}
return false;
}
void KnownObjectNames::add(const String & name, bool case_insensitive)
{
std::lock_guard lock{mutex};
if (case_insensitive)
case_insensitive_names.emplace(Poco::toLower(name));
else
names.emplace(name);
}
KnownTableFunctionNames & KnownTableFunctionNames::instance()
{
static KnownTableFunctionNames the_instance;
return the_instance;
}
KnownFormatNames & KnownFormatNames::instance()
{
static KnownFormatNames the_instance;
return the_instance;
}
}

View File

@ -0,0 +1,37 @@
#pragma once
#include <base/types.h>
#include <mutex>
#include <unordered_set>
namespace DB
{
class KnownObjectNames
{
public:
bool exists(const String & name) const;
void add(const String & name, bool case_insensitive = false);
private:
mutable std::mutex mutex;
std::unordered_set<String> names;
std::unordered_set<String> case_insensitive_names;
};
class KnownTableFunctionNames : public KnownObjectNames
{
public:
static KnownTableFunctionNames & instance();
};
class KnownFormatNames : public KnownObjectNames
{
public:
static KnownFormatNames & instance();
};
}

View File

@ -13,12 +13,19 @@
#include <Common/Exception.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/ProfileEvents.h>
#ifndef NDEBUG
# include <iostream>
#endif
namespace ProfileEvents
{
extern const Event QueryMaskingRulesMatch;
}
namespace DB
{
namespace ErrorCodes
@ -165,6 +172,10 @@ size_t SensitiveDataMasker::wipeSensitiveData(std::string & data) const
size_t matches = 0;
for (const auto & rule : all_masking_rules)
matches += rule->apply(data);
if (matches)
ProfileEvents::increment(ProfileEvents::QueryMaskingRulesMatch, matches);
return matches;
}
@ -184,4 +195,18 @@ size_t SensitiveDataMasker::rulesCount() const
return all_masking_rules.size();
}
std::string wipeSensitiveDataAndCutToLength(const std::string & str, size_t max_length)
{
std::string res = str;
if (auto * masker = SensitiveDataMasker::getInstance())
masker->wipeSensitiveData(res);
if (max_length && (res.length() > max_length))
res.resize(max_length);
return res;
}
}

View File

@ -69,4 +69,8 @@ public:
size_t rulesCount() const;
};
/// Wipes sensitive data and cuts to a specified maximum length in one function call.
/// If the maximum length is zero then the function doesn't cut to the maximum length.
std::string wipeSensitiveDataAndCutToLength(const std::string & str, size_t max_length);
}

View File

@ -13,6 +13,7 @@
#include <Processors/Formats/Impl/ValuesBlockInputFormat.h>
#include <Poco/URI.h>
#include <Common/Exception.h>
#include <Common/KnownObjectNames.h>
#include <fcntl.h>
#include <unistd.h>
@ -445,6 +446,7 @@ void FormatFactory::registerInputFormat(const String & name, InputCreator input_
throw Exception("FormatFactory: Input format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
target = std::move(input_creator);
registerFileExtension(name, name);
KnownFormatNames::instance().add(name);
}
void FormatFactory::registerNonTrivialPrefixAndSuffixChecker(const String & name, NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker)
@ -483,6 +485,7 @@ void FormatFactory::registerOutputFormat(const String & name, OutputCreator outp
throw Exception("FormatFactory: Output format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
target = std::move(output_creator);
registerFileExtension(name, name);
KnownFormatNames::instance().add(name);
}
void FormatFactory::registerFileExtension(const String & extension, const String & format_name)

View File

@ -32,13 +32,8 @@ QueryPipeline InterpreterShowAccessQuery::executeImpl() const
/// Build the result column.
MutableColumnPtr column = ColumnString::create();
WriteBufferFromOwnString buf;
for (const auto & query : queries)
{
buf.restart();
formatAST(*query, buf, false, true);
column->insert(buf.str());
}
column->insert(query->formatWithSecretsHidden());
String desc = "ACCESS";
return QueryPipeline(std::make_shared<SourceFromSingleChunk>(Block{{std::move(column), std::make_shared<DataTypeString>(), desc}}));

View File

@ -62,10 +62,7 @@ namespace
}
if (user.auth_data.getType() != AuthenticationType::NO_PASSWORD)
{
query->auth_data = user.auth_data;
query->show_password = attach_mode; /// We don't show password unless it's an ATTACH statement.
}
if (!user.settings.empty())
{
@ -256,19 +253,12 @@ QueryPipeline InterpreterShowCreateAccessEntityQuery::executeImpl()
/// Build the result column.
MutableColumnPtr column = ColumnString::create();
WriteBufferFromOwnString create_query_buf;
for (const auto & create_query : create_queries)
{
formatAST(*create_query, create_query_buf, false, true);
column->insert(create_query_buf.str());
create_query_buf.restart();
}
column->insert(create_query->formatWithSecretsHidden());
/// Prepare description of the result column.
WriteBufferFromOwnString desc_buf;
const auto & show_query = query_ptr->as<const ASTShowCreateAccessEntityQuery &>();
formatAST(show_query, desc_buf, false, true);
String desc = desc_buf.str();
String desc = serializeAST(show_query);
String prefix = "SHOW ";
if (startsWith(desc, prefix))
desc = desc.substr(prefix.length()); /// `desc` always starts with "SHOW ", so we can trim this prefix.

View File

@ -15,7 +15,6 @@
#include <Parsers/queryToString.h>
#include <Parsers/ASTQueryWithTableAndOutput.h>
#include <Databases/DatabaseReplicated.h>
#include <Interpreters/maskSensitiveInfoInQueryForLogging.h>
namespace DB
@ -174,7 +173,7 @@ void DDLTaskBase::formatRewrittenQuery(ContextPtr context)
{
/// Convert rewritten AST back to string.
query_str = queryToString(*query);
query_for_logging = maskSensitiveInfoInQueryForLogging(query_str, query, context);
query_for_logging = query->formatForLogging(context->getSettingsRef().log_queries_cut_to_length);
}
ContextMutablePtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & /*zookeeper*/)

View File

@ -92,9 +92,7 @@ QueryPipeline InterpreterShowCreateQuery::executeImpl()
create.to_inner_uuid = UUIDHelpers::Nil;
}
WriteBufferFromOwnString buf;
formatAST(*create_query, buf, false, false);
String res = buf.str();
String res = create_query->formatWithSecretsHidden(/* max_length= */ 0, /* one_line= */ false);
MutableColumnPtr column = ColumnString::create();
column->insert(res);

View File

@ -75,7 +75,7 @@ namespace
const char * getName() const override { return "storage definition with comment"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
ParserStorage storage_p;
ParserStorage storage_p{ParserStorage::TABLE_ENGINE};
ASTPtr storage;
if (!storage_p.parse(pos, storage, expected))

View File

@ -3,6 +3,7 @@
#include <Common/typeid_cast.h>
#include <Common/ThreadProfileEvents.h>
#include <Common/MemoryTrackerBlockerInThread.h>
#include <Common/SensitiveDataMasker.h>
#include <Interpreters/AsynchronousInsertQueue.h>
#include <IO/WriteBufferFromFile.h>
@ -55,7 +56,6 @@
#include <Interpreters/SelectQueryOptions.h>
#include <Interpreters/TransactionLog.h>
#include <Interpreters/executeQuery.h>
#include <Interpreters/maskSensitiveInfoInQueryForLogging.h>
#include <Common/ProfileEvents.h>
#include <IO/CompressionMethod.h>
@ -352,6 +352,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
ASTPtr ast;
String query;
String query_for_logging;
size_t log_queries_cut_to_length = context->getSettingsRef().log_queries_cut_to_length;
/// Parse the query from string.
try
@ -392,15 +393,23 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
/// 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 = maskSensitiveInfoInQueryForLogging(query, ast, context);
if (ast->hasSecretParts())
{
/// IAST::formatForLogging() wipes secret parts in AST and then calls wipeSensitiveDataAndCutToLength().
query_for_logging = ast->formatForLogging(log_queries_cut_to_length);
}
else
{
query_for_logging = wipeSensitiveDataAndCutToLength(query, log_queries_cut_to_length);
}
}
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 = maskSensitiveInfoInQueryForLogging(query, ast, context);
query_for_logging = wipeSensitiveDataAndCutToLength(query, log_queries_cut_to_length);
logQuery(query_for_logging, context, internal, stage);
if (!internal)

View File

@ -1,623 +0,0 @@
#include <Interpreters/maskSensitiveInfoInQueryForLogging.h>
#include <Formats/FormatFactory.h>
#include <Interpreters/Context.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTBackupQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/Access/ASTCreateUserQuery.h>
#include <Parsers/formatAST.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/ProfileEvents.h>
#include <Common/SensitiveDataMasker.h>
#include <Common/typeid_cast.h>
namespace ProfileEvents
{
extern const Event QueryMaskingRulesMatch;
}
namespace DB
{
namespace
{
enum class PasswordWipingMode
{
Query,
BackupName,
};
template <bool check_only>
class PasswordWipingVisitor
{
public:
struct Data
{
bool can_contain_password = false;
bool password_was_hidden = false;
bool is_create_table_query = false;
bool is_create_database_query = false;
bool is_create_dictionary_query = false;
ContextPtr context;
PasswordWipingMode mode = PasswordWipingMode::Query;
};
using Visitor = std::conditional_t<
check_only,
ConstInDepthNodeVisitor<PasswordWipingVisitor, /* top_to_bottom= */ true, /* need_child_accept_data= */ true>,
InDepthNodeVisitor<PasswordWipingVisitor, /* top_to_bottom= */ true, /* need_child_accept_data= */ true>>;
static bool needChildVisit(const ASTPtr & /* ast */, const ASTPtr & /* child */, Data & data)
{
if constexpr (check_only)
{
return !data.can_contain_password;
}
else
{
return true;
}
}
static void visit(ASTPtr ast, Data & data)
{
if (auto * create_user_query = ast->as<ASTCreateUserQuery>())
{
visitCreateUserQuery(*create_user_query, data);
}
else if (auto * create_query = ast->as<ASTCreateQuery>())
{
visitCreateQuery(*create_query, data);
}
else if (auto * backup_query = ast->as<ASTBackupQuery>())
{
visitBackupQuery(*backup_query, data);
}
else if (auto * storage = ast->as<ASTStorage>())
{
if (data.is_create_table_query)
visitTableEngine(*storage, data);
else if (data.is_create_database_query)
visitDatabaseEngine(*storage, data);
}
else if (auto * dictionary = ast->as<ASTDictionary>())
{
if (data.is_create_dictionary_query)
visitDictionaryDef(*dictionary, data);
}
else if (auto * function = ast->as<ASTFunction>())
{
if (data.mode == PasswordWipingMode::BackupName)
wipePasswordFromBackupEngineArguments(*function, data);
else
visitFunction(*function, data);
}
}
private:
static void visitCreateUserQuery(ASTCreateUserQuery & query, Data & data)
{
if (!query.auth_data)
return;
auto auth_type = query.auth_data->getType();
if (auth_type == AuthenticationType::NO_PASSWORD || auth_type == AuthenticationType::LDAP
|| auth_type == AuthenticationType::KERBEROS || auth_type == AuthenticationType::SSL_CERTIFICATE)
return; /// No password, nothing to hide.
if constexpr (check_only)
{
data.can_contain_password = true;
return;
}
query.show_password = false;
data.password_was_hidden = true;
}
static void visitCreateQuery(ASTCreateQuery & query, Data & data)
{
if (query.is_dictionary)
data.is_create_dictionary_query = true;
else if (query.table)
data.is_create_table_query = true;
else if (query.database)
data.is_create_database_query = true;
}
static void visitTableEngine(ASTStorage & storage, Data & data)
{
if (!storage.engine)
return;
const String & engine_name = storage.engine->name;
if (engine_name == "ExternalDistributed")
{
/// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password')
wipePasswordFromArgument(*storage.engine, data, 5);
}
else if (engine_name == "MySQL")
{
/// MySQL('host:port', 'database', 'table', 'user', 'password', ...)
wipePasswordFromArgument(*storage.engine, data, 4);
}
else if (engine_name == "PostgreSQL")
{
/// PostgreSQL('host:port', 'database', 'table', 'user', 'password', ...)
wipePasswordFromArgument(*storage.engine, data, 4);
}
else if (engine_name == "MaterializedPostgreSQL")
{
/// MaterializedPostgreSQL('host:port', 'database', 'table', 'user', 'password', ...)
wipePasswordFromArgument(*storage.engine, data, 4);
}
else if (engine_name == "MongoDB")
{
/// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...)
wipePasswordFromArgument(*storage.engine, data, 4);
}
else if (engine_name == "S3" || engine_name == "COSN" || engine_name == "OSS")
{
/// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...)
wipePasswordFromS3TableEngineArguments(*storage.engine, data);
}
}
static void wipePasswordFromS3TableEngineArguments(ASTFunction & engine, Data & data)
{
/// We replace 'aws_secret_access_key' with '[HIDDEN'] for the following signatures:
/// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format')
/// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression')
/// But we should check the number of arguments first because we don't need to do that replacements in case of
/// S3('url' [, 'format' [, 'compression']])
size_t num_arguments;
if (!tryGetNumArguments(engine, &num_arguments) || (num_arguments < 4))
return;
wipePasswordFromArgument(engine, data, 2);
}
static void visitDatabaseEngine(ASTStorage & storage, Data & data)
{
if (!storage.engine)
return;
const String & engine_name = storage.engine->name;
if (engine_name == "MySQL" || engine_name == "MaterializeMySQL" || engine_name == "MaterializedMySQL")
{
/// MySQL('host:port', 'database', 'user', 'password')
wipePasswordFromArgument(*storage.engine, data, 3);
}
else if (engine_name == "PostgreSQL" || engine_name == "MaterializedPostgreSQL")
{
/// PostgreSQL('host:port', 'database', 'user', 'password', ...)
wipePasswordFromArgument(*storage.engine, data, 3);
}
}
static void visitFunction(ASTFunction & function, Data & data)
{
if (function.name == "mysql")
{
/// mysql('host:port', 'database', 'table', 'user', 'password', ...)
wipePasswordFromArgument(function, data, 4);
}
else if (function.name == "postgresql")
{
/// postgresql('host:port', 'database', 'table', 'user', 'password', ...)
wipePasswordFromArgument(function, data, 4);
}
else if (function.name == "mongodb")
{
/// mongodb('host:port', 'database', 'collection', 'user', 'password', ...)
wipePasswordFromArgument(function, data, 4);
}
else if (function.name == "s3" || function.name == "cosn" || function.name == "oss")
{
/// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...)
wipePasswordFromS3FunctionArguments(function, data, /* is_cluster_function= */ false);
}
else if (function.name == "s3Cluster")
{
/// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...)
wipePasswordFromS3FunctionArguments(function, data, /* is_cluster_function= */ true);
}
else if (function.name == "remote" || function.name == "remoteSecure")
{
/// remote('addresses_expr', 'db', 'table', 'user', 'password', ...)
wipePasswordFromRemoteFunctionArguments(function, data);
}
else if (
function.name == "encrypt" || function.name == "decrypt" || function.name == "aes_encrypt_mysql"
|| function.name == "aes_decrypt_mysql" || function.name == "tryDecrypt")
{
/// encrypt('mode', 'plaintext', 'key' [, iv, aad])
wipePasswordFromEncryptionFunctionArguments(function, data);
}
}
static void wipePasswordFromS3FunctionArguments(ASTFunction & function, Data & data, bool is_cluster_function)
{
/// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument.
size_t url_arg_idx = is_cluster_function ? 1 : 0;
/// We're going to replace 'aws_secret_access_key' with '[HIDDEN'] for the following signatures:
/// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...)
/// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression')
/// But we should check the number of arguments first because we don't need to do any replacements in case of
/// s3('url' [, 'format']) or s3Cluster('cluster_name', 'url' [, 'format'])
size_t num_arguments;
if (!tryGetNumArguments(function, &num_arguments) || (num_arguments < url_arg_idx + 3))
return;
if (num_arguments >= url_arg_idx + 5)
{
/// s3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'structure', ...)
wipePasswordFromArgument(function, data, url_arg_idx + 2);
}
else
{
/// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...)
/// We need to distinguish that from s3('url', 'format', 'structure' [, 'compression_method']).
/// So we will check whether the argument after 'url' is a format.
String format;
if (!tryGetEvaluatedConstStringFromArgument(function, data, url_arg_idx + 1, &format))
return;
if (FormatFactory::instance().getAllFormats().contains(format))
return; /// The argument after 'url' is a format: s3('url', 'format', ...)
/// The argument after 'url' is not a format so we do our replacement:
/// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) -> s3('url', 'aws_access_key_id', '[HIDDEN]', ...)
wipePasswordFromArgument(function, data, url_arg_idx + 2);
}
}
static void wipePasswordFromRemoteFunctionArguments(ASTFunction & function, Data & data)
{
/// We're going to replace 'password' with '[HIDDEN'] for the following signatures:
/// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key])
/// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key])
/// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key])
/// But we should check the number of arguments first because we don't need to do any replacements in case of
/// remote('addresses_expr', db.table)
size_t num_arguments;
if (!tryGetNumArguments(function, &num_arguments) || (num_arguments < 3))
return;
auto & arguments = assert_cast<ASTExpressionList &>(*function.arguments).children;
size_t arg_num = 1;
/// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'.
const auto * table_function = arguments[arg_num]->as<ASTFunction>();
if (table_function && TableFunctionFactory::instance().isTableFunctionName(table_function->name))
{
++arg_num;
}
else
{
String database;
if (!tryGetEvaluatedConstDatabaseNameFromArgument(function, data, arg_num, &database))
return;
++arg_num;
auto qualified_name = QualifiedTableName::parseFromString(database);
if (qualified_name.database.empty())
++arg_num; /// skip 'table' argument
}
/// Check if username and password are specified
/// (sharding_key can be of any type so while we're getting string literals they're username & password).
String username, password;
bool username_specified = tryGetStringFromArgument(function, arg_num, &username);
bool password_specified = username_specified && tryGetStringFromArgument(function, arg_num + 1, &password);
if (password_specified)
{
/// Password is specified so we do our replacement:
/// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...)
wipePasswordFromArgument(function, data, arg_num + 1);
}
}
static void wipePasswordFromEncryptionFunctionArguments(ASTFunction & function, Data & data)
{
/// We replace all arguments after 'mode' with '[HIDDEN]':
/// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]')
wipePasswordFromArgument(function, data, 1);
removeArgumentsAfter(function, data, 2);
}
static void visitBackupQuery(ASTBackupQuery & query, Data & data)
{
if (query.backup_name)
{
if (auto * backup_engine = query.backup_name->as<ASTFunction>())
wipePasswordFromBackupEngineArguments(*backup_engine, data);
}
if (query.base_backup_name)
{
if (auto * base_backup_engine = query.base_backup_name->as<ASTFunction>())
wipePasswordFromBackupEngineArguments(*base_backup_engine, data);
}
}
static void wipePasswordFromBackupEngineArguments(ASTFunction & engine, Data & data)
{
if (engine.name == "S3")
{
/// BACKUP ... TO S3(url, [aws_access_key_id, aws_secret_access_key])
wipePasswordFromArgument(engine, data, 2);
}
}
static void wipePasswordFromArgument(ASTFunction & function, Data & data, size_t arg_idx)
{
if (!function.arguments)
return;
auto * expr_list = function.arguments->as<ASTExpressionList>();
if (!expr_list)
return; /// return because we don't want to validate query here
auto & arguments = expr_list->children;
if (arg_idx >= arguments.size())
return;
if constexpr (check_only)
{
data.can_contain_password = true;
return;
}
arguments[arg_idx] = std::make_shared<ASTLiteral>("[HIDDEN]");
data.password_was_hidden = true;
}
static void removeArgumentsAfter(ASTFunction & function, Data & data, size_t new_num_arguments)
{
if (!function.arguments)
return;
auto * expr_list = function.arguments->as<ASTExpressionList>();
if (!expr_list)
return; /// return because we don't want to validate query here
auto & arguments = expr_list->children;
if (new_num_arguments >= arguments.size())
return;
if constexpr (check_only)
{
data.can_contain_password = true;
return;
}
arguments.resize(new_num_arguments);
data.password_was_hidden = true;
}
static bool tryGetNumArguments(const ASTFunction & function, size_t * num_arguments)
{
if (!function.arguments)
return false;
auto * expr_list = function.arguments->as<ASTExpressionList>();
if (!expr_list)
return false; /// return false because we don't want to validate query here
const auto & arguments = expr_list->children;
*num_arguments = arguments.size();
return true;
}
static bool tryGetStringFromArgument(const ASTFunction & function, size_t arg_idx, String * value)
{
if (!function.arguments)
return false;
const auto * expr_list = function.arguments->as<ASTExpressionList>();
if (!expr_list)
return false; /// return false because we don't want to validate query here
const auto & arguments = expr_list->children;
if (arg_idx >= arguments.size())
return false;
const auto * literal = arguments[arg_idx]->as<ASTLiteral>();
if (!literal || literal->value.getType() != Field::Types::String)
return false;
*value = literal->value.safeGet<String>();
return true;
}
static bool tryGetEvaluatedConstStringFromArgument(const ASTFunction & function, Data & data, size_t arg_idx, String * value)
{
if (!function.arguments)
return false;
const auto * expr_list = function.arguments->as<ASTExpressionList>();
if (!expr_list)
return false; /// return false because we don't want to validate query here
const auto & arguments = expr_list->children;
if (arg_idx >= arguments.size())
return false;
if constexpr (check_only)
{
data.can_contain_password = true;
return false;
}
ASTPtr argument = arguments[arg_idx];
try
{
argument = evaluateConstantExpressionOrIdentifierAsLiteral(argument, data.context);
}
catch (...)
{
return false;
}
const auto & literal = assert_cast<const ASTLiteral &>(*argument);
if (literal.value.getType() != Field::Types::String)
return false;
*value = literal.value.safeGet<String>();
return true;
}
static bool tryGetEvaluatedConstDatabaseNameFromArgument(const ASTFunction & function, Data & data, size_t arg_idx, String * value)
{
if (!function.arguments)
return false;
const auto * expr_list = function.arguments->as<ASTExpressionList>();
if (!expr_list)
return false; /// return false because we don't want to validate query here
const auto & arguments = expr_list->children;
if (arg_idx >= arguments.size())
return false;
if constexpr (check_only)
{
data.can_contain_password = true;
return false;
}
ASTPtr argument = arguments[arg_idx];
try
{
argument = evaluateConstantExpressionForDatabaseName(argument, data.context);
}
catch (...)
{
return false;
}
const auto & literal = assert_cast<const ASTLiteral &>(*argument);
if (literal.value.getType() != Field::Types::String)
return false;
*value = literal.value.safeGet<String>();
return true;
}
static void visitDictionaryDef(ASTDictionary & dictionary, Data & data)
{
if (!dictionary.source || !dictionary.source->elements)
return;
const auto * expr_list = dictionary.source->elements->as<ASTExpressionList>();
if (!expr_list)
return; /// return because we don't want to validate query here
const auto & elements = expr_list->children;
/// We replace password in the dictionary's definition:
/// SOURCE(CLICKHOUSE(host 'example01-01-1' port 9000 user 'default' password 'qwe123' db 'default' table 'ids')) ->
/// SOURCE(CLICKHOUSE(host 'example01-01-1' port 9000 user 'default' password '[HIDDEN]' db 'default' table 'ids'))
for (const auto & element : elements)
{
auto * pair = element->as<ASTPair>();
if (!pair)
continue; /// just skip because we don't want to validate query here
if (pair->first == "password")
{
if constexpr (check_only)
{
data.can_contain_password = true;
return;
}
pair->set(pair->second, std::make_shared<ASTLiteral>("[HIDDEN]"));
data.password_was_hidden = true;
}
}
}
};
/// Checks the type of a specified AST and returns true if it can contain a password.
bool canContainPassword(const IAST & ast, PasswordWipingMode mode)
{
using WipingVisitor = PasswordWipingVisitor</*check_only= */ true>;
WipingVisitor::Data data;
data.mode = mode;
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).
bool wipePasswordFromQuery(ASTPtr ast, PasswordWipingMode mode, const ContextPtr & context)
{
using WipingVisitor = PasswordWipingVisitor</*check_only= */ false>;
WipingVisitor::Data data;
data.context = context;
data.mode = mode;
WipingVisitor::Visitor visitor{data};
visitor.visit(ast);
return data.password_was_hidden;
}
/// Common utility for masking sensitive information.
String maskSensitiveInfoImpl(const String & query, const ASTPtr & parsed_query, PasswordWipingMode mode, const ContextPtr & context)
{
String res = query;
// Wiping a password or hash from the query because we don't want it to go to logs.
if (parsed_query && canContainPassword(*parsed_query, mode))
{
ASTPtr ast_without_password = parsed_query->clone();
if (wipePasswordFromQuery(ast_without_password, mode, context))
res = serializeAST(*ast_without_password);
}
// 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;
}
}
String maskSensitiveInfoInQueryForLogging(const String & query, const ASTPtr & parsed_query, const ContextPtr & context)
{
return maskSensitiveInfoImpl(query, parsed_query, PasswordWipingMode::Query, context);
}
String maskSensitiveInfoInBackupNameForLogging(const String & backup_name, const ASTPtr & ast, const ContextPtr & context)
{
return maskSensitiveInfoImpl(backup_name, ast, PasswordWipingMode::BackupName, context);
}
}

View File

@ -1,19 +0,0 @@
#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.
/// Does not validate AST, works a best-effort way.
String maskSensitiveInfoInQueryForLogging(const String & query, const ASTPtr & parsed_query, const ContextPtr & context);
/// Makes a version of backup name without sensitive information (e.g. passwords) for logging.
/// Does not validate AST, works a best-effort way.
String maskSensitiveInfoInBackupNameForLogging(const String & backup_name, const ASTPtr & ast, const ContextPtr & context);
}

View File

@ -1,4 +1,5 @@
#include <Parsers/ASTBackupQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSetQuery.h>
#include <IO/Operators.h>
#include <Common/assert_cast.h>
@ -141,7 +142,7 @@ namespace
}
}
void formatSettings(const ASTPtr & settings, const ASTPtr & base_backup_name, const ASTPtr & cluster_host_ids, const IAST::FormatSettings & format)
void formatSettings(const ASTPtr & settings, const ASTFunction * base_backup_name, const ASTPtr & cluster_host_ids, const IAST::FormatSettings & format)
{
if (!settings && !base_backup_name && !cluster_host_ids)
return;
@ -246,12 +247,13 @@ String ASTBackupQuery::getID(char) const
ASTPtr ASTBackupQuery::clone() const
{
auto res = std::make_shared<ASTBackupQuery>(*this);
res->children.clear();
if (backup_name)
res->backup_name = backup_name->clone();
res->set(res->backup_name, backup_name->clone());
if (base_backup_name)
res->base_backup_name = base_backup_name->clone();
res->set(res->base_backup_name, base_backup_name->clone());
if (cluster_host_ids)
res->cluster_host_ids = cluster_host_ids->clone();

View File

@ -8,6 +8,7 @@ namespace DB
{
using Strings = std::vector<String>;
using DatabaseAndTableName = std::pair<String, String>;
class ASTFunction;
/** BACKUP { TABLE [db.]table_name [AS [db.]table_name_in_backup] [PARTITION[S] partition_expr [,...]] |
@ -77,13 +78,13 @@ public:
Elements elements;
ASTPtr backup_name;
ASTFunction * backup_name = nullptr;
ASTPtr settings;
/// Base backup. Only differences made after the base backup will be included in a newly created backup,
/// so this setting allows to make an incremental backup.
ASTPtr base_backup_name;
ASTFunction * base_backup_name = nullptr;
/// List of cluster's hosts' IDs if this is a BACKUP/RESTORE ON CLUSTER command.
ASTPtr cluster_host_ids;

View File

@ -4,16 +4,21 @@
#include <Common/quoteString.h>
#include <Common/FieldVisitorToString.h>
#include <Common/KnownObjectNames.h>
#include <Common/SipHash.h>
#include <Common/typeid_cast.h>
#include <IO/Operators.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/queryToString.h>
#include <Parsers/ASTSetQuery.h>
#include <Core/QualifiedTableName.h>
using namespace std::literals;
@ -27,6 +32,338 @@ namespace ErrorCodes
extern const int UNEXPECTED_AST_STRUCTURE;
}
namespace
{
/// Finds arguments of a specified function which should not be displayed for most users for security reasons.
/// That involves passwords and secret keys.
/// The member function getRange() returns a pair of numbers [first, last) specifying arguments
/// which must be hidden. If the function returns {-1, -1} that means no arguments must be hidden.
class FunctionSecretArgumentsFinder
{
public:
explicit FunctionSecretArgumentsFinder(const ASTFunction & function_) : function(function_)
{
if (function.arguments)
{
if (const auto * expr_list = function.arguments->as<ASTExpressionList>())
arguments = &expr_list->children;
}
}
std::pair<size_t, size_t> getRange() const
{
if (!arguments)
return npos;
switch (function.kind)
{
case ASTFunction::Kind::ORDINARY_FUNCTION: return findOrdinaryFunctionSecretArguments();
case ASTFunction::Kind::WINDOW_FUNCTION: return npos;
case ASTFunction::Kind::LAMBDA_FUNCTION: return npos;
case ASTFunction::Kind::TABLE_ENGINE: return findTableEngineSecretArguments();
case ASTFunction::Kind::DATABASE_ENGINE: return findDatabaseEngineSecretArguments();
case ASTFunction::Kind::BACKUP_NAME: return findBackupNameSecretArguments();
}
}
static const constexpr std::pair<size_t, size_t> npos{static_cast<size_t>(-1), static_cast<size_t>(-1)};
private:
std::pair<size_t, size_t> findOrdinaryFunctionSecretArguments() const
{
if ((function.name == "mysql") || (function.name == "postgresql") || (function.name == "mongodb"))
{
/// mysql('host:port', 'database', 'table', 'user', 'password', ...)
/// postgresql('host:port', 'database', 'table', 'user', 'password', ...)
/// mongodb('host:port', 'database', 'collection', 'user', 'password', ...)
return {4, 5};
}
else if ((function.name == "s3") || (function.name == "cosn") || (function.name == "oss"))
{
/// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...)
return findS3FunctionSecretArguments(/* is_cluster_function= */ false);
}
else if (function.name == "s3Cluster")
{
/// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...)
return findS3FunctionSecretArguments(/* is_cluster_function= */ true);
}
else if ((function.name == "remote") || (function.name == "remoteSecure"))
{
/// remote('addresses_expr', 'db', 'table', 'user', 'password', ...)
return findRemoteFunctionSecretArguments();
}
else if ((function.name == "encrypt") || (function.name == "decrypt") ||
(function.name == "aes_encrypt_mysql") || (function.name == "aes_decrypt_mysql") ||
(function.name == "tryDecrypt"))
{
/// encrypt('mode', 'plaintext', 'key' [, iv, aad])
return findEncryptionFunctionSecretArguments();
}
else
{
return npos;
}
}
std::pair<size_t, size_t> findS3FunctionSecretArguments(bool is_cluster_function) const
{
/// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument.
size_t url_arg_idx = is_cluster_function ? 1 : 0;
/// We're going to replace 'aws_secret_access_key' with '[HIDDEN'] for the following signatures:
/// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...)
/// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression')
/// But we should check the number of arguments first because we don't need to do any replacements in case of
/// s3('url' [, 'format']) or s3Cluster('cluster_name', 'url' [, 'format'])
if (arguments->size() < url_arg_idx + 3)
return npos;
if (arguments->size() >= url_arg_idx + 5)
{
/// s3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'structure', ...)
return {url_arg_idx + 2, url_arg_idx + 3};
}
else
{
/// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...)
/// We need to distinguish that from s3('url', 'format', 'structure' [, 'compression_method']).
/// So we will check whether the argument after 'url' is a format.
String format;
if (!tryGetStringFromArgument(url_arg_idx + 1, &format, /* allow_identifier= */ false))
{
/// We couldn't evaluate the argument after 'url' so we don't know whether it is a format or `aws_access_key_id`.
/// So it's safer to wipe the next argument just in case.
return {url_arg_idx + 2, url_arg_idx + 3}; /// Wipe either `aws_secret_access_key` or `structure`.
}
if (KnownFormatNames::instance().exists(format))
return npos; /// The argument after 'url' is a format: s3('url', 'format', ...)
/// The argument after 'url' is not a format so we do our replacement:
/// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) -> s3('url', 'aws_access_key_id', '[HIDDEN]', ...)
return {url_arg_idx + 2, url_arg_idx + 3};
}
}
bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const
{
if (arg_idx >= arguments->size())
return false;
ASTPtr argument = (*arguments)[arg_idx];
if (const auto * literal = argument->as<ASTLiteral>())
{
if (literal->value.getType() != Field::Types::String)
return false;
if (res)
*res = literal->value.safeGet<String>();
return true;
}
if (allow_identifier)
{
if (const auto * id = argument->as<ASTIdentifier>())
{
if (res)
*res = id->name();
return true;
}
}
return false;
}
std::pair<size_t, size_t> findRemoteFunctionSecretArguments() const
{
/// We're going to replace 'password' with '[HIDDEN'] for the following signatures:
/// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key])
/// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key])
/// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key])
/// But we should check the number of arguments first because we don't need to do any replacements in case of
/// remote('addresses_expr', db.table)
if (arguments->size() < 3)
return npos;
size_t arg_num = 1;
/// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'.
const auto * table_function = (*arguments)[arg_num]->as<ASTFunction>();
if (table_function && KnownTableFunctionNames::instance().exists(table_function->name))
{
++arg_num;
}
else
{
std::optional<String> database;
std::optional<QualifiedTableName> qualified_table_name;
if (!tryGetDatabaseNameOrQualifiedTableName(arg_num, database, qualified_table_name))
{
/// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'.
/// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user'
/// before the argument 'password'. So it's safer to wipe two arguments just in case.
/// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string
/// before wiping it (because the `password` argument is always a literal string).
auto res = npos;
if (tryGetStringFromArgument(arg_num + 2, nullptr, /* allow_identifier= */ false))
{
/// Wipe either `password` or `user`.
res = {arg_num + 2, arg_num + 3};
}
if (tryGetStringFromArgument(arg_num + 3, nullptr, /* allow_identifier= */ false))
{
/// Wipe either `password` or `sharding_key`.
if (res == npos)
res.first = arg_num + 3;
res.second = arg_num + 4;
}
return res;
}
/// Skip the current argument (which is either a database name or a qualified table name).
++arg_num;
if (database)
{
/// Skip the 'table' argument if the previous argument was a database name.
++arg_num;
}
}
/// Skip username.
++arg_num;
/// Do our replacement:
/// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...)
/// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string
/// before wiping it (because the `password` argument is always a literal string).
bool can_be_password = tryGetStringFromArgument(arg_num, nullptr, /* allow_identifier= */ false);
if (can_be_password)
return {arg_num, arg_num + 1};
return npos;
}
/// Tries to get either a database name or a qualified table name from an argument.
/// Empty string is also allowed (it means the default database).
/// The function is used by findRemoteFunctionSecretArguments() to determine how many arguments to skip before a password.
bool tryGetDatabaseNameOrQualifiedTableName(
size_t arg_idx,
std::optional<String> & res_database,
std::optional<QualifiedTableName> & res_qualified_table_name) const
{
res_database.reset();
res_qualified_table_name.reset();
String str;
if (!tryGetStringFromArgument(arg_idx, &str, /* allow_identifier= */ true))
return false;
if (str.empty())
{
res_database = "";
return true;
}
auto qualified_table_name = QualifiedTableName::tryParseFromString(str);
if (!qualified_table_name)
return false;
if (qualified_table_name->database.empty())
res_database = std::move(qualified_table_name->table);
else
res_qualified_table_name = std::move(qualified_table_name);
return true;
}
std::pair<size_t, size_t> findEncryptionFunctionSecretArguments() const
{
/// We replace all arguments after 'mode' with '[HIDDEN]':
/// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]')
return {1, arguments->size()};
}
std::pair<size_t, size_t> findTableEngineSecretArguments() const
{
const String & engine_name = function.name;
if (engine_name == "ExternalDistributed")
{
/// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password')
return {5, 6};
}
else if ((engine_name == "MySQL") || (engine_name == "PostgreSQL") ||
(engine_name == "MaterializedPostgreSQL") || (engine_name == "MongoDB"))
{
/// MySQL('host:port', 'database', 'table', 'user', 'password', ...)
/// PostgreSQL('host:port', 'database', 'table', 'user', 'password', ...)
/// MaterializedPostgreSQL('host:port', 'database', 'table', 'user', 'password', ...)
/// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...)
return {4, 5};
}
else if ((engine_name == "S3") || (engine_name == "COSN") || (engine_name == "OSS"))
{
/// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...)
return findS3TableEngineSecretArguments();
}
else
{
return npos;
}
}
std::pair<size_t, size_t> findS3TableEngineSecretArguments() const
{
/// We replace 'aws_secret_access_key' with '[HIDDEN'] for the following signatures:
/// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format')
/// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression')
/// But we should check the number of arguments first because we don't need to do that replacements in case of
/// S3('url' [, 'format' [, 'compression']])
if (arguments->size() < 4)
return npos;
return {2, 3};
}
std::pair<size_t, size_t> findDatabaseEngineSecretArguments() const
{
const String & engine_name = function.name;
if ((engine_name == "MySQL") || (engine_name == "MaterializeMySQL") ||
(engine_name == "MaterializedMySQL") || (engine_name == "PostgreSQL") ||
(engine_name == "MaterializedPostgreSQL"))
{
/// MySQL('host:port', 'database', 'user', 'password')
/// PostgreSQL('host:port', 'database', 'user', 'password', ...)
return {3, 4};
}
else
{
return npos;
}
}
std::pair<size_t, size_t> findBackupNameSecretArguments() const
{
const String & engine_name = function.name;
if (engine_name == "S3")
{
/// BACKUP ... TO S3(url, [aws_access_key_id, aws_secret_access_key])
return {2, 3};
}
else
{
return npos;
}
}
const ASTFunction & function;
const ASTs * arguments = nullptr;
};
}
void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const
{
if (name == "view")
@ -629,6 +966,10 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
&& (name == "match" || name == "extract" || name == "extractAll" || name == "replaceRegexpOne"
|| name == "replaceRegexpAll");
auto secret_arguments = std::make_pair(static_cast<size_t>(-1), static_cast<size_t>(-1));
if (!settings.show_secrets)
secret_arguments = FunctionSecretArgumentsFinder(*this).getRange();
for (size_t i = 0, size = arguments->children.size(); i < size; ++i)
{
if (i != 0)
@ -636,12 +977,21 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
if (arguments->children[i]->as<ASTSetQuery>())
settings.ostr << "SETTINGS ";
bool special_hilite = false;
if (i == 1 && special_hilite_regexp)
special_hilite = highlightStringLiteralWithMetacharacters(arguments->children[i], settings, "|()^$.[]?*+{:-");
if (!settings.show_secrets && (secret_arguments.first <= i) && (i < secret_arguments.second))
{
settings.ostr << "'[HIDDEN]'";
if (size - 1 < secret_arguments.second)
break; /// All other arguments should also be hidden.
continue;
}
if (!special_hilite)
arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens);
if ((i == 1) && special_hilite_regexp
&& highlightStringLiteralWithMetacharacters(arguments->children[i], settings, "|()^$.[]?*+{:-"))
{
continue;
}
arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens);
}
}
@ -653,6 +1003,18 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
return finishFormatWithWindow(settings, state, frame);
}
bool ASTFunction::hasSecretParts() const
{
if (arguments)
{
size_t num_arguments = arguments->children.size();
auto secret_arguments = FunctionSecretArgumentsFinder(*this).getRange();
if ((secret_arguments.first < num_arguments) && (secret_arguments.first < secret_arguments.second))
return true;
}
return childrenHaveSecretParts();
}
String getFunctionName(const IAST * ast)
{
String res;

View File

@ -42,6 +42,18 @@ public:
/// do not print empty parentheses if there are no args - compatibility with new AST for data types and engine names.
bool no_empty_args = false;
/// Specifies where this function-like expression is used.
enum class Kind
{
ORDINARY_FUNCTION,
WINDOW_FUNCTION,
LAMBDA_FUNCTION,
TABLE_ENGINE,
DATABASE_ENGINE,
BACKUP_NAME,
};
Kind kind = Kind::ORDINARY_FUNCTION;
/** Get text identifying the AST node. */
String getID(char delim) const override;
@ -55,6 +67,8 @@ public:
std::string getWindowDescription() const;
bool hasSecretParts() const override;
protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void appendColumnNameImpl(WriteBuffer & ostr) const override;

View File

@ -29,7 +29,16 @@ void ASTPair::formatImpl(const FormatSettings & settings, FormatState & state, F
if (second_with_brackets)
settings.ostr << (settings.hilite ? hilite_keyword : "") << "(";
second->formatImpl(settings, state, frame);
if (!settings.show_secrets && (first == "password"))
{
/// Hide password in the definition of a dictionary:
/// SOURCE(CLICKHOUSE(host 'example01-01-1' port 9000 user 'default' password '[HIDDEN]' db 'default' table 'ids'))
settings.ostr << "'[HIDDEN]'";
}
else
{
second->formatImpl(settings, state, frame);
}
if (second_with_brackets)
settings.ostr << (settings.hilite ? hilite_keyword : "") << ")";
@ -38,6 +47,12 @@ void ASTPair::formatImpl(const FormatSettings & settings, FormatState & state, F
}
bool ASTPair::hasSecretParts() const
{
return first == "password";
}
void ASTPair::updateTreeHashImpl(SipHash & hash_state) const
{
hash_state.update(first.size());

View File

@ -30,6 +30,8 @@ public:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
bool hasSecretParts() const override;
void updateTreeHashImpl(SipHash & hash_state) const override;
};

View File

@ -23,7 +23,7 @@ namespace
}
void formatAuthenticationData(const AuthenticationData & auth_data, bool show_password, const IAST::FormatSettings & settings)
void formatAuthenticationData(const AuthenticationData & auth_data, const IAST::FormatSettings & settings)
{
auto auth_type = auth_data.getType();
if (auth_type == AuthenticationType::NO_PASSWORD)
@ -93,7 +93,7 @@ namespace
throw Exception("AST: Unexpected authentication type " + toString(auth_type), ErrorCodes::LOGICAL_ERROR);
}
if (password && !show_password)
if (password && !settings.show_secrets)
{
prefix = "";
password.reset();
@ -324,7 +324,7 @@ void ASTCreateUserQuery::formatImpl(const FormatSettings & format, FormatState &
formatRenameTo(*new_name, format);
if (auth_data)
formatAuthenticationData(*auth_data, show_password, format);
formatAuthenticationData(*auth_data, format);
if (hosts)
formatHosts(nullptr, *hosts, format);
@ -345,4 +345,18 @@ void ASTCreateUserQuery::formatImpl(const FormatSettings & format, FormatState &
if (grantees)
formatGrantees(*grantees, format);
}
bool ASTCreateUserQuery::hasSecretParts() const
{
if (auth_data)
{
auto auth_type = auth_data->getType();
if ((auth_type == AuthenticationType::PLAINTEXT_PASSWORD)
|| (auth_type == AuthenticationType::SHA256_PASSWORD)
|| (auth_type == AuthenticationType::DOUBLE_SHA1_PASSWORD))
return true;
}
return childrenHaveSecretParts();
}
}

View File

@ -45,7 +45,6 @@ public:
std::optional<String> new_name;
std::optional<AuthenticationData> auth_data;
bool show_password = true; /// formatImpl() shows a password or hash by default
std::optional<AllowedClientHosts> hosts;
std::optional<AllowedClientHosts> add_hosts;
@ -60,6 +59,7 @@ public:
String getID(char) const override;
ASTPtr clone() const override;
void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override;
bool hasSecretParts() const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTCreateUserQuery>(clone()); }
};
}

View File

@ -494,7 +494,12 @@ template <typename... Args>
static std::shared_ptr<ASTFunction> makeASTFunction(Operator & op, Args &&... args)
{
auto ast_function = makeASTFunction(op.function_name, std::forward<Args>(args)...);
ast_function->is_lambda_function = op.type == OperatorType::Lambda;
if (op.type == OperatorType::Lambda)
{
ast_function->is_lambda_function = true;
ast_function->kind = ASTFunction::Kind::LAMBDA_FUNCTION;
}
return ast_function;
}
@ -999,6 +1004,7 @@ public:
if (over.ignore(pos, expected))
{
function_node->is_window_function = true;
function_node->kind = ASTFunction::Kind::WINDOW_FUNCTION;
ASTPtr function_node_as_iast = function_node;

View File

@ -1,8 +1,10 @@
#include <Parsers/IAST.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <Common/SensitiveDataMasker.h>
#include <Common/SipHash.h>
#include <Parsers/IAST.h>
namespace DB
@ -165,11 +167,25 @@ size_t IAST::checkDepthImpl(size_t max_depth) const
return res;
}
std::string IAST::formatForErrorMessage() const
String IAST::formatWithSecretsHidden(size_t max_length, bool one_line) const
{
WriteBufferFromOwnString buf;
format(FormatSettings(buf, true /* one line */));
return buf.str();
FormatSettings settings{buf, one_line};
settings.show_secrets = false;
format(settings);
return wipeSensitiveDataAndCutToLength(buf.str(), max_length);
}
bool IAST::childrenHaveSecretParts() const
{
for (const auto & child : children)
{
if (child->hasSecretParts())
return true;
}
return false;
}
void IAST::cloneChildren()

View File

@ -185,6 +185,7 @@ public:
bool one_line;
bool always_quote_identifiers = false;
IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks;
bool show_secrets = true; /// Show secret parts of the AST (e.g. passwords, encryption keys).
// Newline or whitespace.
char nl_or_ws;
@ -240,9 +241,12 @@ public:
}
// A simple way to add some user-readable context to an error message.
std::string formatForErrorMessage() const;
template <typename AstArray>
static std::string formatForErrorMessage(const AstArray & array);
String formatWithSecretsHidden(size_t max_length = 0, bool one_line = true) const;
String formatForLogging(size_t max_length = 0) const { return formatWithSecretsHidden(max_length, true); }
String formatForErrorMessage() const { return formatWithSecretsHidden(0, true); }
/// If an AST has secret parts then formatForLogging() will replace them with the placeholder '[HIDDEN]'.
virtual bool hasSecretParts() const { return childrenHaveSecretParts(); }
void cloneChildren();
@ -272,6 +276,9 @@ public:
static const char * hilite_substitution;
static const char * hilite_none;
protected:
bool childrenHaveSecretParts() const;
private:
size_t checkDepthImpl(size_t max_depth) const;
@ -282,20 +289,4 @@ private:
ASTPtr * next_to_delete_list_head = nullptr;
};
template <typename AstArray>
std::string IAST::formatForErrorMessage(const AstArray & array)
{
WriteBufferFromOwnString buf;
for (size_t i = 0; i < array.size(); ++i)
{
if (i > 0)
{
const char * delim = ", ";
buf.write(delim, strlen(delim));
}
array[i]->format(IAST::FormatSettings(buf, true /* one line */));
}
return buf.str();
}
}

View File

@ -1,5 +1,6 @@
#include <Parsers/ParserBackupQuery.h>
#include <Parsers/ASTBackupQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier_fwd.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/CommonParsers.h>
@ -207,7 +208,11 @@ namespace
bool parseBackupName(IParser::Pos & pos, Expected & expected, ASTPtr & backup_name)
{
return ParserIdentifierWithOptionalParameters{}.parse(pos, backup_name, expected);
if (!ParserIdentifierWithOptionalParameters{}.parse(pos, backup_name, expected))
return false;
backup_name->as<ASTFunction &>().kind = ASTFunction::Kind::BACKUP_NAME;
return true;
}
bool parseBaseBackupSetting(IParser::Pos & pos, Expected & expected, ASTPtr & base_backup_name)
@ -358,11 +363,16 @@ bool ParserBackupQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
query->kind = kind;
query->elements = std::move(elements);
query->cluster = std::move(cluster);
query->backup_name = std::move(backup_name);
if (backup_name)
query->set(query->backup_name, backup_name);
query->settings = std::move(settings);
query->base_backup_name = std::move(base_backup_name);
query->cluster_host_ids = std::move(cluster_host_ids);
if (base_backup_name)
query->set(query->base_backup_name, base_backup_name);
return true;
}

View File

@ -442,6 +442,20 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!storage_like)
return false;
if (engine)
{
switch (engine_kind)
{
case EngineKind::TABLE_ENGINE:
engine->as<ASTFunction &>().kind = ASTFunction::Kind::TABLE_ENGINE;
break;
case EngineKind::DATABASE_ENGINE:
engine->as<ASTFunction &>().kind = ASTFunction::Kind::DATABASE_ENGINE;
break;
}
}
auto storage = std::make_shared<ASTStorage>();
storage->set(storage->engine, engine);
storage->set(storage->partition_by, partition_by);
@ -449,7 +463,6 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
storage->set(storage->order_by, order_by);
storage->set(storage->sample_by, sample_by);
storage->set(storage->ttl_table, ttl_table);
storage->set(storage->settings, settings);
node = storage;
@ -473,7 +486,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
ParserToken s_comma(TokenType::Comma);
ParserToken s_lparen(TokenType::OpeningRoundBracket);
ParserToken s_rparen(TokenType::ClosingRoundBracket);
ParserStorage storage_p;
ParserStorage storage_p{ParserStorage::TABLE_ENGINE};
ParserIdentifier name_p;
ParserTablePropertiesDeclarationList table_properties_p;
ParserSelectWithUnionQuery select_p;
@ -635,6 +648,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
}
}
}
auto comment = parseComment(pos, expected);
auto query = std::make_shared<ASTCreateQuery>();
@ -697,6 +711,8 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
ParserToken s_dot(TokenType::Dot);
ParserToken s_lparen(TokenType::OpeningRoundBracket);
ParserToken s_rparen(TokenType::ClosingRoundBracket);
ParserStorage storage_p{ParserStorage::TABLE_ENGINE};
ParserStorage storage_inner{ParserStorage::TABLE_ENGINE};
ParserTablePropertiesDeclarationList table_properties_p;
ParserSelectWithUnionQuery select_p;
@ -835,8 +851,8 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
ParserToken s_eq(TokenType::Equals);
ParserToken s_lparen(TokenType::OpeningRoundBracket);
ParserToken s_rparen(TokenType::ClosingRoundBracket);
ParserStorage storage_p;
ParserStorage storage_inner;
ParserStorage storage_p{ParserStorage::TABLE_ENGINE};
ParserStorage storage_inner{ParserStorage::TABLE_ENGINE};
ParserTablePropertiesDeclarationList table_properties_p;
ParserExpression watermark_p;
ParserExpression lateness_p;
@ -1135,7 +1151,7 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
ParserKeyword s_attach("ATTACH");
ParserKeyword s_database("DATABASE");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserStorage storage_p;
ParserStorage storage_p{ParserStorage::DATABASE_ENGINE};
ParserIdentifier name_p(true);
ParserTableOverridesDeclarationList table_overrides_p;
@ -1222,7 +1238,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
ParserToken s_dot(TokenType::Dot);
ParserToken s_lparen(TokenType::OpeningRoundBracket);
ParserToken s_rparen(TokenType::ClosingRoundBracket);
ParserStorage storage_p;
ParserStorage storage_p{ParserStorage::TABLE_ENGINE};
ParserIdentifier name_p;
ParserTablePropertiesDeclarationList table_properties_p;
ParserSelectWithUnionQuery select_p;

View File

@ -420,9 +420,20 @@ protected:
*/
class ParserStorage : public IParserBase
{
public:
/// What kind of engine we're going to parse.
enum EngineKind
{
TABLE_ENGINE,
DATABASE_ENGINE,
};
ParserStorage(EngineKind engine_kind_) : engine_kind(engine_kind_) {}
protected:
const char * getName() const override { return "storage definition"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
EngineKind engine_kind;
};
/** Query like this:

View File

@ -10,7 +10,6 @@
#include <Interpreters/Context.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/queryToString.h>
#include <Common/typeid_cast.h>
#include <Common/StringUtils/StringUtils.h>
#include <DataTypes/DataTypesNumber.h>
@ -232,7 +231,7 @@ protected:
{
auto temp_db = DatabaseCatalog::instance().getDatabaseForTemporaryTables();
ASTPtr ast = temp_db ? temp_db->tryGetCreateTableQuery(table.second->getStorageID().getTableName(), context) : nullptr;
res_columns[res_index++]->insert(ast ? queryToString(ast) : "");
res_columns[res_index++]->insert(ast ? ast->formatWithSecretsHidden() : "");
}
// engine_full
@ -383,7 +382,7 @@ protected:
}
if (columns_mask[src_index++])
res_columns[res_index++]->insert(ast ? queryToString(ast) : "");
res_columns[res_index++]->insert(ast ? ast->formatWithSecretsHidden() : "");
if (columns_mask[src_index++])
{
@ -391,7 +390,7 @@ protected:
if (ast_create && ast_create->storage)
{
engine_full = queryToString(*ast_create->storage);
engine_full = ast_create->storage->formatWithSecretsHidden();
static const char * const extra_head = " ENGINE = ";
if (startsWith(engine_full, extra_head))
@ -405,7 +404,7 @@ protected:
{
String as_select;
if (ast_create && ast_create->select)
as_select = queryToString(*ast_create->select);
as_select = ast_create->select->formatWithSecretsHidden();
res_columns[res_index++]->insert(as_select);
}
}
@ -420,7 +419,7 @@ protected:
if (columns_mask[src_index++])
{
if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPartitionKeyAST()))
res_columns[res_index++]->insert(queryToString(expression_ptr));
res_columns[res_index++]->insert(expression_ptr->formatWithSecretsHidden());
else
res_columns[res_index++]->insertDefault();
}
@ -428,7 +427,7 @@ protected:
if (columns_mask[src_index++])
{
if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSortingKey().expression_list_ast))
res_columns[res_index++]->insert(queryToString(expression_ptr));
res_columns[res_index++]->insert(expression_ptr->formatWithSecretsHidden());
else
res_columns[res_index++]->insertDefault();
}
@ -436,7 +435,7 @@ protected:
if (columns_mask[src_index++])
{
if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPrimaryKey().expression_list_ast))
res_columns[res_index++]->insert(queryToString(expression_ptr));
res_columns[res_index++]->insert(expression_ptr->formatWithSecretsHidden());
else
res_columns[res_index++]->insertDefault();
}
@ -444,7 +443,7 @@ protected:
if (columns_mask[src_index++])
{
if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSamplingKeyAST()))
res_columns[res_index++]->insert(queryToString(expression_ptr));
res_columns[res_index++]->insert(expression_ptr->formatWithSecretsHidden());
else
res_columns[res_index++]->insertDefault();
}

View File

@ -3,6 +3,7 @@
#include <Interpreters/Context.h>
#include <Common/CurrentThread.h>
#include <Common/Exception.h>
#include <Common/KnownObjectNames.h>
#include <IO/WriteHelpers.h>
#include <Parsers/ASTFunction.h>
@ -27,6 +28,8 @@ void TableFunctionFactory::registerFunction(
&& !case_insensitive_table_functions.emplace(Poco::toLower(name), value).second)
throw Exception("TableFunctionFactory: the case insensitive table function name '" + name + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
KnownTableFunctionNames::instance().add(name, (case_sensitiveness == CaseInsensitive));
}
TableFunctionPtr TableFunctionFactory::get(

View File

@ -1,6 +1,7 @@
import pytest
import random, string
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance("node", with_zookeeper=True)
@ -110,6 +111,22 @@ def test_create_table():
for i, table_engine in enumerate(table_engines):
node.query(f"CREATE TABLE table{i} (x int) ENGINE = {table_engine}")
assert (
node.query("SHOW CREATE TABLE table0")
== "CREATE TABLE default.table0\\n(\\n `x` Int32\\n)\\nENGINE = MySQL(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')\n"
)
assert node.query(
"SELECT create_table_query, engine_full FROM system.tables WHERE name = 'table0'"
) == TSV(
[
[
"CREATE TABLE default.table0 (`x` Int32) ENGINE = MySQL(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')",
"MySQL(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')",
],
]
)
check_logs(
must_contain=[
"CREATE TABLE table0 (`x` int) ENGINE = MySQL('mysql57:3306', 'mysql_db', 'mysql_table', 'mysql_user', '[HIDDEN]')",
@ -189,6 +206,22 @@ def test_table_functions():
for i, table_function in enumerate(table_functions):
node.query(f"CREATE TABLE tablefunc{i} (x int) AS {table_function}")
assert (
node.query("SHOW CREATE TABLE tablefunc0")
== "CREATE TABLE default.tablefunc0\\n(\\n `x` Int32\\n) AS mysql(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')\n"
)
assert node.query(
"SELECT create_table_query, engine_full FROM system.tables WHERE name = 'tablefunc0'"
) == TSV(
[
[
"CREATE TABLE default.tablefunc0 (`x` Int32) AS mysql(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')",
"",
],
]
)
check_logs(
must_contain=[
"CREATE TABLE tablefunc0 (`x` int) AS mysql('mysql57:3306', 'mysql_db', 'mysql_table', 'mysql_user', '[HIDDEN]')",
@ -268,6 +301,16 @@ def test_create_dictionary():
f"LIFETIME(MIN 0 MAX 10) LAYOUT(FLAT())"
)
assert (
node.query("SHOW CREATE TABLE dict1")
== "CREATE DICTIONARY default.dict1\\n(\\n `n` int DEFAULT 0,\\n `m` int DEFAULT 1\\n)\\nPRIMARY KEY n\\nSOURCE(CLICKHOUSE(HOST \\'localhost\\' PORT 9000 USER \\'user1\\' TABLE \\'test\\' PASSWORD \\'[HIDDEN]\\' DB \\'default\\'))\\nLIFETIME(MIN 0 MAX 10)\\nLAYOUT(FLAT())\n"
)
assert (
node.query("SELECT create_table_query FROM system.tables WHERE name = 'dict1'")
== "CREATE DICTIONARY default.dict1 (`n` int DEFAULT 0, `m` int DEFAULT 1) PRIMARY KEY n SOURCE(CLICKHOUSE(HOST \\'localhost\\' PORT 9000 USER \\'user1\\' TABLE \\'test\\' PASSWORD \\'[HIDDEN]\\' DB \\'default\\')) LIFETIME(MIN 0 MAX 10) LAYOUT(FLAT())\n"
)
check_logs(
must_contain=[
"CREATE DICTIONARY dict1 (`n` int DEFAULT 0, `m` int DEFAULT 1) PRIMARY KEY n "

View File

@ -1,5 +1,5 @@
=DICTIONARY in Ordinary DB
CREATE DICTIONARY db_01018.dict1\n(\n `key_column` UInt64 DEFAULT 0,\n `second_column` UInt8 DEFAULT 1,\n `third_column` String DEFAULT \'qqq\'\n)\nPRIMARY KEY key_column\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'database_for_dict_01018\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT())
CREATE DICTIONARY db_01018.dict1\n(\n `key_column` UInt64 DEFAULT 0,\n `second_column` UInt8 DEFAULT 1,\n `third_column` String DEFAULT \'qqq\'\n)\nPRIMARY KEY key_column\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'[HIDDEN]\' DB \'database_for_dict_01018\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT())
dict1
1
db_01018 dict1
@ -12,7 +12,7 @@ db_01018 dict1
==DROP DICTIONARY
0
=DICTIONARY in Memory DB
CREATE DICTIONARY memory_db.dict2\n(\n `key_column` UInt64 DEFAULT 0 INJECTIVE,\n `second_column` UInt8 DEFAULT 1 EXPRESSION rand() % 222,\n `third_column` String DEFAULT \'qqq\'\n)\nPRIMARY KEY key_column\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'database_for_dict_01018\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT())
CREATE DICTIONARY memory_db.dict2\n(\n `key_column` UInt64 DEFAULT 0 INJECTIVE,\n `second_column` UInt8 DEFAULT 1 EXPRESSION rand() % 222,\n `third_column` String DEFAULT \'qqq\'\n)\nPRIMARY KEY key_column\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'[HIDDEN]\' DB \'database_for_dict_01018\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT())
dict2
1
memory_db dict2

View File

@ -1,5 +1,5 @@
CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT())
CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT())
CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'[HIDDEN]\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT())
CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'[HIDDEN]\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT())
CREATE TABLE test_01190.log\n(\n `s` String\n)\nENGINE = Log
CREATE TABLE test_01190.log\n(\n `s` String\n)\nENGINE = Log
test

View File

@ -6,7 +6,7 @@ CREATE DICTIONARY dict_db_01224.dict
`val` UInt64 DEFAULT 10
)
PRIMARY KEY key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'dict_data' PASSWORD '' DB 'dict_db_01224'))
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'dict_data' PASSWORD '[HIDDEN]' DB 'dict_db_01224'))
LIFETIME(MIN 0 MAX 0)
LAYOUT(FLAT())
NOT_LOADED
@ -17,7 +17,7 @@ CREATE TABLE dict_db_01224_dictionary.`dict_db_01224.dict`
)
ENGINE = Dictionary(`dict_db_01224.dict`)
NOT_LOADED
Dictionary 1 CREATE DICTIONARY dict_db_01224.dict (`key` UInt64 DEFAULT 0, `val` UInt64 DEFAULT 10) PRIMARY KEY key SOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'dict_data\' PASSWORD \'\' DB \'dict_db_01224\')) LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT())
Dictionary 1 CREATE DICTIONARY dict_db_01224.dict (`key` UInt64 DEFAULT 0, `val` UInt64 DEFAULT 10) PRIMARY KEY key SOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'dict_data\' PASSWORD \'[HIDDEN]\' DB \'dict_db_01224\')) LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT())
NOT_LOADED
key UInt64
val UInt64