Merge pull request #51038 from evillique/headers-blacklist

Add HTTP header filtering
This commit is contained in:
Nikita Mikhaylov 2023-07-19 15:49:12 +02:00 committed by GitHub
commit 4a3f5234af
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
19 changed files with 177 additions and 5 deletions

View File

@ -887,6 +887,7 @@ try
#endif
global_context->setRemoteHostFilter(config());
global_context->setHTTPHeaderFilter(config());
std::string path_str = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH));
fs::path path = path_str;
@ -1200,6 +1201,7 @@ try
}
global_context->setRemoteHostFilter(*config);
global_context->setHTTPHeaderFilter(*config);
global_context->setMaxTableSizeToDrop(server_settings_.max_table_size_to_drop);
global_context->setMaxPartitionSizeToDrop(server_settings_.max_partition_size_to_drop);

View File

@ -866,6 +866,14 @@
-->
<!--</remote_url_allow_hosts>-->
<!-- The list of HTTP headers forbidden to use in HTTP-related storage engines and table functions.
If this section is not present in configuration, all headers are allowed.
-->
<!-- <http_forbid_headers>
<header>exact_header</header>
<header_regexp>(?i)(case_insensitive_header)</header_regexp>
</http_forbid_headers> -->
<!-- If element has 'incl' attribute, then for it's value will be used corresponding substitution from another file.
By default, path to file with substitutions is /etc/metrika.xml. It could be changed in config in 'include_from' element.
Values for substitutions are specified in /clickhouse/name_of_substitution elements in that file.

View File

@ -0,0 +1,56 @@
#include <Common/HTTPHeaderFilter.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/Exception.h>
#include <re2/re2.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
void HTTPHeaderFilter::checkHeaders(const HTTPHeaderEntries & entries) const
{
std::lock_guard guard(mutex);
for (const auto & entry : entries)
{
if (forbidden_headers.contains(entry.name))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "HTTP header \"{}\" is forbidden in configuration file, "
"see <http_forbid_headers>", entry.name);
for (const auto & header_regex : forbidden_headers_regexp)
if (re2::RE2::FullMatch(entry.name, header_regex))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "HTTP header \"{}\" is forbidden in configuration file, "
"see <http_forbid_headers>", entry.name);
}
}
void HTTPHeaderFilter::setValuesFromConfig(const Poco::Util::AbstractConfiguration & config)
{
std::lock_guard guard(mutex);
if (config.has("http_forbid_headers"))
{
std::vector<std::string> keys;
config.keys("http_forbid_headers", keys);
for (const auto & key : keys)
{
if (startsWith(key, "header_regexp"))
forbidden_headers_regexp.push_back(config.getString("http_forbid_headers." + key));
else if (startsWith(key, "header"))
forbidden_headers.insert(config.getString("http_forbid_headers." + key));
}
}
else
{
forbidden_headers.clear();
forbidden_headers_regexp.clear();
}
}
}

View File

@ -0,0 +1,27 @@
#pragma once
#include <IO/HTTPHeaderEntries.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <vector>
#include <unordered_set>
#include <mutex>
namespace DB
{
class HTTPHeaderFilter
{
public:
void setValuesFromConfig(const Poco::Util::AbstractConfiguration & config);
void checkHeaders(const HTTPHeaderEntries & entries) const;
private:
std::unordered_set<std::string> forbidden_headers;
std::vector<std::string> forbidden_headers_regexp;
mutable std::mutex mutex;
};
}

View File

@ -257,7 +257,6 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
const auto & headers_prefix = settings_config_prefix + ".headers";
if (config.has(headers_prefix))
{
Poco::Util::AbstractConfiguration::Keys config_keys;
@ -297,7 +296,10 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix);
if (created_from_ddl)
{
context->getRemoteHostFilter().checkURL(Poco::URI(configuration.url));
context->getHTTPHeaderFilter().checkHeaders(configuration.header_entries);
}
return std::make_unique<HTTPDictionarySource>(dict_struct, configuration, credentials, sample_block, context);
};

View File

