Mask azure connection string sensitive info

This commit is contained in:
Alex Katsman 2024-09-12 18:17:41 +00:00
parent 7fd2207626
commit b88cd79959
4 changed files with 97 additions and 22 deletions

View File

@ -722,7 +722,14 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
assert_cast<const ASTFunction *>(argument.get())->arguments->children[0]->formatImpl(settings, state, nested_dont_need_parens);
settings.ostr << (settings.hilite ? hilite_operator : "") << " = " << (settings.hilite ? hilite_none : "");
}
settings.ostr << "'[HIDDEN]'";
if (!secret_arguments.replacement.empty())
{
settings.ostr << "'" << secret_arguments.replacement << "'";
}
else
{
settings.ostr << "'[HIDDEN]'";
}
if (size <= secret_arguments.start + secret_arguments.count && !secret_arguments.are_named)
break; /// All other arguments should also be hidden.
continue;

View File

@ -1,6 +1,7 @@
#pragma once
#include <Common/KnownObjectNames.h>
#include <Common/re2.h>
#include <Core/QualifiedTableName.h>
#include <base/defines.h>
#include <boost/algorithm/string/predicate.hpp>
@ -49,6 +50,11 @@ public:
bool are_named = false; /// Arguments like `password = 'password'` are considered as named arguments.
/// E.g. "headers" in `url('..', headers('foo' = '[HIDDEN]'))`
std::vector<std::string> nested_maps;
/// Full replacement of an argument. Only supported when count is 1, otherwise all arguments will be replaced with this string.
/// It's needed in cases when we don't want to hide the entire parameter, but some part of it, e.g. "connection_string" in
/// `azureBlobStorage('DefaultEndpointsProtocol=https;AccountKey=secretkey;...', ...)` should be replaced with
/// `azureBlobStorage('DefaultEndpointsProtocol=https;AccountKey=[HIDDEN];...', ...)`.
std::string replacement;
bool hasSecrets() const
{
@ -74,6 +80,7 @@ protected:
result.are_named = argument_is_named;
}
chassert(index >= result.start); /// We always check arguments consecutively
chassert(result.replacement.empty()); /// We shouldn't use replacement with masking other arguments
result.count = index + 1 - result.start;
if (!argument_is_named)
result.are_named = false;
@ -199,32 +206,39 @@ protected:
void findAzureBlobStorageFunctionSecretArguments(bool is_cluster_function)
{
/// azureBlobStorage('cluster_name', 'conn_string/storage_account_url', ...) has 'conn_string/storage_account_url' as its second argument.
/// azureBlobStorageCluster('cluster_name', 'conn_string/storage_account_url', ...) has 'conn_string/storage_account_url' as its second argument.
size_t url_arg_idx = is_cluster_function ? 1 : 0;
if (!is_cluster_function && isNamedCollectionName(0))
{
/// azureBlobStorage(named_collection, ..., account_key = 'account_key', ...)
if (maskAzureConnectionString(-1, true, 1))
return;
findSecretNamedArgument("account_key", 1);
return;
}
else if (is_cluster_function && isNamedCollectionName(1))
{
/// azureBlobStorageCluster(cluster, named_collection, ..., account_key = 'account_key', ...)
if (maskAzureConnectionString(-1, true, 2))
return;
findSecretNamedArgument("account_key", 2);
return;
}
/// We should check other arguments first because we don't need to do any replacement in case storage_account_url is not used
/// azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure)
/// azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])
if (maskAzureConnectionString(url_arg_idx))
return;
/// We should check other arguments first because we don't need to do any replacement in case of
/// azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, format, [account_name, account_key, ...])
/// azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, format, [account_name, account_key, ...])
size_t count = function->arguments->size();
if ((url_arg_idx + 4 <= count) && (count <= url_arg_idx + 7))
{
String second_arg;
if (tryGetStringFromArgument(url_arg_idx + 3, &second_arg))
String fourth_arg;
if (tryGetStringFromArgument(url_arg_idx + 3, &fourth_arg))
{
if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg))
if (fourth_arg == "auto" || KnownFormatNames::instance().exists(fourth_arg))
return; /// The argument after 'url' is a format: s3('url', 'format', ...)
}
}
@ -234,6 +248,40 @@ protected:
markSecretArgument(url_arg_idx + 4);
}
bool maskAzureConnectionString(ssize_t url_arg_idx, bool argument_is_named = false, size_t start = 0)
{
String url_arg;
if (argument_is_named)
{
url_arg_idx = findNamedArgument(&url_arg, "connection_string", start);
if (url_arg_idx == -1 || url_arg.empty())
url_arg_idx = findNamedArgument(&url_arg, "storage_account_url", start);
if (url_arg_idx == -1 || url_arg.empty())
return false;
}
else
{
if (!tryGetStringFromArgument(url_arg_idx, &url_arg))
return false;
}
if (!url_arg.starts_with("http"))
{
static re2::RE2 account_key_pattern = "AccountKey=.*?(;|$)";
if (RE2::Replace(&url_arg, account_key_pattern, "AccountKey=[HIDDEN]\\1"))
{
chassert(result.count == 0); /// We shouldn't use replacement with masking other arguments
result.start = url_arg_idx;
result.are_named = argument_is_named;
result.count = 1;
result.replacement = url_arg;
return true;
}
}
return false;
}
void findURLSecretArguments()
{
if (!isNamedCollectionName(0))
@ -513,8 +561,9 @@ protected:
return function->arguments->at(arg_idx)->isIdentifier();
}
/// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified.
void findSecretNamedArgument(const std::string_view & key, size_t start = 0)
/// Looks for an argument with a specified name. This function looks for arguments in format `key=value` where the key is specified.
/// Returns -1 if no argument was found.
ssize_t findNamedArgument(String * res, const std::string_view & key, size_t start = 0)
{
for (size_t i = start; i < function->arguments->size(); ++i)
{
@ -531,8 +580,22 @@ protected:
continue;
if (found_key == key)
markSecretArgument(i, /* argument_is_named= */ true);
{
tryGetStringFromArgument(*equals_func->arguments->at(1), res);
return i;
}
}
return -1;
}
/// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified.
/// If the argument is found, it is marked as a secret.
void findSecretNamedArgument(const std::string_view & key, size_t start = 0)
{
ssize_t arg_idx = findNamedArgument(nullptr, key, start);
if (arg_idx >= 0)
markSecretArgument(arg_idx, /* argument_is_named= */ true);
}
};

