clickhouse-server: loading users, profiles and quotas form separate config file. [#METR-8956]

This commit is contained in:
Michael Kolupaev 2014-02-13 07:17:22 +00:00
parent c74a21d4fc
commit 1b7ed4dba1
10 changed files with 278 additions and 87 deletions

View File

@ -50,6 +50,8 @@ typedef std::pair<String, String> DatabaseAndTableName;
typedef std::map<DatabaseAndTableName, std::set<DatabaseAndTableName> > ViewDependencies;
typedef std::vector<DatabaseAndTableName> Dependencies;
typedef Poco::AutoPtr<Poco::Util::AbstractConfiguration> ConfigurationPtr;
/** Набор известных объектов, которые могут быть использованы в запросе.
* Разделяемая часть. Порядок членов (порядок их уничтожения) очень важен.
@ -89,6 +91,7 @@ struct ContextShared
mutable MarkCachePtr mark_cache; /// Кэш засечек в сжатых файлах.
ProcessList process_list; /// Исполняющиеся в данный момент запросы.
ViewDependencies view_dependencies; /// Текущие зависимости
ConfigurationPtr users_config; /// Конфиг с секциями users, profiles и quotas.
/// Кластеры для distributed таблиц
/// Создаются при создании Distributed таблиц, так как нужно дождаться пока будут выставлены Settings
@ -143,7 +146,7 @@ private:
String user; /// Текущий пользователь.
Poco::Net::IPAddress ip_address; /// IP-адрес, с которого задан запрос.
QuotaForIntervals * quota; /// Текущая квота.
QuotaForIntervalsPtr quota; /// Текущая квота.
String current_database; /// Текущая БД.
String current_query_id; /// Id текущего запроса.
NamesAndTypesList columns; /// Столбцы текущей обрабатываемой таблицы.
@ -163,12 +166,18 @@ public:
String getPath() const;
void setPath(const String & path);
void initUsersFromConfig();
/** Забрать список пользователей, квот и профилей настроек из этого конфига.
* Список пользователей полностью заменяется.
* Накопленные значения у квоты не сбрасываются, если квота не удалена.
*/
void setUsersConfig(ConfigurationPtr config);
ConfigurationPtr getUsersConfig();
void setUser(const String & name, const String & password, const Poco::Net::IPAddress & address, const String & quota_key);
String getUser() const { return user; }
Poco::Net::IPAddress getIPAddress() const { return ip_address; }
void initQuotasFromConfig();
void setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address);
QuotaForIntervals & getQuota();
@ -211,7 +220,8 @@ public:
/// Установить настройку по имени.
void setSetting(const String & name, const Field & value);
void setSetting(const String & name, const std::string & value);
const TableFunctionFactory & getTableFunctionFactory() const { return shared->table_function_factory; }
const FunctionFactory & getFunctionFactory() const { return shared->function_factory; }
const AggregateFunctionFactory & getAggregateFunctionFactory() const { return shared->aggregate_function_factory; }

View File

