mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #35294 from ClickHouse/reload-remote_url_allow_hosts
Reload remote_url_allow_hosts after config update.
This commit is contained in:
commit
a380aa6b8a
@ -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"));
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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
|
||||
|
||||
|
@ -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));
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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),
|
||||
|
Loading…
Reference in New Issue
Block a user