ClickHouse/src/Storages/StorageDictionary.cpp

300 lines
11 KiB
C++
Raw Normal View History

2017-04-25 09:10:27 +00:00
#include <Storages/StorageDictionary.h>
#include <Storages/StorageFactory.h>
#include <DataTypes/DataTypesNumber.h>
#include <Dictionaries/DictionaryStructure.h>
2017-04-25 09:10:27 +00:00
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/ExternalDictionariesLoader.h>
2021-04-21 13:45:13 +00:00
#include <Interpreters/ExternalLoaderDictionaryStorageConfigRepository.h>
#include <Parsers/ASTLiteral.h>
#include <Common/quoteString.h>
2021-10-16 14:03:50 +00:00
#include <QueryPipeline/Pipe.h>
2020-11-10 18:22:26 +00:00
#include <IO/Operators.h>
2021-04-21 13:45:13 +00:00
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
2017-04-25 09:10:27 +00:00
namespace DB
{
namespace ErrorCodes
2017-06-22 15:44:19 +00:00
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int THERE_IS_NO_COLUMN;
extern const int CANNOT_DETACH_DICTIONARY_AS_TABLE;
2021-04-25 14:05:48 +00:00
extern const int DICTIONARY_ALREADY_EXISTS;
2017-06-22 15:44:19 +00:00
}
namespace
2017-04-25 09:10:27 +00:00
{
void checkNamesAndTypesCompatibleWithDictionary(const String & dictionary_name, const ColumnsDescription & columns, const DictionaryStructure & dictionary_structure)
{
auto dictionary_names_and_types = StorageDictionary::getNamesAndTypes(dictionary_structure);
std::set<NameAndTypePair> names_and_types_set(dictionary_names_and_types.begin(), dictionary_names_and_types.end());
2017-04-25 09:10:27 +00:00
for (const auto & column : columns.getOrdinary())
{
if (names_and_types_set.find(column) == names_and_types_set.end())
{
2020-11-10 18:22:26 +00:00
throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Not found column {} {} in dictionary {}. There are only columns {}",
column.name, column.type->getName(), backQuote(dictionary_name),
StorageDictionary::generateNamesAndTypesDescription(dictionary_names_and_types));
}
}
}
}
2017-04-25 09:10:27 +00:00
NamesAndTypesList StorageDictionary::getNamesAndTypes(const DictionaryStructure & dictionary_structure)
2017-04-25 09:10:27 +00:00
{
NamesAndTypesList dictionary_names_and_types;
2017-04-28 18:33:31 +00:00
if (dictionary_structure.id)
dictionary_names_and_types.emplace_back(dictionary_structure.id->name, std::make_shared<DataTypeUInt64>());
/// In old-style (XML) configuration we don't have this attributes in the
/// main attribute list, so we have to add them to columns list explicitly.
/// In the new configuration (DDL) we have them both in range_* nodes and
/// main attribute list, but for compatibility we add them before main
/// attributes list.
if (dictionary_structure.range_min)
dictionary_names_and_types.emplace_back(dictionary_structure.range_min->name, dictionary_structure.range_min->type);
if (dictionary_structure.range_max)
dictionary_names_and_types.emplace_back(dictionary_structure.range_max->name, dictionary_structure.range_max->type);
if (dictionary_structure.key)
{
for (const auto & attribute : *dictionary_structure.key)
dictionary_names_and_types.emplace_back(attribute.name, attribute.type);
}
2017-04-28 18:33:31 +00:00
for (const auto & attribute : dictionary_structure.attributes)
{
/// Some attributes can be already added (range_min and range_max)
if (!dictionary_names_and_types.contains(attribute.name))
dictionary_names_and_types.emplace_back(attribute.name, attribute.type);
}
2017-04-28 18:33:31 +00:00
return dictionary_names_and_types;
2017-04-28 18:33:31 +00:00
}
String StorageDictionary::generateNamesAndTypesDescription(const NamesAndTypesList & list)
{
2020-11-10 18:22:26 +00:00
WriteBufferFromOwnString ss;
bool first = true;
for (const auto & name_and_type : list)
{
if (!std::exchange(first, false))
ss << ", ";
ss << name_and_type.name << ' ' << name_and_type.type->getName();
2017-04-25 09:10:27 +00:00
}
return ss.str();
2017-04-25 09:10:27 +00:00
}
StorageDictionary::StorageDictionary(
const StorageID & table_id_,
const String & dictionary_name_,
const ColumnsDescription & columns_,
2021-04-23 12:18:23 +00:00
const String & comment,
2021-04-21 13:45:13 +00:00
Location location_,
ContextPtr context_)
2021-05-02 16:33:45 +00:00
: IStorage(table_id_), WithContext(context_->getGlobalContext()), dictionary_name(dictionary_name_), location(location_)
{
2020-06-19 15:39:41 +00:00
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_);
2021-04-23 12:18:23 +00:00
storage_metadata.setComment(comment);
2020-06-19 15:39:41 +00:00
setInMemoryMetadata(storage_metadata);
}
StorageDictionary::StorageDictionary(
2021-04-21 13:45:13 +00:00
const StorageID & table_id_,
const String & dictionary_name_,
const DictionaryStructure & dictionary_structure_,
Location location_,
ContextPtr context_)
: StorageDictionary(
2021-05-02 16:33:45 +00:00
table_id_, dictionary_name_, ColumnsDescription{getNamesAndTypes(dictionary_structure_)}, String{}, location_, context_)
{
}
2021-04-21 13:45:13 +00:00
StorageDictionary::StorageDictionary(
const StorageID & table_id,
LoadablesConfigurationPtr dictionary_configuration,
ContextPtr context_)
: StorageDictionary(
table_id,
2021-04-22 10:32:28 +00:00
table_id.getFullNameNotQuoted(),
2021-04-21 13:45:13 +00:00
context_->getExternalDictionariesLoader().getDictionaryStructure(*dictionary_configuration),
Location::SameDatabaseAndNameAsDictionary,
context_)
{
configuration = dictionary_configuration;
auto repository = std::make_unique<ExternalLoaderDictionaryStorageConfigRepository>(*this);
remove_repository_callback = context_->getExternalDictionariesLoader().addConfigRepository(std::move(repository));
}
2021-04-22 07:53:44 +00:00
StorageDictionary::~StorageDictionary()
{
2021-04-24 11:02:25 +00:00
removeDictionaryConfigurationFromRepository();
2021-04-22 07:53:44 +00:00
}
void StorageDictionary::checkTableCanBeDropped() const
{
if (location == Location::SameDatabaseAndNameAsDictionary)
2021-04-22 09:19:59 +00:00
throw Exception(ErrorCodes::CANNOT_DETACH_DICTIONARY_AS_TABLE,
"Cannot drop/detach dictionary {} as table, use DROP DICTIONARY or DETACH DICTIONARY query instead",
dictionary_name);
if (location == Location::DictionaryDatabase)
2021-04-22 09:19:59 +00:00
throw Exception(ErrorCodes::CANNOT_DETACH_DICTIONARY_AS_TABLE,
2021-04-26 10:20:57 +00:00
"Cannot drop/detach table from a database with DICTIONARY engine, use DROP DICTIONARY or DETACH DICTIONARY query instead",
2021-04-22 09:19:59 +00:00
dictionary_name);
}
void StorageDictionary::checkTableCanBeDetached() const
{
checkTableCanBeDropped();
}
2020-08-03 13:54:14 +00:00
Pipe StorageDictionary::read(
const Names & column_names,
const StorageMetadataPtr & /*metadata_snapshot*/,
SelectQueryInfo & /*query_info*/,
2021-04-21 13:45:13 +00:00
ContextPtr local_context,
QueryProcessingStage::Enum /*processed_stage*/,
const size_t max_block_size,
const unsigned threads)
{
2021-07-20 08:35:10 +00:00
auto registered_dictionary_name = location == Location::SameDatabaseAndNameAsDictionary ? getStorageID().getInternalDictionaryName() : dictionary_name;
auto dictionary = getContext()->getExternalDictionariesLoader().getDictionary(registered_dictionary_name, local_context);
return dictionary->read(column_names, max_block_size, threads);
}
2021-04-21 13:45:13 +00:00
void StorageDictionary::shutdown()
2021-04-22 07:53:44 +00:00
{
2021-04-24 11:02:25 +00:00
removeDictionaryConfigurationFromRepository();
}
2021-04-26 20:35:56 +00:00
void StorageDictionary::startup()
{
auto global_context = getContext();
2021-04-26 20:35:56 +00:00
bool lazy_load = global_context->getConfigRef().getBool("dictionaries_lazy_load", true);
2021-04-26 20:35:56 +00:00
if (!lazy_load)
{
2021-06-01 09:00:19 +00:00
const auto & external_dictionaries_loader = global_context->getExternalDictionariesLoader();
2021-04-26 20:35:56 +00:00
/// reloadConfig() is called here to force loading the dictionary.
external_dictionaries_loader.reloadConfig(getStorageID().getInternalDictionaryName());
}
}
2021-04-24 11:02:25 +00:00
void StorageDictionary::removeDictionaryConfigurationFromRepository()
{
remove_repository_callback.reset();
2021-04-22 07:53:44 +00:00
}
Poco::Timestamp StorageDictionary::getUpdateTime() const
2021-04-21 13:45:13 +00:00
{
std::lock_guard<std::mutex> lock(dictionary_config_mutex);
2021-04-22 07:53:44 +00:00
return update_time;
}
LoadablesConfigurationPtr StorageDictionary::getConfiguration() const
{
std::lock_guard<std::mutex> lock(dictionary_config_mutex);
return configuration;
2021-04-21 13:45:13 +00:00
}
void StorageDictionary::renameInMemory(const StorageID & new_table_id)
{
2021-06-18 20:59:35 +00:00
auto old_table_id = getStorageID();
IStorage::renameInMemory(new_table_id);
bool has_configuration = false;
2021-04-21 13:45:13 +00:00
{
std::lock_guard<std::mutex> lock(dictionary_config_mutex);
2021-04-21 13:45:13 +00:00
if (configuration)
{
has_configuration = true;
configuration->setString("dictionary.database", new_table_id.database_name);
configuration->setString("dictionary.name", new_table_id.table_name);
}
}
if (has_configuration)
{
2021-06-01 09:00:19 +00:00
const auto & external_dictionaries_loader = getContext()->getExternalDictionariesLoader();
2021-06-18 20:59:35 +00:00
auto result = external_dictionaries_loader.getLoadResult(old_table_id.getInternalDictionaryName());
2021-04-21 13:45:13 +00:00
2021-06-18 20:59:35 +00:00
if (result.object)
{
const auto dictionary = std::static_pointer_cast<const IDictionary>(result.object);
dictionary->updateDictionaryName(new_table_id);
}
2021-04-21 13:45:13 +00:00
2021-06-18 20:59:35 +00:00
external_dictionaries_loader.reloadConfig(old_table_id.getInternalDictionaryName());
dictionary_name = new_table_id.getFullNameNotQuoted();
2021-04-21 13:45:13 +00:00
}
}
void registerStorageDictionary(StorageFactory & factory)
{
factory.registerStorage("Dictionary", [](const StorageFactory::Arguments & args)
{
2021-04-21 13:45:13 +00:00
auto query = args.query;
2021-04-22 07:53:44 +00:00
auto local_context = args.getLocalContext();
2021-04-21 13:45:13 +00:00
if (query.is_dictionary)
{
2021-04-25 14:05:48 +00:00
auto dictionary_id = args.table_id;
auto & external_dictionaries_loader = local_context->getExternalDictionariesLoader();
/// A dictionary with the same full name could be defined in *.xml config files.
if (external_dictionaries_loader.getCurrentStatus(dictionary_id.getFullNameNotQuoted()) != ExternalLoader::Status::NOT_EXIST)
throw Exception(ErrorCodes::DICTIONARY_ALREADY_EXISTS,
"Dictionary {} already exists.", dictionary_id.getFullNameNotQuoted());
2021-04-21 13:45:13 +00:00
/// Create dictionary storage that owns underlying dictionary
2021-04-25 14:05:48 +00:00
auto abstract_dictionary_configuration = getDictionaryConfigurationFromAST(args.query, local_context, dictionary_id.database_name);
auto result_storage = StorageDictionary::create(dictionary_id, abstract_dictionary_configuration, local_context);
bool lazy_load = local_context->getConfigRef().getBool("dictionaries_lazy_load", true);
if (!args.attach && !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.
external_dictionaries_loader.load(dictionary_id.getInternalDictionaryName());
}
2021-04-25 14:05:48 +00:00
return result_storage;
}
2021-04-21 13:45:13 +00:00
else
{
/// Create dictionary storage that is view of underlying dictionary
2021-04-21 13:45:13 +00:00
if (args.engine_args.size() != 1)
throw Exception("Storage Dictionary requires single parameter: name of dictionary",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
2021-04-22 07:53:44 +00:00
args.engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(args.engine_args[0], local_context);
2021-04-21 13:45:13 +00:00
String dictionary_name = args.engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
if (!args.attach)
{
const auto & dictionary = args.getContext()->getExternalDictionariesLoader().getDictionary(dictionary_name, args.getContext());
const DictionaryStructure & dictionary_structure = dictionary->getStructure();
checkNamesAndTypesCompatibleWithDictionary(dictionary_name, args.columns, dictionary_structure);
}
2021-05-02 16:33:45 +00:00
return StorageDictionary::create(
args.table_id, dictionary_name, args.columns, args.comment, StorageDictionary::Location::Custom, local_context);
2021-04-21 13:45:13 +00:00
}
});
}
}