2019-11-05 20:26:14 +00:00
|
|
|
#include <Databases/DatabaseWithDictionaries.h>
|
2020-03-16 10:32:23 +00:00
|
|
|
#include <Common/StatusInfo.h>
|
2020-03-17 09:24:24 +00:00
|
|
|
#include <Common/ExternalLoaderStatus.h>
|
2019-11-05 20:26:14 +00:00
|
|
|
#include <Interpreters/ExternalDictionariesLoader.h>
|
2019-12-30 23:30:06 +00:00
|
|
|
#include <Interpreters/ExternalLoaderTempConfigRepository.h>
|
|
|
|
#include <Interpreters/ExternalLoaderDatabaseConfigRepository.h>
|
2019-11-06 16:05:04 +00:00
|
|
|
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
|
2020-04-12 20:50:32 +00:00
|
|
|
#include <Dictionaries/DictionaryStructure.h>
|
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
2019-11-05 20:26:14 +00:00
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Storages/StorageDictionary.h>
|
2019-11-06 16:05:04 +00:00
|
|
|
#include <IO/WriteBufferFromFile.h>
|
|
|
|
#include <Poco/File.h>
|
2020-04-29 22:38:22 +00:00
|
|
|
#include <boost/smart_ptr/make_shared_object.hpp>
|
2019-11-05 20:26:14 +00:00
|
|
|
|
|
|
|
|
2020-03-12 12:09:15 +00:00
|
|
|
namespace CurrentStatusInfo
|
|
|
|
{
|
2020-03-12 12:29:28 +00:00
|
|
|
extern const Status DictionaryStatus;
|
2020-03-12 12:09:15 +00:00
|
|
|
}
|
|
|
|
|
2019-11-05 20:26:14 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2020-02-25 18:02:41 +00:00
|
|
|
extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY;
|
2019-11-05 20:26:14 +00:00
|
|
|
extern const int TABLE_ALREADY_EXISTS;
|
2020-04-12 20:50:32 +00:00
|
|
|
extern const int UNKNOWN_DICTIONARY;
|
2019-11-05 20:26:14 +00:00
|
|
|
extern const int DICTIONARY_ALREADY_EXISTS;
|
2020-04-09 22:32:59 +00:00
|
|
|
extern const int FILE_DOESNT_EXIST;
|
2019-11-05 20:26:14 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-04-12 20:50:32 +00:00
|
|
|
void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name, const DictionaryAttachInfo & attach_info)
|
2019-11-05 20:26:14 +00:00
|
|
|
{
|
|
|
|
String full_name = getDatabaseName() + "." + dictionary_name;
|
|
|
|
{
|
2020-04-22 20:43:10 +00:00
|
|
|
std::unique_lock lock(mutex);
|
2020-04-12 20:50:32 +00:00
|
|
|
auto [it, inserted] = dictionaries.emplace(dictionary_name, attach_info);
|
|
|
|
if (!inserted)
|
2019-11-05 20:26:14 +00:00
|
|
|
throw Exception("Dictionary " + full_name + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS);
|
2020-04-12 20:50:32 +00:00
|
|
|
|
|
|
|
/// Attach the dictionary as table too.
|
|
|
|
try
|
|
|
|
{
|
|
|
|
attachTableUnlocked(
|
|
|
|
dictionary_name,
|
|
|
|
StorageDictionary::create(
|
2020-07-07 12:11:58 +00:00
|
|
|
StorageID(database_name, dictionary_name),
|
2020-04-12 20:50:32 +00:00
|
|
|
full_name,
|
2020-07-03 13:36:08 +00:00
|
|
|
ExternalDictionariesLoader::getDictionaryStructure(*attach_info.config),
|
2020-07-05 14:14:20 +00:00
|
|
|
StorageDictionary::Location::SameDatabaseAndNameAsDictionary),
|
2020-04-22 20:43:10 +00:00
|
|
|
lock);
|
2020-04-12 20:50:32 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
dictionaries.erase(it);
|
|
|
|
throw;
|
|
|
|
}
|
2019-11-05 20:26:14 +00:00
|
|
|
}
|
|
|
|
|
2020-03-17 09:24:24 +00:00
|
|
|
CurrentStatusInfo::set(CurrentStatusInfo::DictionaryStatus, full_name, static_cast<Int8>(ExternalLoaderStatus::NOT_LOADED));
|
2020-04-12 20:50:32 +00:00
|
|
|
|
2020-04-29 22:38:22 +00:00
|
|
|
/// We want ExternalLoader::reloadConfig() to find out that the dictionary's config
|
|
|
|
/// has been added and in case `dictionaries_lazy_load == false` to load the dictionary.
|
|
|
|
reloadDictionaryConfig(full_name);
|
2019-11-05 20:26:14 +00:00
|
|
|
}
|
|
|
|
|
2020-04-12 20:50:32 +00:00
|
|
|
void DatabaseWithDictionaries::detachDictionary(const String & dictionary_name)
|
|
|
|
{
|
|
|
|
DictionaryAttachInfo attach_info;
|
|
|
|
detachDictionaryImpl(dictionary_name, attach_info);
|
|
|
|
}
|
|
|
|
|
|
|
|
void DatabaseWithDictionaries::detachDictionaryImpl(const String & dictionary_name, DictionaryAttachInfo & attach_info)
|
2019-11-05 20:26:14 +00:00
|
|
|
{
|
2020-07-07 12:11:58 +00:00
|
|
|
String full_name = getDatabaseName() + "." + dictionary_name; //FIXME
|
2020-04-12 20:50:32 +00:00
|
|
|
|
2019-11-05 20:26:14 +00:00
|
|
|
{
|
2020-04-22 20:43:10 +00:00
|
|
|
std::unique_lock lock(mutex);
|
2019-11-05 20:26:14 +00:00
|
|
|
auto it = dictionaries.find(dictionary_name);
|
|
|
|
if (it == dictionaries.end())
|
2020-04-12 20:50:32 +00:00
|
|
|
throw Exception("Dictionary " + full_name + " doesn't exist.", ErrorCodes::UNKNOWN_DICTIONARY);
|
|
|
|
attach_info = std::move(it->second);
|
2019-11-05 20:26:14 +00:00
|
|
|
dictionaries.erase(it);
|
2020-04-12 20:50:32 +00:00
|
|
|
|
|
|
|
/// Detach the dictionary as table too.
|
|
|
|
try
|
|
|
|
{
|
2020-04-22 20:43:10 +00:00
|
|
|
detachTableUnlocked(dictionary_name, lock);
|
2020-04-12 20:50:32 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
dictionaries.emplace(dictionary_name, std::move(attach_info));
|
|
|
|
throw;
|
|
|
|
}
|
2019-11-05 20:26:14 +00:00
|
|
|
}
|
|
|
|
|
2020-03-12 12:09:15 +00:00
|
|
|
CurrentStatusInfo::unset(CurrentStatusInfo::DictionaryStatus, full_name);
|
2020-04-12 20:50:32 +00:00
|
|
|
|
2020-04-29 22:38:22 +00:00
|
|
|
/// We want ExternalLoader::reloadConfig() to find out that the dictionary's config
|
|
|
|
/// has been removed and to unload the dictionary.
|
|
|
|
reloadDictionaryConfig(full_name);
|
2019-11-05 20:26:14 +00:00
|
|
|
}
|
|
|
|
|
2019-11-06 16:05:04 +00:00
|
|
|
void DatabaseWithDictionaries::createDictionary(const Context & context, const String & dictionary_name, const ASTPtr & query)
|
|
|
|
{
|
|
|
|
const auto & settings = context.getSettingsRef();
|
|
|
|
|
|
|
|
/** 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.
|
|
|
|
*/
|
|
|
|
|
|
|
|
/// A race condition would be possible if a dictionary with the same name is simultaneously created using CREATE and using ATTACH.
|
|
|
|
/// But there is protection from it - see using DDLGuard in InterpreterCreateQuery.
|
2020-04-23 16:51:48 +00:00
|
|
|
if (isDictionaryExist(dictionary_name))
|
2019-11-06 16:05:04 +00:00
|
|
|
throw Exception("Dictionary " + backQuote(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 = getDatabaseName() + "." + dictionary_name;
|
2020-04-29 22:38:22 +00:00
|
|
|
if (external_loader.getCurrentStatus(full_name) != ExternalLoader::Status::NOT_EXIST)
|
2019-11-06 16:05:04 +00:00
|
|
|
throw Exception(
|
|
|
|
"Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.",
|
|
|
|
ErrorCodes::DICTIONARY_ALREADY_EXISTS);
|
|
|
|
|
2020-05-28 20:10:45 +00:00
|
|
|
if (isTableExist(dictionary_name, global_context))
|
2019-11-06 16:05:04 +00:00
|
|
|
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
|
|
|
|
|
|
|
|
|
|
|
String dictionary_metadata_path = getObjectMetadataPath(dictionary_name);
|
|
|
|
String dictionary_metadata_tmp_path = dictionary_metadata_path + ".tmp";
|
|
|
|
String 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);
|
|
|
|
out.next();
|
|
|
|
if (settings.fsync_metadata)
|
|
|
|
out.sync();
|
|
|
|
out.close();
|
|
|
|
}
|
|
|
|
|
|
|
|
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.
|
2020-04-29 22:38:22 +00:00
|
|
|
auto temp_repository = external_loader.addConfigRepository(std::make_unique<ExternalLoaderTempConfigRepository>(
|
2020-04-12 20:50:32 +00:00
|
|
|
getDatabaseName(), dictionary_metadata_tmp_path, getDictionaryConfigurationFromAST(query->as<const ASTCreateQuery &>())));
|
2019-11-06 16:05:04 +00:00
|
|
|
|
|
|
|
bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true);
|
|
|
|
if (!lazy_load)
|
|
|
|
{
|
|
|
|
/// load() is called here to force loading the dictionary, wait until the loading is finished,
|
|
|
|
/// and throw an exception if the loading is failed.
|
2020-04-29 22:38:22 +00:00
|
|
|
external_loader.load(full_name);
|
2019-11-06 16:05:04 +00:00
|
|
|
}
|
|
|
|
|
2020-04-12 20:50:32 +00:00
|
|
|
auto config = getDictionaryConfigurationFromAST(query->as<const ASTCreateQuery &>());
|
|
|
|
attachDictionary(dictionary_name, DictionaryAttachInfo{query, config, time(nullptr)});
|
2019-11-06 16:05:04 +00:00
|
|
|
SCOPE_EXIT({
|
|
|
|
if (!succeeded)
|
2020-04-12 20:50:32 +00:00
|
|
|
detachDictionary(dictionary_name);
|
2019-11-06 16:05:04 +00:00
|
|
|
});
|
|
|
|
|
|
|
|
/// 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.
|
2020-04-29 22:38:22 +00:00
|
|
|
/// That's why we have to call ExternalLoader::reloadConfig() here.
|
|
|
|
reloadDictionaryConfig(full_name);
|
2019-11-06 16:05:04 +00:00
|
|
|
|
|
|
|
/// Everything's ok.
|
|
|
|
succeeded = true;
|
|
|
|
}
|
|
|
|
|
2020-04-12 20:50:32 +00:00
|
|
|
void DatabaseWithDictionaries::removeDictionary(const Context &, const String & dictionary_name)
|
2019-11-06 16:05:04 +00:00
|
|
|
{
|
2020-04-12 20:50:32 +00:00
|
|
|
DictionaryAttachInfo attach_info;
|
|
|
|
detachDictionaryImpl(dictionary_name, attach_info);
|
2019-11-06 16:05:04 +00:00
|
|
|
|
|
|
|
try
|
|
|
|
{
|
2020-04-12 20:50:32 +00:00
|
|
|
String dictionary_metadata_path = getObjectMetadataPath(dictionary_name);
|
2019-11-06 16:05:04 +00:00
|
|
|
Poco::File(dictionary_metadata_path).remove();
|
2020-07-07 12:11:58 +00:00
|
|
|
CurrentStatusInfo::unset(CurrentStatusInfo::DictionaryStatus, getDatabaseName() + "." + dictionary_name); //FIXME
|
2019-11-06 16:05:04 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
/// If remove was not possible for some reason
|
2020-04-12 20:50:32 +00:00
|
|
|
attachDictionary(dictionary_name, attach_info);
|
2019-11-06 16:05:04 +00:00
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-04-20 11:09:09 +00:00
|
|
|
DatabaseDictionariesIteratorPtr DatabaseWithDictionaries::getDictionariesIterator(const FilterByNameFunction & filter_by_dictionary_name)
|
2019-11-05 20:26:14 +00:00
|
|
|
{
|
2020-04-12 20:50:32 +00:00
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
if (!filter_by_dictionary_name)
|
|
|
|
return std::make_unique<DatabaseDictionariesSnapshotIterator>(dictionaries);
|
2019-11-05 20:26:14 +00:00
|
|
|
|
2020-04-12 20:50:32 +00:00
|
|
|
Dictionaries filtered_dictionaries;
|
|
|
|
for (const auto & dictionary_name : dictionaries | boost::adaptors::map_keys)
|
|
|
|
if (filter_by_dictionary_name(dictionary_name))
|
|
|
|
filtered_dictionaries.emplace_back(dictionary_name);
|
|
|
|
return std::make_unique<DatabaseDictionariesSnapshotIterator>(std::move(filtered_dictionaries));
|
2019-11-05 20:26:14 +00:00
|
|
|
}
|
2020-04-12 20:50:32 +00:00
|
|
|
|
2020-04-23 16:51:48 +00:00
|
|
|
bool DatabaseWithDictionaries::isDictionaryExist(const String & dictionary_name) const
|
2020-04-09 22:32:59 +00:00
|
|
|
{
|
2020-04-12 20:50:32 +00:00
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
return dictionaries.find(dictionary_name) != dictionaries.end();
|
2020-04-09 22:32:59 +00:00
|
|
|
}
|
|
|
|
|
2020-04-12 20:50:32 +00:00
|
|
|
ASTPtr DatabaseWithDictionaries::getCreateDictionaryQueryImpl(
|
|
|
|
const String & dictionary_name,
|
|
|
|
bool throw_on_error) const
|
2020-04-09 22:32:59 +00:00
|
|
|
{
|
2020-04-12 20:50:32 +00:00
|
|
|
{
|
|
|
|
/// Try to get create query ifg for an attached dictionary.
|
|
|
|
std::lock_guard lock{mutex};
|
|
|
|
auto it = dictionaries.find(dictionary_name);
|
|
|
|
if (it != dictionaries.end())
|
|
|
|
{
|
|
|
|
ASTPtr ast = it->second.create_query->clone();
|
|
|
|
auto & create_query = ast->as<ASTCreateQuery &>();
|
|
|
|
create_query.attach = false;
|
2020-07-07 12:11:58 +00:00
|
|
|
create_query.database = database_name;
|
2020-04-12 20:50:32 +00:00
|
|
|
return ast;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Try to get create query for non-attached dictionary.
|
2020-04-09 22:32:59 +00:00
|
|
|
ASTPtr ast;
|
|
|
|
try
|
|
|
|
{
|
2020-04-12 20:50:32 +00:00
|
|
|
auto dictionary_metadata_path = getObjectMetadataPath(dictionary_name);
|
2020-04-23 16:51:48 +00:00
|
|
|
ast = getCreateQueryFromMetadata(dictionary_metadata_path, throw_on_error);
|
2020-04-09 22:32:59 +00:00
|
|
|
}
|
|
|
|
catch (const Exception & e)
|
|
|
|
{
|
2020-04-12 20:50:32 +00:00
|
|
|
if (throw_on_error && (e.code() != ErrorCodes::FILE_DOESNT_EXIST))
|
2020-04-09 22:32:59 +00:00
|
|
|
throw;
|
|
|
|
}
|
2019-11-05 20:26:14 +00:00
|
|
|
|
2020-04-12 20:50:32 +00:00
|
|
|
if (ast)
|
2019-11-05 20:26:14 +00:00
|
|
|
{
|
2020-04-12 20:50:32 +00:00
|
|
|
const auto * create_query = ast->as<const ASTCreateQuery>();
|
|
|
|
if (create_query && create_query->is_dictionary)
|
|
|
|
return ast;
|
2019-11-05 20:26:14 +00:00
|
|
|
}
|
2020-04-12 20:50:32 +00:00
|
|
|
if (throw_on_error)
|
|
|
|
throw Exception{"Dictionary " + backQuote(dictionary_name) + " doesn't exist",
|
|
|
|
ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY};
|
|
|
|
return nullptr;
|
2019-11-05 20:26:14 +00:00
|
|
|
}
|
|
|
|
|
2020-04-12 20:50:32 +00:00
|
|
|
Poco::AutoPtr<Poco::Util::AbstractConfiguration> DatabaseWithDictionaries::getDictionaryConfiguration(const String & dictionary_name) const
|
2019-11-05 20:26:14 +00:00
|
|
|
{
|
|
|
|
std::lock_guard lock(mutex);
|
2020-04-12 20:50:32 +00:00
|
|
|
auto it = dictionaries.find(dictionary_name);
|
|
|
|
if (it != dictionaries.end())
|
|
|
|
return it->second.config;
|
|
|
|
throw Exception("Dictionary " + backQuote(dictionary_name) + " doesn't exist", ErrorCodes::UNKNOWN_DICTIONARY);
|
2019-11-05 20:26:14 +00:00
|
|
|
}
|
|
|
|
|
2020-04-12 20:50:32 +00:00
|
|
|
time_t DatabaseWithDictionaries::getObjectMetadataModificationTime(const String & object_name) const
|
2019-11-05 20:26:14 +00:00
|
|
|
{
|
|
|
|
{
|
2020-04-12 20:50:32 +00:00
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
auto it = dictionaries.find(object_name);
|
|
|
|
if (it != dictionaries.end())
|
|
|
|
return it->second.modification_time;
|
2019-11-05 20:26:14 +00:00
|
|
|
}
|
2020-04-12 20:50:32 +00:00
|
|
|
return DatabaseOnDisk::getObjectMetadataModificationTime(object_name);
|
2019-11-05 20:26:14 +00:00
|
|
|
}
|
|
|
|
|
2019-11-06 16:05:04 +00:00
|
|
|
|
2020-04-23 16:51:48 +00:00
|
|
|
bool DatabaseWithDictionaries::empty() const
|
2020-04-12 20:50:32 +00:00
|
|
|
{
|
|
|
|
std::lock_guard lock{mutex};
|
|
|
|
return tables.empty() && dictionaries.empty();
|
2019-11-06 16:05:04 +00:00
|
|
|
}
|
|
|
|
|
2020-04-29 22:38:22 +00:00
|
|
|
void DatabaseWithDictionaries::reloadDictionaryConfig(const String & full_name)
|
|
|
|
{
|
|
|
|
/// Ensure that this database is attached to ExternalLoader as a config repository.
|
|
|
|
if (!database_as_config_repo_for_external_loader.load())
|
|
|
|
database_as_config_repo_for_external_loader = boost::make_shared<ext::scope_guard>(
|
|
|
|
external_loader.addConfigRepository(std::make_unique<ExternalLoaderDatabaseConfigRepository>(*this)));
|
|
|
|
|
|
|
|
external_loader.reloadConfig(getDatabaseName(), full_name);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-12-30 23:30:06 +00:00
|
|
|
void DatabaseWithDictionaries::shutdown()
|
|
|
|
{
|
2020-04-12 20:50:32 +00:00
|
|
|
{
|
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
dictionaries.clear();
|
|
|
|
}
|
2020-04-29 22:38:22 +00:00
|
|
|
|
|
|
|
/// Invoke removing the database from ExternalLoader.
|
|
|
|
database_as_config_repo_for_external_loader = nullptr;
|
|
|
|
|
2019-12-30 23:30:06 +00:00
|
|
|
DatabaseOnDisk::shutdown();
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-04-29 22:38:22 +00:00
|
|
|
DatabaseWithDictionaries::DatabaseWithDictionaries(
|
|
|
|
const String & name, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context)
|
|
|
|
: DatabaseOnDisk(name, metadata_path_, data_path_, logger, context)
|
|
|
|
, external_loader(context.getExternalDictionariesLoader())
|
2019-12-30 23:30:06 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2020-04-29 22:38:22 +00:00
|
|
|
DatabaseWithDictionaries::~DatabaseWithDictionaries() = default;
|
2019-12-30 23:30:06 +00:00
|
|
|
|
2019-11-05 20:26:14 +00:00
|
|
|
}
|