Merge pull request #63314 from arthurpassos/no_proxy_support

Add `no_proxy` support
This commit is contained in:
Sema Checherinda 2024-06-13 15:48:12 +00:00 committed by GitHub
commit e8c0caa5ab
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
25 changed files with 538 additions and 231 deletions

View File

@ -2924,6 +2924,8 @@ Define proxy servers for HTTP and HTTPS requests, currently supported by S3 stor
There are three ways to define proxy servers: environment variables, proxy lists, and remote proxy resolvers.
Bypassing proxy servers for specific hosts is also supported with the use of `no_proxy`.
### Environment variables
The `http_proxy` and `https_proxy` environment variables allow you to specify a
@ -3033,6 +3035,29 @@ This also allows a mix of resolver types can be used.
By default, tunneling (i.e, `HTTP CONNECT`) is used to make `HTTPS` requests over `HTTP` proxy. This setting can be used to disable it.
### no_proxy
By default, all requests will go through the proxy. In order to disable it for specific hosts, the `no_proxy` variable must be set.
It can be set inside the `<proxy>` clause for list and remote resolvers and as an environment variable for environment resolver.
It supports IP addresses, domains, subdomains and `'*'` wildcard for full bypass. Leading dots are stripped just like curl does.
Example:
The below configuration bypasses proxy requests to `clickhouse.cloud` and all of its subdomains (e.g, `auth.clickhouse.cloud`).
The same applies to GitLab, even though it has a leading dot. Both `gitlab.com` and `about.gitlab.com` would bypass the proxy.
``` xml
<proxy>
<no_proxy>clickhouse.cloud,.gitlab.com</no_proxy>
<http>
<uri>http://proxy1</uri>
<uri>http://proxy2:3128</uri>
</http>
<https>
<uri>http://proxy1:3128</uri>
</https>
</proxy>
```
## max_materialized_views_count_for_table {#max_materialized_views_count_for_table}
A limit on the number of materialized views attached to a table.

View File

@ -1,6 +1,7 @@
#include "EnvironmentProxyConfigurationResolver.h"
#include <Common/logger_useful.h>
#include <Common/proxyConfigurationToPocoProxyConfig.h>
#include <Poco/URI.h>
namespace DB
@ -12,6 +13,7 @@ namespace DB
* */
static constexpr auto PROXY_HTTP_ENVIRONMENT_VARIABLE = "http_proxy";
static constexpr auto PROXY_HTTPS_ENVIRONMENT_VARIABLE = "https_proxy";
static constexpr auto NO_PROXY_ENVIRONMENT_VARIABLE = "no_proxy";
EnvironmentProxyConfigurationResolver::EnvironmentProxyConfigurationResolver(
Protocol request_protocol_, bool disable_tunneling_for_https_requests_over_http_proxy_)
@ -34,31 +36,60 @@ namespace
return std::getenv(PROXY_HTTPS_ENVIRONMENT_VARIABLE); // NOLINT(concurrency-mt-unsafe)
}
}
const char * getNoProxyHosts()
{
return std::getenv(NO_PROXY_ENVIRONMENT_VARIABLE); // NOLINT(concurrency-mt-unsafe)
}
ProxyConfiguration buildProxyConfiguration(
ProxyConfiguration::Protocol request_protocol,
const Poco::URI & uri,
const std::string & no_proxy_hosts_string,
bool disable_tunneling_for_https_requests_over_http_proxy)
{
if (uri.empty())
{
return {};
}
const auto & host = uri.getHost();
const auto & scheme = uri.getScheme();
const auto port = uri.getPort();
const bool use_tunneling_for_https_requests_over_http_proxy = ProxyConfiguration::useTunneling(
request_protocol,
ProxyConfiguration::protocolFromString(scheme),
disable_tunneling_for_https_requests_over_http_proxy);
LOG_TRACE(getLogger("EnvironmentProxyConfigurationResolver"), "Use proxy from environment: {}://{}:{}", scheme, host, port);
return ProxyConfiguration {
host,
ProxyConfiguration::protocolFromString(scheme),
port,
use_tunneling_for_https_requests_over_http_proxy,
request_protocol,
no_proxy_hosts_string
};
}
}
ProxyConfiguration EnvironmentProxyConfigurationResolver::resolve()
{
const auto * proxy_host = getProxyHost(request_protocol);
static const auto * http_proxy_host = getProxyHost(Protocol::HTTP);
static const auto * https_proxy_host = getProxyHost(Protocol::HTTPS);
static const auto * no_proxy = getNoProxyHosts();
static const auto poco_no_proxy_hosts = no_proxy ? buildPocoNonProxyHosts(no_proxy) : "";
if (!proxy_host)
{
return {};
}
static const Poco::URI http_proxy_uri(http_proxy_host ? http_proxy_host : "");
static const Poco::URI https_proxy_uri(https_proxy_host ? https_proxy_host : "");
auto uri = Poco::URI(proxy_host);
auto host = uri.getHost();
auto scheme = uri.getScheme();
auto port = uri.getPort();
LOG_TRACE(getLogger("EnvironmentProxyConfigurationResolver"), "Use proxy from environment: {}://{}:{}", scheme, host, port);
return ProxyConfiguration {
host,
ProxyConfiguration::protocolFromString(scheme),
port,
useTunneling(request_protocol, ProxyConfiguration::protocolFromString(scheme), disable_tunneling_for_https_requests_over_http_proxy),
request_protocol
};
return buildProxyConfiguration(
request_protocol,
request_protocol == Protocol::HTTP ? http_proxy_uri : https_proxy_uri,
poco_no_proxy_hosts,
disable_tunneling_for_https_requests_over_http_proxy);
}
}

View File

