ExternalLoader.h 8.8 KB
Newer Older
1 2 3
#pragma once

#include <chrono>
4 5 6
#include <functional>
#include <unordered_map>
#include <Core/Types.h>
7
#include <Interpreters/IExternalLoadable.h>
8
#include <Interpreters/IExternalLoaderConfigRepository.h>
9
#include <common/logger_useful.h>
10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28


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;
};

29

30
/** Interface for manage user-defined objects.
31 32 33 34 35 36 37 38 39 40 41 42 43
  * 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.
  */
44 45
class ExternalLoader
{
N
Nikolai Kochetov 已提交
46
public:
47 48
    using LoadablePtr = std::shared_ptr<const IExternalLoadable>;
    using Loadables = std::vector<LoadablePtr>;
N
Nikolai Kochetov 已提交
49

50
    enum class Status
N
Nikolai Kochetov 已提交
51
    {
52 53 54 55 56 57 58
        NOT_LOADED, /// Object hasn't been tried to load. This is an initial state.
        LOADED, /// Object has been loaded successfully.
        FAILED, /// Object has been failed to load.
        LOADING, /// Object is being loaded right now for the first time.
        FAILED_AND_RELOADING, /// Object was failed to load before and it's being reloaded right now.
        LOADED_AND_RELOADING, /// Object was loaded successfully before and it's being reloaded right now.
        NOT_EXIST, /// Object with this name wasn't found in the configuration.
N
Nikolai Kochetov 已提交
59 60
    };

61 62 63 64 65 66
    static std::vector<std::pair<String, Int8>> getStatusEnumAllPossibleValues();

    using Duration = std::chrono::milliseconds;
    using TimePoint = std::chrono::system_clock::time_point;

    struct LoadResult
N
Nikolai Kochetov 已提交
67
    {
68 69 70 71 72 73 74
        LoadResult(Status status_) : status(status_) {}
        Status status;
        LoadablePtr object;
        String origin;
        TimePoint loading_start_time;
        Duration loading_duration;
        std::exception_ptr exception;
A
alesapin 已提交
75
        std::string repository_name;
N
Nikolai Kochetov 已提交
76 77
    };

78 79
    using LoadResults = std::vector<std::pair<String, LoadResult>>;

80
    ExternalLoader(const String & type_name_, Logger * log);
N
Nikolai Kochetov 已提交
81
    virtual ~ExternalLoader();
82

83 84
    /// Adds a repository which will be used to read configurations from.
    void addConfigRepository(
A
alesapin 已提交
85 86 87 88 89 90
        const std::string & repository_name,
        std::unique_ptr<IExternalLoaderConfigRepository> config_repository,
        const ExternalLoaderConfigSettings & config_settings);

