mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Mask sensitive information in logs.
This commit is contained in:
parent
05e178b6d3
commit
5d2a222fe4
@ -34,7 +34,6 @@
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/toOneLineQuery.h>
|
||||
#include <Parsers/wipePasswordFromQuery.h>
|
||||
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Storages/StorageInput.h>
|
||||
@ -56,6 +55,7 @@
|
||||
#include <Interpreters/SelectQueryOptions.h>
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Interpreters/wipePasswordFromQuery.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
|
||||
#include <Common/SensitiveDataMasker.h>
|
||||
@ -119,7 +119,7 @@ static String prepareQueryForLogging(const String & query, const ASTPtr & parsed
|
||||
if (parsed_query && canContainPassword(*parsed_query))
|
||||
{
|
||||
ASTPtr ast_for_logging = parsed_query->clone();
|
||||
wipePasswordFromQuery(ast_for_logging);
|
||||
wipePasswordFromQuery(ast_for_logging, context);
|
||||
res = serializeAST(*ast_for_logging);
|
||||
}
|
||||
|
||||
|
538
src/Interpreters/wipePasswordFromQuery.cpp
Normal file
538
src/Interpreters/wipePasswordFromQuery.cpp
Normal file
@ -0,0 +1,538 @@
|
||||
#include <Interpreters/wipePasswordFromQuery.h>
|
||||
|
||||
#include <Formats/FormatFactory.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 <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
template <bool check_only>
|
||||
class PasswordWipingVisitor
|
||||
{
|
||||
public:
|
||||
struct Data
|
||||
{
|
||||
bool can_contain_password = false;
|
||||
bool is_create_table_query = false;
|
||||
bool is_create_database_query = false;
|
||||
bool is_create_dictionary_query = false;
|
||||
ContextPtr context;
|
||||
};
|
||||
|
||||
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>())
|
||||
{
|
||||
visitFunction(*function, data);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
static void visitCreateUserQuery(ASTCreateUserQuery & query, Data & data)
|
||||
{
|
||||
if constexpr (check_only)
|
||||
{
|
||||
data.can_contain_password = true;
|
||||
return;
|
||||
}
|
||||
query.show_password = false;
|
||||
}
|
||||
|
||||
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
|
||||
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")
|
||||
{
|
||||
/// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...)
|
||||
wipePasswordFromS3TableEngineArguments(*storage.engine, data);
|
||||
}
|
||||
}
|
||||
|
||||
static void wipePasswordFromS3TableEngineArguments(ASTFunction & engine, Data & data)
|
||||
{
|
||||
if constexpr (check_only)
|
||||
{
|
||||
data.can_contain_password = true;
|
||||
return;
|
||||
}
|
||||
|
||||
/// 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")
|
||||
{
|
||||
/// 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)
|
||||
{
|
||||
if constexpr (check_only)
|
||||
{
|
||||
data.can_contain_password = true;
|
||||
return;
|
||||
}
|
||||
|
||||
/// 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, url_arg_idx + 1, data.context, &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)
|
||||
{
|
||||
if constexpr (check_only)
|
||||
{
|
||||
data.can_contain_password = true;
|
||||
return;
|
||||
}
|
||||
|
||||
/// 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 = function.arguments->as<ASTExpressionList>()->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, arg_num, data.context, &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)
|
||||
{
|
||||
if constexpr (check_only)
|
||||
{
|
||||
data.can_contain_password = true;
|
||||
return;
|
||||
}
|
||||
|
||||
/// We replace all arguments after 'mode' with '[HIDDEN]':
|
||||
/// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]')
|
||||
|
||||
size_t num_arguments;
|
||||
if (!tryGetNumArguments(function, &num_arguments) || (num_arguments < 2))
|
||||
return;
|
||||
|
||||
wipePasswordFromArgument(function, data, 1);
|
||||
function.arguments->as<ASTExpressionList>()->children.resize(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 * backup_engine = query.base_backup_name->as<ASTFunction>())
|
||||
wipePasswordFromBackupEngineArguments(*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 constexpr (check_only)
|
||||
{
|
||||
data.can_contain_password = true;
|
||||
return;
|
||||
}
|
||||
|
||||
if (!function.arguments)
|
||||
return;
|
||||
|
||||
auto * expr_list = function.arguments->as<ASTExpressionList>();
|
||||
if (!expr_list)
|
||||
return;
|
||||
|
||||
auto & arguments = expr_list->children;
|
||||
|
||||
if (arg_idx < arguments.size())
|
||||
arguments[arg_idx] = std::make_shared<ASTLiteral>("[HIDDEN]");
|
||||
}
|
||||
|
||||
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;
|
||||
|
||||
*num_arguments = expr_list->children.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;
|
||||
|
||||
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, size_t arg_idx, const ContextPtr & context, String * value)
|
||||
{
|
||||
if (!function.arguments)
|
||||
return false;
|
||||
|
||||
const auto * expr_list = function.arguments->as<ASTExpressionList>();
|
||||
if (!expr_list)
|
||||
return false;
|
||||
|
||||
const auto & arguments = expr_list->children;
|
||||
|
||||
if (arg_idx >= arguments.size())
|
||||
return false;
|
||||
|
||||
ASTPtr argument = arguments[arg_idx];
|
||||
try
|
||||
{
|
||||
argument = evaluateConstantExpressionOrIdentifierAsLiteral(argument, context);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
const auto * literal = argument->as<ASTLiteral>();
|
||||
if (!literal || literal->value.getType() != Field::Types::String)
|
||||
return false;
|
||||
|
||||
*value = literal->value.safeGet<String>();
|
||||
return true;
|
||||
}
|
||||
|
||||
static bool tryGetEvaluatedConstDatabaseNameFromArgument(
|
||||
const ASTFunction & function, size_t arg_idx, const ContextPtr & context, String * value)
|
||||
{
|
||||
if (!function.arguments)
|
||||
return false;
|
||||
|
||||
const auto * expr_list = function.arguments->as<ASTExpressionList>();
|
||||
if (!expr_list)
|
||||
return false;
|
||||
|
||||
const auto & arguments = expr_list->children;
|
||||
|
||||
if (arg_idx >= arguments.size())
|
||||
return false;
|
||||
|
||||
ASTPtr argument = arguments[arg_idx];
|
||||
try
|
||||
{
|
||||
argument = evaluateConstantExpressionForDatabaseName(argument, context);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
const auto * literal = argument->as<ASTLiteral>();
|
||||
if (!literal || 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 * elements = dictionary.source->elements->as<ASTExpressionList>();
|
||||
if (!elements)
|
||||
return;
|
||||
|
||||
/// 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->children)
|
||||
{
|
||||
auto * pair = element->as<ASTPair>();
|
||||
if (!pair)
|
||||
continue;
|
||||
|
||||
if (pair->first == "password")
|
||||
{
|
||||
if constexpr (check_only)
|
||||
{
|
||||
data.can_contain_password = true;
|
||||
return;
|
||||
}
|
||||
pair->set(pair->second, std::make_shared<ASTLiteral>("[HIDDEN]"));
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -13,6 +14,6 @@ bool canContainPassword(const IAST & ast);
|
||||
/// 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);
|
||||
void wipePasswordFromQuery(ASTPtr ast, const ContextPtr & context);
|
||||
|
||||
}
|
@ -1,22 +0,0 @@
|
||||
#include <Parsers/Access/ASTCreateUserQuery.h>
|
||||
#include <Parsers/wipePasswordFromQuery.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool canContainPassword(const IAST & ast)
|
||||
{
|
||||
return ast.as<ASTCreateUserQuery>();
|
||||
}
|
||||
|
||||
void wipePasswordFromQuery(ASTPtr ast)
|
||||
{
|
||||
if (auto * create_query = ast->as<ASTCreateUserQuery>())
|
||||
{
|
||||
create_query->show_password = false;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -1151,6 +1151,14 @@ StorageS3Configuration StorageS3::getConfiguration(ASTs & engine_args, ContextPt
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Supported signatures:
|
||||
///
|
||||
/// S3('url')
|
||||
/// S3('url', 'format')
|
||||
/// S3('url', 'format', 'compression')
|
||||
/// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format')
|
||||
/// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression')
|
||||
|
||||
if (engine_args.empty() || engine_args.size() > 5)
|
||||
throw Exception(
|
||||
"Storage S3 requires 1 to 5 arguments: url, [access_key_id, secret_access_key], name of used format and [compression_method].",
|
||||
|
@ -94,6 +94,30 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Supported signatures:
|
||||
///
|
||||
/// remote('addresses_expr', db.table)
|
||||
/// remote('addresses_expr', 'db', 'table')
|
||||
/// remote('addresses_expr', db.table, 'user')
|
||||
/// remote('addresses_expr', 'db', 'table', 'user')
|
||||
/// remote('addresses_expr', db.table, 'user', 'password')
|
||||
/// remote('addresses_expr', 'db', 'table', 'user', 'password')
|
||||
/// remote('addresses_expr', db.table, sharding_key)
|
||||
/// remote('addresses_expr', 'db', 'table', sharding_key)
|
||||
/// remote('addresses_expr', db.table, 'user', sharding_key)
|
||||
/// remote('addresses_expr', 'db', 'table', 'user', sharding_key)
|
||||
/// remote('addresses_expr', db.table, 'user', 'password', sharding_key)
|
||||
/// remote('addresses_expr', 'db', 'table', 'user', 'password', sharding_key)
|
||||
///
|
||||
/// remoteSecure() - same as remote()
|
||||
///
|
||||
/// cluster('cluster_name', db.table)
|
||||
/// cluster('cluster_name', 'db', 'table')
|
||||
/// cluster('cluster_name', db.table, sharding_key)
|
||||
/// cluster('cluster_name', 'db', 'table', sharding_key)
|
||||
///
|
||||
/// clusterAllReplicas() - same as cluster()
|
||||
|
||||
if (args.size() < 2 || args.size() > max_args)
|
||||
throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
@ -318,7 +342,6 @@ TableFunctionRemote::TableFunctionRemote(const std::string & name_, bool secure_
|
||||
is_cluster_function ? " [, sharding_key]" : " [, username[, password], sharding_key]");
|
||||
}
|
||||
|
||||
|
||||
void registerTableFunctionRemote(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction("remote", [] () -> TableFunctionPtr { return std::make_shared<TableFunctionRemote>("remote"); });
|
||||
|
@ -151,6 +151,10 @@ def subprocess_check_call(args, detach=False, nothrow=False):
|
||||
return run_and_check(args, detach=detach, nothrow=nothrow)
|
||||
|
||||
|
||||
def escape_substring_for_grep(substring):
|
||||
return substring.replace("`", "\\`").replace("[", "\\[").replace("]", "\\]")
|
||||
|
||||
|
||||
def get_odbc_bridge_path():
|
||||
path = os.environ.get("CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH")
|
||||
if path is None:
|
||||
@ -3419,13 +3423,14 @@ class ClickHouseInstance:
|
||||
def contains_in_log(
|
||||
self, substring, from_host=False, filename="clickhouse-server.log"
|
||||
):
|
||||
escaped_substring = escape_substring_for_grep(substring)
|
||||
if from_host:
|
||||
# We check fist file exists but want to look for all rotated logs as well
|
||||
result = subprocess_check_call(
|
||||
[
|
||||
"bash",
|
||||
"-c",
|
||||
f'[ -f {self.logs_dir}/{filename} ] && zgrep -aH "{substring}" {self.logs_dir}/{filename}* || true',
|
||||
f'[ -f {self.logs_dir}/{filename} ] && zgrep -aH "{escaped_substring}" {self.logs_dir}/{filename}* || true',
|
||||
]
|
||||
)
|
||||
else:
|
||||
@ -3433,20 +3438,21 @@ class ClickHouseInstance:
|
||||
[
|
||||
"bash",
|
||||
"-c",
|
||||
f'[ -f /var/log/clickhouse-server/{filename} ] && zgrep -aH "{substring}" /var/log/clickhouse-server/{filename} || true',
|
||||
f'[ -f /var/log/clickhouse-server/{filename} ] && zgrep -aH "{escaped_substring}" /var/log/clickhouse-server/{filename} || true',
|
||||
]
|
||||
)
|
||||
return len(result) > 0
|
||||
|
||||
def grep_in_log(self, substring, from_host=False, filename="clickhouse-server.log"):
|
||||
logging.debug(f"grep in log called %s", substring)
|
||||
escaped_substring = escape_substring_for_grep(substring)
|
||||
if from_host:
|
||||
# We check fist file exists but want to look for all rotated logs as well
|
||||
result = subprocess_check_call(
|
||||
[
|
||||
"bash",
|
||||
"-c",
|
||||
f'[ -f {self.logs_dir}/{filename} ] && zgrep -a "{substring}" {self.logs_dir}/{filename}* || true',
|
||||
f'[ -f {self.logs_dir}/{filename} ] && zgrep -a "{escaped_substring}" {self.logs_dir}/{filename}* || true',
|
||||
]
|
||||
)
|
||||
else:
|
||||
@ -3454,19 +3460,20 @@ class ClickHouseInstance:
|
||||
[
|
||||
"bash",
|
||||
"-c",
|
||||
f'[ -f /var/log/clickhouse-server/{filename} ] && zgrep -a "{substring}" /var/log/clickhouse-server/{filename}* || true',
|
||||
f'[ -f /var/log/clickhouse-server/{filename} ] && zgrep -a "{escaped_substring}" /var/log/clickhouse-server/{filename}* || true',
|
||||
]
|
||||
)
|
||||
logging.debug("grep result %s", result)
|
||||
return result
|
||||
|
||||
def count_in_log(self, substring):
|
||||
escaped_substring = escape_substring_for_grep(substring)
|
||||
result = self.exec_in_container(
|
||||
[
|
||||
"bash",
|
||||
"-c",
|
||||
'grep -a "{}" /var/log/clickhouse-server/clickhouse-server.log | wc -l'.format(
|
||||
substring
|
||||
escaped_substring
|
||||
),
|
||||
]
|
||||
)
|
||||
|
@ -15,26 +15,32 @@ def started_cluster():
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def check_logs(must_contain, must_not_contain):
|
||||
def check_logs(must_contain=[], must_not_contain=[]):
|
||||
node.query("SYSTEM FLUSH LOGS")
|
||||
|
||||
for str in must_contain:
|
||||
assert node.contains_in_log(str)
|
||||
|
||||
for str in must_not_contain:
|
||||
assert not node.contains_in_log(str)
|
||||
|
||||
for str in must_contain:
|
||||
escaped_str = str.replace("'", "\\'")
|
||||
assert (
|
||||
int(
|
||||
node.query(
|
||||
f"SELECT COUNT() FROM system.query_log WHERE query LIKE '%{str}%'"
|
||||
f"SELECT COUNT() FROM system.query_log WHERE query LIKE '%{escaped_str}%'"
|
||||
).strip()
|
||||
)
|
||||
>= 1
|
||||
)
|
||||
|
||||
for str in must_not_contain:
|
||||
assert not node.contains_in_log(str)
|
||||
escaped_str = str.replace("'", "\\'")
|
||||
assert (
|
||||
int(
|
||||
node.query(
|
||||
f"SELECT COUNT() FROM system.query_log WHERE query LIKE '%{str}%'"
|
||||
f"SELECT COUNT() FROM system.query_log WHERE query LIKE '%{escaped_str}%'"
|
||||
).strip()
|
||||
)
|
||||
== 0
|
||||
@ -73,3 +79,200 @@ def test_create_alter_user():
|
||||
"IDENTIFIED WITH plaintext_password BY",
|
||||
],
|
||||
)
|
||||
|
||||
|
||||
def test_create_table():
|
||||
table_engines = [
|
||||
"MySQL('mysql57:3306', 'mysql_db', 'mysql_table', 'mysql_user', 'qwe124')",
|
||||
"PostgreSQL('postgres1:5432', 'postgres_db', 'postgres_table', 'postgres_user', 'qwe124')",
|
||||
"MongoDB('mongo1:27017', 'mongo_db', 'mongo_col', 'mongo_user', 'qwe124')",
|
||||
"S3('http://minio1:9001/root/data/test1.csv')",
|
||||
"S3('http://minio1:9001/root/data/test2.csv', 'CSV')",
|
||||
"S3('http://minio1:9001/root/data/test3.csv.gz', 'CSV', 'gzip')",
|
||||
"S3('http://minio1:9001/root/data/test4.csv', 'minio', 'qwe124', 'CSV')",
|
||||
"S3('http://minio1:9001/root/data/test5.csv.gz', 'minio', 'qwe124', 'CSV', 'gzip')",
|
||||
]
|
||||
|
||||
for i, table_engine in enumerate(table_engines):
|
||||
node.query(f"CREATE TABLE table{i} (x int) ENGINE = {table_engine}")
|
||||
|
||||
check_logs(
|
||||
must_contain=[
|
||||
"CREATE TABLE table0 (`x` int) ENGINE = MySQL('mysql57:3306', 'mysql_db', 'mysql_table', 'mysql_user', '[HIDDEN]')",
|
||||
"CREATE TABLE table1 (`x` int) ENGINE = PostgreSQL('postgres1:5432', 'postgres_db', 'postgres_table', 'postgres_user', '[HIDDEN]')",
|
||||
"CREATE TABLE table2 (`x` int) ENGINE = MongoDB('mongo1:27017', 'mongo_db', 'mongo_col', 'mongo_user', '[HIDDEN]')",
|
||||
"CREATE TABLE table3 (`x` int) ENGINE = S3('http://minio1:9001/root/data/test1.csv')",
|
||||
"CREATE TABLE table4 (`x` int) ENGINE = S3('http://minio1:9001/root/data/test2.csv', 'CSV')",
|
||||
"CREATE TABLE table5 (`x` int) ENGINE = S3('http://minio1:9001/root/data/test3.csv.gz', 'CSV', 'gzip')",
|
||||
"CREATE TABLE table6 (`x` int) ENGINE = S3('http://minio1:9001/root/data/test4.csv', 'minio', '[HIDDEN]', 'CSV')",
|
||||
"CREATE TABLE table7 (`x` int) ENGINE = S3('http://minio1:9001/root/data/test5.csv.gz', 'minio', '[HIDDEN]', 'CSV', 'gzip')",
|
||||
],
|
||||
must_not_contain=["qwe124"],
|
||||
)
|
||||
|
||||
for i in range(0, len(table_engines)):
|
||||
node.query(f"DROP TABLE IF EXISTS table{i}")
|
||||
|
||||
|
||||
def test_create_database():
|
||||
database_engines = [
|
||||
"MySQL('mysql57:3306', 'mysql_db', 'mysql_user', 'qwe125')",
|
||||
"PostgreSQL('postgres1:5432', 'postgres_db', 'postgres_user', 'qwe125')",
|
||||
]
|
||||
|
||||
for i, database_engine in enumerate(database_engines):
|
||||
# query_and_get_answer_with_error() is used here because we don't want to stop on error "Cannot connect to MySQL server".
|
||||
# We test logging here and not actual work with MySQL server.
|
||||
node.query_and_get_answer_with_error(
|
||||
f"CREATE DATABASE database{i} ENGINE = {database_engine}"
|
||||
)
|
||||
|
||||
check_logs(
|
||||
must_contain=[
|
||||
"CREATE DATABASE database0 ENGINE = MySQL('mysql57:3306', 'mysql_db', 'mysql_user', '[HIDDEN]')",
|
||||
"CREATE DATABASE database1 ENGINE = PostgreSQL('postgres1:5432', 'postgres_db', 'postgres_user', '[HIDDEN]')",
|
||||
],
|
||||
must_not_contain=["qwe125"],
|
||||
)
|
||||
|
||||
for i in range(0, len(database_engines)):
|
||||
node.query(f"DROP DATABASE IF EXISTS database{i}")
|
||||
|
||||
|
||||
def test_table_functions():
|
||||
table_functions = [
|
||||
"mysql('mysql57:3306', 'mysql_db', 'mysql_table', 'mysql_user', 'qwe126')",
|
||||
"postgresql('postgres1:5432', 'postgres_db', 'postgres_table', 'postgres_user', 'qwe126')",
|
||||
"mongodb('mongo1:27017', 'mongo_db', 'mongo_col', 'mongo_user', 'qwe126', 'x int')",
|
||||
"s3('http://minio1:9001/root/data/test1.csv')",
|
||||
"s3('http://minio1:9001/root/data/test2.csv', 'CSV')",
|
||||
"s3('http://minio1:9001/root/data/test3.csv', 'minio', 'qwe126')",
|
||||
"s3('http://minio1:9001/root/data/test4.csv', 'CSV', 'x int')",
|
||||
"s3('http://minio1:9001/root/data/test5.csv.gz', 'CSV', 'x int', 'gzip')",
|
||||
"s3('http://minio1:9001/root/data/test6.csv', 'minio', 'qwe126', 'CSV')",
|
||||
"s3('http://minio1:9001/root/data/test7.csv', 'minio', 'qwe126', 'CSV', 'x int')",
|
||||
"s3('http://minio1:9001/root/data/test8.csv.gz', 'minio', 'qwe126', 'CSV', 'x int', 'gzip')",
|
||||
"s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test1.csv', 'minio', 'qwe126')",
|
||||
"s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test2.csv', 'CSV', 'x int')",
|
||||
"s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test3.csv', 'minio', 'qwe126', 'CSV')",
|
||||
"remote('127.{2..11}', default.remote_table)",
|
||||
"remote('127.{2..11}', default.remote_table, rand())",
|
||||
"remote('127.{2..11}', default.remote_table, 'remote_user')",
|
||||
"remote('127.{2..11}', default.remote_table, 'remote_user', 'qwe126')",
|
||||
"remote('127.{2..11}', default.remote_table, 'remote_user', rand())",
|
||||
"remote('127.{2..11}', default.remote_table, 'remote_user', 'qwe126', rand())",
|
||||
"remote('127.{2..11}', 'default.remote_table', 'remote_user', 'qwe126', rand())",
|
||||
"remote('127.{2..11}', 'default', 'remote_table', 'remote_user', 'qwe126', rand())",
|
||||
"remote('127.{2..11}', numbers(10), 'remote_user', 'qwe126', rand())",
|
||||
"remoteSecure('127.{2..11}', 'default', 'remote_table', 'remote_user', 'qwe126')",
|
||||
"remoteSecure('127.{2..11}', 'default', 'remote_table', 'remote_user', rand())",
|
||||
]
|
||||
|
||||
for i, table_function in enumerate(table_functions):
|
||||
node.query(f"CREATE TABLE tablefunc{i} (x int) AS {table_function}")
|
||||
|
||||
check_logs(
|
||||
must_contain=[
|
||||
"CREATE TABLE tablefunc0 (`x` int) AS mysql('mysql57:3306', 'mysql_db', 'mysql_table', 'mysql_user', '[HIDDEN]')",
|
||||
"CREATE TABLE tablefunc1 (`x` int) AS postgresql('postgres1:5432', 'postgres_db', 'postgres_table', 'postgres_user', '[HIDDEN]')",
|
||||
"CREATE TABLE tablefunc2 (`x` int) AS mongodb('mongo1:27017', 'mongo_db', 'mongo_col', 'mongo_user', '[HIDDEN]', 'x int')",
|
||||
"CREATE TABLE tablefunc3 (`x` int) AS s3('http://minio1:9001/root/data/test1.csv')",
|
||||
"CREATE TABLE tablefunc4 (`x` int) AS s3('http://minio1:9001/root/data/test2.csv', 'CSV')",
|
||||
"CREATE TABLE tablefunc5 (`x` int) AS s3('http://minio1:9001/root/data/test3.csv', 'minio', '[HIDDEN]')",
|
||||
"CREATE TABLE tablefunc6 (`x` int) AS s3('http://minio1:9001/root/data/test4.csv', 'CSV', 'x int')",
|
||||
"CREATE TABLE tablefunc7 (`x` int) AS s3('http://minio1:9001/root/data/test5.csv.gz', 'CSV', 'x int', 'gzip')",
|
||||
"CREATE TABLE tablefunc8 (`x` int) AS s3('http://minio1:9001/root/data/test6.csv', 'minio', '[HIDDEN]', 'CSV')",
|
||||
"CREATE TABLE tablefunc9 (`x` int) AS s3('http://minio1:9001/root/data/test7.csv', 'minio', '[HIDDEN]', 'CSV', 'x int')",
|
||||
"CREATE TABLE tablefunc10 (`x` int) AS s3('http://minio1:9001/root/data/test8.csv.gz', 'minio', '[HIDDEN]', 'CSV', 'x int', 'gzip')",
|
||||
"CREATE TABLE tablefunc11 (`x` int) AS s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test1.csv', 'minio', '[HIDDEN]')",
|
||||
"CREATE TABLE tablefunc12 (`x` int) AS s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test2.csv', 'CSV', 'x int')",
|
||||
"CREATE TABLE tablefunc13 (`x` int) AS s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test3.csv', 'minio', '[HIDDEN]', 'CSV')",
|
||||
"CREATE TABLE tablefunc14 (`x` int) AS remote('127.{2..11}', default.remote_table)",
|
||||
"CREATE TABLE tablefunc15 (`x` int) AS remote('127.{2..11}', default.remote_table, rand())",
|
||||
"CREATE TABLE tablefunc16 (`x` int) AS remote('127.{2..11}', default.remote_table, 'remote_user')",
|
||||
"CREATE TABLE tablefunc17 (`x` int) AS remote('127.{2..11}', default.remote_table, 'remote_user', '[HIDDEN]')",
|
||||
"CREATE TABLE tablefunc18 (`x` int) AS remote('127.{2..11}', default.remote_table, 'remote_user', rand())",
|
||||
"CREATE TABLE tablefunc19 (`x` int) AS remote('127.{2..11}', default.remote_table, 'remote_user', '[HIDDEN]', rand())",
|
||||
"CREATE TABLE tablefunc20 (`x` int) AS remote('127.{2..11}', 'default.remote_table', 'remote_user', '[HIDDEN]', rand())",
|
||||
"CREATE TABLE tablefunc21 (`x` int) AS remote('127.{2..11}', 'default', 'remote_table', 'remote_user', '[HIDDEN]', rand())",
|
||||
"CREATE TABLE tablefunc22 (`x` int) AS remote('127.{2..11}', numbers(10), 'remote_user', '[HIDDEN]', rand())",
|
||||
"CREATE TABLE tablefunc23 (`x` int) AS remoteSecure('127.{2..11}', 'default', 'remote_table', 'remote_user', '[HIDDEN]')",
|
||||
"CREATE TABLE tablefunc24 (`x` int) AS remoteSecure('127.{2..11}', 'default', 'remote_table', 'remote_user', rand())",
|
||||
],
|
||||
must_not_contain=["qwe126"],
|
||||
)
|
||||
|
||||
for i in range(0, len(table_functions)):
|
||||
node.query(f"DROP TABLE IF EXISTS tablefunc{i}")
|
||||
|
||||
|
||||
def test_encryption_functions():
|
||||
encryption_functions = [
|
||||
"encrypt('aes-256-ofb', 'qwe127', 'encryptionkey_encryptionkey_encr')",
|
||||
"encrypt('aes-256-ofb', 'qwe127', 'encryptionkey_encryptionkey_encr', 'iv_iv_iv_iv_iv_i')",
|
||||
"encrypt('aes-256-gcm', 'qwe127', 'encryptionkey_encryptionkey_encr', 'iv_iv_iv')",
|
||||
"encrypt('aes-256-gcm', 'qwe127', 'encryptionkey_encryptionkey_encr', 'iv_iv_iv', 'add')",
|
||||
"decrypt('aes-256-ofb', unhex('3AC43029BF24'), 'encryptionkey_encryptionkey_encr', 'iv_iv_iv_iv_iv_i')",
|
||||
"aes_encrypt_mysql('aes-256-ofb', 'qwe127', 'encryptionkey_encryptionkey_encr', 'iv_iv_iv_iv_iv_i')",
|
||||
"aes_decrypt_mysql('aes-256-ofb', unhex('3AC43029BF24'), 'encryptionkey_encryptionkey_encr', 'iv_iv_iv_iv_iv_i')",
|
||||
"tryDecrypt('aes-256-ofb', unhex('3AC43029BF24'), 'encryptionkey_encryptionkey_encr', 'iv_iv_iv_iv_iv_i')",
|
||||
]
|
||||
|
||||
for encryption_function in encryption_functions:
|
||||
node.query(f"SELECT {encryption_function}")
|
||||
|
||||
check_logs(
|
||||
must_contain=[
|
||||
"SELECT encrypt('aes-256-ofb', '[HIDDEN]')",
|
||||
"SELECT encrypt('aes-256-gcm', '[HIDDEN]')",
|
||||
"SELECT decrypt('aes-256-ofb', '[HIDDEN]')",
|
||||
"SELECT aes_encrypt_mysql('aes-256-ofb', '[HIDDEN]')",
|
||||
"SELECT aes_decrypt_mysql('aes-256-ofb', '[HIDDEN]')",
|
||||
"SELECT tryDecrypt('aes-256-ofb', '[HIDDEN]')",
|
||||
],
|
||||
must_not_contain=["qwe127", "3AC43029BF24"],
|
||||
)
|
||||
|
||||
|
||||
def test_create_dictionary():
|
||||
node.query(
|
||||
"CREATE DICTIONARY dict1 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n "
|
||||
"SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'user1' TABLE 'test' PASSWORD 'qwe128' DB 'default')) "
|
||||
"LIFETIME(MIN 0 MAX 10) LAYOUT(FLAT())"
|
||||
)
|
||||
|
||||
check_logs(
|
||||
must_contain=[
|
||||
"CREATE DICTIONARY 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())"
|
||||
],
|
||||
must_not_contain=["qwe128"],
|
||||
)
|
||||
|
||||
node.query("DROP DICTIONARY IF EXISTS dict1")
|
||||
|
||||
|
||||
def test_backup_to_s3():
|
||||
node.query("CREATE TABLE temptbl (x int) ENGINE=Log")
|
||||
|
||||
queries = [
|
||||
"BACKUP TABLE temptbl TO S3('http://minio1:9001/root/data/backups/backup1', 'minio', 'qwe129')",
|
||||
"RESTORE TABLE temptbl AS temptbl2 FROM S3('http://minio1:9001/root/data/backups/backup1', 'minio', 'qwe129')",
|
||||
]
|
||||
|
||||
for query in queries:
|
||||
# query_and_get_answer_with_error() is used here because we don't want to stop on error "Cannot connect to AWS".
|
||||
# We test logging here and not actual work with AWS server.
|
||||
node.query_and_get_answer_with_error(query)
|
||||
|
||||
check_logs(
|
||||
must_contain=[
|
||||
"BACKUP TABLE temptbl TO S3('http://minio1:9001/root/data/backups/backup1', 'minio', '[HIDDEN]')",
|
||||
"RESTORE TABLE temptbl AS temptbl2 FROM S3('http://minio1:9001/root/data/backups/backup1', 'minio', '[HIDDEN]')",
|
||||
],
|
||||
must_not_contain=["qwe129"],
|
||||
)
|
||||
|
||||
node.query("DROP TABLE IF EXISTS temptbl")
|
||||
node.query("DROP TABLE IF EXISTS temptbl2")
|
||||
|
@ -1,10 +1,10 @@
|
||||
===http===
|
||||
{"query":"select 1 from remote('127.0.0.2', system, one) format Null\n","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1}
|
||||
{"query":"SELECT 1 FROM remote('127.0.0.2', system, one) FORMAT `Null`","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1}
|
||||
{"query":"DESC TABLE system.one","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1}
|
||||
{"query":"SELECT 1 FROM `system`.`one`","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1}
|
||||
{"query":"DESC TABLE system.one","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1}
|
||||
{"query":"SELECT 1 FROM `system`.`one`","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1}
|
||||
{"query":"select 1 from remote('127.0.0.2', system, one) format Null\n","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1}
|
||||
{"query":"SELECT 1 FROM remote('127.0.0.2', system, one) FORMAT `Null`","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1}
|
||||
{"total spans":"3","unique spans":"3","unique non-zero parent spans":"3"}
|
||||
{"initial query spans with proper parent":"1"}
|
||||
{"unique non-empty tracestate values":"1"}
|
||||
|
Loading…
Reference in New Issue
Block a user