@ -98,6 +98,7 @@
#include <Common/logger_useful.h>
#include <base/EnumReflection.h>
#include <Common/RemoteHostFilter.h>
#include <Common/HTTPHeaderFilter.h>
#include <Interpreters/AsynchronousInsertQueue.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/JIT/CompiledExpressionCache.h>
@ -327,9 +328,10 @@ struct ContextSharedPart : boost::noncopyable
OrdinaryBackgroundExecutorPtr fetch_executor;
OrdinaryBackgroundExecutorPtr common_executor;
RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml
RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml
HTTPHeaderFilter http_header_filter; /// Forbidden HTTP headers from config.xml
std::optional<TraceCollector> trace_collector; /// Thread collecting traces from threads executing queries
std::optional<TraceCollector> trace_collector; /// Thread collecting traces from threads executing queries
/// Clusters for distributed tables
/// Initialized on demand (on distributed storages initialization) since Settings should be initialized
@ -2963,6 +2965,16 @@ const RemoteHostFilter & Context::getRemoteHostFilter() const
return shared->remote_host_filter;
}
void Context::setHTTPHeaderFilter(const Poco::Util::AbstractConfiguration & config)
{
shared->http_header_filter.setValuesFromConfig(config);
}
const HTTPHeaderFilter & Context::getHTTPHeaderFilter() const
{
return shared->http_header_filter;
}
UInt16 Context::getTCPPort() const
{
auto lock = getLock();

View File

@ -6,6 +6,7 @@
#include <Common/isLocalAddress.h>
#include <Common/MultiVersion.h>
#include <Common/RemoteHostFilter.h>
#include <Common/HTTPHeaderFilter.h>
#include <Common/ThreadPool_fwd.h>
#include <Common/Throttler_fwd.h>
#include <Core/NamesAndTypes.h>
@ -766,6 +767,10 @@ public:
void setRemoteHostFilter(const Poco::Util::AbstractConfiguration & config);
const RemoteHostFilter & getRemoteHostFilter() const;
/// Storage of forbidden HTTP headers from config.xml
void setHTTPHeaderFilter(const Poco::Util::AbstractConfiguration & config);
const HTTPHeaderFilter & getHTTPHeaderFilter() const;
/// The port that the server listens for executing SQL queries.
UInt16 getTCPPort() const;

View File

@ -974,6 +974,7 @@ StorageS3::StorageS3(
FormatFactory::instance().checkFormatName(configuration.format);
context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration.url.uri);
context_->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(configuration.headers_from_ast);
StorageInMemoryMetadata storage_metadata;
if (columns_.empty())

View File

@ -44,6 +44,8 @@ StorageS3Cluster::StorageS3Cluster(
, s3_configuration{configuration_}
{
context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.url.uri);
context_->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(configuration_.headers_from_ast);
StorageInMemoryMetadata storage_metadata;
updateConfigurationIfChanged(context_);

View File

@ -1019,6 +1019,7 @@ StorageURL::StorageURL(
distributed_processing_)
{
context_->getRemoteHostFilter().checkURL(Poco::URI(uri));
context_->getHTTPHeaderFilter().checkHeaders(headers);
}

View File

@ -48,6 +48,7 @@ StorageURLCluster::StorageURLCluster(
, uri(uri_)
{
context_->getRemoteHostFilter().checkURL(Poco::URI(uri));
context_->getHTTPHeaderFilter().checkHeaders(configuration_.headers);
StorageInMemoryMetadata storage_metadata;

View File

@ -18,6 +18,8 @@
#include <Storages/NamedCollectionsHelpers.h>
#include <Formats/FormatFactory.h>
#include "registerTableFunctions.h"
#include <Analyzer/FunctionNode.h>
#include <Analyzer/TableFunctionNode.h>
#include <boost/algorithm/string.hpp>
@ -32,6 +34,24 @@ namespace ErrorCodes
}
std::vector<size_t> TableFunctionS3::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr) const
{
auto & table_function_node = query_node_table_function->as<TableFunctionNode &>();
auto & table_function_arguments_nodes = table_function_node.getArguments().getNodes();
size_t table_function_arguments_size = table_function_arguments_nodes.size();
std::vector<size_t> result;
for (size_t i = 0; i < table_function_arguments_size; ++i)
{
auto * function_node = table_function_arguments_nodes[i]->as<FunctionNode>();
if (function_node && function_node->getFunctionName() == "headers")
result.push_back(i);
}
return result;
}
/// This is needed to avoid copy-pase. Because s3Cluster arguments only differ in additional argument (first) - cluster name
void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context)
{
@ -41,13 +61,14 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context
}
else
{
if (args.empty() || args.size() > 6)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "The signature of table function {} shall be the following:\n{}", getName(), getSignature());
auto * header_it = StorageURL::collectHeaders(args, configuration.headers_from_ast, context);
if (header_it != args.end())
args.erase(header_it);
if (args.empty() || args.size() > 6)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "The signature of table function {} shall be the following:\n{}", getName(), getSignature());
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);

