mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #51038 from evillique/headers-blacklist
Add HTTP header filtering
This commit is contained in:
commit
4a3f5234af
@ -887,6 +887,7 @@ try
|
|||||||
#endif
|
#endif
|
||||||
|
|
||||||
global_context->setRemoteHostFilter(config());
|
global_context->setRemoteHostFilter(config());
|
||||||
|
global_context->setHTTPHeaderFilter(config());
|
||||||
|
|
||||||
std::string path_str = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH));
|
std::string path_str = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH));
|
||||||
fs::path path = path_str;
|
fs::path path = path_str;
|
||||||
@ -1200,6 +1201,7 @@ try
|
|||||||
}
|
}
|
||||||
|
|
||||||
global_context->setRemoteHostFilter(*config);
|
global_context->setRemoteHostFilter(*config);
|
||||||
|
global_context->setHTTPHeaderFilter(*config);
|
||||||
|
|
||||||
global_context->setMaxTableSizeToDrop(server_settings_.max_table_size_to_drop);
|
global_context->setMaxTableSizeToDrop(server_settings_.max_table_size_to_drop);
|
||||||
global_context->setMaxPartitionSizeToDrop(server_settings_.max_partition_size_to_drop);
|
global_context->setMaxPartitionSizeToDrop(server_settings_.max_partition_size_to_drop);
|
||||||
|
@ -866,6 +866,14 @@
|
|||||||
-->
|
-->
|
||||||
<!--</remote_url_allow_hosts>-->
|
<!--</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.
|
<!-- 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.
|
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.
|
Values for substitutions are specified in /clickhouse/name_of_substitution elements in that file.
|
||||||
|
56
src/Common/HTTPHeaderFilter.cpp
Normal file
56
src/Common/HTTPHeaderFilter.cpp
Normal 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();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
27
src/Common/HTTPHeaderFilter.h
Normal file
27
src/Common/HTTPHeaderFilter.h
Normal 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;
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -257,7 +257,6 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
|
|||||||
|
|
||||||
const auto & headers_prefix = settings_config_prefix + ".headers";
|
const auto & headers_prefix = settings_config_prefix + ".headers";
|
||||||
|
|
||||||
|
|
||||||
if (config.has(headers_prefix))
|
if (config.has(headers_prefix))
|
||||||
{
|
{
|
||||||
Poco::Util::AbstractConfiguration::Keys config_keys;
|
Poco::Util::AbstractConfiguration::Keys config_keys;
|
||||||
@ -297,7 +296,10 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
|
|||||||
auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix);
|
auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix);
|
||||||
|
|
||||||
if (created_from_ddl)
|
if (created_from_ddl)
|
||||||
|
{
|
||||||
context->getRemoteHostFilter().checkURL(Poco::URI(configuration.url));
|
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);
|
return std::make_unique<HTTPDictionarySource>(dict_struct, configuration, credentials, sample_block, context);
|
||||||
};
|
};
|
||||||
|
@ -98,6 +98,7 @@
|
|||||||
#include <Common/logger_useful.h>
|
#include <Common/logger_useful.h>
|
||||||
#include <base/EnumReflection.h>
|
#include <base/EnumReflection.h>
|
||||||
#include <Common/RemoteHostFilter.h>
|
#include <Common/RemoteHostFilter.h>
|
||||||
|
#include <Common/HTTPHeaderFilter.h>
|
||||||
#include <Interpreters/AsynchronousInsertQueue.h>
|
#include <Interpreters/AsynchronousInsertQueue.h>
|
||||||
#include <Interpreters/DatabaseCatalog.h>
|
#include <Interpreters/DatabaseCatalog.h>
|
||||||
#include <Interpreters/JIT/CompiledExpressionCache.h>
|
#include <Interpreters/JIT/CompiledExpressionCache.h>
|
||||||
@ -328,6 +329,7 @@ struct ContextSharedPart : boost::noncopyable
|
|||||||
OrdinaryBackgroundExecutorPtr common_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
|
||||||
|
|
||||||
@ -2963,6 +2965,16 @@ const RemoteHostFilter & Context::getRemoteHostFilter() const
|
|||||||
return shared->remote_host_filter;
|
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
|
UInt16 Context::getTCPPort() const
|
||||||
{
|
{
|
||||||
auto lock = getLock();
|
auto lock = getLock();
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
#include <Common/isLocalAddress.h>
|
#include <Common/isLocalAddress.h>
|
||||||
#include <Common/MultiVersion.h>
|
#include <Common/MultiVersion.h>
|
||||||
#include <Common/RemoteHostFilter.h>
|
#include <Common/RemoteHostFilter.h>
|
||||||
|
#include <Common/HTTPHeaderFilter.h>
|
||||||
#include <Common/ThreadPool_fwd.h>
|
#include <Common/ThreadPool_fwd.h>
|
||||||
#include <Common/Throttler_fwd.h>
|
#include <Common/Throttler_fwd.h>
|
||||||
#include <Core/NamesAndTypes.h>
|
#include <Core/NamesAndTypes.h>
|
||||||
@ -766,6 +767,10 @@ public:
|
|||||||
void setRemoteHostFilter(const Poco::Util::AbstractConfiguration & config);
|
void setRemoteHostFilter(const Poco::Util::AbstractConfiguration & config);
|
||||||
const RemoteHostFilter & getRemoteHostFilter() const;
|
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.
|
/// The port that the server listens for executing SQL queries.
|
||||||
UInt16 getTCPPort() const;
|
UInt16 getTCPPort() const;
|
||||||
|
|
||||||
|
@ -974,6 +974,7 @@ StorageS3::StorageS3(
|
|||||||
|
|
||||||
FormatFactory::instance().checkFormatName(configuration.format);
|
FormatFactory::instance().checkFormatName(configuration.format);
|
||||||
context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration.url.uri);
|
context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration.url.uri);
|
||||||
|
context_->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(configuration.headers_from_ast);
|
||||||
|
|
||||||
StorageInMemoryMetadata storage_metadata;
|
StorageInMemoryMetadata storage_metadata;
|
||||||
if (columns_.empty())
|
if (columns_.empty())
|
||||||
|
@ -44,6 +44,8 @@ StorageS3Cluster::StorageS3Cluster(
|
|||||||
, s3_configuration{configuration_}
|
, s3_configuration{configuration_}
|
||||||
{
|
{
|
||||||
context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.url.uri);
|
context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.url.uri);
|
||||||
|
context_->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(configuration_.headers_from_ast);
|
||||||
|
|
||||||
StorageInMemoryMetadata storage_metadata;
|
StorageInMemoryMetadata storage_metadata;
|
||||||
updateConfigurationIfChanged(context_);
|
updateConfigurationIfChanged(context_);
|
||||||
|
|
||||||
|
@ -1019,6 +1019,7 @@ StorageURL::StorageURL(
|
|||||||
distributed_processing_)
|
distributed_processing_)
|
||||||
{
|
{
|
||||||
context_->getRemoteHostFilter().checkURL(Poco::URI(uri));
|
context_->getRemoteHostFilter().checkURL(Poco::URI(uri));
|
||||||
|
context_->getHTTPHeaderFilter().checkHeaders(headers);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -48,6 +48,7 @@ StorageURLCluster::StorageURLCluster(
|
|||||||
, uri(uri_)
|
, uri(uri_)
|
||||||
{
|
{
|
||||||
context_->getRemoteHostFilter().checkURL(Poco::URI(uri));
|
context_->getRemoteHostFilter().checkURL(Poco::URI(uri));
|
||||||
|
context_->getHTTPHeaderFilter().checkHeaders(configuration_.headers);
|
||||||
|
|
||||||
StorageInMemoryMetadata storage_metadata;
|
StorageInMemoryMetadata storage_metadata;
|
||||||
|
|
||||||
|
@ -18,6 +18,8 @@
|
|||||||
#include <Storages/NamedCollectionsHelpers.h>
|
#include <Storages/NamedCollectionsHelpers.h>
|
||||||
#include <Formats/FormatFactory.h>
|
#include <Formats/FormatFactory.h>
|
||||||
#include "registerTableFunctions.h"
|
#include "registerTableFunctions.h"
|
||||||
|
#include <Analyzer/FunctionNode.h>
|
||||||
|
#include <Analyzer/TableFunctionNode.h>
|
||||||
|
|
||||||
#include <boost/algorithm/string.hpp>
|
#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
|
/// 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)
|
void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context)
|
||||||
{
|
{
|
||||||
@ -41,13 +61,14 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context
|
|||||||
}
|
}
|
||||||
else
|
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);
|
auto * header_it = StorageURL::collectHeaders(args, configuration.headers_from_ast, context);
|
||||||
if (header_it != args.end())
|
if (header_it != args.end())
|
||||||
args.erase(header_it);
|
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)
|
for (auto & arg : args)
|
||||||
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
|
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
|
||||||
|
|
||||||
|
@ -73,6 +73,10 @@ protected:
|
|||||||
|
|
||||||
mutable StorageS3::Configuration configuration;
|
mutable StorageS3::Configuration configuration;
|
||||||
ColumnsDescription structure_hint;
|
ColumnsDescription structure_hint;
|
||||||
|
|
||||||
|
private:
|
||||||
|
|
||||||
|
std::vector<size_t> skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
6
tests/config/config.d/forbidden_headers.xml
Normal file
6
tests/config/config.d/forbidden_headers.xml
Normal 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>
|
@ -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/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/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/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/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/custom_disks_base_path.xml $DEST_SERVER_PATH/config.d/
|
||||||
ln -sf $SRC_PATH/config.d/display_name.xml $DEST_SERVER_PATH/config.d/
|
ln -sf $SRC_PATH/config.d/display_name.xml $DEST_SERVER_PATH/config.d/
|
||||||
|
18
tests/queries/0_stateless/02752_forbidden_headers.sql
Normal file
18
tests/queries/0_stateless/02752_forbidden_headers.sql
Normal 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 }
|
0
tests/queries/0_stateless/02772_s3_crash.reference
Normal file
0
tests/queries/0_stateless/02772_s3_crash.reference
Normal file
5
tests/queries/0_stateless/02772_s3_crash.sql
Normal file
5
tests/queries/0_stateless/02772_s3_crash.sql
Normal 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 }
|
Loading…
Reference in New Issue
Block a user