2017-10-06 10:31:06 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <chrono>
|
2019-06-02 12:11:01 +00:00
|
|
|
#include <functional>
|
|
|
|
#include <unordered_map>
|
2021-10-02 07:13:14 +00:00
|
|
|
#include <base/types.h>
|
2017-10-06 10:31:06 +00:00
|
|
|
#include <Interpreters/IExternalLoadable.h>
|
2019-09-30 16:12:08 +00:00
|
|
|
#include <Interpreters/IExternalLoaderConfigRepository.h>
|
2021-10-02 07:13:14 +00:00
|
|
|
#include <base/logger_useful.h>
|
|
|
|
#include <base/scope_guard.h>
|
2020-03-17 09:24:24 +00:00
|
|
|
#include <Common/ExternalLoaderStatus.h>
|
2017-10-06 10:31:06 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
/* 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;
|
2019-12-25 23:12:12 +00:00
|
|
|
std::string external_database;
|
2020-07-15 19:25:31 +00:00
|
|
|
std::string external_uuid;
|
2017-10-06 10:31:06 +00:00
|
|
|
};
|
|
|
|
|
2019-11-07 11:44:02 +00:00
|
|
|
/** Interface for manage user-defined objects.
|
2019-06-02 12:11:01 +00:00
|
|
|
* Monitors configuration file and automatically reloads objects in separate threads.
|
|
|
|
* The monitoring thread wakes up every 'check_period_sec' seconds and checks
|
|
|
|
* modification time of objects' configuration file. If said time is greater than
|
|
|
|
* 'config_last_modified', the objects are created from scratch using configuration file,
|
|
|
|
* 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.
|
|
|
|
*/
|
2017-10-06 10:31:06 +00:00
|
|
|
class ExternalLoader
|
|
|
|
{
|
2017-10-06 11:10:01 +00:00
|
|
|
public:
|
2019-06-02 12:11:01 +00:00
|
|
|
using LoadablePtr = std::shared_ptr<const IExternalLoadable>;
|
|
|
|
using Loadables = std::vector<LoadablePtr>;
|
2020-03-17 09:24:24 +00:00
|
|
|
using Status = ExternalLoaderStatus;
|
2017-10-06 11:10:01 +00:00
|
|
|
|
2019-06-02 12:11:01 +00:00
|
|
|
using Duration = std::chrono::milliseconds;
|
|
|
|
using TimePoint = std::chrono::system_clock::time_point;
|
|
|
|
|
2020-04-12 20:50:32 +00:00
|
|
|
struct ObjectConfig
|
|
|
|
{
|
|
|
|
Poco::AutoPtr<Poco::Util::AbstractConfiguration> config;
|
|
|
|
String key_in_config;
|
|
|
|
String repository_name;
|
|
|
|
bool from_temp_repository = false;
|
|
|
|
String path;
|
|
|
|
};
|
|
|
|
|
2019-06-02 12:11:01 +00:00
|
|
|
struct LoadResult
|
2017-10-06 11:10:01 +00:00
|
|
|
{
|
2019-12-12 18:33:43 +00:00
|
|
|
Status status = Status::NOT_EXIST;
|
|
|
|
String name;
|
2019-06-02 12:11:01 +00:00
|
|
|
LoadablePtr object;
|
|
|
|
TimePoint loading_start_time;
|
2020-02-27 10:07:50 +00:00
|
|
|
TimePoint last_successful_update_time;
|
2019-06-02 12:11:01 +00:00
|
|
|
Duration loading_duration;
|
|
|
|
std::exception_ptr exception;
|
2020-04-12 20:50:32 +00:00
|
|
|
std::shared_ptr<const ObjectConfig> config;
|
2017-10-06 11:10:01 +00:00
|
|
|
};
|
|
|
|
|
2019-12-12 18:33:43 +00:00
|
|
|
using LoadResults = std::vector<LoadResult>;
|
|
|
|
|
|
|
|
template <typename T>
|
|
|
|
static constexpr bool is_scalar_load_result_type = std::is_same_v<T, LoadResult> || std::is_same_v<T, LoadablePtr>;
|
|
|
|
|
|
|
|
template <typename T>
|
|
|
|
static constexpr bool is_vector_load_result_type = std::is_same_v<T, LoadResults> || std::is_same_v<T, Loadables>;
|
2019-06-02 12:11:01 +00:00
|
|
|
|
2020-05-30 21:57:37 +00:00
|
|
|
ExternalLoader(const String & type_name_, Poco::Logger * log);
|
2017-10-06 11:10:01 +00:00
|
|
|
virtual ~ExternalLoader();
|
2017-10-06 10:31:06 +00:00
|
|
|
|
2019-06-02 12:11:01 +00:00
|
|
|
/// Adds a repository which will be used to read configurations from.
|
2021-06-15 19:55:21 +00:00
|
|
|
scope_guard addConfigRepository(std::unique_ptr<IExternalLoaderConfigRepository> config_repository) const;
|
2019-10-15 18:04:17 +00:00
|
|
|
|
2019-12-30 23:30:06 +00:00
|
|
|
void setConfigSettings(const ExternalLoaderConfigSettings & settings_);
|
2019-04-10 18:36:52 +00:00
|
|
|
|
2019-06-02 12:11:01 +00:00
|
|
|
/// Sets whether all the objects from the configuration should be always loaded (even those which are never used).
|
|
|
|
void enableAlwaysLoadEverything(bool enable);
|
2017-10-06 10:31:06 +00:00
|
|
|
|
2019-06-02 12:11:01 +00:00
|
|
|
/// Sets whether the objects should be loaded asynchronously, each loading in a new thread (from the thread pool).
|
|
|
|
void enableAsyncLoading(bool enable);
|
2017-10-06 10:31:06 +00:00
|
|
|
|
2019-06-02 12:11:01 +00:00
|
|
|
/// Sets settings for periodic updates.
|
2019-08-30 09:50:38 +00:00
|
|
|
void enablePeriodicUpdates(bool enable);
|
2017-10-06 10:31:06 +00:00
|
|
|
|
2019-06-02 12:11:01 +00:00
|
|
|
/// Returns the status of the object.
|
|
|
|
/// If the object has not been loaded yet then the function returns Status::NOT_LOADED.
|
|
|
|
/// If the specified name isn't found in the configuration then the function returns Status::NOT_EXIST.
|
|
|
|
Status getCurrentStatus(const String & name) const;
|
2017-10-26 14:26:03 +00:00
|
|
|
|
2019-06-02 12:11:01 +00:00
|
|
|
/// Returns the result of loading the object.
|
|
|
|
/// The function doesn't load anything, it just returns the current load result as is.
|
2019-12-12 18:33:43 +00:00
|
|
|
template <typename ReturnType = LoadResult, typename = std::enable_if_t<is_scalar_load_result_type<ReturnType>, void>>
|
2020-04-12 20:50:32 +00:00
|
|
|
ReturnType getLoadResult(const String & name) const;
|
2017-10-06 11:10:01 +00:00
|
|
|
|
2019-06-02 12:11:01 +00:00
|
|
|
using FilterByNameFunction = std::function<bool(const String &)>;
|
2017-10-06 11:10:01 +00:00
|
|
|
|
2019-06-02 12:11:01 +00:00
|
|
|
/// Returns all the load results as a map.
|
|
|
|
/// The function doesn't load anything, it just returns the current load results as is.
|
2019-12-12 18:33:43 +00:00
|
|
|
template <typename ReturnType = LoadResults, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>>
|
2020-04-12 20:50:32 +00:00
|
|
|
ReturnType getLoadResults() const { return getLoadResults<ReturnType>(FilterByNameFunction{}); }
|
2019-12-12 18:33:43 +00:00
|
|
|
|
|
|
|
template <typename ReturnType = LoadResults, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>>
|
2020-04-12 20:50:32 +00:00
|
|
|
ReturnType getLoadResults(const FilterByNameFunction & filter) const;
|
2017-10-06 10:31:06 +00:00
|
|
|
|
2019-06-02 12:11:01 +00:00
|
|
|
/// Returns all loaded objects as a map.
|
|
|
|
/// The function doesn't load anything, it just returns the current load results as is.
|
2020-04-12 20:50:32 +00:00
|
|
|
Loadables getLoadedObjects() const;
|
|
|
|
Loadables getLoadedObjects(const FilterByNameFunction & filter) const;
|
2017-10-06 10:31:06 +00:00
|
|
|
|
2019-06-29 17:27:32 +00:00
|
|
|
/// Returns true if any object was loaded.
|
2020-04-12 20:50:32 +00:00
|
|
|
bool hasLoadedObjects() const;
|
|
|
|
size_t getNumberOfLoadedObjects() const;
|
|
|
|
|
|
|
|
/// Returns true if there is no object.
|
|
|
|
bool hasObjects() const { return getNumberOfObjects() == 0; }
|
|
|
|
|
|
|
|
/// Returns number of objects.
|
|
|
|
size_t getNumberOfObjects() const;
|
2019-06-29 17:27:32 +00:00
|
|
|
|
2019-12-12 18:33:43 +00:00
|
|
|
static constexpr Duration NO_WAIT = Duration::zero();
|
|
|
|
static constexpr Duration WAIT = Duration::max();
|
|
|
|
|
|
|
|
/// Loads a specified object.
|
|
|
|
/// The function does nothing if it's already loaded.
|
|
|
|
/// The function doesn't throw an exception if it's failed to load.
|
|
|
|
template <typename ReturnType = LoadablePtr, typename = std::enable_if_t<is_scalar_load_result_type<ReturnType>, void>>
|
|
|
|
ReturnType tryLoad(const String & name, Duration timeout = WAIT) const;
|
|
|
|
|
|
|
|
/// Loads objects by filter.
|
|
|
|
/// The function does nothing for already loaded objects, it just returns them.
|
|
|
|
/// The function doesn't throw an exception if it's failed to load something.
|
|
|
|
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>>
|
|
|
|
ReturnType tryLoad(const FilterByNameFunction & filter, Duration timeout = WAIT) const;
|
|
|
|
|
|
|
|
/// Loads all objects.
|
|
|
|
/// The function does nothing for already loaded objects, it just returns them.
|
|
|
|
/// The function doesn't throw an exception if it's failed to load something.
|
|
|
|
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>>
|
2020-04-12 20:50:32 +00:00
|
|
|
ReturnType tryLoadAll(Duration timeout = WAIT) const { return tryLoad<ReturnType>(FilterByNameFunction{}, timeout); }
|
2019-12-12 18:33:43 +00:00
|
|
|
|
|
|
|
/// Loads a specified object.
|
|
|
|
/// The function does nothing if it's already loaded.
|
|
|
|
/// The function throws an exception if it's failed to load.
|
|
|
|
template <typename ReturnType = LoadablePtr, typename = std::enable_if_t<is_scalar_load_result_type<ReturnType>, void>>
|
|
|
|
ReturnType load(const String & name) const;
|
|
|
|
|
|
|
|
/// Loads objects by filter.
|
|
|
|
/// The function does nothing for already loaded objects, it just returns them.
|
|
|
|
/// The function throws an exception if it's failed to load something.
|
|
|
|
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>>
|
|
|
|
ReturnType load(const FilterByNameFunction & filter) const;
|
|
|
|
|
|
|
|
/// Loads all objects. Not recommended to use.
|
|
|
|
/// The function does nothing for already loaded objects, it just returns them.
|
|
|
|
/// The function throws an exception if it's failed to load something.
|
|
|
|
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>>
|
2020-04-12 20:50:32 +00:00
|
|
|
ReturnType loadAll() const { return load<ReturnType>(FilterByNameFunction{}); }
|
2019-12-12 18:33:43 +00:00
|
|
|
|
|
|
|
/// Loads or reloads a specified object.
|
|
|
|
/// The function reloads the object if it's already loaded.
|
|
|
|
/// The function throws an exception if it's failed to load or reload.
|
|
|
|
template <typename ReturnType = LoadablePtr, typename = std::enable_if_t<is_scalar_load_result_type<ReturnType>, void>>
|
|
|
|
ReturnType loadOrReload(const String & name) const;
|
|
|
|
|
|
|
|
/// Loads or reloads objects by filter.
|
|
|
|
/// The function reloads the objects which are already loaded.
|
|
|
|
/// The function throws an exception if it's failed to load or reload something.
|
|
|
|
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>>
|
|
|
|
ReturnType loadOrReload(const FilterByNameFunction & filter) const;
|
|
|
|
|
|
|
|
/// Load or reloads all objects. Not recommended to use.
|
|
|
|
/// The function throws an exception if it's failed to load or reload something.
|
|
|
|
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>>
|
2020-04-12 20:50:32 +00:00
|
|
|
ReturnType loadOrReloadAll() const { return loadOrReload<ReturnType>(FilterByNameFunction{}); }
|
2019-12-12 18:33:43 +00:00
|
|
|
|
|
|
|
/// Reloads objects by filter which were tried to load before (successfully or not).
|
|
|
|
/// The function throws an exception if it's failed to load or reload something.
|
|
|
|
template <typename ReturnType = Loadables, typename = std::enable_if_t<is_vector_load_result_type<ReturnType>, void>>
|
|
|
|
ReturnType reloadAllTriedToLoad() const;
|
2019-11-25 22:48:23 +00:00
|
|
|
|
2021-03-18 14:03:22 +00:00
|
|
|
/// Check if object with name exists in configuration
|
|
|
|
bool has(const String & name) const;
|
|
|
|
|
2019-12-12 16:41:41 +00:00
|
|
|
/// Reloads all config repositories.
|
|
|
|
void reloadConfig() const;
|
|
|
|
|
|
|
|
/// Reloads only a specified config repository.
|
|
|
|
void reloadConfig(const String & repository_name) const;
|
|
|
|
|
|
|
|
/// Reload only a specified path in a specified config repository.
|
|
|
|
void reloadConfig(const String & repository_name, const String & path) const;
|
|
|
|
|
2019-06-02 12:11:01 +00:00
|
|
|
protected:
|
2019-12-11 11:09:21 +00:00
|
|
|
virtual LoadablePtr create(const String & name, const Poco::Util::AbstractConfiguration & config, const String & key_in_config, const String & repository_name) const = 0;
|
2017-10-06 10:31:06 +00:00
|
|
|
|
2019-06-02 12:11:01 +00:00
|
|
|
private:
|
2019-12-12 18:33:43 +00:00
|
|
|
void checkLoaded(const LoadResult & result, bool check_no_errors) const;
|
|
|
|
void checkLoaded(const LoadResults & results, bool check_no_errors) const;
|
2019-04-10 18:36:52 +00:00
|
|
|
|
2019-12-12 18:33:43 +00:00
|
|
|
Strings getAllTriedToLoadNames() const;
|
|
|
|
|
2019-12-04 23:53:06 +00:00
|
|
|
LoadablePtr createObject(const String & name, const ObjectConfig & config, const LoadablePtr & previous_version) const;
|
2019-04-10 18:36:52 +00:00
|
|
|
|
2021-09-10 09:54:22 +00:00
|
|
|
/// We have to read configuration from LoadablesConfigReader and load configuration using LoadingDispatcher atomically.
|
|
|
|
/// Otherwise we can read configuration in one thread, then read and load newer configuration in another thread,
|
|
|
|
/// and then load outdated configuration from the first thread.
|
|
|
|
/// Remarkably, each class (LoadablesConfigReader, LoadingDispatcher, PeriodicUpdater) has own mutex for own purposes,
|
|
|
|
/// but it does not save from complex logical race conditions.
|
|
|
|
/// TODO Refactor dictionaries loading and get rid of this.
|
|
|
|
mutable std::recursive_mutex config_mutex;
|
|
|
|
|
2019-09-30 16:12:08 +00:00
|
|
|
class LoadablesConfigReader;
|
|
|
|
std::unique_ptr<LoadablesConfigReader> config_files_reader;
|
2017-10-06 10:31:06 +00:00
|
|
|
|
2019-06-02 12:11:01 +00:00
|
|
|
class LoadingDispatcher;
|
|
|
|
std::unique_ptr<LoadingDispatcher> loading_dispatcher;
|
2018-03-23 19:56:24 +00:00
|
|
|
|
2019-06-02 12:11:01 +00:00
|
|
|
class PeriodicUpdater;
|
|
|
|
std::unique_ptr<PeriodicUpdater> periodic_updater;
|
2019-04-15 19:53:46 +00:00
|
|
|
|
2019-06-02 12:11:01 +00:00
|
|
|
const String type_name;
|
2019-12-12 18:33:43 +00:00
|
|
|
Poco::Logger * log;
|
2017-10-06 10:31:06 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|