提交 4c157007 编写于 作者: V Vitaly Baranov

Refactoring of using ExternalLoader in dictionary DDL:

Instead of using ExternalLoader::reload() now it's used reloadConfig() which reloads only what necessary.
Functions attachDictionary() and detachDictionary() are simplified and have lesser number of parameters.
Instead of injecting into LoadablesConfigReader's internals for creating dictionary a temp repository is used.
上级 575da235
......@@ -474,6 +474,7 @@ namespace ErrorCodes
extern const int NOT_ENOUGH_PRIVILEGES = 497;
extern const int LIMIT_BY_WITH_TIES_IS_NOT_SUPPORTED = 498;
extern const int S3_ERROR = 499;
extern const int CANNOT_CREATE_DICTIONARY_FROM_METADATA = 500;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;
......
......@@ -102,13 +102,12 @@ void DatabaseDictionary::removeDictionary(
}
void DatabaseDictionary::attachDictionary(
const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/)
const String & /*dictionary_name*/, const Context & /*context*/)
{
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
}
void DatabaseDictionary::detachDictionary(
const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/)
void DatabaseDictionary::detachDictionary(const String & /*dictionary_name*/, const Context & /*context*/)
{
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
}
......
......@@ -89,9 +89,9 @@ public:
ASTPtr tryGetCreateDictionaryQuery(const Context & context, const String & table_name) const override;
void attachDictionary(const String & dictionary_name, const Context & context, bool reload) override;
void attachDictionary(const String & dictionary_name, const Context & context) override;
void detachDictionary(const String & dictionary_name, const Context & context, bool reload) override;
void detachDictionary(const String & dictionary_name, const Context & context) override;
void shutdown() override;
......
......@@ -123,13 +123,12 @@ DatabaseDictionariesIteratorPtr DatabaseLazy::getDictionariesIterator(
void DatabaseLazy::attachDictionary(
const String & /*dictionary_name*/,
const Context & /*context*/,
bool /*load*/)
const Context & /*context*/)
{
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
}
void DatabaseLazy::detachDictionary(const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/)
void DatabaseLazy::detachDictionary(const String & /*dictionary_name*/, const Context & /*context*/)
{
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
}
......
......@@ -111,9 +111,9 @@ public:
StoragePtr detachTable(const String & table_name) override;
void attachDictionary(const String & dictionary_name, const Context & context, bool reload) override;
void attachDictionary(const String & dictionary_name, const Context & context) override;
void detachDictionary(const String & dictionary_name, const Context & context, bool reload) override;
void detachDictionary(const String & dictionary_name, const Context & context) override;
void shutdown() override;
......
......@@ -35,7 +35,7 @@ void DatabaseMemory::createTable(
}
void DatabaseMemory::attachDictionary(const String & /*name*/, const Context & /*context*/, bool /*reload*/)
void DatabaseMemory::attachDictionary(const String & /*name*/, const Context & /*context*/)
{
throw Exception("There is no ATTACH DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD);
}
......@@ -57,7 +57,7 @@ void DatabaseMemory::removeTable(
}
void DatabaseMemory::detachDictionary(const String & /*name*/, const Context & /*context*/, bool /*reload*/)
void DatabaseMemory::detachDictionary(const String & /*name*/, const Context & /*context*/)
{
throw Exception("There is no DETACH DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD);
}
......
......@@ -40,8 +40,7 @@ public:
void attachDictionary(
const String & name,
const Context & context,
bool reload) override;
const Context & context) override;
void removeTable(
const Context & context,
......@@ -53,8 +52,7 @@ public:
void detachDictionary(
const String & name,
const Context & context,
bool reload) override;
const Context & context) override;
time_t getObjectMetadataModificationTime(const Context & context, const String & table_name) override;
......
......@@ -64,7 +64,7 @@ public:
throw Exception("MySQL database engine does not support detach table.", ErrorCodes::NOT_IMPLEMENTED);
}
void detachDictionary(const String &, const Context &, bool) override
void detachDictionary(const String &, const Context &) override
{
throw Exception("MySQL database engine does not support detach dictionary.", ErrorCodes::NOT_IMPLEMENTED);
}
......@@ -90,7 +90,7 @@ public:
throw Exception("MySQL database engine does not support attach table.", ErrorCodes::NOT_IMPLEMENTED);
}
void attachDictionary(const String &, const Context &, bool) override
void attachDictionary(const String &, const Context &) override
{
throw Exception("MySQL database engine does not support attach dictionary.", ErrorCodes::NOT_IMPLEMENTED);
}
......
......@@ -7,6 +7,7 @@
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/ExternalLoaderPresetConfigRepository.h>
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ParserCreateQuery.h>
......@@ -18,6 +19,7 @@
#include <Common/escapeForFileName.h>
#include <common/logger_useful.h>
#include <ext/scope_guard.h>
#include <Poco/DirectoryIterator.h>
......@@ -267,9 +269,11 @@ void DatabaseOnDisk::createDictionary(
{
const auto & settings = context.getSettingsRef();
/** The code is based on the assumption that all threads share the same order of operations
* - creating the .sql.tmp file;
* - adding a dictionary to `dictionaries`;
/** The code is based on the assumption that all threads share the same order of operations:
* - create the .sql.tmp file;
* - add the dictionary to ExternalDictionariesLoader;
* - load the dictionary in case dictionaries_lazy_load == false;
* - attach the dictionary;
* - rename .sql.tmp to .sql.
*/
......@@ -278,17 +282,20 @@ void DatabaseOnDisk::createDictionary(
if (database.isDictionaryExist(context, dictionary_name))
throw Exception("Dictionary " + backQuote(database.getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS);
/// A dictionary with the same full name could be defined in *.xml config files.
String full_name = database.getDatabaseName() + "." + dictionary_name;
auto & external_loader = const_cast<ExternalDictionariesLoader &>(context.getExternalDictionariesLoader());
if (external_loader.getCurrentStatus(full_name) != ExternalLoader::Status::NOT_EXIST)
throw Exception("Dictionary " + backQuote(full_name) + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS);
if (database.isTableExist(context, dictionary_name))
throw Exception("Table " + backQuote(database.getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
String dictionary_metadata_path = database.getObjectMetadataPath(dictionary_name);
String dictionary_metadata_tmp_path = dictionary_metadata_path + ".tmp";
String statement;
String statement = getObjectDefinitionFromCreateQuery(query);
{
statement = getObjectDefinitionFromCreateQuery(query);
/// Exclusive flags guarantees, that table is not created right now in another thread. Otherwise, exception will be thrown.
WriteBufferFromFile out(dictionary_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
writeString(statement, out);
......@@ -298,27 +305,48 @@ void DatabaseOnDisk::createDictionary(
out.close();
}
try
bool succeeded = false;
SCOPE_EXIT({
if (!succeeded)
Poco::File(dictionary_metadata_tmp_path).remove();
});
/// Add a temporary repository containing the dictionary.
/// We need this temp repository to try loading the dictionary before actually attaching it to the database.
static std::atomic<size_t> counter = 0;
String temp_repository_name = String(IExternalLoaderConfigRepository::INTERNAL_REPOSITORY_NAME_PREFIX) + " creating " + full_name + " "
+ std::to_string(++counter);
external_loader.addConfigRepository(
temp_repository_name,
std::make_unique<ExternalLoaderPresetConfigRepository>(
std::vector{std::pair{dictionary_metadata_tmp_path,
getDictionaryConfigurationFromAST(query->as<const ASTCreateQuery &>(), database.getDatabaseName())}}));
SCOPE_EXIT({ external_loader.removeConfigRepository(temp_repository_name); });
bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true);
if (!lazy_load)
{
/// Do not load it now because we want more strict loading
database.attachDictionary(dictionary_name, context, false);
/// Load dictionary
bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true);
String dict_name = database.getDatabaseName() + "." + dictionary_name;
context.getExternalDictionariesLoader().addDictionaryWithConfig(
dict_name, database.getDatabaseName(), query->as<const ASTCreateQuery &>(), !lazy_load);
/// If it was ATTACH query and file with dictionary metadata already exist
/// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one.
Poco::File(dictionary_metadata_tmp_path).renameTo(dictionary_metadata_path);
}
catch (...)
{
database.detachDictionary(dictionary_name, context);
Poco::File(dictionary_metadata_tmp_path).remove();
throw;
/// loadStrict() is called here to force loading the dictionary, wait until the loading is finished,
/// and throw an exception if the loading is failed.
external_loader.loadStrict(full_name);
}
database.attachDictionary(dictionary_name, context);
SCOPE_EXIT({
if (!succeeded)
database.detachDictionary(dictionary_name, context);
});
/// If it was ATTACH query and file with dictionary metadata already exist
/// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one.
Poco::File(dictionary_metadata_tmp_path).renameTo(dictionary_metadata_path);
/// ExternalDictionariesLoader doesn't know we renamed the metadata path.
/// So we have to manually call reloadConfig() here.
external_loader.reloadConfig(database.getDatabaseName(), full_name);
/// Everything's ok.
succeeded = true;
}
......@@ -362,16 +390,18 @@ void DatabaseOnDisk::removeDictionary(
database.detachDictionary(dictionary_name, context);
String dictionary_metadata_path = database.getObjectMetadataPath(dictionary_name);
try
{
Poco::File(dictionary_metadata_path).remove();
}
catch (...)
if (Poco::File(dictionary_metadata_path).exists())
{
/// If remove was not possible for some reason
database.attachDictionary(dictionary_name, context);
throw;
try
{
Poco::File(dictionary_metadata_path).remove();
}
catch (...)
{
/// If remove was not possible for some reason
database.attachDictionary(dictionary_name, context);
throw;
}
}
}
......
......@@ -52,52 +52,68 @@ static constexpr size_t PRINT_MESSAGE_EACH_N_OBJECTS = 256;
static constexpr size_t PRINT_MESSAGE_EACH_N_SECONDS = 5;
static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
namespace
{
void loadObject(
Context & context,
const ASTCreateQuery & query,
DatabaseOrdinary & database,
const String database_data_path,
const String & database_name,
bool has_force_restore_data_flag)
try
namespace
{
if (query.is_dictionary)
void tryAttachTable(
Context & context,
const ASTCreateQuery & query,
DatabaseOrdinary & database,
const String database_data_path,
const String & database_name,
bool has_force_restore_data_flag)
{
String dictionary_name = query.table;
database.attachDictionary(dictionary_name, context, false);
assert(!query.is_dictionary);
try
{
String table_name;
StoragePtr table;
std::tie(table_name, table)
= createTableFromAST(query, database_name, database_data_path, context, has_force_restore_data_flag);
database.attachTable(table_name, table);
}
catch (const Exception & e)
{
throw Exception(
"Cannot attach table '" + query.table + "' from query " + serializeAST(query)
+ ". Error: " + DB::getCurrentExceptionMessage(true),
e,
DB::ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA);
}
}
else
void tryAttachDictionary(
Context & context,
const ASTCreateQuery & query,
DatabaseOrdinary & database)
{
String table_name;
StoragePtr table;
std::tie(table_name, table)
= createTableFromAST(query, database_name, database_data_path, context, has_force_restore_data_flag);
database.attachTable(table_name, table);
assert(query.is_dictionary);
try
{
database.attachDictionary(query.table, context);
}
catch (const Exception & e)
{
throw Exception(
"Cannot create dictionary '" + query.table + "' from query " + serializeAST(query)
+ ". Error: " + DB::getCurrentExceptionMessage(true),
e,
DB::ErrorCodes::CANNOT_CREATE_DICTIONARY_FROM_METADATA);
}
}
}
catch (const Exception & e)
{
throw Exception(
"Cannot create object '" + query.table + "' from query " + serializeAST(query) + ". Error: " + DB::getCurrentExceptionMessage(true),
e, DB::ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA);
}
void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, AtomicStopwatch & watch)
{
if (processed % PRINT_MESSAGE_EACH_N_OBJECTS == 0 || watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))
void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, AtomicStopwatch & watch)
{
LOG_INFO(log, std::fixed << std::setprecision(2) << processed * 100.0 / total << "%");
watch.restart();
if (processed % PRINT_MESSAGE_EACH_N_OBJECTS == 0 || watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))
{
LOG_INFO(log, std::fixed << std::setprecision(2) << processed * 100.0 / total << "%");
watch.restart();
}
}
}
}
DatabaseOrdinary::DatabaseOrdinary(String name_, const String & metadata_path_, const Context & context)
: DatabaseWithOwnTablesBase(std::move(name_))
......@@ -151,22 +167,20 @@ void DatabaseOrdinary::loadStoredObjects(
std::atomic<size_t> tables_processed{0};
std::atomic<size_t> dictionaries_processed{0};
auto loadOneObject = [&](const ASTCreateQuery & query)
{
loadObject(context, query, *this, getDataPath(), getDatabaseName(), has_force_restore_data_flag);
/// Messages, so that it's not boring to wait for the server to load for a long time.
if (query.is_dictionary)
logAboutProgress(log, ++dictionaries_processed, total_dictionaries, watch);
else
logAboutProgress(log, ++tables_processed, total_tables, watch);
};
ThreadPool pool(SettingMaxThreads().getAutoValue());
/// Attach tables.
for (const auto & name_with_query : file_names)
{
pool.scheduleOrThrowOnError([&]() { loadOneObject(name_with_query.second->as<const ASTCreateQuery &>()); });
const auto & create_query = name_with_query.second->as<const ASTCreateQuery &>();
if (!create_query.is_dictionary)
pool.scheduleOrThrowOnError([&]()
{
tryAttachTable(context, create_query, *this, getDataPath(), getDatabaseName(), has_force_restore_data_flag);
/// Messages, so that it's not boring to wait for the server to load for a long time.
logAboutProgress(log, ++tables_processed, total_tables, watch);
});
}
pool.wait();
......@@ -178,16 +192,19 @@ void DatabaseOrdinary::loadStoredObjects(
auto dictionaries_repository = std::make_unique<ExternalLoaderDatabaseConfigRepository>(shared_from_this(), context);
auto & external_loader = context.getExternalDictionariesLoader();
external_loader.addConfigRepository(getDatabaseName(), std::move(dictionaries_repository));
bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true);
auto filter = [this](const std::string & dictionary_name) -> bool
/// Attach dictionaries.
for (const auto & name_with_query : file_names)
{
if (!startsWith(dictionary_name, name + "." /* db name */))
return false;
LOG_INFO(log, "Loading dictionary " << backQuote(dictionary_name) << ", for database " << backQuote(name));
return true;
};
external_loader.reload(filter, !lazy_load);
auto create_query = name_with_query.second->as<const ASTCreateQuery &>();
if (create_query.is_dictionary)
{
tryAttachDictionary(context, create_query, *this);
/// Messages, so that it's not boring to wait for the server to load for a long time.
logAboutProgress(log, ++dictionaries_processed, total_dictionaries, watch);
}
}
}
......
......@@ -161,19 +161,21 @@ StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name)
return res;
}
void DatabaseWithOwnTablesBase::detachDictionary(const String & dictionary_name, const Context & context, bool reload)
void DatabaseWithOwnTablesBase::detachDictionary(const String & dictionary_name, const Context & context)
{
String full_name = getDatabaseName() + "." + dictionary_name;
{
std::lock_guard lock(mutex);
auto it = dictionaries.find(dictionary_name);
if (it == dictionaries.end())
throw Exception("Dictionary " + name + "." + dictionary_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
throw Exception("Dictionary " + full_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
dictionaries.erase(it);
}
if (reload)
context.getExternalDictionariesLoader().reload(getDatabaseName() + "." + dictionary_name);
/// ExternalLoader::reloadConfig() will find out that the dictionary's config has been removed
/// and therefore it will unload the dictionary.
const auto & external_loader = context.getExternalDictionariesLoader();
external_loader.reloadConfig(getDatabaseName(), full_name);
}
void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const StoragePtr & table)
......@@ -184,22 +186,19 @@ void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const Sto
}
void DatabaseWithOwnTablesBase::attachDictionary(const String & dictionary_name, const Context & context, bool load)
void DatabaseWithOwnTablesBase::attachDictionary(const String & dictionary_name, const Context & context)
{
const auto & external_loader = context.getExternalDictionariesLoader();
String full_name = getDatabaseName() + "." + dictionary_name;
{
std::lock_guard lock(mutex);
auto status = external_loader.getCurrentStatus(full_name);
if (status != ExternalLoader::Status::NOT_EXIST || !dictionaries.emplace(dictionary_name).second)
throw Exception(
"Dictionary " + full_name + " already exists.",
ErrorCodes::DICTIONARY_ALREADY_EXISTS);
if (!dictionaries.emplace(dictionary_name).second)
throw Exception("Dictionary " + full_name + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS);
}
if (load)
external_loader.reload(full_name, true);
/// ExternalLoader::reloadConfig() will find out that the dictionary's config has been added
/// and in case `dictionaries_lazy_load == false` it will load the dictionary.
const auto & external_loader = context.getExternalDictionariesLoader();
external_loader.reloadConfig(getDatabaseName(), full_name);
}
void DatabaseWithOwnTablesBase::shutdown()
......
......@@ -33,11 +33,11 @@ public:
void attachTable(const String & table_name, const StoragePtr & table) override;
void attachDictionary(const String & name, const Context & context, bool reload) override;
void attachDictionary(const String & name, const Context & context) override;
StoragePtr detachTable(const String & table_name) override;
void detachDictionary(const String & name, const Context & context, bool reload) override;
void detachDictionary(const String & name, const Context & context) override;
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
......
......@@ -165,14 +165,14 @@ public:
virtual void attachTable(const String & name, const StoragePtr & table) = 0;
/// Add dictionary to the database, but do not add it to the metadata. The database may not support this method.
/// load is false when we starting up and lazy_load is true, so we don't want to load dictionaries synchronously.
virtual void attachDictionary(const String & name, const Context & context, bool reload = true) = 0;
/// If dictionaries_lazy_load is false it also starts loading the dictionary asynchronously.
virtual void attachDictionary(const String & name, const Context & context) = 0;
/// Forget about the table without deleting it, and return it. The database may not support this method.
virtual StoragePtr detachTable(const String & name) = 0;
/// Forget about the dictionary without deleting it, and return it. The database may not support this method.
virtual void detachDictionary(const String & name, const Context & context, bool reload = true) = 0;
/// Forget about the dictionary without deleting it. The database may not support this method.
virtual void detachDictionary(const String & name, const Context & context) = 0;
/// Rename the table and possibly move the table to another database.
virtual void renameTable(
......
......@@ -414,7 +414,7 @@ void checkPrimaryKey(const std::unordered_set<std::string> & all_attrs, const Na
}
DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuery & query)
DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuery & query, const String & database_name)
{
checkAST(query);
......@@ -427,7 +427,8 @@ DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuer
AutoPtr<Poco::XML::Element> name_element(xml_document->createElement("name"));
current_dictionary->appendChild(name_element);
AutoPtr<Text> name(xml_document->createTextNode(query.database + "." + query.table));
String full_name = (!database_name.empty() ? database_name : query.database) + "." + query.table;
AutoPtr<Text> name(xml_document->createTextNode(full_name));
name_element->appendChild(name);
AutoPtr<Element> structure_element(xml_document->createElement("structure"));
......
......@@ -10,6 +10,6 @@ using DictionaryConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfigurati
/// Convert dictionary AST to Poco::AbstractConfiguration
/// This function is necessary because all loadable objects configuration are Poco::AbstractConfiguration
/// Can throw exception if query is ill-formed
DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuery & query);
DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuery & query, const String & database_name = {});
}
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/Context.h>
#include <Dictionaries/DictionaryFactory.h>
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
namespace DB
{
......@@ -28,15 +26,4 @@ void ExternalDictionariesLoader::addConfigRepository(
ExternalLoader::addConfigRepository(repository_name, std::move(config_repository), {"dictionary", "name"});
}
void ExternalDictionariesLoader::addDictionaryWithConfig(
const String & dictionary_name, const String & repo_name, const ASTCreateQuery & query, bool load_never_loading) const
{
ExternalLoader::addObjectAndLoad(
dictionary_name, /// names are equal
dictionary_name,
repo_name,
getDictionaryConfigurationFromAST(query),
"dictionary", load_never_loading);
}
}
#pragma once
#include <Dictionaries/IDictionary.h>
#include <Interpreters/IExternalLoaderConfigRepository.h>
#include <Interpreters/ExternalLoader.h>
#include <common/logger_useful.h>
#include <Parsers/ASTCreateQuery.h>
#include <memory>
namespace DB
{
class Context;
class IExternalLoaderConfigRepository;
/// Manages user-defined dictionaries.
class ExternalDictionariesLoader : public ExternalLoader
......@@ -36,14 +33,6 @@ public:
const std::string & repository_name,
std::unique_ptr<IExternalLoaderConfigRepository> config_repository);
/// Starts reloading of a specified object.
void addDictionaryWithConfig(
const String & dictionary_name,
const String & repo_name,
const ASTCreateQuery & query,
bool load_never_loading = false) const;
protected:
LoadablePtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config,
const std::string & key_in_config) const override;
......@@ -52,7 +41,6 @@ protected:
friend class DatabaseDictionary;
private:
Context & context;
};
......
......@@ -87,7 +87,7 @@ public:
const ExternalLoaderConfigSettings & config_settings);
/// Removes a repository which were used to read configurations.
void removeConfigRepository(const std::string & repository_name);
std::unique_ptr<IExternalLoaderConfigRepository> removeConfigRepository(const std::string & repository_name);
/// Sets whether all the objects from the configuration should be always loaded (even those which are never used).
void enableAlwaysLoadEverything(bool enable);
......@@ -128,15 +128,18 @@ public:
/// 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;
void load(const String & name) const { LoadablePtr object; load(name, object, Duration::zero()); }
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); }
/// 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;
void loadStrict(const String & name) const { LoadablePtr object; loadStrict(name, object); }
LoadablePtr getLoadable(const String & name) const { LoadablePtr object; loadStrict(name, object); return object; }
/// Tries to finish loading of the objects for which the specified function returns true.
void load(const FilterByNameFunction & filter_by_name) const { Loadables objects; load(filter_by_name, objects, Duration::zero()); }
void load(const FilterByNameFunction & filter_by_name, Loadables & loaded_objects, Duration timeout = NO_TIMEOUT) const;
void load(const FilterByNameFunction & filter_by_name, LoadResults & load_results, Duration timeout = NO_TIMEOUT) const;
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; }
......@@ -160,18 +163,18 @@ public:
/// 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;
/// 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;
protected:
virtual LoadablePtr create(const String & name, const Poco::Util::AbstractConfiguration & config, const String & key_in_config) const = 0;
/// Reload object with already parsed configuration
void addObjectAndLoad(
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)
bool load_never_loading = false) const;
private:
struct ObjectConfig;
......
#include <Interpreters/ExternalLoaderPresetConfigRepository.h>
#include <Common/Exception.h>
#include <boost/range/algorithm/copy.hpp>
#include <boost/range/adaptor/map.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
ExternalLoaderPresetConfigRepository::ExternalLoaderPresetConfigRepository(const std::vector<std::pair<String, LoadablesConfigurationPtr>> & preset_)
{
boost::range::copy(preset_, std::inserter(preset, preset.end()));
}
ExternalLoaderPresetConfigRepository::~ExternalLoaderPresetConfigRepository() = default;
std::set<String> ExternalLoaderPresetConfigRepository::getAllLoadablesDefinitionNames() const
{
std::set<String> paths;
boost::range::copy(preset | boost::adaptors::map_keys, std::inserter(paths, paths.end()));
return paths;
}
bool ExternalLoaderPresetConfigRepository::exists(const String& path) const
{
return preset.count(path);
}
Poco::Timestamp ExternalLoaderPresetConfigRepository::getUpdateTime(const String & path)
{
if (!exists(path))
throw Exception("Loadable " + path + " not found", ErrorCodes::BAD_ARGUMENTS);
return creation_time;
}
/// May contain definition about several entities (several dictionaries in one .xml file)
LoadablesConfigurationPtr ExternalLoaderPresetConfigRepository::load(const String & path) const
{
auto it = preset.find(path);
if (it == preset.end())
throw Exception("Loadable " + path + " not found", ErrorCodes::BAD_ARGUMENTS);
return it->second;
}
}
#pragma once
#include <Core/Types.h>
#include <unordered_map>
#include <Interpreters/IExternalLoaderConfigRepository.h>
#include <Poco/Timestamp.h>
namespace DB
{
/// A config repository filled with preset loadables used by ExternalLoader.
class ExternalLoaderPresetConfigRepository : public IExternalLoaderConfigRepository
{
public:
ExternalLoaderPresetConfigRepository(const std::vector<std::pair<String, LoadablesConfigurationPtr>> & preset_);
~ExternalLoaderPresetConfigRepository() override;
std::set<String> getAllLoadablesDefinitionNames() const override;
bool exists(const String & path) const override;
Poco::Timestamp getUpdateTime(const String & path) override;
LoadablesConfigurationPtr load(const String & path) const override;
private:
std::unordered_map<String, LoadablesConfigurationPtr> preset;
Poco::Timestamp creation_time;
};
}
#include <Interpreters/IExternalLoaderConfigRepository.h>
namespace DB
{
const char * IExternalLoaderConfigRepository::INTERNAL_REPOSITORY_NAME_PREFIX = "\xFF internal repo ";
}
......@@ -36,6 +36,8 @@ public:
virtual LoadablesConfigurationPtr load(const std::string & loadable_definition_name) const = 0;
virtual ~IExternalLoaderConfigRepository() = default;
static const char * INTERNAL_REPOSITORY_NAME_PREFIX;
};
using ExternalLoaderConfigRepositoryPtr = std::unique_ptr<IExternalLoaderConfigRepository>;
......
......@@ -50,14 +50,21 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Con
const auto & external_dictionaries = context.getExternalDictionariesLoader();
for (const auto & [dict_name, load_result] : external_dictionaries.getCurrentLoadResults())
{
if (startsWith(load_result.repository_name, IExternalLoaderConfigRepository::INTERNAL_REPOSITORY_NAME_PREFIX))
continue;
size_t i = 0;
String database;
String short_name = dict_name;
res_columns[i++]->insert(load_result.repository_name);
if (!load_result.repository_name.empty())
res_columns[i++]->insert(dict_name.substr(load_result.repository_name.length() + 1));
else
res_columns[i++]->insert(dict_name);
if (!load_result.repository_name.empty() && startsWith(dict_name, load_result.repository_name + "."))
{
database = load_result.repository_name;
short_name = dict_name.substr(load_result.repository_name.length() + 1);
}
res_columns[i++]->insert(database);
res_columns[i++]->insert(short_name);
res_columns[i++]->insert(static_cast<Int8>(load_result.status));
res_columns[i++]->insert(load_result.origin);
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册