View File

@ -223,7 +223,7 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context,
{
account_name = fourth_arg;
account_key = checkAndGetLiteralArgument<String>(engine_args[4], "account_key");
auto sixth_arg = checkAndGetLiteralArgument<String>(engine_args[5], "format/account_name");
auto sixth_arg = checkAndGetLiteralArgument<String>(engine_args[5], "format/structure");
if (is_format_arg(sixth_arg))
{
format = sixth_arg;
@ -257,10 +257,10 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context,
}
else if (with_structure && engine_args.size() == 8)
{
auto fourth_arg = checkAndGetLiteralArgument<String>(engine_args[3], "format/account_name");
auto fourth_arg = checkAndGetLiteralArgument<String>(engine_args[3], "account_name");
account_name = fourth_arg;
account_key = checkAndGetLiteralArgument<String>(engine_args[4], "account_key");
auto sixth_arg = checkAndGetLiteralArgument<String>(engine_args[5], "format/account_name");
auto sixth_arg = checkAndGetLiteralArgument<String>(engine_args[5], "format");
if (!is_format_arg(sixth_arg))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg);
format = sixth_arg;

View File

@ -1,5 +1,6 @@
import pytest
import random, string
import re
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
@ -336,6 +337,10 @@ def test_create_database():
def test_table_functions():
password = new_password()
azure_conn_string = cluster.env_variables["AZURITE_CONNECTION_STRING"]
account_key_pattern = re.compile("AccountKey=.*?(;|$)")
masked_azure_conn_string = re.sub(
account_key_pattern, "AccountKey=[HIDDEN]\\1", azure_conn_string
)
azure_storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]
azure_account_name = "devstoreaccount1"
azure_account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw=="
@ -467,23 +472,23 @@ def test_table_functions():
"CREATE TABLE tablefunc30 (x int) AS s3('http://minio1:9001/root/data/test9.csv.gz', 'NOSIGN', 'CSV')",
"CREATE TABLE tablefunc31 (`x` int) AS s3('http://minio1:9001/root/data/test10.csv.gz', 'minio', '[HIDDEN]')",
"CREATE TABLE tablefunc32 (`x` int) AS deltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')",
f"CREATE TABLE tablefunc33 (x int) AS azureBlobStorage('{azure_conn_string}', 'cont', 'test_simple.csv', 'CSV')",
f"CREATE TABLE tablefunc34 (x int) AS azureBlobStorage('{azure_conn_string}', 'cont', 'test_simple_1.csv', 'CSV', 'none')",
f"CREATE TABLE tablefunc35 (x int) AS azureBlobStorage('{azure_conn_string}', 'cont', 'test_simple_2.csv', 'CSV', 'none', 'auto')",
f"CREATE TABLE tablefunc33 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple.csv', 'CSV')",
f"CREATE TABLE tablefunc34 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple_1.csv', 'CSV', 'none')",
f"CREATE TABLE tablefunc35 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple_2.csv', 'CSV', 'none', 'auto')",
f"CREATE TABLE tablefunc36 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_3.csv', '{azure_account_name}', '[HIDDEN]')",
f"CREATE TABLE tablefunc37 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_4.csv', '{azure_account_name}', '[HIDDEN]', 'CSV')",
f"CREATE TABLE tablefunc38 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_5.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none')",
f"CREATE TABLE tablefunc39 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')",
f"CREATE TABLE tablefunc40 (x int) AS azureBlobStorage(named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')",
f"CREATE TABLE tablefunc40 (`x` int) AS azureBlobStorage(named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')",
f"CREATE TABLE tablefunc41 (`x` int) AS azureBlobStorage(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')",
f"CREATE TABLE tablefunc42 (x int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_conn_string}', 'cont', 'test_simple_9.csv', 'CSV')",
f"CREATE TABLE tablefunc43 (x int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_conn_string}', 'cont', 'test_simple_10.csv', 'CSV', 'none')",
f"CREATE TABLE tablefunc44 (x int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_conn_string}', 'cont', 'test_simple_11.csv', 'CSV', 'none', 'auto')",
f"CREATE TABLE tablefunc42 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{masked_azure_conn_string}', 'cont', 'test_simple_9.csv', 'CSV')",
f"CREATE TABLE tablefunc43 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{masked_azure_conn_string}', 'cont', 'test_simple_10.csv', 'CSV', 'none')",
f"CREATE TABLE tablefunc44 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{masked_azure_conn_string}', 'cont', 'test_simple_11.csv', 'CSV', 'none', 'auto')",
f"CREATE TABLE tablefunc45 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_12.csv', '{azure_account_name}', '[HIDDEN]')",
f"CREATE TABLE tablefunc46 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_13.csv', '{azure_account_name}', '[HIDDEN]', 'CSV')",
f"CREATE TABLE tablefunc47 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_14.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none')",
f"CREATE TABLE tablefunc48 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_15.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')",
f"CREATE TABLE tablefunc49 (x int) AS azureBlobStorageCluster('test_shard_localhost', named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_16.csv', format = 'CSV')",
f"CREATE TABLE tablefunc49 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_16.csv', format = 'CSV')",
f"CREATE TABLE tablefunc50 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_17.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')",
"CREATE TABLE tablefunc51 (`x` int) AS iceberg('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')",
],