@ -50,7 +50,19 @@ struct QuotaValues
memset(this, 0, sizeof(*this));
}
void initFromConfig(const String & config_elem);
void initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config);
bool operator== (const QuotaValues & rhs) const
{
return
queries == rhs.queries &&
errors == rhs.errors &&
result_rows == rhs.result_rows &&
result_bytes == rhs.result_bytes &&
read_rows == rhs.read_rows &&
read_bytes == rhs.read_bytes &&
execution_time == rhs.execution_time;
}
};
@ -65,7 +77,7 @@ struct QuotaForInterval
QuotaForInterval() : rounded_time() {}
QuotaForInterval(time_t duration_) : duration(duration_) {}
void initFromConfig(const String & config_elem, time_t duration_);
void initFromConfig(const String & config_elem, time_t duration_, Poco::Util::AbstractConfiguration & config);
/// Увеличить соответствующее значение.
void addQuery(time_t current_time, const String & quota_name);
@ -82,6 +94,14 @@ struct QuotaForInterval
/// Получить текст, описывающий, какая часть квоты израсходована.
String toString() const;
bool operator== (const QuotaForInterval & rhs) const
{
return
rounded_time == rhs.rounded_time &&
duration == rhs.duration &&
max == rhs.max &&
used == rhs.used;
}
private:
/// Сбросить счётчик использованных ресурсов, если соответствующий интервал, за который считается квота, прошёл.
void updateTime(time_t current_time);
@ -99,10 +119,10 @@ private:
typedef std::map<size_t, QuotaForInterval> Container;
Container cont;
Quota * parent;
std::string name;
public:
QuotaForIntervals(Quota * parent_) : parent(parent_) {}
QuotaForIntervals(const std::string & name_ = "") : name(name_) {}
/// Есть ли хотя бы один интервал, за который считается квота?
bool empty() const
@ -110,7 +130,11 @@ public:
return cont.empty();
}
void initFromConfig(const String & config_elem);
void initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config);
/// Обновляет максимальные значения значениями из quota.
/// Удаляет интервалы, которых нет в quota, добавляет интревалы, которых нет здесь, но есть в quota.
void setMax(const QuotaForIntervals & quota);
void addQuery(time_t current_time);
void addError(time_t current_time);
@ -123,29 +147,37 @@ public:
/// Получить текст, описывающий, какая часть квоты израсходована.
String toString() const;
bool operator== (const QuotaForIntervals & rhs) const
{
return cont == rhs.cont && name == rhs.name;
}
};
typedef Poco::SharedPtr<QuotaForIntervals> QuotaForIntervalsPtr;
/// Ключ квоты -> квоты за интервалы. Если квота не допускает ключей, то накопленные значения хранятся по ключу 0.
struct Quota
{
typedef std::unordered_map<UInt64, QuotaForIntervals> Container;
typedef std::unordered_map<UInt64, QuotaForIntervalsPtr> Container;
String name;
/// Максимальные значения из конфига.
QuotaForIntervals max;
/// Максимальные и накопленные значения для разных ключей.
/// Для всех ключей максимальные значения одинаковы и взяты из max.
Container quota_for_keys;
Poco::FastMutex mutex;
bool is_keyed;
bool keyed_by_ip;
Quota() : max(this), is_keyed(false), keyed_by_ip(false) {}
Quota() : is_keyed(false), keyed_by_ip(false) {}
void initFromConfig(const String & config_elem, const String & name_);
QuotaForIntervals & get(const String & quota_key, const String & user_name, const Poco::Net::IPAddress & ip);
void loadFromConfig(const String & config_elem, const String & name_, Poco::Util::AbstractConfiguration & config);
QuotaForIntervalsPtr get(const String & quota_key, const String & user_name, const Poco::Net::IPAddress & ip);
};
@ -157,8 +189,9 @@ private:
Container cont;
public:
void initFromConfig();
QuotaForIntervals & get(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & ip);
void loadFromConfig(Poco::Util::AbstractConfiguration & config);
QuotaForIntervalsPtr get(const String & name, const String & quota_key,
const String & user_name, const Poco::Net::IPAddress & ip);
};
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <Poco/Timespan.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <DB/Core/Defines.h>
#include <DB/Core/Field.h>
@ -102,7 +103,7 @@ struct Settings
/** Установить настройки из профиля (в конфиге сервера, в одном профиле может быть перечислено много настроек).
* Профиль также может быть установлен с помощью функций set, как настройка profile.
*/
void setProfile(const String & profile_name);
void setProfile(const String & profile_name, Poco::Util::AbstractConfiguration & config);
/// Прочитать настройки из буфера. Они записаны как набор name-value пар, идущих подряд, заканчивающихся пустым name.
void deserialize(ReadBuffer & buf);

View File

