2017-10-06 10:31:06 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <common/logger_useful.h>
|
|
|
|
#include <Poco/Event.h>
|
|
|
|
#include <mutex>
|
|
|
|
#include <thread>
|
|
|
|
#include <unordered_map>
|
2018-08-03 19:49:03 +00:00
|
|
|
#include <unordered_set>
|
2017-10-06 10:31:06 +00:00
|
|
|
#include <chrono>
|
2017-10-06 11:10:01 +00:00
|
|
|
#include <tuple>
|
2017-10-06 10:31:06 +00:00
|
|
|
#include <Interpreters/IExternalLoadable.h>
|
2017-11-28 11:00:07 +00:00
|
|
|
#include <Interpreters/IExternalLoaderConfigRepository.h>
|
2017-10-06 11:10:01 +00:00
|
|
|
#include <Core/Types.h>
|
|
|
|
#include <pcg_random.hpp>
|
|
|
|
#include <Common/randomSeed.h>
|
2019-01-14 19:22:09 +00:00
|
|
|
#include <Common/ThreadPool.h>
|
2017-10-06 10:31:06 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
struct ExternalLoaderUpdateSettings
|
|
|
|
{
|
|
|
|
UInt64 check_period_sec = 5;
|
|
|
|
UInt64 backoff_initial_sec = 5;
|
|
|
|
/// 10 minutes
|
|
|
|
UInt64 backoff_max_sec = 10 * 60;
|
2017-10-26 18:39:55 +00:00
|
|
|
|
|
|
|
ExternalLoaderUpdateSettings() = default;
|
|
|
|
ExternalLoaderUpdateSettings(UInt64 check_period_sec, UInt64 backoff_initial_sec, UInt64 backoff_max_sec)
|
|
|
|
: check_period_sec(check_period_sec),
|
|
|
|
backoff_initial_sec(backoff_initial_sec),
|
|
|
|
backoff_max_sec(backoff_max_sec) {}
|
2017-10-06 10:31:06 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
/* External configuration structure.
|
|
|
|
*
|
|
|
|
* <external_group>
|
|
|
|
* <external_config>
|
|
|
|
* <external_name>name</external_name>
|
|
|
|
* ....
|
|
|
|
* </external_config>
|
|
|
|
* </external_group>
|
|
|
|
*/
|
|
|
|
struct ExternalLoaderConfigSettings
|
|
|
|
{
|
|
|
|
std::string external_config;
|
|
|
|
std::string external_name;
|
|
|
|
|
|
|
|
std::string path_setting_name;
|
|
|
|
};
|
|
|
|
|
|
|
|
/** Manages user-defined objects.
|
|
|
|
* Monitors configuration file and automatically reloads objects in a separate thread.
|
2018-06-03 16:51:31 +00:00
|
|
|
* The monitoring thread wakes up every 'check_period_sec' seconds and checks
|
2017-10-06 10:31:06 +00:00
|
|
|
* modification time of objects' configuration file. If said time is greater than
|
2018-06-03 16:51:31 +00:00
|
|
|
* 'config_last_modified', the objects are created from scratch using configuration file,
|
2017-10-06 10:31:06 +00:00
|
|
|
* possibly overriding currently existing objects with the same name (previous versions of
|
|
|
|
* overridden objects will live as long as there are any users retaining them).
|
|
|
|
*
|
|
|
|
* Apart from checking configuration file for modifications, each object
|
|
|
|
* has a lifetime of its own and may be updated if it supportUpdates.
|
|
|
|
* The time of next update is calculated by choosing uniformly a random number
|
|
|
|
* distributed between lifetime.min_sec and lifetime.max_sec.
|
|
|
|
* If either of lifetime.min_sec and lifetime.max_sec is zero, such object is never updated.
|
|
|
|
*/
|
|
|
|
class ExternalLoader
|
|
|
|
{
|
2017-10-06 11:10:01 +00:00
|
|
|
public:
|
|
|
|
using LoadablePtr = std::shared_ptr<IExternalLoadable>;
|
|
|
|
|
|
|
|
private:
|
|
|
|
struct LoadableInfo final
|
|
|
|
{
|
|
|
|
LoadablePtr loadable;
|
|
|
|
std::string origin;
|
|
|
|
std::exception_ptr exception;
|
|
|
|
};
|
|
|
|
|
|
|
|
struct FailedLoadableInfo final
|
|
|
|
{
|
|
|
|
std::unique_ptr<IExternalLoadable> loadable;
|
|
|
|
std::chrono::system_clock::time_point next_attempt_time;
|
|
|
|
UInt64 error_count;
|
|
|
|
};
|
|
|
|
|
2017-10-06 10:31:06 +00:00
|
|
|
public:
|
|
|
|
using Configuration = Poco::Util::AbstractConfiguration;
|
|
|
|
using ObjectsMap = std::unordered_map<std::string, LoadableInfo>;
|
|
|
|
|
2019-04-10 18:36:52 +00:00
|
|
|
/// Call init() after constructing the instance of any derived class.
|
2018-11-27 16:11:46 +00:00
|
|
|
ExternalLoader(const Configuration & config_main,
|
2017-10-06 10:31:06 +00:00
|
|
|
const ExternalLoaderUpdateSettings & update_settings,
|
|
|
|
const ExternalLoaderConfigSettings & config_settings,
|
2017-11-28 11:00:07 +00:00
|
|
|
std::unique_ptr<IExternalLoaderConfigRepository> config_repository,
|
2017-10-06 11:10:01 +00:00
|
|
|
Logger * log, const std::string & loadable_object_name);
|
|
|
|
virtual ~ExternalLoader();
|
2017-10-06 10:31:06 +00:00
|
|
|
|
2019-04-10 18:36:52 +00:00
|
|
|
/// Should be called after creating an instance of a derived class.
|
|
|
|
/// Loads the objects immediately and starts a separate thread to update them once in each 'reload_period' seconds.
|
|
|
|
/// This function does nothing if called again.
|
|
|
|
void init(bool throw_on_error);
|
|
|
|
|
2017-10-06 10:31:06 +00:00
|
|
|
/// Forcibly reloads all loadable objects.
|
|
|
|
void reload();
|
|
|
|
|
|
|
|
/// Forcibly reloads specified loadable object.
|
|
|
|
void reload(const std::string & name);
|
|
|
|
|
|
|
|
LoadablePtr getLoadable(const std::string & name) const;
|
2018-03-23 19:56:24 +00:00
|
|
|
LoadablePtr tryGetLoadable(const std::string & name) const;
|
2017-10-06 10:31:06 +00:00
|
|
|
|
|
|
|
protected:
|
2017-10-06 11:10:01 +00:00
|
|
|
virtual std::unique_ptr<IExternalLoadable> create(const std::string & name, const Configuration & config,
|
2019-04-10 18:36:52 +00:00
|
|
|
const std::string & config_prefix) const = 0;
|
2017-10-06 10:31:06 +00:00
|
|
|
|
2017-10-26 14:26:03 +00:00
|
|
|
class LockedObjectsMap
|
|
|
|
{
|
|
|
|
public:
|
2018-03-08 01:50:58 +00:00
|
|
|
LockedObjectsMap(std::mutex & mutex, const ObjectsMap & objects_map) : lock(mutex), objects_map(objects_map) {}
|
|
|
|
const ObjectsMap & get() { return objects_map; }
|
2017-10-26 14:26:03 +00:00
|
|
|
private:
|
|
|
|
std::unique_lock<std::mutex> lock;
|
2018-03-08 01:50:58 +00:00
|
|
|
const ObjectsMap & objects_map;
|
2017-10-26 14:26:03 +00:00
|
|
|
};
|
|
|
|
|
2017-10-06 10:31:06 +00:00
|
|
|
/// Direct access to objects.
|
2017-10-26 14:26:03 +00:00
|
|
|
LockedObjectsMap getObjectsMap() const;
|
2017-10-06 11:10:01 +00:00
|
|
|
|
2017-10-06 10:31:06 +00:00
|
|
|
private:
|
2019-04-10 18:36:52 +00:00
|
|
|
std::once_flag is_initialized_flag;
|
2017-10-06 11:10:01 +00:00
|
|
|
|
|
|
|
/// Protects only objects map.
|
2018-12-30 01:09:06 +00:00
|
|
|
/** Reading and assignment of "loadable" should be done under mutex.
|
|
|
|
* Creating new versions of "loadable" should not be done under mutex.
|
|
|
|
*/
|
2017-10-06 10:31:06 +00:00
|
|
|
mutable std::mutex map_mutex;
|
|
|
|
|
2019-04-10 18:36:52 +00:00
|
|
|
/// Protects all data, currently used to avoid races between updating thread and SYSTEM queries.
|
|
|
|
/// The mutex is recursive because creating of objects might be recursive, i.e.
|
|
|
|
/// creating objects might cause creating other objects.
|
|
|
|
mutable std::recursive_mutex all_mutex;
|
2017-10-06 10:31:06 +00:00
|
|
|
|
|
|
|
/// name -> loadable.
|
2019-04-10 18:36:52 +00:00
|
|
|
mutable ObjectsMap loadable_objects;
|
|
|
|
|
|
|
|
struct LoadableCreationInfo
|
|
|
|
{
|
|
|
|
std::string name;
|
|
|
|
Poco::AutoPtr<Poco::Util::AbstractConfiguration> config;
|
|
|
|
std::string config_path;
|
|
|
|
std::string config_prefix;
|
|
|
|
};
|
|
|
|
|
|
|
|
/// Objects which should be reloaded soon.
|
|
|
|
mutable std::unordered_map<std::string, LoadableCreationInfo> objects_to_reload;
|
|
|
|
|
|
|
|
/// Should some objects be reloaded?
|
|
|
|
mutable std::atomic<bool> has_objects_to_reload = true;
|
2017-10-06 10:31:06 +00:00
|
|
|
|
2017-10-06 11:10:01 +00:00
|
|
|
/// Here are loadable objects, that has been never loaded successfully.
|
|
|
|
/// They are also in 'loadable_objects', but with nullptr as 'loadable'.
|
2019-04-10 18:36:52 +00:00
|
|
|
mutable std::unordered_map<std::string, FailedLoadableInfo> failed_loadable_objects;
|
2017-10-06 10:31:06 +00:00
|
|
|
|
2017-10-06 11:10:01 +00:00
|
|
|
/// Both for loadable_objects and failed_loadable_objects.
|
2019-04-10 18:36:52 +00:00
|
|
|
mutable std::unordered_map<std::string, std::chrono::system_clock::time_point> update_times;
|
2017-10-06 10:31:06 +00:00
|
|
|
|
2018-08-03 19:49:03 +00:00
|
|
|
std::unordered_map<std::string, std::unordered_set<std::string>> loadable_objects_defined_in_config;
|
|
|
|
|
2019-04-10 18:36:52 +00:00
|
|
|
mutable pcg64 rnd_engine{randomSeed()};
|
2017-10-06 10:31:06 +00:00
|
|
|
|
2018-11-27 16:11:46 +00:00
|
|
|
const Configuration & config_main;
|
2017-10-06 10:31:06 +00:00
|
|
|
const ExternalLoaderUpdateSettings & update_settings;
|
|
|
|
const ExternalLoaderConfigSettings & config_settings;
|
|
|
|
|
2017-11-28 11:00:07 +00:00
|
|
|
std::unique_ptr<IExternalLoaderConfigRepository> config_repository;
|
|
|
|
|
2019-01-14 19:22:09 +00:00
|
|
|
ThreadFromGlobalPool reloading_thread;
|
2017-10-06 10:31:06 +00:00
|
|
|
Poco::Event destroy;
|
|
|
|
|
|
|
|
Logger * log;
|
|
|
|
/// Loadable object name to use in log messages.
|
|
|
|
std::string object_name;
|
|
|
|
|
|
|
|
std::unordered_map<std::string, Poco::Timestamp> last_modification_times;
|
|
|
|
|
2019-04-10 18:36:52 +00:00
|
|
|
void initImpl(bool throw_on_error);
|
|
|
|
|
2017-10-06 11:10:01 +00:00
|
|
|
/// Check objects definitions in config files and reload or/and add new ones if the definition is changed
|
2017-10-06 10:31:06 +00:00
|
|
|
/// If loadable_name is not empty, load only loadable object with name loadable_name
|
|
|
|
void reloadFromConfigFiles(bool throw_on_error, bool force_reload = false, const std::string & loadable_name = "");
|
2019-04-10 18:36:52 +00:00
|
|
|
void reloadFromConfigFile(const std::string & config_path, const bool force_reload, const std::string & loadable_name);
|
|
|
|
|
|
|
|
void ensureReloadFinished(const std::string & loadable_name, bool throw_on_error) const;
|
|
|
|
void ensureReloadFinished(bool throw_on_error) const;
|
|
|
|
void ensureReloadFinishedImpl(const LoadableCreationInfo & creation_info, bool throw_on_error) const;
|
2017-10-06 10:31:06 +00:00
|
|
|
|
|
|
|
/// Check config files and update expired loadable objects
|
|
|
|
void reloadAndUpdate(bool throw_on_error = false);
|
|
|
|
|
|
|
|
void reloadPeriodically();
|
2018-03-23 19:56:24 +00:00
|
|
|
|
|
|
|
LoadablePtr getLoadableImpl(const std::string & name, bool throw_on_error) const;
|
2017-10-06 10:31:06 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|