Skip to content

Commit

Permalink
Merge pull request #51038 from evillique/headers-blacklist
Browse files Browse the repository at this point in the history
Add HTTP header filtering
  • Loading branch information
nikitamikhaylov committed Jul 19, 2023
2 parents 06fe08d + ee0bbc0 commit 4a3f523
Show file tree
Hide file tree
Showing 19 changed files with 177 additions and 5 deletions.
2 changes: 2 additions & 0 deletions programs/server/Server.cpp
Expand Up @@ -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;
Expand Down Expand Up @@ -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);
Expand Down
8 changes: 8 additions & 0 deletions programs/server/config.xml
Expand Up @@ -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.
Expand Down
56 changes: 56 additions & 0 deletions src/Common/HTTPHeaderFilter.cpp
@@ -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();
}
}

}
27 changes: 27 additions & 0 deletions src/Common/HTTPHeaderFilter.h
@@ -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;
};

}
4 changes: 3 additions & 1 deletion src/Dictionaries/HTTPDictionarySource.cpp
Expand Up @@ -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;
Expand Down Expand Up @@ -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);
};
Expand Down
16 changes: 14 additions & 2 deletions src/Interpreters/Context.cpp
Expand Up @@ -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>
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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();
Expand Down
5 changes: 5 additions & 0 deletions src/Interpreters/Context.h
Expand Up @@ -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>
Expand Down Expand Up @@ -766,6 +767,10 @@ class Context: public std::enable_shared_from_this<Context>
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;

Expand Down
1 change: 1 addition & 0 deletions src/Storages/StorageS3.cpp
Expand Up @@ -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())
Expand Down
2 changes: 2 additions & 0 deletions src/Storages/StorageS3Cluster.cpp
Expand Up @@ -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_);

Expand Down
1 change: 1 addition & 0 deletions src/Storages/StorageURL.cpp
Expand Up @@ -1019,6 +1019,7 @@ StorageURL::StorageURL(
distributed_processing_)
{
context_->getRemoteHostFilter().checkURL(Poco::URI(uri));
context_->getHTTPHeaderFilter().checkHeaders(headers);
}


Expand Down
1 change: 1 addition & 0 deletions src/Storages/StorageURLCluster.cpp
Expand Up @@ -48,6 +48,7 @@ StorageURLCluster::StorageURLCluster(
, uri(uri_)
{
context_->getRemoteHostFilter().checkURL(Poco::URI(uri));
context_->getHTTPHeaderFilter().checkHeaders(configuration_.headers);

StorageInMemoryMetadata storage_metadata;

Expand Down
25 changes: 23 additions & 2 deletions src/TableFunctions/TableFunctionS3.cpp
Expand Up @@ -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>

Expand All @@ -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)
{
Expand All @@ -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);

Expand Down
4 changes: 4 additions & 0 deletions src/TableFunctions/TableFunctionS3.h
Expand Up @@ -73,6 +73,10 @@ class TableFunctionS3 : public ITableFunction

mutable StorageS3::Configuration configuration;
ColumnsDescription structure_hint;

private:

std::vector<size_t> skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override;
};

}
Expand Down
6 changes: 6 additions & 0 deletions tests/config/config.d/forbidden_headers.xml
@@ -0,0 +1,6 @@
<clickhouse>
<http_forbid_headers>
<header>exact_header</header>
<header_regexp>(?i)(case_insensitive_header)</header_regexp>
</http_forbid_headers>
</clickhouse>
1 change: 1 addition & 0 deletions tests/config/install.sh
Expand Up @@ -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/
Expand Down
Empty file.
18 changes: 18 additions & 0 deletions tests/queries/0_stateless/02752_forbidden_headers.sql
@@ -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 }
Empty file.
5 changes: 5 additions & 0 deletions tests/queries/0_stateless/02772_s3_crash.sql
@@ -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 }

0 comments on commit 4a3f523

Please sign in to comment.