@ -9,6 +9,7 @@
#include <Common/ProxyConfiguration.h>
#include <Common/MemoryTrackerSwitcher.h>
#include <Common/SipHash.h>
#include <Common/proxyConfigurationToPocoProxyConfig.h>
#include <Poco/Net/HTTPChunkedStream.h>
#include <Poco/Net/HTTPClientSession.h>
@ -70,20 +71,6 @@ namespace CurrentMetrics
namespace
{
Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const DB::ProxyConfiguration & proxy_configuration)
{
Poco::Net::HTTPClientSession::ProxyConfig poco_proxy_config;
poco_proxy_config.host = proxy_configuration.host;
poco_proxy_config.port = proxy_configuration.port;
poco_proxy_config.protocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.protocol);
poco_proxy_config.tunnel = proxy_configuration.tunneling;
poco_proxy_config.originalRequestProtocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.original_request_protocol);
return poco_proxy_config;
}
constexpr size_t roundUp(size_t x, size_t rounding)
{
chassert(rounding > 0);
@ -696,7 +683,8 @@ struct EndpointPoolKey
proxy_config.port,
proxy_config.protocol,
proxy_config.tunneling,
proxy_config.original_request_protocol)
proxy_config.original_request_protocol,
proxy_config.no_proxy_hosts)
== std::tie(
rhs.connection_group,
rhs.target_host,
@ -706,7 +694,8 @@ struct EndpointPoolKey
rhs.proxy_config.port,
rhs.proxy_config.protocol,
rhs.proxy_config.tunneling,
rhs.proxy_config.original_request_protocol);
rhs.proxy_config.original_request_protocol,
rhs.proxy_config.no_proxy_hosts);
}
};

View File

@ -44,11 +44,18 @@ struct ProxyConfiguration
}
}
static bool useTunneling(Protocol request_protocol, Protocol proxy_protocol, bool disable_tunneling_for_https_requests_over_http_proxy)
{
bool is_https_request_over_http_proxy = request_protocol == Protocol::HTTPS && proxy_protocol == Protocol::HTTP;
return is_https_request_over_http_proxy && !disable_tunneling_for_https_requests_over_http_proxy;
}
std::string host = std::string{};
Protocol protocol = Protocol::HTTP;
uint16_t port = 0;
bool tunneling = false;
Protocol original_request_protocol = Protocol::HTTP;
std::string no_proxy_hosts = std::string{};
bool isEmpty() const { return host.empty(); }
};

View File

@ -19,13 +19,6 @@ struct ProxyConfigurationResolver
virtual void errorReport(const ProxyConfiguration & config) = 0;
protected:
static bool useTunneling(Protocol request_protocol, Protocol proxy_protocol, bool disable_tunneling_for_https_requests_over_http_proxy)
{
bool is_https_request_over_http_proxy = request_protocol == Protocol::HTTPS && proxy_protocol == Protocol::HTTP;
return is_https_request_over_http_proxy && !disable_tunneling_for_https_requests_over_http_proxy;
}
Protocol request_protocol;
bool disable_tunneling_for_https_requests_over_http_proxy = false;
};

View File

@ -1,6 +1,7 @@
#include <Common/ProxyConfigurationResolverProvider.h>
#include <Common/EnvironmentProxyConfigurationResolver.h>
#include <Common/proxyConfigurationToPocoProxyConfig.h>
#include <Common/Exception.h>
#include <Common/ProxyListConfigurationResolver.h>
#include <Common/RemoteProxyConfigurationResolver.h>
@ -17,6 +18,11 @@ namespace ErrorCodes
namespace
{
std::string getNoProxyHosts(const Poco::Util::AbstractConfiguration & configuration)
{
return configuration.getString("proxy.no_proxy", "");
}
bool isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(
const Poco::Util::AbstractConfiguration & configuration)
{
@ -49,6 +55,7 @@ namespace
return std::make_shared<RemoteProxyConfigurationResolver>(
server_configuration,
request_protocol,
buildPocoNonProxyHosts(getNoProxyHosts(configuration)),
std::make_shared<RemoteProxyHostFetcherImpl>(),
isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration));
}
@ -88,7 +95,11 @@ namespace
return uris.empty()
? nullptr
: std::make_shared<ProxyListConfigurationResolver>(uris, request_protocol, isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration));
: std::make_shared<ProxyListConfigurationResolver>(
uris,
request_protocol,
buildPocoNonProxyHosts(getNoProxyHosts(configuration)),
isTunnelingDisabledForHTTPSRequestsOverHTTPProxy(configuration));
}
bool hasRemoteResolver(const String & config_prefix, const Poco::Util::AbstractConfiguration & configuration)

View File

@ -1,7 +1,6 @@
#include <Common/ProxyListConfigurationResolver.h>
#include <Common/StringUtils.h>
#include <Common/logger_useful.h>
#include <Poco/URI.h>
namespace DB
@ -9,8 +8,11 @@ namespace DB
ProxyListConfigurationResolver::ProxyListConfigurationResolver(
std::vector<Poco::URI> proxies_,
Protocol request_protocol_, bool disable_tunneling_for_https_requests_over_http_proxy_)
: ProxyConfigurationResolver(request_protocol_, disable_tunneling_for_https_requests_over_http_proxy_), proxies(std::move(proxies_))
Protocol request_protocol_,
const std::string & no_proxy_hosts_,
bool disable_tunneling_for_https_requests_over_http_proxy_)
: ProxyConfigurationResolver(request_protocol_, disable_tunneling_for_https_requests_over_http_proxy_),
proxies(std::move(proxies_)), no_proxy_hosts(no_proxy_hosts_)
{
}
@ -26,12 +28,18 @@ ProxyConfiguration ProxyListConfigurationResolver::resolve()
auto & proxy = proxies[index];
bool use_tunneling_for_https_requests_over_http_proxy = ProxyConfiguration::useTunneling(
request_protocol,
ProxyConfiguration::protocolFromString(proxy.getScheme()),
disable_tunneling_for_https_requests_over_http_proxy);
return ProxyConfiguration {
proxy.getHost(),
ProxyConfiguration::protocolFromString(proxy.getScheme()),
proxy.getPort(),
useTunneling(request_protocol, ProxyConfiguration::protocolFromString(proxy.getScheme()), disable_tunneling_for_https_requests_over_http_proxy),
request_protocol
use_tunneling_for_https_requests_over_http_proxy,
request_protocol,
no_proxy_hosts
};
}

View File

@ -15,7 +15,11 @@ namespace DB
class ProxyListConfigurationResolver : public ProxyConfigurationResolver
{
public:
ProxyListConfigurationResolver(std::vector<Poco::URI> proxies_, Protocol request_protocol_, bool disable_tunneling_for_https_requests_over_http_proxy_ = false);
ProxyListConfigurationResolver(
std::vector<Poco::URI> proxies_,
Protocol request_protocol_,
const std::string & no_proxy_hosts_,
bool disable_tunneling_for_https_requests_over_http_proxy_ = false);
ProxyConfiguration resolve() override;
@ -23,6 +27,7 @@ public:
private:
std::vector<Poco::URI> proxies;
std::string no_proxy_hosts;
/// Access counter to get proxy using round-robin strategy.
std::atomic<size_t> access_counter;

View File

@ -42,11 +42,12 @@ std::string RemoteProxyHostFetcherImpl::fetch(const Poco::URI & endpoint, const
RemoteProxyConfigurationResolver::RemoteProxyConfigurationResolver(
const RemoteServerConfiguration & remote_server_configuration_,
Protocol request_protocol_,
const std::string & no_proxy_hosts_,
std::shared_ptr<RemoteProxyHostFetcher> fetcher_,
bool disable_tunneling_for_https_requests_over_http_proxy_
)
: ProxyConfigurationResolver(request_protocol_, disable_tunneling_for_https_requests_over_http_proxy_),
remote_server_configuration(remote_server_configuration_), fetcher(fetcher_)
remote_server_configuration(remote_server_configuration_), no_proxy_hosts(no_proxy_hosts_), fetcher(fetcher_)
{
}
@ -84,7 +85,7 @@ ProxyConfiguration RemoteProxyConfigurationResolver::resolve()
auto proxy_protocol = ProxyConfiguration::protocolFromString(proxy_protocol_string);
bool use_tunneling_for_https_requests_over_http_proxy = useTunneling(
bool use_tunneling_for_https_requests_over_http_proxy = ProxyConfiguration::useTunneling(
request_protocol,
proxy_protocol,
disable_tunneling_for_https_requests_over_http_proxy);
@ -94,6 +95,7 @@ ProxyConfiguration RemoteProxyConfigurationResolver::resolve()
cached_config.port = proxy_port;
cached_config.tunneling = use_tunneling_for_https_requests_over_http_proxy;
cached_config.original_request_protocol = request_protocol;
cached_config.no_proxy_hosts = no_proxy_hosts;
cache_timestamp = std::chrono::system_clock::now();
cache_valid = true;

View File

@ -41,6 +41,7 @@ public:
RemoteProxyConfigurationResolver(
const RemoteServerConfiguration & remote_server_configuration_,
Protocol request_protocol_,
const std::string & no_proxy_hosts_,
std::shared_ptr<RemoteProxyHostFetcher> fetcher_,
bool disable_tunneling_for_https_requests_over_http_proxy_ = false);
@ -50,6 +51,7 @@ public:
private:
RemoteServerConfiguration remote_server_configuration;
std::string no_proxy_hosts;
std::shared_ptr<RemoteProxyHostFetcher> fetcher;
std::mutex cache_mutex;

View File

@ -0,0 +1,117 @@
#include <Common/proxyConfigurationToPocoProxyConfig.h>
#include <Common/StringUtils.h>
#include <base/find_symbols.h>
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#pragma clang diagnostic ignored "-Wgnu-anonymous-struct"
#pragma clang diagnostic ignored "-Wnested-anon-types"
#pragma clang diagnostic ignored "-Wunused-parameter"
#pragma clang diagnostic ignored "-Wshadow-field-in-constructor"
#pragma clang diagnostic ignored "-Wdtor-name"
#include <re2/re2.h>
#pragma clang diagnostic pop
namespace DB
{
namespace
{
/*
* Copy `curl` behavior instead of `wget` as it seems to be more flexible.
* `curl` strips leading dot and accepts url gitlab.com as a match for no_proxy .gitlab.com,
* while `wget` does an exact match.
* */
std::string buildPocoRegexpEntryWithoutLeadingDot(const std::string & host)
{
std::string_view view_without_leading_dot = host;
if (host[0] == '.')
{
view_without_leading_dot = std::string_view {host.begin() + 1u, host.end()};
}
return RE2::QuoteMeta(view_without_leading_dot);
}
}
/*
* Even though there is not an RFC that defines NO_PROXY, it is usually a comma-separated list of domains.
* Different tools implement their own versions of `NO_PROXY` support. Some support CIDR blocks, some support wildcard etc.
* Opting for a simple implementation that covers most use cases:
* * Support only single wildcard * (match anything)
* * Match subdomains
* * Strip leading dots
* * No regex
* * No CIDR blocks
* * No fancy stuff about loopback IPs
* https://about.gitlab.com/blog/2021/01/27/we-need-to-talk-no-proxy/
* Open for discussions
* */
std::string buildPocoNonProxyHosts(const std::string & no_proxy_hosts_string)
{
if (no_proxy_hosts_string.empty())
{
return "";
}
static constexpr auto OR_SEPARATOR = "|";
static constexpr auto MATCH_ANYTHING = R"(.*)";
static constexpr auto MATCH_SUBDOMAINS_REGEX = R"((?:.*\.)?)";
bool match_any_host = no_proxy_hosts_string.size() == 1 && no_proxy_hosts_string[0] == '*';
if (match_any_host)
{
return MATCH_ANYTHING;
}
std::vector<std::string> no_proxy_hosts;
splitInto<','>(no_proxy_hosts, no_proxy_hosts_string);
bool first = true;
std::string result;
for (auto & host : no_proxy_hosts)
{
trim(host);
if (host.empty())
{
continue;
}
if (!first)
{
result.append(OR_SEPARATOR);
}
auto escaped_host_without_leading_dot = buildPocoRegexpEntryWithoutLeadingDot(host);
result.append(MATCH_SUBDOMAINS_REGEX);
result.append(escaped_host_without_leading_dot);
first = false;
}
return result;
}
Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const DB::ProxyConfiguration & proxy_configuration)
{
Poco::Net::HTTPClientSession::ProxyConfig poco_proxy_config;
poco_proxy_config.host = proxy_configuration.host;
poco_proxy_config.port = proxy_configuration.port;
poco_proxy_config.protocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.protocol);
poco_proxy_config.tunnel = proxy_configuration.tunneling;
poco_proxy_config.originalRequestProtocol = DB::ProxyConfiguration::protocolToString(proxy_configuration.original_request_protocol);
poco_proxy_config.nonProxyHosts = proxy_configuration.no_proxy_hosts;
return poco_proxy_config;
}
}

View File

@ -0,0 +1,13 @@
#pragma once
#include <Poco/Net/HTTPClientSession.h>
#include <Common/ProxyConfiguration.h>
namespace DB
{
Poco::Net::HTTPClientSession::ProxyConfig proxyConfigurationToPocoProxyConfig(const DB::ProxyConfiguration & proxy_configuration);
std::string buildPocoNonProxyHosts(const std::string & no_proxy_hosts_string);
}

View File

@ -76,22 +76,28 @@ inline std::string xmlNodeAsString(Poco::XML::Node *pNode)
struct EnvironmentProxySetter
{
EnvironmentProxySetter(const Poco::URI & http_proxy, const Poco::URI & https_proxy)
{
if (!http_proxy.empty())
{
setenv("http_proxy", http_proxy.toString().c_str(), 1); // NOLINT(concurrency-mt-unsafe)
}
static constexpr auto * NO_PROXY = "*";
static constexpr auto * HTTP_PROXY = "http://proxy_server:3128";
static constexpr auto * HTTPS_PROXY = "https://proxy_server:3128";
if (!https_proxy.empty())
{
setenv("https_proxy", https_proxy.toString().c_str(), 1); // NOLINT(concurrency-mt-unsafe)
}
EnvironmentProxySetter()
{
setenv("http_proxy", HTTP_PROXY, 1); // NOLINT(concurrency-mt-unsafe)
setenv("https_proxy", HTTPS_PROXY, 1); // NOLINT(concurrency-mt-unsafe)
// Some other tests rely on HTTP clients (e.g, gtest_aws_s3_client), which depend on proxy configuration
// since in https://github.com/ClickHouse/ClickHouse/pull/63314 the environment proxy resolver reads only once
// from the environment, the proxy configuration will always be there.
// The problem is that the proxy server does not exist, causing the test to fail.
// To work around this issue, `no_proxy` is set to bypass all domains.
setenv("no_proxy", NO_PROXY, 1); // NOLINT(concurrency-mt-unsafe)
}
~EnvironmentProxySetter()
{
unsetenv("http_proxy"); // NOLINT(concurrency-mt-unsafe)
unsetenv("https_proxy"); // NOLINT(concurrency-mt-unsafe)
unsetenv("no_proxy"); // NOLINT(concurrency-mt-unsafe)
}
};

View File

@ -0,0 +1,24 @@
#include <gtest/gtest.h>
#include <Common/proxyConfigurationToPocoProxyConfig.h>
TEST(ProxyConfigurationToPocoProxyConfiguration, TestNoProxyHostRegexBuild)
{
ASSERT_EQ(
DB::buildPocoNonProxyHosts("localhost,127.0.0.1,some_other_domain:8080,sub-domain.domain.com"),
R"((?:.*\.)?localhost|(?:.*\.)?127\.0\.0\.1|(?:.*\.)?some_other_domain\:8080|(?:.*\.)?sub\-domain\.domain\.com)");
}
TEST(ProxyConfigurationToPocoProxyConfiguration, TestNoProxyHostRegexBuildMatchAnything)
{
ASSERT_EQ(
DB::buildPocoNonProxyHosts("*"),
".*");
}
TEST(ProxyConfigurationToPocoProxyConfiguration, TestNoProxyHostRegexBuildEmpty)
{
ASSERT_EQ(
DB::buildPocoNonProxyHosts(""),
"");
}

View File

@ -1,6 +1,9 @@
#include <gtest/gtest.h>
#include <Common/ProxyConfigurationResolverProvider.h>
#include <Common/RemoteProxyConfigurationResolver.h>
#include <Common/ProxyListConfigurationResolver.h>
#include <Common/EnvironmentProxyConfigurationResolver.h>
#include <Common/tests/gtest_global_context.h>
#include <Common/tests/gtest_helper_functions.h>
@ -25,27 +28,19 @@ protected:
DB::ContextMutablePtr ProxyConfigurationResolverProviderTests::context;
Poco::URI http_env_proxy_server = Poco::URI("http://http_environment_proxy:3128");
Poco::URI https_env_proxy_server = Poco::URI("http://https_environment_proxy:3128");
Poco::URI http_list_proxy_server = Poco::URI("http://http_list_proxy:3128");
Poco::URI https_list_proxy_server = Poco::URI("http://https_list_proxy:3128");
TEST_F(ProxyConfigurationResolverProviderTests, EnvironmentResolverShouldBeUsedIfNoSettings)
{
EnvironmentProxySetter setter(http_env_proxy_server, https_env_proxy_server);
EnvironmentProxySetter setter;
const auto & config = getContext().context->getConfigRef();
auto http_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, config)->resolve();
auto https_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, config)->resolve();
auto http_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, config);
auto https_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, config);
ASSERT_EQ(http_configuration.host, http_env_proxy_server.getHost());
ASSERT_EQ(http_configuration.port, http_env_proxy_server.getPort());
ASSERT_EQ(http_configuration.protocol, DB::ProxyConfiguration::protocolFromString(http_env_proxy_server.getScheme()));
ASSERT_EQ(https_configuration.host, https_env_proxy_server.getHost());
ASSERT_EQ(https_configuration.port, https_env_proxy_server.getPort());
ASSERT_EQ(https_configuration.protocol, DB::ProxyConfiguration::protocolFromString(https_env_proxy_server.getScheme()));
ASSERT_TRUE(std::dynamic_pointer_cast<DB::EnvironmentProxyConfigurationResolver>(http_resolver));
ASSERT_TRUE(std::dynamic_pointer_cast<DB::EnvironmentProxyConfigurationResolver>(https_resolver));
}
TEST_F(ProxyConfigurationResolverProviderTests, ListHTTPOnly)
@ -57,17 +52,11 @@ TEST_F(ProxyConfigurationResolverProviderTests, ListHTTPOnly)
config->setString("proxy.http.uri", http_list_proxy_server.toString());
context->setConfig(config);
auto http_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config)->resolve();
auto http_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config);
auto https_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config);
ASSERT_EQ(http_proxy_configuration.host, http_list_proxy_server.getHost());
ASSERT_EQ(http_proxy_configuration.port, http_list_proxy_server.getPort());
ASSERT_EQ(http_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(http_list_proxy_server.getScheme()));
auto https_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config)->resolve();
// No https configuration since it's not set
ASSERT_EQ(https_proxy_configuration.host, "");
ASSERT_EQ(https_proxy_configuration.port, 0);
ASSERT_TRUE(std::dynamic_pointer_cast<DB::ProxyListConfigurationResolver>(http_resolver));
ASSERT_TRUE(std::dynamic_pointer_cast<DB::EnvironmentProxyConfigurationResolver>(https_resolver));
}
TEST_F(ProxyConfigurationResolverProviderTests, ListHTTPSOnly)
@ -79,18 +68,11 @@ TEST_F(ProxyConfigurationResolverProviderTests, ListHTTPSOnly)
config->setString("proxy.https.uri", https_list_proxy_server.toString());
context->setConfig(config);
auto http_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config)->resolve();
auto http_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config);
auto https_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config);
ASSERT_EQ(http_proxy_configuration.host, "");
ASSERT_EQ(http_proxy_configuration.port, 0);
auto https_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config)->resolve();
ASSERT_EQ(https_proxy_configuration.host, https_list_proxy_server.getHost());
// still HTTP because the proxy host is not HTTPS
ASSERT_EQ(https_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(https_list_proxy_server.getScheme()));
ASSERT_EQ(https_proxy_configuration.port, https_list_proxy_server.getPort());
ASSERT_TRUE(std::dynamic_pointer_cast<DB::EnvironmentProxyConfigurationResolver>(http_resolver));
ASSERT_TRUE(std::dynamic_pointer_cast<DB::ProxyListConfigurationResolver>(https_resolver));
}
TEST_F(ProxyConfigurationResolverProviderTests, ListBoth)
@ -107,70 +89,15 @@ TEST_F(ProxyConfigurationResolverProviderTests, ListBoth)
context->setConfig(config);
auto http_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config)->resolve();
auto http_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config);
auto https_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config);
ASSERT_EQ(http_proxy_configuration.host, http_list_proxy_server.getHost());
ASSERT_EQ(http_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(http_list_proxy_server.getScheme()));
ASSERT_EQ(http_proxy_configuration.port, http_list_proxy_server.getPort());
auto https_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config)->resolve();
ASSERT_EQ(https_proxy_configuration.host, https_list_proxy_server.getHost());
// still HTTP because the proxy host is not HTTPS
ASSERT_EQ(https_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(https_list_proxy_server.getScheme()));
ASSERT_EQ(https_proxy_configuration.port, https_list_proxy_server.getPort());
}
TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverIsBasedOnProtocolConfigurationHTTP)
{
/*
* Since there is no way to call `ProxyConfigurationResolver::resolve` on remote resolver,
* it is hard to verify the remote resolver was actually picked. One hackish way to assert
* the remote resolver was OR was not picked based on the configuration, is to use the
* environment resolver. Since the environment resolver is always returned as a fallback,
* we can assert the remote resolver was not picked if `ProxyConfigurationResolver::resolve`
* succeeds and returns an environment proxy configuration.
* */
EnvironmentProxySetter setter(http_env_proxy_server, https_env_proxy_server);
ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration());
config->setString("proxy", "");
config->setString("proxy.https", "");
config->setString("proxy.https.resolver", "");
config->setString("proxy.https.resolver.endpoint", "http://resolver:8080/hostname");
// even tho proxy protocol / scheme is http, it should not be picked (prior to this PR, it would be picked)
config->setString("proxy.https.resolver.proxy_scheme", "http");
config->setString("proxy.https.resolver.proxy_port", "80");
config->setString("proxy.https.resolver.proxy_cache_time", "10");
context->setConfig(config);
auto http_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config)->resolve();
/*
* Asserts env proxy is used and not the remote resolver. If the remote resolver is picked, it is an error because
* there is no `http` specification for remote resolver
* */
ASSERT_EQ(http_proxy_configuration.host, http_env_proxy_server.getHost());
ASSERT_EQ(http_proxy_configuration.port, http_env_proxy_server.getPort());
ASSERT_EQ(http_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(http_env_proxy_server.getScheme()));
ASSERT_TRUE(std::dynamic_pointer_cast<DB::ProxyListConfigurationResolver>(http_resolver));
ASSERT_TRUE(std::dynamic_pointer_cast<DB::ProxyListConfigurationResolver>(https_resolver));
}
TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverIsBasedOnProtocolConfigurationHTTPS)
{
/*
* Since there is no way to call `ProxyConfigurationResolver::resolve` on remote resolver,
* it is hard to verify the remote resolver was actually picked. One hackish way to assert
* the remote resolver was OR was not picked based on the configuration, is to use the
* environment resolver. Since the environment resolver is always returned as a fallback,
* we can assert the remote resolver was not picked if `ProxyConfigurationResolver::resolve`
* succeeds and returns an environment proxy configuration.
* */
EnvironmentProxySetter setter(http_env_proxy_server, https_env_proxy_server);
ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration());
config->setString("proxy", "");
@ -185,27 +112,44 @@ TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverIsBasedOnProtocolC
context->setConfig(config);
auto http_proxy_configuration = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config)->resolve();
auto http_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config);
auto https_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config);
/*
* Asserts env proxy is used and not the remote resolver. If the remote resolver is picked, it is an error because
* there is no `http` specification for remote resolver
* */
ASSERT_EQ(http_proxy_configuration.host, https_env_proxy_server.getHost());
ASSERT_EQ(http_proxy_configuration.port, https_env_proxy_server.getPort());
ASSERT_EQ(http_proxy_configuration.protocol, DB::ProxyConfiguration::protocolFromString(https_env_proxy_server.getScheme()));
ASSERT_TRUE(std::dynamic_pointer_cast<DB::RemoteProxyConfigurationResolver>(http_resolver));
ASSERT_TRUE(std::dynamic_pointer_cast<DB::EnvironmentProxyConfigurationResolver>(https_resolver));
}
// remote resolver is tricky to be tested in unit tests
TEST_F(ProxyConfigurationResolverProviderTests, RemoteResolverHTTPSOnly)
{
ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration());
config->setString("proxy", "");
config->setString("proxy.https", "");
config->setString("proxy.https.resolver", "");
config->setString("proxy.https.resolver.endpoint", "http://resolver:8080/hostname");
// even tho proxy protocol / scheme is http, it should not be picked (prior to this PR, it would be picked)
config->setString("proxy.https.resolver.proxy_scheme", "http");
config->setString("proxy.https.resolver.proxy_port", "80");
config->setString("proxy.https.resolver.proxy_cache_time", "10");
context->setConfig(config);
auto http_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTP, *config);
auto https_resolver = DB::ProxyConfigurationResolverProvider::get(DB::ProxyConfiguration::Protocol::HTTPS, *config);
ASSERT_TRUE(std::dynamic_pointer_cast<DB::EnvironmentProxyConfigurationResolver>(http_resolver));
ASSERT_TRUE(std::dynamic_pointer_cast<DB::RemoteProxyConfigurationResolver>(https_resolver));
}
template <bool DISABLE_TUNNELING_FOR_HTTPS_REQUESTS_OVER_HTTP_PROXY, bool STRING>
void test_tunneling(DB::ContextMutablePtr context)
{
EnvironmentProxySetter setter(http_env_proxy_server, https_env_proxy_server);
ConfigurationPtr config = Poco::AutoPtr(new Poco::Util::MapConfiguration());
config->setString("proxy", "");
config->setString("proxy.https", "");
config->setString("proxy.https.uri", http_list_proxy_server.toString());
if constexpr (STRING)
{
@ -230,4 +174,3 @@ TEST_F(ProxyConfigurationResolverProviderTests, TunnelingForHTTPSRequestsOverHTT
test_tunneling<true, false>(context);
test_tunneling<true, true>(context);
}

View File

@ -2,81 +2,38 @@
#include <Common/EnvironmentProxyConfigurationResolver.h>
#include <Common/tests/gtest_helper_functions.h>
#include <Common/proxyConfigurationToPocoProxyConfig.h>
#include <Poco/URI.h>
namespace DB
{
namespace
TEST(EnvironmentProxyConfigurationResolver, TestHTTPandHTTPS)
{
auto http_proxy_server = Poco::URI("http://proxy_server:3128");
auto https_proxy_server = Poco::URI("https://proxy_server:3128");
}
const auto http_proxy_server = Poco::URI(EnvironmentProxySetter::HTTP_PROXY);
const auto https_proxy_server = Poco::URI(EnvironmentProxySetter::HTTPS_PROXY);
TEST(EnvironmentProxyConfigurationResolver, TestHTTP)
{
EnvironmentProxySetter setter(http_proxy_server, {});
std::string poco_no_proxy_regex = buildPocoNonProxyHosts(EnvironmentProxySetter::NO_PROXY);
EnvironmentProxyConfigurationResolver resolver(ProxyConfiguration::Protocol::HTTP);
EnvironmentProxySetter setter;
auto configuration = resolver.resolve();
EnvironmentProxyConfigurationResolver http_resolver(ProxyConfiguration::Protocol::HTTP);
ASSERT_EQ(configuration.host, http_proxy_server.getHost());
ASSERT_EQ(configuration.port, http_proxy_server.getPort());
ASSERT_EQ(configuration.protocol, ProxyConfiguration::protocolFromString(http_proxy_server.getScheme()));
}
auto http_configuration = http_resolver.resolve();
TEST(EnvironmentProxyConfigurationResolver, TestHTTPNoEnv)
{
EnvironmentProxyConfigurationResolver resolver(ProxyConfiguration::Protocol::HTTP);
ASSERT_EQ(http_configuration.host, http_proxy_server.getHost());
ASSERT_EQ(http_configuration.port, http_proxy_server.getPort());
ASSERT_EQ(http_configuration.protocol, ProxyConfiguration::protocolFromString(http_proxy_server.getScheme()));
ASSERT_EQ(http_configuration.no_proxy_hosts, poco_no_proxy_regex);
auto configuration = resolver.resolve();
EnvironmentProxyConfigurationResolver https_resolver(ProxyConfiguration::Protocol::HTTPS);
ASSERT_EQ(configuration.host, "");
ASSERT_EQ(configuration.protocol, ProxyConfiguration::Protocol::HTTP);
ASSERT_EQ(configuration.port, 0u);
}
auto https_configuration = https_resolver.resolve();
TEST(EnvironmentProxyConfigurationResolver, TestHTTPs)
{
EnvironmentProxySetter setter({}, https_proxy_server);
EnvironmentProxyConfigurationResolver resolver(ProxyConfiguration::Protocol::HTTPS);
auto configuration = resolver.resolve();
ASSERT_EQ(configuration.host, https_proxy_server.getHost());
ASSERT_EQ(configuration.port, https_proxy_server.getPort());
ASSERT_EQ(configuration.protocol, ProxyConfiguration::protocolFromString(https_proxy_server.getScheme()));
}
TEST(EnvironmentProxyConfigurationResolver, TestHTTPsNoEnv)
{
EnvironmentProxyConfigurationResolver resolver(ProxyConfiguration::Protocol::HTTPS);
auto configuration = resolver.resolve();
ASSERT_EQ(configuration.host, "");
ASSERT_EQ(configuration.protocol, ProxyConfiguration::Protocol::HTTP);
ASSERT_EQ(configuration.port, 0u);
}
TEST(EnvironmentProxyConfigurationResolver, TestHTTPsOverHTTPTunnelingDisabled)
{
// use http proxy for https, this would use connect protocol by default
EnvironmentProxySetter setter({}, http_proxy_server);
bool disable_tunneling_for_https_requests_over_http_proxy = true;
EnvironmentProxyConfigurationResolver resolver(
ProxyConfiguration::Protocol::HTTPS, disable_tunneling_for_https_requests_over_http_proxy);
auto configuration = resolver.resolve();
ASSERT_EQ(configuration.host, http_proxy_server.getHost());
ASSERT_EQ(configuration.port, http_proxy_server.getPort());
ASSERT_EQ(configuration.protocol, ProxyConfiguration::protocolFromString(http_proxy_server.getScheme()));
ASSERT_EQ(configuration.tunneling, false);
ASSERT_EQ(https_configuration.host, https_proxy_server.getHost());
ASSERT_EQ(https_configuration.port, https_proxy_server.getPort());
ASSERT_EQ(https_configuration.protocol, ProxyConfiguration::protocolFromString(https_proxy_server.getScheme()));
ASSERT_EQ(https_configuration.no_proxy_hosts, poco_no_proxy_regex);
}
}

View File

@ -10,6 +10,8 @@ namespace
{
auto proxy_server1 = Poco::URI("http://proxy_server1:3128");
auto proxy_server2 = Poco::URI("http://proxy_server2:3128");
std::string no_proxy_hosts = "localhost,,127.0.0.1,some_other_domain,,,, sub-domain.domain.com,";
}
TEST(ProxyListConfigurationResolver, SimpleTest)
@ -17,7 +19,8 @@ TEST(ProxyListConfigurationResolver, SimpleTest)
ProxyListConfigurationResolver resolver(
{proxy_server1, proxy_server2},
ProxyConfiguration::Protocol::HTTP);
ProxyConfiguration::Protocol::HTTP,
no_proxy_hosts);
auto configuration1 = resolver.resolve();
auto configuration2 = resolver.resolve();
@ -25,10 +28,12 @@ TEST(ProxyListConfigurationResolver, SimpleTest)
ASSERT_EQ(configuration1.host, proxy_server1.getHost());
ASSERT_EQ(configuration1.port, proxy_server1.getPort());
ASSERT_EQ(configuration1.protocol, ProxyConfiguration::protocolFromString(proxy_server1.getScheme()));
ASSERT_EQ(configuration1.no_proxy_hosts, no_proxy_hosts);
ASSERT_EQ(configuration2.host, proxy_server2.getHost());
ASSERT_EQ(configuration2.port, proxy_server2.getPort());
ASSERT_EQ(configuration2.protocol, ProxyConfiguration::protocolFromString(proxy_server2.getScheme()));
ASSERT_EQ(configuration2.no_proxy_hosts, no_proxy_hosts);
}
TEST(ProxyListConfigurationResolver, HTTPSRequestsOverHTTPProxyDefault)
@ -36,7 +41,8 @@ TEST(ProxyListConfigurationResolver, HTTPSRequestsOverHTTPProxyDefault)
ProxyListConfigurationResolver resolver(
{proxy_server1, proxy_server2},
ProxyConfiguration::Protocol::HTTPS);
ProxyConfiguration::Protocol::HTTPS,
"");
auto configuration1 = resolver.resolve();
auto configuration2 = resolver.resolve();
@ -45,11 +51,12 @@ TEST(ProxyListConfigurationResolver, HTTPSRequestsOverHTTPProxyDefault)
ASSERT_EQ(configuration1.port, proxy_server1.getPort());
ASSERT_EQ(configuration1.protocol, ProxyConfiguration::protocolFromString(proxy_server1.getScheme()));
ASSERT_EQ(configuration1.tunneling, true);
ASSERT_EQ(configuration1.no_proxy_hosts, "");
ASSERT_EQ(configuration2.host, proxy_server2.getHost());
ASSERT_EQ(configuration2.port, proxy_server2.getPort());
ASSERT_EQ(configuration2.protocol, ProxyConfiguration::protocolFromString(proxy_server2.getScheme()));
ASSERT_EQ(configuration1.tunneling, true);
ASSERT_EQ(configuration2.no_proxy_hosts, "");
}
TEST(ProxyListConfigurationResolver, SimpleTestTunnelingDisabled)
@ -58,6 +65,7 @@ TEST(ProxyListConfigurationResolver, SimpleTestTunnelingDisabled)
ProxyListConfigurationResolver resolver(
{proxy_server1, proxy_server2},
ProxyConfiguration::Protocol::HTTPS,
"",
disable_tunneling_for_https_requests_over_http_proxy);
auto configuration1 = resolver.resolve();

View File

@ -42,6 +42,7 @@ TEST(RemoteProxyConfigurationResolver, HTTPOverHTTP)
RemoteProxyConfigurationResolver resolver(
remote_server_configuration,
ProxyConfiguration::Protocol::HTTP,
"",
std::make_shared<RemoteProxyHostFetcherMock>(proxy_server_mock)
);
@ -68,6 +69,7 @@ TEST(RemoteProxyConfigurationResolver, HTTPSOverHTTPS)
RemoteProxyConfigurationResolver resolver(
remote_server_configuration,
ProxyConfiguration::Protocol::HTTPS,
"",
std::make_shared<RemoteProxyHostFetcherMock>(proxy_server_mock)
);
@ -95,6 +97,7 @@ TEST(RemoteProxyConfigurationResolver, HTTPSOverHTTP)
RemoteProxyConfigurationResolver resolver(
remote_server_configuration,
ProxyConfiguration::Protocol::HTTPS,
"",
std::make_shared<RemoteProxyHostFetcherMock>(proxy_server_mock)
);
@ -122,6 +125,7 @@ TEST(RemoteProxyConfigurationResolver, HTTPSOverHTTPNoTunneling)
RemoteProxyConfigurationResolver resolver(
remote_server_configuration,
ProxyConfiguration::Protocol::HTTPS,
"",
std::make_shared<RemoteProxyHostFetcherMock>(proxy_server_mock),
true /* disable_tunneling_for_https_requests_over_http_proxy_ */
);
@ -153,6 +157,7 @@ TEST(RemoteProxyConfigurationResolver, SimpleCacheTest)
RemoteProxyConfigurationResolver resolver(
remote_server_configuration,
ProxyConfiguration::Protocol::HTTP,
"",
fetcher_mock
);

View File

@ -30,7 +30,7 @@ def check_proxy_logs(
False
), f"{http_method} method not found in logs of {proxy_instance} for bucket {bucket}"
time.sleep(1)
time.sleep(1)
def wait_resolver(cluster):
@ -124,3 +124,13 @@ def simple_storage_test(cluster, node, proxies, policy):
# not checking for POST because it is in a different format
check_proxy_logs(cluster, proxies, "http", policy, ["PUT", "GET"])
def simple_test_assert_no_proxy(cluster, proxies, protocol, bucket):
minio_endpoint = build_s3_endpoint(protocol, bucket)
node = cluster.instances[bucket]
perform_simple_queries(node, minio_endpoint)
# No HTTP method should be found in proxy logs if no proxy is active
empty_method_list = []
check_proxy_logs(cluster, proxies, protocol, bucket, empty_method_list)

View File

@ -0,0 +1,9 @@
<clickhouse>
<proxy>
<no_proxy>not_important_host,, minio1 ,</no_proxy>
<http>
<uri>http://proxy1</uri>
<uri>http://proxy2</uri>
</http>
</proxy>
</clickhouse>

View File

@ -0,0 +1,18 @@
<clickhouse>
<proxy>
<no_proxy>not_important_host,, minio1 ,</no_proxy>
<!--
At each interaction with S3 resolver sends empty GET request to specified endpoint URL to obtain proxy host.
Proxy host is returned as string in response body.
Then S3 client uses proxy URL formed as proxy_scheme://proxy_host:proxy_port to make request.
-->
<http>
<resolver>
<endpoint>http://resolver:8080/hostname</endpoint>
<proxy_scheme>http</proxy_scheme>
<proxy_port>80</proxy_port>
<proxy_cache_time>10</proxy_cache_time>
</resolver>
</http>
</proxy>
</clickhouse>

View File

@ -19,6 +19,14 @@ def cluster():
with_minio=True,
)
cluster.add_instance(
"remote_proxy_node_no_proxy",
main_configs=[
"configs/config.d/proxy_remote_no_proxy.xml",
],
with_minio=True,
)
cluster.add_instance(
"proxy_list_node",
main_configs=[
@ -27,6 +35,14 @@ def cluster():
with_minio=True,
)
cluster.add_instance(
"proxy_list_node_no_proxy",
main_configs=[
"configs/config.d/proxy_list_no_proxy.xml",
],
with_minio=True,
)
cluster.add_instance(
"env_node",
with_minio=True,
@ -36,6 +52,16 @@ def cluster():
instance_env_variables=True,
)
cluster.add_instance(
"env_node_no_proxy",
with_minio=True,
env_variables={
"http_proxy": "http://proxy1",
"no_proxy": "not_important_host,, minio1 ,",
},
instance_env_variables=True,
)
logging.info("Starting cluster...")
cluster.start()
logging.info("Cluster started")
@ -48,6 +74,24 @@ def cluster():
cluster.shutdown()
def test_s3_with_http_proxy_list_no_proxy(cluster):
proxy_util.simple_test_assert_no_proxy(
cluster, ["proxy1", "proxy2"], "http", "proxy_list_node_no_proxy"
)
def test_s3_with_http_remote_proxy_no_proxy(cluster):
proxy_util.simple_test_assert_no_proxy(
cluster, ["proxy1"], "http", "remote_proxy_node_no_proxy"
)
def test_s3_with_http_env_no_proxy(cluster):
proxy_util.simple_test_assert_no_proxy(
cluster, ["proxy1"], "http", "env_node_no_proxy"
)
def test_s3_with_http_proxy_list(cluster):
proxy_util.simple_test(cluster, ["proxy1", "proxy2"], "http", "proxy_list_node")

View File

@ -0,0 +1,13 @@
<clickhouse>
<proxy>
<no_proxy>not_important_host,, minio1 ,</no_proxy>
<http>
<uri>http://proxy1</uri>
<uri>http://proxy2</uri>
</http>
<https>
<uri>https://proxy1</uri>
<uri>https://proxy2</uri>
</https>
</proxy>
</clickhouse>

View File

@ -0,0 +1,18 @@
<clickhouse>
<proxy>
<no_proxy>not_important_host,, minio1 ,</no_proxy>
<!--
At each interaction with S3 resolver sends empty GET request to specified endpoint URL to obtain proxy host.
Proxy host is returned as string in response body.
Then S3 client uses proxy URL formed as proxy_scheme://proxy_host:proxy_port to make request.
-->
<https>
<resolver>
<endpoint>http://resolver:8080/hostname</endpoint>
<proxy_scheme>https</proxy_scheme>
<proxy_port>443</proxy_port>
<proxy_cache_time>10</proxy_cache_time>
</resolver>
</https>
</proxy>
</clickhouse>

View File

@ -23,6 +23,15 @@ def cluster():
minio_certs_dir="minio_certs",
)
cluster.add_instance(
"remote_proxy_node_no_proxy",
main_configs=[
"configs/config.d/proxy_remote_no_proxy.xml",
"configs/config.d/ssl.xml",
],
with_minio=True,
)
cluster.add_instance(
"proxy_list_node",
main_configs=[
@ -32,6 +41,15 @@ def cluster():
with_minio=True,
)
cluster.add_instance(
"proxy_list_node_no_proxy",
main_configs=[
"configs/config.d/proxy_list_no_proxy.xml",
"configs/config.d/ssl.xml",
],
with_minio=True,
)
cluster.add_instance(
"env_node",
main_configs=[
@ -44,6 +62,19 @@ def cluster():
instance_env_variables=True,
)
cluster.add_instance(
"env_node_no_proxy",
main_configs=[
"configs/config.d/ssl.xml",
],
with_minio=True,
env_variables={
"https_proxy": "https://proxy1",
"no_proxy": "not_important_host,, minio1 ,",
},
instance_env_variables=True,
)
logging.info("Starting cluster...")
cluster.start()
logging.info("Cluster started")
@ -56,6 +87,24 @@ def cluster():
cluster.shutdown()
def test_s3_with_https_proxy_list_no_proxy(cluster):
proxy_util.simple_test_assert_no_proxy(
cluster, ["proxy1", "proxy2"], "https", "proxy_list_node_no_proxy"
)
def test_s3_with_https_env_no_proxy(cluster):
proxy_util.simple_test_assert_no_proxy(
cluster, ["proxy1"], "https", "env_node_no_proxy"
)
def test_s3_with_https_remote_no_proxy(cluster):
proxy_util.simple_test_assert_no_proxy(
cluster, ["proxy1"], "https", "remote_proxy_node_no_proxy"
)
def test_s3_with_https_proxy_list(cluster):
proxy_util.simple_test(cluster, ["proxy1", "proxy2"], "https", "proxy_list_node")