    /// Removes a repository which were used to read configurations.
    void removeConfigRepository(const std::string & repository_name);
91

92 93
    /// Sets whether all the objects from the configuration should be always loaded (even those which are never used).
    void enableAlwaysLoadEverything(bool enable);
94

95 96
    /// Sets whether the objects should be loaded asynchronously, each loading in a new thread (from the thread pool).
    void enableAsyncLoading(bool enable);
97

98
    /// Sets settings for periodic updates.
99
    void enablePeriodicUpdates(bool enable);
100

101 102 103 104
    /// 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;
105

106 107 108
    /// Returns the result of loading the object.
    /// The function doesn't load anything, it just returns the current load result as is.
    LoadResult getCurrentLoadResult(const String & name) const;
N
Nikolai Kochetov 已提交
109

110
    using FilterByNameFunction = std::function<bool(const String &)>;
N
Nikolai Kochetov 已提交
111

112 113 114 115
    /// Returns all the load results as a map.
    /// The function doesn't load anything, it just returns the current load results as is.
    LoadResults getCurrentLoadResults() const;
    LoadResults getCurrentLoadResults(const FilterByNameFunction & filter_by_name) const;
116

117 118 119 120 121
    /// Returns all loaded objects as a map.
    /// The function doesn't load anything, it just returns the current load results as is.
    Loadables getCurrentlyLoadedObjects() const;
    Loadables getCurrentlyLoadedObjects(const FilterByNameFunction & filter_by_name) const;
    size_t getNumberOfCurrentlyLoadedObjects() const;
122

123 124 125
    /// Returns true if any object was loaded.
    bool hasCurrentlyLoadedObjects() const;

126
    static constexpr Duration NO_TIMEOUT = Duration::max();
127

128 129 130 131 132
    /// Tries to finish loading of a specified object during the timeout.
    /// Returns nullptr if the loading is unsuccessful or if there is no such object.
    void load(const String & name, LoadablePtr & loaded_object, Duration timeout = NO_TIMEOUT) const;
    LoadablePtr loadAndGet(const String & name, Duration timeout = NO_TIMEOUT) const { LoadablePtr object; load(name, object, timeout); return object; }
    LoadablePtr tryGetLoadable(const String & name) const { return loadAndGet(name); }
133

134 135 136
    /// Tries to finish loading of a specified object during the timeout.
    /// Throws an exception if the loading is unsuccessful or if there is no such object.
    void loadStrict(const String & name, LoadablePtr & loaded_object) const;
A
alesapin 已提交
137
    LoadablePtr getLoadable(const String & name) const { LoadablePtr object; loadStrict(name, object); return object; }
138

139 140
    /// Tries to finish loading of the objects for which the specified function returns true.
    void load(const FilterByNameFunction & filter_by_name, Loadables & loaded_objects, Duration timeout = NO_TIMEOUT) const;
A
alesapin 已提交
141
    void load(const FilterByNameFunction & filter_by_name, LoadResults & load_results, Duration timeout = NO_TIMEOUT) const;
142
    Loadables loadAndGet(const FilterByNameFunction & filter_by_name, Duration timeout = NO_TIMEOUT) const { Loadables loaded_objects; load(filter_by_name, loaded_objects, timeout); return loaded_objects; }
143

144 145
    /// Tries to finish loading of all the objects during the timeout.
    void load(Loadables & loaded_objects, Duration timeout = NO_TIMEOUT) const;
146

147 148 149
    /// Starts reloading of a specified object.
    /// `load_never_loading` specifies what to do if the object has never been loading before.
    /// The function can either skip it (false) or load for the first time (true).
150
    /// Also function can load dictionary synchronously
A
alesapin 已提交
151
    void reload(const String & name, bool load_never_loading = false) const;
152

153 154 155
    /// Starts reloading of all the objects.
    /// `load_never_loading` specifies what to do with the objects which have never been loading before.
    /// The function can either skip them (false) or load for the first time (true).
A
alesapin 已提交
156
    void reload(bool load_never_loading = false) const;
157

158 159 160 161 162
    /// Starts reloading of all objects matched `filter_by_name`.
    /// `load_never_loading` specifies what to do with the objects which have never been loading before.
    /// The function can either skip them (false) or load for the first time (true).
    void reload(const FilterByNameFunction & filter_by_name, bool load_never_loading = false) const;

163 164
protected:
    virtual LoadablePtr create(const String & name, const Poco::Util::AbstractConfiguration & config, const String & key_in_config) const = 0;
165

166
    /// Reload object with already parsed configuration
A
alesapin 已提交
167
    void addObjectAndLoad(
168 169 170 171 172
        const String & name, /// name of dictionary
        const String & external_name, /// name of source (example xml-file, may contain more than dictionary)
        const String & repo_name, /// name of repository (database name, or all xml files)
        const Poco::AutoPtr<Poco::Util::AbstractConfiguration> & config,
        const String & key_in_config, /// key where we can start search of loadables (<dictionary>, <model>, etc)
A
alesapin 已提交
173
        bool load_never_loading = false) const;
174

175 176
private:
    struct ObjectConfig;
177

178
    LoadablePtr createObject(const String & name, const ObjectConfig & config, bool config_changed, const LoadablePtr & previous_version) const;
179

180 181
    class LoadablesConfigReader;
    std::unique_ptr<LoadablesConfigReader> config_files_reader;
182

183 184
    class LoadingDispatcher;
    std::unique_ptr<LoadingDispatcher> loading_dispatcher;
N
Nikolai Kochetov 已提交
185

186 187
    class PeriodicUpdater;
    std::unique_ptr<PeriodicUpdater> periodic_updater;
188

189
    const String type_name;
190 191
};

192 193 194
String toString(ExternalLoader::Status status);
std::ostream & operator<<(std::ostream & out, ExternalLoader::Status status);

195
}