View File

@ -73,6 +73,10 @@ protected:
mutable StorageS3::Configuration configuration;
ColumnsDescription structure_hint;
private:
std::vector<size_t> skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override;
};
}

View File

@ -0,0 +1,6 @@
<clickhouse>
<http_forbid_headers>
<header>exact_header</header>
<header_regexp>(?i)(case_insensitive_header)</header_regexp>
</http_forbid_headers>
</clickhouse>

View File

@ -51,6 +51,7 @@ ln -sf $SRC_PATH/config.d/session_log.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/system_unfreeze.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/enable_zero_copy_replication.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/nlp.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/forbidden_headers.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/enable_keeper_map.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/custom_disks_base_path.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/display_name.xml $DEST_SERVER_PATH/config.d/

View File

@ -0,0 +1,18 @@
-- Tags: no-fasttest
-- Tag no-fasttest: Depends on AWS
SELECT * FROM url('http://localhost:8123/', LineAsString, headers('exact_header' = 'value')); -- { serverError BAD_ARGUMENTS }
SELECT * FROM url('http://localhost:8123/', LineAsString, headers('cAsE_INSENSITIVE_header' = 'value')); -- { serverError BAD_ARGUMENTS }
SELECT * FROM url('http://localhost:8123/', LineAsString, headers('random_header' = 'value')) FORMAT Null;
SELECT * FROM urlCluster('test_cluster_two_shards_localhost', 'http://localhost:8123/', LineAsString, headers('exact_header' = 'value')); -- { serverError BAD_ARGUMENTS }
SELECT * FROM urlCluster('test_cluster_two_shards_localhost', 'http://localhost:8123/', LineAsString, headers('cAsE_INSENSITIVE_header' = 'value')); -- { serverError BAD_ARGUMENTS }
SELECT * FROM urlCluster('test_cluster_two_shards_localhost', 'http://localhost:8123/', LineAsString, headers('random_header' = 'value')) FORMAT Null;
SELECT * FROM s3('http://localhost:8123/123/4', LineAsString, headers('exact_header' = 'value')); -- { serverError BAD_ARGUMENTS }
SELECT * FROM s3('http://localhost:8123/123/4', LineAsString, headers('cAsE_INSENSITIVE_header' = 'value')); -- { serverError BAD_ARGUMENTS }
SELECT * FROM s3('http://localhost:8123/123/4', LineAsString, headers('random_header' = 'value')); -- { serverError S3_ERROR }
SELECT * FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:8123/123/4', LineAsString, headers('exact_header' = 'value')); -- { serverError BAD_ARGUMENTS }
SELECT * FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:8123/123/4', LineAsString, headers('cAsE_INSENSITIVE_header' = 'value')); -- { serverError BAD_ARGUMENTS }
SELECT * FROM s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:8123/123/4', LineAsString, headers('random_header' = 'value')); -- { serverError S3_ERROR }

View File

@ -0,0 +1,5 @@
-- Tags: no-fasttest
-- Tag no-fasttest: Depends on AWS
SELECT * FROM s3(headers('random_header' = 'value')); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT * FROM s3Cluster('test_cluster_two_shards_localhost', headers('random_header' = 'value')); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }