Merge pull request #35294 from ClickHouse/reload-remote_url_allow_hosts

Reload remote_url_allow_hosts after config update.
This commit is contained in:
Nikolai Kochetov 2022-03-15 22:07:16 +01:00 committed by GitHub
commit a380aa6b8a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 46 additions and 24 deletions

View File

@ -932,6 +932,8 @@ if (ThreadFuzzer::instance().isEffective())
global_context->loadOrReloadModels(*config);
global_context->loadOrReloadUserDefinedExecutableFunctions(*config);
global_context->setRemoteHostFilter(*config);
/// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default)
if (config->has("max_table_size_to_drop"))
global_context->setMaxTableSizeToDrop(config->getUInt64("max_table_size_to_drop"));

View File

@ -34,6 +34,11 @@ void RemoteHostFilter::setValuesFromConfig(const Poco::Util::AbstractConfigurati
{
std::vector<std::string> keys;
config.keys("remote_url_allow_hosts", keys);
std::lock_guard guard(hosts_mutex);
primary_hosts.clear();
regexp_hosts.clear();
for (const auto & key : keys)
{
if (startsWith(key, "host_regexp"))
@ -41,23 +46,33 @@ void RemoteHostFilter::setValuesFromConfig(const Poco::Util::AbstractConfigurati
else if (startsWith(key, "host"))
primary_hosts.insert(config.getString("remote_url_allow_hosts." + key));
}
is_allow_by_default = false;
is_initialized = true;
}
else
{
is_initialized = false;
std::lock_guard guard(hosts_mutex);
primary_hosts.clear();
regexp_hosts.clear();
}
}
bool RemoteHostFilter::checkForDirectEntry(const std::string & str) const
{
if (!primary_hosts.empty() || !regexp_hosts.empty())
{
if (primary_hosts.find(str) == primary_hosts.end())
{
for (const auto & regexp : regexp_hosts)
if (re2::RE2::FullMatch(str, regexp))
return true;
return false;
}
if (!is_initialized)
/// Allow everything by default.
return true;
}
return is_allow_by_default;
std::lock_guard guard(hosts_mutex);
if (primary_hosts.contains(str))
return true;
for (const auto & regexp : regexp_hosts)
if (re2::RE2::FullMatch(str, regexp))
return true;
return false;
}
}

View File

@ -2,6 +2,7 @@
#include <string>
#include <vector>
#include <mutex>
#include <unordered_set>
@ -24,7 +25,9 @@ public:
void checkHostAndPort(const std::string & host, const std::string & port) const; /// Does the same as checkURL, but for host and port.
private:
bool is_allow_by_default = true;
std::atomic_bool is_initialized = false;
mutable std::mutex hosts_mutex;
std::unordered_set<std::string> primary_hosts; /// Allowed primary (<host>) URL from config.xml
std::vector<std::string> regexp_hosts; /// Allowed regexp (<hots_regexp>) URL from config.xml

View File

@ -104,7 +104,7 @@ Pipe HTTPDictionarySource::loadAll()
context->getReadSettings(),
configuration.header_entries,
ReadWriteBufferFromHTTP::Range{},
RemoteHostFilter{}, false);
nullptr, false);
return createWrappedBuffer(std::move(in_ptr));
}
@ -125,7 +125,7 @@ Pipe HTTPDictionarySource::loadUpdatedAll()
context->getReadSettings(),
configuration.header_entries,
ReadWriteBufferFromHTTP::Range{},
RemoteHostFilter{}, false);
nullptr, false);
return createWrappedBuffer(std::move(in_ptr));
}
@ -155,7 +155,7 @@ Pipe HTTPDictionarySource::loadIds(const std::vector<UInt64> & ids)
context->getReadSettings(),
configuration.header_entries,
ReadWriteBufferFromHTTP::Range{},
RemoteHostFilter{}, false);
nullptr, false);
return createWrappedBuffer(std::move(in_ptr));
}
@ -185,7 +185,7 @@ Pipe HTTPDictionarySource::loadKeys(const Columns & key_columns, const std::vect
context->getReadSettings(),
configuration.header_entries,
ReadWriteBufferFromHTTP::Range{},
RemoteHostFilter{}, false);
nullptr, false);
return createWrappedBuffer(std::move(in_ptr));
}

View File

@ -76,7 +76,7 @@ std::unique_ptr<ReadBuffer> ReadBufferFromWebServer::initialize()
read_settings,
ReadWriteBufferFromHTTP::HTTPHeaderEntries{},
range,
context->getRemoteHostFilter(),
&context->getRemoteHostFilter(),
/* delay_initialization */true,
use_external_buffer);
}

View File

@ -115,7 +115,7 @@ namespace detail
const Poco::Net::HTTPBasicCredentials & credentials;
std::vector<Poco::Net::HTTPCookie> cookies;
HTTPHeaderEntries http_header_entries;
RemoteHostFilter remote_host_filter;
const RemoteHostFilter * remote_host_filter = nullptr;
std::function<void(size_t)> next_callback;
size_t buffer_size;
@ -219,7 +219,8 @@ namespace detail
while (isRedirect(response.getStatus()))
{
Poco::URI uri_redirect(response.get("Location"));
remote_host_filter.checkURL(uri_redirect);
if (remote_host_filter)
remote_host_filter->checkURL(uri_redirect);
session->updateSession(uri_redirect);
@ -265,7 +266,7 @@ namespace detail
const ReadSettings & settings_ = {},
HTTPHeaderEntries http_header_entries_ = {},
Range read_range_ = {},
const RemoteHostFilter & remote_host_filter_ = {},
const RemoteHostFilter * remote_host_filter_ = nullptr,
bool delay_initialization = false,
bool use_external_buffer_ = false,
bool http_skip_not_found_url_ = false)
@ -359,7 +360,8 @@ namespace detail
while (isRedirect(response.getStatus()))
{
Poco::URI uri_redirect(response.get("Location"));
remote_host_filter.checkURL(uri_redirect);
if (remote_host_filter)
remote_host_filter->checkURL(uri_redirect);
session->updateSession(uri_redirect);
@ -640,7 +642,7 @@ public:
const ReadSettings & settings_ = {},
const HTTPHeaderEntries & http_header_entries_ = {},
Range read_range_ = {},
const RemoteHostFilter & remote_host_filter_ = {},
const RemoteHostFilter * remote_host_filter_ = nullptr,
bool delay_initialization_ = true,
bool use_external_buffer_ = false,
bool skip_not_found_url_ = false)

View File

@ -269,7 +269,7 @@ namespace
read_settings,
headers,
ReadWriteBufferFromHTTP::Range{},
context->getRemoteHostFilter(),
&context->getRemoteHostFilter(),
delay_initialization,
/* use_external_buffer */false,
/* skip_url_not_found_error */skip_url_not_found_error),