Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
alesapin 2019-02-05 14:07:53 +03:00
commit 5e8ad5d419
19 changed files with 101 additions and 104 deletions

View File

@ -153,6 +153,4 @@ private:
void attachToThreadGroup();
};
using BackgroundSchedulePoolPtr = std::shared_ptr<BackgroundSchedulePool>;
}

View File

@ -183,7 +183,8 @@ private:
try
{
setThreadName("ParalInputsProc");
CurrentThread::attachTo(thread_group);
if (thread_group)
CurrentThread::attachTo(thread_group);
while (!finish)
{

View File

@ -20,9 +20,8 @@ namespace ErrorCodes
extern const int SYNTAX_ERROR;
}
DatabaseDictionary::DatabaseDictionary(const String & name_, const Context & context)
DatabaseDictionary::DatabaseDictionary(const String & name_)
: name(name_),
external_dictionaries(context.getExternalDictionaries()),
log(&Logger::get("DatabaseDictionary(" + name + ")"))
{
}
@ -31,23 +30,21 @@ void DatabaseDictionary::loadTables(Context &, ThreadPool *, bool)
{
}
Tables DatabaseDictionary::loadTables()
Tables DatabaseDictionary::listTables(const Context & context)
{
auto objects_map = external_dictionaries.getObjectsMap();
auto objects_map = context.getExternalDictionaries().getObjectsMap();
const auto & dictionaries = objects_map.get();
Tables tables;
for (const auto & pair : dictionaries)
{
const std::string & dict_name = pair.first;
if (deleted_tables.count(dict_name))
continue;
auto dict_ptr = std::static_pointer_cast<IDictionaryBase>(pair.second.loadable);
if (dict_ptr)
{
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
tables[dict_name] = StorageDictionary::create(dict_name, ColumnsDescription{columns}, dictionary_structure, dict_name);
const std::string & dict_name = pair.first;
tables[dict_name] = StorageDictionary::create(dict_name, ColumnsDescription{columns}, context, true, dict_name);
}
}
@ -55,23 +52,21 @@ Tables DatabaseDictionary::loadTables()
}
bool DatabaseDictionary::isTableExist(
const Context & /*context*/,
const Context & context,
const String & table_name) const
{
auto objects_map = external_dictionaries.getObjectsMap();
auto objects_map = context.getExternalDictionaries().getObjectsMap();
const auto & dictionaries = objects_map.get();
return dictionaries.count(table_name) && !deleted_tables.count(table_name);
return dictionaries.count(table_name);
}
StoragePtr DatabaseDictionary::tryGetTable(
const Context & /*context*/,
const Context & context,
const String & table_name) const
{
auto objects_map = external_dictionaries.getObjectsMap();
auto objects_map = context.getExternalDictionaries().getObjectsMap();
const auto & dictionaries = objects_map.get();
if (deleted_tables.count(table_name))
return {};
{
auto it = dictionaries.find(table_name);
if (it != dictionaries.end())
@ -81,7 +76,7 @@ StoragePtr DatabaseDictionary::tryGetTable(
{
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
return StorageDictionary::create(table_name, ColumnsDescription{columns}, dictionary_structure, table_name);
return StorageDictionary::create(table_name, ColumnsDescription{columns}, context, true, table_name);
}
}
}
@ -89,17 +84,17 @@ StoragePtr DatabaseDictionary::tryGetTable(
return {};
}
DatabaseIteratorPtr DatabaseDictionary::getIterator(const Context & /*context*/)
DatabaseIteratorPtr DatabaseDictionary::getIterator(const Context & context)
{
return std::make_unique<DatabaseSnapshotIterator>(loadTables());
return std::make_unique<DatabaseSnapshotIterator>(listTables(context));
}
bool DatabaseDictionary::empty(const Context & /*context*/) const
bool DatabaseDictionary::empty(const Context & context) const
{
auto objects_map = external_dictionaries.getObjectsMap();
auto objects_map = context.getExternalDictionaries().getObjectsMap();
const auto & dictionaries = objects_map.get();
for (const auto & pair : dictionaries)
if (pair.second.loadable && !deleted_tables.count(pair.first))
if (pair.second.loadable)
return false;
return true;
}
@ -115,23 +110,19 @@ void DatabaseDictionary::attachTable(const String & /*table_name*/, const Storag
}
void DatabaseDictionary::createTable(
const Context & /*context*/,
const String & /*table_name*/,
const StoragePtr & /*table*/,
const ASTPtr & /*query*/)
const Context &,
const String &,
const StoragePtr &,
const ASTPtr &)
{
throw Exception("DatabaseDictionary: createTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
}
void DatabaseDictionary::removeTable(
const Context & context,
const String & table_name)
const Context &,
const String &)
{
if (!isTableExist(context, table_name))
throw Exception("Table " + name + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
auto objects_map = external_dictionaries.getObjectsMap();
deleted_tables.insert(table_name);
throw Exception("DatabaseDictionary: removeTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
}
void DatabaseDictionary::renameTable(

View File

@ -15,7 +15,6 @@ namespace Poco
namespace DB
{
class ExternalDictionaries;
/* Database to store StorageDictionary tables
* automatically creates tables for all dictionaries
@ -23,7 +22,7 @@ class ExternalDictionaries;
class DatabaseDictionary : public IDatabase
{
public:
DatabaseDictionary(const String & name_, const Context & context);
DatabaseDictionary(const String & name_);
String getDatabaseName() const override;
@ -93,13 +92,10 @@ public:
private:
const String name;
mutable std::mutex mutex;
const ExternalDictionaries & external_dictionaries;
std::unordered_set<String> deleted_tables;
Poco::Logger * log;
Tables loadTables();
Tables listTables(const Context & context);
ASTPtr getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const;
};

View File

@ -23,7 +23,7 @@ DatabasePtr DatabaseFactory::get(
else if (engine_name == "Memory")
return std::make_shared<DatabaseMemory>(database_name);
else if (engine_name == "Dictionary")
return std::make_shared<DatabaseDictionary>(database_name, context);
return std::make_shared<DatabaseDictionary>(database_name);
throw Exception("Unknown database engine: " + engine_name, ErrorCodes::UNKNOWN_DATABASE_ENGINE);
}

View File

@ -54,7 +54,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const Block & sample_block,
Context & context)
Context & context_)
: update_time{std::chrono::system_clock::from_time_t(0)}
, dict_struct{dict_struct_}
, host{config.getString(config_prefix + ".host")}
@ -69,11 +69,13 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
, invalidate_query{config.getString(config_prefix + ".invalidate_query", "")}
, query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks}
, sample_block{sample_block}
, context(context)
, context(context_)
, is_local{isLocalAddress({host, port}, context.getTCPPort())}
, pool{is_local ? nullptr : createPool(host, port, secure, db, user, password, context)}
, load_all_query{query_builder.composeLoadAllQuery()}
{
/// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication).
context.setUser(user, password, Poco::Net::SocketAddress("127.0.0.1", 0), {});
}
@ -182,7 +184,8 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re
{
if (is_local)
{
auto input_block = executeQuery(request, context, true).in;
Context query_context = context;
auto input_block = executeQuery(request, query_context, true).in;
return readInvalidateQuery(*input_block);
}
else
@ -201,7 +204,8 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
Context & context) -> DictionarySourcePtr {
Context & context) -> DictionarySourcePtr
{
return std::make_unique<ClickHouseDictionarySource>(dict_struct, config, config_prefix + ".clickhouse", sample_block, context);
};
factory.registerSource("clickhouse", createTableSource);

View File

@ -2,6 +2,7 @@
#include <memory>
#include <Client/ConnectionPoolWithFailover.h>
#include <Interpreters/Context.h>
#include "DictionaryStructure.h"
#include "ExternalQueryBuilder.h"
#include "IDictionarySource.h"
@ -65,7 +66,7 @@ private:
mutable std::string invalidate_query_response;
ExternalQueryBuilder query_builder;
Block sample_block;
Context & context;
Context context;
const bool is_local;
ConnectionPoolWithFailoverPtr pool;
const std::string load_all_query;

View File

@ -14,7 +14,6 @@ namespace ErrorCodes
void DictionaryFactory::registerLayout(const std::string & layout_type, Creator create_layout)
{
//LOG_DEBUG(log, "Register dictionary layout type `" + layout_type + "`");
if (!registered_layouts.emplace(layout_type, std::move(create_layout)).second)
throw Exception("DictionaryFactory: the layout name '" + layout_type + "' is not unique", ErrorCodes::LOGICAL_ERROR);
}

View File

@ -234,7 +234,8 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
const Context & context) -> DictionarySourcePtr {
Context & context) -> DictionarySourcePtr
{
if (dict_struct.has_expressions)
throw Exception{"Dictionary source of type `executable` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};

View File

@ -56,7 +56,8 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
const Context & context) -> DictionarySourcePtr {
Context & context) -> DictionarySourcePtr
{
if (dict_struct.has_expressions)
throw Exception{"Dictionary source of type `file` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};

View File

@ -157,7 +157,8 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
const Context & context) -> DictionarySourcePtr {
Context & context) -> DictionarySourcePtr
{
if (dict_struct.has_expressions)
throw Exception{"Dictionary source of type `http` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};

View File

@ -121,14 +121,12 @@ LibraryDictionarySource::LibraryDictionarySource(
const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
const Context & context)
Block & sample_block)
: log(&Logger::get("LibraryDictionarySource"))
, dict_struct{dict_struct_}
, config_prefix{config_prefix}
, path{config.getString(config_prefix + ".path", "")}
, sample_block{sample_block}
, context(context)
{
if (!Poco::File(path).exists())
throw Exception(
@ -152,7 +150,6 @@ LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource &
, config_prefix{other.config_prefix}
, path{other.path}
, sample_block{other.sample_block}
, context(other.context)
, library{other.library}
, description{other.description}
, settings{other.settings}
@ -288,8 +285,9 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
const Context & context) -> DictionarySourcePtr {
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, context);
const Context &) -> DictionarySourcePtr
{
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block);
};
factory.registerSource("library", createTableSource);
}

View File

@ -32,8 +32,7 @@ public:
const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
const Context & context);
Block & sample_block);
LibraryDictionarySource(const LibraryDictionarySource & other);
@ -70,7 +69,6 @@ private:
const std::string config_prefix;
const std::string path;
Block sample_block;
const Context & context;
SharedLibraryPtr library;
ExternalResultDescription description;
std::shared_ptr<CStringsHolder> settings;

View File

@ -186,7 +186,7 @@ public:
: owned_dict(owned_dict_)
{
if (!owned_dict)
throw Exception("Dictionaries was not loaded. You need to check configuration file.", ErrorCodes::DICTIONARIES_WAS_NOT_LOADED);
throw Exception("Embedded dictionaries were not loaded. You need to check configuration file.", ErrorCodes::DICTIONARIES_WAS_NOT_LOADED);
}
String getName() const override
@ -280,7 +280,7 @@ public:
: owned_dict(owned_dict_)
{
if (!owned_dict)
throw Exception("Dictionaries was not loaded. You need to check configuration file.", ErrorCodes::DICTIONARIES_WAS_NOT_LOADED);
throw Exception("Embedded dictionaries were not loaded. You need to check configuration file.", ErrorCodes::DICTIONARIES_WAS_NOT_LOADED);
}
String getName() const override
@ -418,7 +418,7 @@ public:
: owned_dict(owned_dict_)
{
if (!owned_dict)
throw Exception("Dictionaries was not loaded. You need to check configuration file.", ErrorCodes::DICTIONARIES_WAS_NOT_LOADED);
throw Exception("Embedded dictionaries were not loaded. You need to check configuration file.", ErrorCodes::DICTIONARIES_WAS_NOT_LOADED);
}
String getName() const override
@ -690,7 +690,7 @@ public:
: owned_dict(owned_dict_)
{
if (!owned_dict)
throw Exception("Dictionaries was not loaded. You need to check configuration file.", ErrorCodes::DICTIONARIES_WAS_NOT_LOADED);
throw Exception("Embedded dictionaries were not loaded. You need to check configuration file.", ErrorCodes::DICTIONARIES_WAS_NOT_LOADED);
}
String getName() const override

View File

@ -1,8 +1,8 @@
#include <map>
#include <set>
#include <boost/functional/hash/hash.hpp>
#include <optional>
#include <memory>
#include <Poco/Mutex.h>
#include <Poco/File.h>
#include <Poco/UUID.h>
#include <Poco/Net/IPAddress.h>
#include <common/logger_useful.h>
@ -98,7 +98,7 @@ struct ContextShared
{
Logger * log = &Logger::get("Context");
std::shared_ptr<IRuntimeComponentsFactory> runtime_components_factory;
std::unique_ptr<IRuntimeComponentsFactory> runtime_components_factory;
/// For access of most of shared objects. Recursive mutex.
mutable std::recursive_mutex mutex;
@ -124,12 +124,12 @@ struct ContextShared
ConfigurationPtr config; /// Global configuration settings.
Databases databases; /// List of databases and tables in them.
mutable std::shared_ptr<EmbeddedDictionaries> embedded_dictionaries; /// Metrica's dictionaries. Have lazy initialization.
mutable std::shared_ptr<ExternalDictionaries> external_dictionaries;
mutable std::shared_ptr<ExternalModels> external_models;
mutable std::optional<EmbeddedDictionaries> embedded_dictionaries; /// Metrica's dictionaries. Have lazy initialization.
mutable std::optional<ExternalDictionaries> external_dictionaries;
mutable std::optional<ExternalModels> external_models;
String default_profile_name; /// Default profile name used for default values.
String system_profile_name; /// Profile used by system processes
std::shared_ptr<ISecurityManager> security_manager; /// Known users.
std::unique_ptr<ISecurityManager> security_manager; /// Known users.
Quotas quotas; /// Known quotas for resource use.
mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks.
mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files.
@ -138,14 +138,14 @@ struct ContextShared
ViewDependencies view_dependencies; /// Current dependencies
ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections.
InterserverIOHandler interserver_io_handler; /// Handler for interserver communication.
BackgroundProcessingPoolPtr background_pool; /// The thread pool for the background work performed by the tables.
BackgroundSchedulePoolPtr schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables)
std::optional<BackgroundProcessingPool> background_pool; /// The thread pool for the background work performed by the tables.
std::optional<BackgroundSchedulePool> schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables)
MultiVersion<Macros> macros; /// Substitutions extracted from config.
std::unique_ptr<Compiler> compiler; /// Used for dynamic compilation of queries' parts if it necessary.
std::optional<Compiler> compiler; /// Used for dynamic compilation of queries' parts if it necessary.
std::shared_ptr<DDLWorker> ddl_worker; /// Process ddl commands from zk.
/// Rules for selecting the compression settings, depending on the size of the part.
mutable std::unique_ptr<CompressionCodecSelector> compression_codec_selector;
std::unique_ptr<MergeTreeSettings> merge_tree_settings; /// Settings of MergeTree* engines.
std::optional<MergeTreeSettings> merge_tree_settings; /// Settings of MergeTree* engines.
size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default)
size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default)
String format_schema_path; /// Path to a directory that contains schema files used by input formats.
@ -207,7 +207,7 @@ struct ContextShared
Context::ConfigReloadCallback config_reload_callback;
ContextShared(std::shared_ptr<IRuntimeComponentsFactory> runtime_components_factory_)
ContextShared(std::unique_ptr<IRuntimeComponentsFactory> runtime_components_factory_)
: runtime_components_factory(std::move(runtime_components_factory_)), macros(std::make_unique<Macros>())
{
/// TODO: make it singleton (?)
@ -266,6 +266,15 @@ struct ContextShared
std::lock_guard lock(mutex);
databases.clear();
}
/// Preemptive destruction is important, because these objects may have a refcount to ContextShared (cyclic reference).
/// TODO: Get rid of this.
embedded_dictionaries.reset();
external_dictionaries.reset();
external_models.reset();
background_pool.reset();
schedule_pool.reset();
}
private:
@ -279,11 +288,10 @@ private:
Context::Context() = default;
Context Context::createGlobal(std::shared_ptr<IRuntimeComponentsFactory> runtime_components_factory)
Context Context::createGlobal(std::unique_ptr<IRuntimeComponentsFactory> runtime_components_factory)
{
Context res;
res.runtime_components_factory = runtime_components_factory;
res.shared = std::make_shared<ContextShared>(runtime_components_factory);
res.shared = std::make_shared<ContextShared>(std::move(runtime_components_factory));
res.quota = std::make_shared<QuotaForIntervals>();
return res;
}
@ -1180,9 +1188,9 @@ EmbeddedDictionaries & Context::getEmbeddedDictionariesImpl(const bool throw_on_
if (!shared->embedded_dictionaries)
{
auto geo_dictionaries_loader = runtime_components_factory->createGeoDictionariesLoader();
auto geo_dictionaries_loader = shared->runtime_components_factory->createGeoDictionariesLoader();
shared->embedded_dictionaries = std::make_shared<EmbeddedDictionaries>(
shared->embedded_dictionaries.emplace(
std::move(geo_dictionaries_loader),
*this->global_context,
throw_on_error);
@ -1201,9 +1209,9 @@ ExternalDictionaries & Context::getExternalDictionariesImpl(const bool throw_on_
if (!this->global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
auto config_repository = runtime_components_factory->createExternalDictionariesConfigRepository();
auto config_repository = shared->runtime_components_factory->createExternalDictionariesConfigRepository();
shared->external_dictionaries = std::make_shared<ExternalDictionaries>(
shared->external_dictionaries.emplace(
std::move(config_repository),
*this->global_context,
throw_on_error);
@ -1221,9 +1229,9 @@ ExternalModels & Context::getExternalModelsImpl(bool throw_on_error) const
if (!this->global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
auto config_repository = runtime_components_factory->createExternalModelsConfigRepository();
auto config_repository = shared->runtime_components_factory->createExternalModelsConfigRepository();
shared->external_models = std::make_shared<ExternalModels>(
shared->external_models.emplace(
std::move(config_repository),
*this->global_context,
throw_on_error);
@ -1341,7 +1349,7 @@ BackgroundProcessingPool & Context::getBackgroundPool()
{
auto lock = getLock();
if (!shared->background_pool)
shared->background_pool = std::make_shared<BackgroundProcessingPool>(settings.background_pool_size);
shared->background_pool.emplace(settings.background_pool_size);
return *shared->background_pool;
}
@ -1349,7 +1357,7 @@ BackgroundSchedulePool & Context::getSchedulePool()
{
auto lock = getLock();
if (!shared->schedule_pool)
shared->schedule_pool = std::make_shared<BackgroundSchedulePool>(settings.background_schedule_pool_size);
shared->schedule_pool.emplace(settings.background_schedule_pool_size);
return *shared->schedule_pool;
}
@ -1528,7 +1536,7 @@ Compiler & Context::getCompiler()
auto lock = getLock();
if (!shared->compiler)
shared->compiler = std::make_unique<Compiler>(shared->path + "build/", 1);
shared->compiler.emplace(shared->path + "build/", 1);
return *shared->compiler;
}
@ -1611,7 +1619,7 @@ const MergeTreeSettings & Context::getMergeTreeSettings() const
if (!shared->merge_tree_settings)
{
auto & config = getConfigRef();
shared->merge_tree_settings = std::make_unique<MergeTreeSettings>();
shared->merge_tree_settings.emplace();
shared->merge_tree_settings->loadFromConfig("merge_tree", config);
}

View File

@ -113,8 +113,6 @@ private:
using Shared = std::shared_ptr<ContextShared>;
Shared shared;
std::shared_ptr<IRuntimeComponentsFactory> runtime_components_factory;
ClientInfo client_info;
ExternalTablesInitializer external_tables_initializer_callback;
@ -148,7 +146,7 @@ private:
public:
/// Create initial Context with ContextShared and etc.
static Context createGlobal(std::shared_ptr<IRuntimeComponentsFactory> runtime_components_factory);
static Context createGlobal(std::unique_ptr<IRuntimeComponentsFactory> runtime_components_factory);
static Context createGlobal();
Context(const Context &) = default;

View File

@ -80,8 +80,6 @@ protected:
void threadFunction();
};
using BackgroundProcessingPoolPtr = std::shared_ptr<BackgroundProcessingPool>;
class BackgroundProcessingPoolTaskInfo
{

View File

@ -26,13 +26,19 @@ namespace ErrorCodes
StorageDictionary::StorageDictionary(
const String & table_name_,
const ColumnsDescription & columns_,
const DictionaryStructure & dictionary_structure_,
const Context & context,
bool attach,
const String & dictionary_name_)
: IStorage{columns_}, table_name(table_name_),
dictionary_name(dictionary_name_),
logger(&Poco::Logger::get("StorageDictionary"))
{
checkNamesAndTypesCompatibleWithDictionary(dictionary_structure_);
if (!attach)
{
const auto & dictionary = context.getExternalDictionaries().getDictionary(dictionary_name);
const DictionaryStructure & dictionary_structure = dictionary->getStructure();
checkNamesAndTypesCompatibleWithDictionary(dictionary_structure);
}
}
BlockInputStreams StorageDictionary::read(
@ -70,11 +76,11 @@ NamesAndTypesList StorageDictionary::getNamesAndTypes(const DictionaryStructure
void StorageDictionary::checkNamesAndTypesCompatibleWithDictionary(const DictionaryStructure & dictionary_structure) const
{
auto dictionary_names_and_types = getNamesAndTypes(dictionary_structure);
std::set<NameAndTypePair> namesAndTypesSet(dictionary_names_and_types.begin(), dictionary_names_and_types.end());
std::set<NameAndTypePair> names_and_types_set(dictionary_names_and_types.begin(), dictionary_names_and_types.end());
for (auto & column : getColumns().ordinary)
for (const auto & column : getColumns().ordinary)
{
if (namesAndTypesSet.find(column) == namesAndTypesSet.end())
if (names_and_types_set.find(column) == names_and_types_set.end())
{
std::string message = "Not found column ";
message += column.name + " " + column.type->getName();
@ -97,11 +103,8 @@ void registerStorageDictionary(StorageFactory & factory)
args.engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(args.engine_args[0], args.local_context);
String dictionary_name = typeid_cast<const ASTLiteral &>(*args.engine_args[0]).value.safeGet<String>();
const auto & dictionary = args.context.getExternalDictionaries().getDictionary(dictionary_name);
const DictionaryStructure & dictionary_structure = dictionary->getStructure();
return StorageDictionary::create(
args.table_name, args.columns, dictionary_structure, dictionary_name);
args.table_name, args.columns, args.context, args.attach, dictionary_name);
});
}

View File

@ -66,7 +66,8 @@ private:
protected:
StorageDictionary(const String & table_name_,
const ColumnsDescription & columns_,
const DictionaryStructure & dictionary_structure_,
const Context & context,
bool attach,
const String & dictionary_name_);
};