@ -225,10 +225,8 @@ public:
return false;
}
void addFromConfig(const String & config_elem)
void addFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config)
{
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(config_elem, config_keys);
@ -266,16 +264,14 @@ struct User
AddressPatterns addresses;
User(const String & name_, const String & config_elem)
User(const String & name_, const String & config_elem, Poco::Util::AbstractConfiguration & config)
: name(name_)
{
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
password = config.getString(config_elem + ".password");
profile = config.getString(config_elem + ".profile");
quota = config.getString(config_elem + ".quota");
addresses.addFromConfig(config_elem + ".networks");
addresses.addFromConfig(config_elem + ".networks", config);
}
/// Для вставки в контейнер.
@ -291,15 +287,15 @@ private:
Container cont;
public:
void initFromConfig()
void loadFromConfig(Poco::Util::AbstractConfiguration & config)
{
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
cont.clear();
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys("users", config_keys);
for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = config_keys.begin(); it != config_keys.end(); ++it)
cont[*it] = User(*it, "users." + *it);
cont[*it] = User(*it, "users." + *it, config);
}
const User & get(const String & name, const String & password, const Poco::Net::IPAddress & address) const

View File

@ -27,10 +27,18 @@ void Context::setPath(const String & path)
}
void Context::initUsersFromConfig()
void Context::setUsersConfig(ConfigurationPtr config)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
shared->users.initFromConfig();
shared->users_config = config;
shared->users.loadFromConfig(*shared->users_config);
shared->quotas.loadFromConfig(*shared->users_config);
}
ConfigurationPtr Context::getUsersConfig()
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
return shared->users_config;
}
@ -47,17 +55,10 @@ void Context::setUser(const String & name, const String & password, const Poco::
}
void Context::initQuotasFromConfig()
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
shared->quotas.initFromConfig();
}
void Context::setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
quota = &shared->quotas.get(name, quota_key, user_name, address);
quota = shared->quotas.get(name, quota_key, user_name, address);
}
@ -343,7 +344,20 @@ void Context::setSettings(const Settings & settings_)
void Context::setSetting(const String & name, const Field & value)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
settings.set(name, value);
if (name == "profile")
settings.setProfile(value.safeGet<String>(), *shared->users_config);
else
settings.set(name, value);
}
void Context::setSetting(const String & name, const std::string & value)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
if (name == "profile")
settings.setProfile(value, *shared->users_config);
else
settings.set(name, value);
}

View File

@ -6,14 +6,14 @@
#include <DB/IO/ReadHelpers.h>
#include <DB/Interpreters/Quota.h>
#include <set>
namespace DB
{
void QuotaValues::initFromConfig(const String & config_elem)
void QuotaValues::initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config)
{
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
queries = parse<UInt64>(config.getString(config_elem + ".queries", "0"));
errors = parse<UInt64>(config.getString(config_elem + ".errors", "0"));
result_rows = parse<UInt64>(config.getString(config_elem + ".result_rows", "0"));
@ -24,11 +24,11 @@ void QuotaValues::initFromConfig(const String & config_elem)
}
void QuotaForInterval::initFromConfig(const String & config_elem, time_t duration_)
void QuotaForInterval::initFromConfig(const String & config_elem, time_t duration_, Poco::Util::AbstractConfiguration & config)
{
rounded_time = 0;
duration = duration_;
max.initFromConfig(config_elem);
max.initFromConfig(config_elem, config);
}
void QuotaForInterval::checkExceeded(time_t current_time, const String & quota_name)
@ -118,7 +118,7 @@ void QuotaForInterval::check(size_t max_amount, size_t used_amount, time_t curre
else
message << duration << " seconds";
message << " has been expired. "
message << " has been exceeded. "
<< resource_name << ": " << used_amount << ", max: " << max_amount << ". "
<< "Interval will end at " << mysqlxx::DateTime(rounded_time + duration) << ".";
@ -127,10 +127,8 @@ void QuotaForInterval::check(size_t max_amount, size_t used_amount, time_t curre
}
void QuotaForIntervals::initFromConfig(const String & config_elem)
void QuotaForIntervals::initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config)
{
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(config_elem, config_keys);
@ -142,44 +140,63 @@ void QuotaForIntervals::initFromConfig(const String & config_elem)
String interval_config_elem = config_elem + "." + *it;
time_t duration = config.getInt(interval_config_elem + ".duration");
cont[duration].initFromConfig(interval_config_elem, duration);
cont[duration].initFromConfig(interval_config_elem, duration, config);
}
}
void QuotaForIntervals::setMax(const QuotaForIntervals & quota)
{
for (Container::iterator it = cont.begin(); it != cont.end();)
{
if (quota.cont.count(it->first))
++it;
else
cont.erase(it++);
}
for (auto & x : quota.cont)
{
if (!cont.count(x.first))
cont[x.first] = x.second;
else
cont[x.first].max = x.second.max;
}
}
void QuotaForIntervals::checkExceeded(time_t current_time)
{
for (Container::reverse_iterator it = cont.rbegin(); it != cont.rend(); ++it)
it->second.checkExceeded(current_time, parent->name);
it->second.checkExceeded(current_time, name);
}
void QuotaForIntervals::addQuery(time_t current_time)
{
for (Container::reverse_iterator it = cont.rbegin(); it != cont.rend(); ++it)
it->second.addQuery(current_time, parent->name);
it->second.addQuery(current_time, name);
}
void QuotaForIntervals::addError(time_t current_time)
{
for (Container::reverse_iterator it = cont.rbegin(); it != cont.rend(); ++it)
it->second.addError(current_time, parent->name);
it->second.addError(current_time, name);
}
void QuotaForIntervals::checkAndAddResultRowsBytes(time_t current_time, size_t rows, size_t bytes)
{
for (Container::reverse_iterator it = cont.rbegin(); it != cont.rend(); ++it)
it->second.checkAndAddResultRowsBytes(current_time, parent->name, rows, bytes);
it->second.checkAndAddResultRowsBytes(current_time, name, rows, bytes);
}
void QuotaForIntervals::checkAndAddReadRowsBytes(time_t current_time, size_t rows, size_t bytes)
{
for (Container::reverse_iterator it = cont.rbegin(); it != cont.rend(); ++it)
it->second.checkAndAddReadRowsBytes(current_time, parent->name, rows, bytes);
it->second.checkAndAddReadRowsBytes(current_time, name, rows, bytes);
}
void QuotaForIntervals::checkAndAddExecutionTime(time_t current_time, Poco::Timespan amount)
{
for (Container::reverse_iterator it = cont.rbegin(); it != cont.rend(); ++it)
it->second.checkAndAddExecutionTime(current_time, parent->name, amount);
it->second.checkAndAddExecutionTime(current_time, name, amount);
}
String QuotaForIntervals::toString() const
@ -193,19 +210,34 @@ String QuotaForIntervals::toString() const
}
void Quota::initFromConfig(const String & config_elem, const String & name_)
void Quota::loadFromConfig(const String & config_elem, const String & name_, Poco::Util::AbstractConfiguration & config)
{
name = name_;
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
bool new_keyed_by_ip = config.has(config_elem + ".keyed_by_ip");
bool new_is_keyed = new_keyed_by_ip || config.has(config_elem + ".keyed");
keyed_by_ip = config.has(config_elem + ".keyed_by_ip");
is_keyed = keyed_by_ip || config.has(config_elem + ".keyed");
if (new_is_keyed != is_keyed || new_keyed_by_ip != keyed_by_ip)
{
keyed_by_ip = new_keyed_by_ip;
is_keyed = new_is_keyed;
/// Смысл ключей поменялся. Выбросим накопленные значения.
quota_for_keys.clear();
}
max.initFromConfig(config_elem);
QuotaForIntervals new_max(name);
new_max.initFromConfig(config_elem, config);
if (!(new_max == max))
{
max = new_max;
for (auto & quota : quota_for_keys)
{
quota.second->setMax(max);
}
}
}
QuotaForIntervals & Quota::get(const String & quota_key, const String & user_name, const Poco::Net::IPAddress & ip)
QuotaForIntervalsPtr Quota::get(const String & quota_key, const String & user_name, const Poco::Net::IPAddress & ip)
{
if (!quota_key.empty() && (!is_keyed || keyed_by_ip))
throw Exception("Quota " + name + " doesn't allow client supplied keys.", ErrorCodes::QUOTA_DOESNT_ALLOW_KEYS);
@ -228,29 +260,37 @@ QuotaForIntervals & Quota::get(const String & quota_key, const String & user_nam
Container::iterator it = quota_for_keys.find(quota_key_hashed);
if (quota_for_keys.end() == it)
{
it = quota_for_keys.insert(std::make_pair(quota_key_hashed, QuotaForIntervals(this))).first;
it->second = max;
it = quota_for_keys.insert(std::make_pair(quota_key_hashed, new QuotaForIntervals(max))).first;
}
return it->second;
}
void Quotas::initFromConfig()
void Quotas::loadFromConfig(Poco::Util::AbstractConfiguration & config)
{
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys("quotas", config_keys);
/// Удалим ключи, которых больше нет в кофиге.
std::set<std::string> keys_set(config_keys.begin(), config_keys.end());
for (Container::iterator it = cont.begin(); it != cont.end();)
{
if (keys_set.count(it->first))
++it;
else
cont.erase(it++);
}
for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = config_keys.begin(); it != config_keys.end(); ++it)
{
cont[*it] = new Quota();
cont[*it]->initFromConfig("quotas." + *it, *it);
if (!cont[*it])
cont[*it] = new Quota();
cont[*it]->loadFromConfig("quotas." + *it, *it, config);
}
}
QuotaForIntervals & Quotas::get(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & ip)
QuotaForIntervalsPtr Quotas::get(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & ip)
{
Container::iterator it = cont.find(name);
if (cont.end() == it)

View File

@ -41,7 +41,6 @@ void Settings::set(const String & name, const Field & value)
else if (name == "extremes") extremes = safeGet<UInt64>(value);
else if (name == "use_uncompressed_cache") use_uncompressed_cache = safeGet<UInt64>(value);
else if (name == "use_splitting_aggregator") use_splitting_aggregator = safeGet<UInt64>(value);
else if (name == "profile") setProfile(get<const String &>(value));
else if (name == "load_balancing") load_balancing = getLoadBalancing(safeGet<const String &>(value));
else if (name == "default_sample")
{
@ -85,12 +84,6 @@ void Settings::set(const String & name, ReadBuffer & buf)
readVarUInt(value, buf);
set(name, value);
}
else if (name == "profile")
{
String value;
readBinary(value, buf);
setProfile(value);
}
else if (name == "load_balancing")
{
String value;
@ -138,17 +131,12 @@ void Settings::set(const String & name, const String & value)
{
set(name, Field(value));
}
else if (name == "profile")
{
setProfile(value);
}
else if (!limits.trySet(name, value))
throw Exception("Unknown setting " + name, ErrorCodes::UNKNOWN_SETTING);
}
void Settings::setProfile(const String & profile_name)
void Settings::setProfile(const String & profile_name, Poco::Util::AbstractConfiguration & config)
{
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
String elem = "profiles." + profile_name;
if (!config.has(elem))

View File

@ -109,7 +109,7 @@ void HTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco::Net
{
}
else /// Все неизвестные параметры запроса рассматриваются, как настройки.
context.getSettingsRef().set(it->first, it->second);
context.setSetting(it->first, it->second);
}
if (readonly)

View File

@ -1,4 +1,5 @@
#include <Poco/Net/HTTPServerRequest.h>
#include <Poco/Util/XMLConfiguration.h>
#include <Yandex/ApplicationServerExt.h>
@ -82,6 +83,88 @@ public:
};
UsersConfigReloader::UsersConfigReloader(const std::string & path_, Poco::SharedPtr<Context> context_)
: path(path_), context(context_), file_modification_time(0), quit(false), log(&Logger::get("UsersConfigReloader"))
{
reloadIfNewer(true);
thread = std::thread(&UsersConfigReloader::run, this);
}
UsersConfigReloader::~UsersConfigReloader()
{
try
{
quit = true;
thread.join();
}
catch(...)
{
tryLogCurrentException("~UsersConfigReloader");
}
}
void UsersConfigReloader::run()
{
while (!quit)
{
std::this_thread::sleep_for(std::chrono::seconds(2));
reloadIfNewer(false);
}
}
void UsersConfigReloader::reloadIfNewer(bool force)
{
Poco::File f(path);
time_t new_modification_time = f.getLastModified().epochTime();
if (!force && new_modification_time == file_modification_time)
return;
file_modification_time = new_modification_time;
LOG_DEBUG(log, "Loading users config");
ConfigurationPtr config;
try
{
config = new Poco::Util::XMLConfiguration(path);
}
catch (Poco::Exception & e)
{
if (force)
throw;
LOG_ERROR(log, "Couldn't parse users config: " << e.what() << ", " << e.displayText());
return;
}
try
{
context->setUsersConfig(config);
}
catch (Exception & e)
{
if (force)
throw;
LOG_ERROR(log, "Error updating users config: " << e.what() << ": " << e.displayText() << "\n" << e.getStackTrace().toString());
}
catch (Poco::Exception & e)
{
if (force)
throw;
LOG_ERROR(log, "Error updating users config: " << e.what() << ": " << e.displayText());
}
catch (...)
{
if (force)
throw;
LOG_ERROR(log, "Error updating users config.");
}
}
int Server::main(const std::vector<std::string> & args)
{
Logger * log = &logger();
@ -99,11 +182,8 @@ int Server::main(const std::vector<std::string> & args)
global_context->setGlobalContext(*global_context);
global_context->setPath(config.getString("path"));
/// Загружаем пользователей.
global_context->initUsersFromConfig();
/// Загружаем квоты.
global_context->initQuotasFromConfig();
std::string users_config_path = config.getString("users_config", "users.xml");
users_config_reloader = new UsersConfigReloader(users_config_path, global_context);
/// Максимальное количество одновременно выполняющихся запросов.
global_context->getProcessList().setMaxSize(config.getInt("max_concurrent_queries", 0));
@ -120,7 +200,7 @@ int Server::main(const std::vector<std::string> & args)
/// Загружаем настройки.
Settings & settings = global_context->getSettingsRef();
settings.setProfile(config.getString("default_profile", "default"));
global_context->setSetting("profile", config.getString("default_profile", "default"));
LOG_INFO(log, "Loading metadata.");
loadMetadata(*global_context);
@ -198,6 +278,9 @@ int Server::main(const std::vector<std::string> & args)
waitForTerminationRequest();
LOG_DEBUG(log, "Received termination signal. Waiting for current connections to close.");
users_config_reloader = NULL;
is_cancelled = true;
http_server.stop();

View File

@ -25,6 +25,9 @@
#include "OLAPQueryParser.h"
#include "OLAPQueryConverter.h"
#include <thread>
#include <atomic>
/** Сервер предоставляет три интерфейса:
* 1. HTTP - простой интерфейс для доступа из любых приложений.
* 2. TCP - интерфейс для доступа из родной библиотеки, родного клиента, и для межсерверного взаимодействия.
@ -40,6 +43,27 @@ namespace DB
{
/// Каждые две секунды проверяет, не изменился ли конфиг. Когда изменился, вызывает setUsersConfig у контекста.
class UsersConfigReloader
{
public:
UsersConfigReloader(const std::string & path, Poco::SharedPtr<Context> context);
~UsersConfigReloader();
private:
std::string path;
Poco::SharedPtr<Context> context;
time_t file_modification_time;
std::atomic<bool> quit;
std::thread thread;
Logger * log;
void reloadIfNewer(bool force);
void run();
};
class Server : public Daemon
{
public:
@ -54,6 +78,8 @@ public:
Server() : config(Application::instance().config()) {}
protected:
Poco::SharedPtr<UsersConfigReloader> users_config_reloader;
void initialize(Application& self)
{
Daemon::initialize(self);