Merge pull request #27817 from vitlibar/use-sessions-more

Use sessions more
This commit is contained in:
Vitaly Baranov 2021-08-19 22:51:08 +03:00 committed by GitHub
commit c257cd61e5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
29 changed files with 131 additions and 141 deletions

View File

@ -12,6 +12,7 @@
#include <Interpreters/executeQuery.h> #include <Interpreters/executeQuery.h>
#include <Interpreters/loadMetadata.h> #include <Interpreters/loadMetadata.h>
#include <Interpreters/DatabaseCatalog.h> #include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/Session.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/Macros.h> #include <Common/Macros.h>
#include <Common/Config/ConfigProcessor.h> #include <Common/Config/ConfigProcessor.h>
@ -374,14 +375,13 @@ void LocalServer::processQueries()
if (!parse_res.second) if (!parse_res.second)
throw Exception("Cannot parse and execute the following part of query: " + String(parse_res.first), ErrorCodes::SYNTAX_ERROR); throw Exception("Cannot parse and execute the following part of query: " + String(parse_res.first), ErrorCodes::SYNTAX_ERROR);
/// we can't mutate global global_context (can lead to races, as it was already passed to some background threads) /// Authenticate and create a context to execute queries.
/// so we can't reuse it safely as a query context and need a copy here Session session{global_context, ClientInfo::Interface::TCP};
auto context = Context::createCopy(global_context); session.authenticate("default", "", Poco::Net::SocketAddress{});
context->makeSessionContext(); /// Use the same context for all queries.
context->makeQueryContext(); auto context = session.makeQueryContext();
context->makeSessionContext(); /// initial_create_query requires a session context to be set.
context->authenticate("default", "", Poco::Net::SocketAddress{});
context->setCurrentQueryId(""); context->setCurrentQueryId("");
applyCmdSettings(context); applyCmdSettings(context);

View File

@ -17,7 +17,7 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory)
[[maybe_unused]] const Poco::Util::AbstractConfiguration & config, [[maybe_unused]] const Poco::Util::AbstractConfiguration & config,
[[maybe_unused]] const std::string & config_prefix, [[maybe_unused]] const std::string & config_prefix,
[[maybe_unused]] Block & sample_block, [[maybe_unused]] Block & sample_block,
ContextPtr /* context */, ContextPtr /* global_context */,
const std::string & /* default_database */, const std::string & /* default_database */,
bool /*created_from_ddl*/) -> DictionarySourcePtr bool /*created_from_ddl*/) -> DictionarySourcePtr
{ {

View File

@ -7,6 +7,7 @@
#include <Interpreters/ExpressionActions.h> #include <Interpreters/ExpressionActions.h>
#include <Processors/Transforms/ExpressionTransform.h> #include <Processors/Transforms/ExpressionTransform.h>
#include <IO/ConnectionTimeouts.h> #include <IO/ConnectionTimeouts.h>
#include <Interpreters/Session.h>
#include <Interpreters/executeQuery.h> #include <Interpreters/executeQuery.h>
#include <Common/isLocalAddress.h> #include <Common/isLocalAddress.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
@ -63,19 +64,18 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
const DictionaryStructure & dict_struct_, const DictionaryStructure & dict_struct_,
const Configuration & configuration_, const Configuration & configuration_,
const Block & sample_block_, const Block & sample_block_,
ContextPtr context_) ContextMutablePtr context_,
std::shared_ptr<Session> local_session_)
: update_time{std::chrono::system_clock::from_time_t(0)} : update_time{std::chrono::system_clock::from_time_t(0)}
, dict_struct{dict_struct_} , dict_struct{dict_struct_}
, configuration{configuration_} , configuration{configuration_}
, query_builder{dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks} , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks}
, sample_block{sample_block_} , sample_block{sample_block_}
, context(Context::createCopy(context_)) , local_session(local_session_)
, context(context_)
, pool{createPool(configuration)} , pool{createPool(configuration)}
, load_all_query{query_builder.composeLoadAllQuery()} , load_all_query{query_builder.composeLoadAllQuery()}
{ {
/// Query context is needed because some code in executeQuery function may assume it exists.
/// Current example is Context::getSampleBlockCache from InterpreterSelectWithUnionQuery::getSampleBlock.
context->makeQueryContext();
} }
ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionarySource & other) ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionarySource & other)
@ -85,11 +85,11 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionar
, invalidate_query_response{other.invalidate_query_response} , invalidate_query_response{other.invalidate_query_response}
, query_builder{dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks} , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks}
, sample_block{other.sample_block} , sample_block{other.sample_block}
, local_session(other.local_session)
, context(Context::createCopy(other.context)) , context(Context::createCopy(other.context))
, pool{createPool(configuration)} , pool{createPool(configuration)}
, load_all_query{other.load_all_query} , load_all_query{other.load_all_query}
{ {
context->makeQueryContext();
} }
std::string ClickHouseDictionarySource::getUpdateFieldAndDate() std::string ClickHouseDictionarySource::getUpdateFieldAndDate()
@ -222,14 +222,13 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, Block & sample_block,
ContextPtr context, ContextPtr global_context,
const std::string & default_database [[maybe_unused]], const std::string & default_database [[maybe_unused]],
bool /* created_from_ddl */) -> DictionarySourcePtr bool /* created_from_ddl */) -> DictionarySourcePtr
{ {
bool secure = config.getBool(config_prefix + ".secure", false); bool secure = config.getBool(config_prefix + ".secure", false);
auto context_copy = Context::createCopy(context);
UInt16 default_port = getPortFromContext(context_copy, secure); UInt16 default_port = getPortFromContext(global_context, secure);
std::string settings_config_prefix = config_prefix + ".clickhouse"; std::string settings_config_prefix = config_prefix + ".clickhouse";
std::string host = config.getString(settings_config_prefix + ".host", "localhost"); std::string host = config.getString(settings_config_prefix + ".host", "localhost");
@ -252,12 +251,18 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
.secure = config.getBool(settings_config_prefix + ".secure", false) .secure = config.getBool(settings_config_prefix + ".secure", false)
}; };
/// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication). ContextMutablePtr context;
std::shared_ptr<Session> local_session;
if (configuration.is_local) if (configuration.is_local)
{ {
context_copy->authenticate(configuration.user, configuration.password, Poco::Net::SocketAddress("127.0.0.1", 0)); /// Start local session in case when the dictionary is loaded in-process (without TCP communication).
context_copy = copyContextAndApplySettings(config_prefix, context_copy, config); local_session = std::make_shared<Session>(global_context, ClientInfo::Interface::TCP);
local_session->authenticate(configuration.user, configuration.password, Poco::Net::SocketAddress{"127.0.0.1", 0});
context = local_session->makeQueryContext();
context->applySettingsChanges(readSettingsFromDictionaryConfig(config, config_prefix));
} }
else
context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix);
String dictionary_name = config.getString(".dictionary.name", ""); String dictionary_name = config.getString(".dictionary.name", "");
String dictionary_database = config.getString(".dictionary.database", ""); String dictionary_database = config.getString(".dictionary.database", "");
@ -265,7 +270,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
if (dictionary_name == configuration.table && dictionary_database == configuration.db) if (dictionary_name == configuration.table && dictionary_database == configuration.db)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ClickHouseDictionarySource table cannot be dictionary table"); throw Exception(ErrorCodes::BAD_ARGUMENTS, "ClickHouseDictionarySource table cannot be dictionary table");
return std::make_unique<ClickHouseDictionarySource>(dict_struct, configuration, sample_block, context_copy); return std::make_unique<ClickHouseDictionarySource>(dict_struct, configuration, sample_block, context, local_session);
}; };
factory.registerSource("clickhouse", create_table_source); factory.registerSource("clickhouse", create_table_source);

View File

@ -39,7 +39,8 @@ public:
const DictionaryStructure & dict_struct_, const DictionaryStructure & dict_struct_,
const Configuration & configuration_, const Configuration & configuration_,
const Block & sample_block_, const Block & sample_block_,
ContextPtr context); ContextMutablePtr context_,
std::shared_ptr<Session> local_session_);
/// copy-constructor is provided in order to support cloneability /// copy-constructor is provided in order to support cloneability
ClickHouseDictionarySource(const ClickHouseDictionarySource & other); ClickHouseDictionarySource(const ClickHouseDictionarySource & other);
@ -81,6 +82,7 @@ private:
mutable std::string invalidate_query_response; mutable std::string invalidate_query_response;
ExternalQueryBuilder query_builder; ExternalQueryBuilder query_builder;
Block sample_block; Block sample_block;
std::shared_ptr<Session> local_session;
ContextMutablePtr context; ContextMutablePtr context;
ConnectionPoolWithFailoverPtr pool; ConnectionPoolWithFailoverPtr pool;
const std::string load_all_query; const std::string load_all_query;

View File

@ -31,7 +31,7 @@ DictionaryPtr DictionaryFactory::create(
const std::string & name, const std::string & name,
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
ContextPtr context, ContextPtr global_context,
bool created_from_ddl) const bool created_from_ddl) const
{ {
Poco::Util::AbstractConfiguration::Keys keys; Poco::Util::AbstractConfiguration::Keys keys;
@ -45,12 +45,9 @@ DictionaryPtr DictionaryFactory::create(
const DictionaryStructure dict_struct{config, config_prefix}; const DictionaryStructure dict_struct{config, config_prefix};
DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create( DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create(
name, config, config_prefix + ".source", dict_struct, context, config.getString(config_prefix + ".database", ""), created_from_ddl); name, config, config_prefix + ".source", dict_struct, global_context, config.getString(config_prefix + ".database", ""), created_from_ddl);
LOG_TRACE(&Poco::Logger::get("DictionaryFactory"), "Created dictionary source '{}' for dictionary '{}'", source_ptr->toString(), name); LOG_TRACE(&Poco::Logger::get("DictionaryFactory"), "Created dictionary source '{}' for dictionary '{}'", source_ptr->toString(), name);
if (context->hasQueryContext() && context->getSettingsRef().log_queries)
context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, name);
const auto & layout_type = keys.front(); const auto & layout_type = keys.front();
{ {
@ -58,7 +55,7 @@ DictionaryPtr DictionaryFactory::create(
if (found != registered_layouts.end()) if (found != registered_layouts.end())
{ {
const auto & layout_creator = found->second.layout_create_function; const auto & layout_creator = found->second.layout_create_function;
return layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr), context, created_from_ddl); return layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl);
} }
} }
@ -68,10 +65,10 @@ DictionaryPtr DictionaryFactory::create(
layout_type); layout_type);
} }
DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, ContextPtr context) const DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, ContextPtr global_context) const
{ {
auto configuration = getDictionaryConfigurationFromAST(ast, context); auto configuration = getDictionaryConfigurationFromAST(ast, global_context);
return DictionaryFactory::create(name, *configuration, "dictionary", context, true); return DictionaryFactory::create(name, *configuration, "dictionary", global_context, true);
} }
bool DictionaryFactory::isComplex(const std::string & layout_type) const bool DictionaryFactory::isComplex(const std::string & layout_type) const

View File

@ -36,13 +36,13 @@ public:
const std::string & name, const std::string & name,
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
ContextPtr context, ContextPtr global_context,
bool created_from_ddl) const; bool created_from_ddl) const;
/// Create dictionary from DDL-query /// Create dictionary from DDL-query
DictionaryPtr create(const std::string & name, DictionaryPtr create(const std::string & name,
const ASTCreateQuery & ast, const ASTCreateQuery & ast,
ContextPtr context) const; ContextPtr global_context) const;
using LayoutCreateFunction = std::function<DictionaryPtr( using LayoutCreateFunction = std::function<DictionaryPtr(
const std::string & name, const std::string & name,
@ -50,7 +50,7 @@ public:
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
DictionarySourcePtr source_ptr, DictionarySourcePtr source_ptr,
ContextPtr context, ContextPtr global_context,
bool created_from_ddl)>; bool created_from_ddl)>;
bool isComplex(const std::string & layout_type) const; bool isComplex(const std::string & layout_type) const;

View File

@ -80,7 +80,7 @@ DictionarySourcePtr DictionarySourceFactory::create(
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
const DictionaryStructure & dict_struct, const DictionaryStructure & dict_struct,
ContextPtr context, ContextPtr global_context,
const std::string & default_database, const std::string & default_database,
bool check_config) const bool check_config) const
{ {
@ -99,7 +99,7 @@ DictionarySourcePtr DictionarySourceFactory::create(
{ {
const auto & create_source = found->second; const auto & create_source = found->second;
auto sample_block = createSampleBlock(dict_struct); auto sample_block = createSampleBlock(dict_struct);
return create_source(dict_struct, config, config_prefix, sample_block, context, default_database, check_config); return create_source(dict_struct, config, config_prefix, sample_block, global_context, default_database, check_config);
} }
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG,

View File

@ -35,7 +35,7 @@ public:
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, Block & sample_block,
ContextPtr context, ContextPtr global_context,
const std::string & default_database, const std::string & default_database,
bool check_config)>; bool check_config)>;
@ -48,7 +48,7 @@ public:
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
const DictionaryStructure & dict_struct, const DictionaryStructure & dict_struct,
ContextPtr context, ContextPtr global_context,
const std::string & default_database, const std::string & default_database,
bool check_config) const; bool check_config) const;

View File

@ -59,30 +59,36 @@ Block blockForKeys(
return block; return block;
} }
ContextMutablePtr copyContextAndApplySettings(
const std::string & config_prefix, SettingsChanges readSettingsFromDictionaryConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
ContextPtr context,
const Poco::Util::AbstractConfiguration & config)
{ {
auto local_context = Context::createCopy(context); if (!config.has(config_prefix + ".settings"))
if (config.has(config_prefix + ".settings")) return {};
const auto prefix = config_prefix + ".settings";
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(prefix, config_keys);
SettingsChanges changes;
for (const std::string & key : config_keys)
{ {
const auto prefix = config_prefix + ".settings"; const auto value = config.getString(prefix + "." + key);
changes.emplace_back(key, value);
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(prefix, config_keys);
SettingsChanges changes;
for (const std::string & key : config_keys)
{
const auto value = config.getString(prefix + "." + key);
changes.emplace_back(key, value);
}
local_context->applySettingsChanges(changes);
} }
return local_context;
return changes;
}
ContextMutablePtr copyContextAndApplySettingsFromDictionaryConfig(
const ContextPtr & context, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
auto context_copy = Context::createCopy(context);
auto changes = readSettingsFromDictionaryConfig(config, config_prefix);
context_copy->applySettingsChanges(changes);
return context_copy;
} }
static Block transformHeader(Block header, Block block_to_add) static Block transformHeader(Block header, Block block_to_add)

View File

@ -14,6 +14,7 @@ namespace DB
{ {
struct DictionaryStructure; struct DictionaryStructure;
class SettingsChanges;
/// For simple key /// For simple key
@ -29,10 +30,8 @@ Block blockForKeys(
const std::vector<size_t> & requested_rows); const std::vector<size_t> & requested_rows);
/// Used for applying settings to copied context in some register[...]Source functions /// Used for applying settings to copied context in some register[...]Source functions
ContextMutablePtr copyContextAndApplySettings( SettingsChanges readSettingsFromDictionaryConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
const std::string & config_prefix, ContextMutablePtr copyContextAndApplySettingsFromDictionaryConfig(const ContextPtr & context, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
ContextPtr context,
const Poco::Util::AbstractConfiguration & config);
/** A stream, adds additional columns to each block that it will read from inner stream. /** A stream, adds additional columns to each block that it will read from inner stream.
* *

View File

@ -307,7 +307,7 @@ namespace
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
DictionarySourcePtr source_ptr, DictionarySourcePtr source_ptr,
ContextPtr /* context */, ContextPtr /* global_context */,
bool /* created_from_ddl */) bool /* created_from_ddl */)
{ {
const auto * layout_name = dictionary_key_type == DictionaryKeyType::Simple ? "direct" : "complex_key_direct"; const auto * layout_name = dictionary_key_type == DictionaryKeyType::Simple ? "direct" : "complex_key_direct";

View File

@ -275,7 +275,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, Block & sample_block,
ContextPtr context, ContextPtr global_context,
const std::string & /* default_database */, const std::string & /* default_database */,
bool created_from_ddl) -> DictionarySourcePtr bool created_from_ddl) -> DictionarySourcePtr
{ {
@ -285,10 +285,10 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
/// Executable dictionaries may execute arbitrary commands. /// Executable dictionaries may execute arbitrary commands.
/// It's OK for dictionaries created by administrator from xml-file, but /// It's OK for dictionaries created by administrator from xml-file, but
/// maybe dangerous for dictionaries created from DDL-queries. /// maybe dangerous for dictionaries created from DDL-queries.
if (created_from_ddl && context->getApplicationType() != Context::ApplicationType::LOCAL) if (created_from_ddl && global_context->getApplicationType() != Context::ApplicationType::LOCAL)
throw Exception(ErrorCodes::DICTIONARY_ACCESS_DENIED, "Dictionaries with executable dictionary source are not allowed to be created from DDL query"); throw Exception(ErrorCodes::DICTIONARY_ACCESS_DENIED, "Dictionaries with executable dictionary source are not allowed to be created from DDL query");
auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config); auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix);
std::string settings_config_prefix = config_prefix + ".executable"; std::string settings_config_prefix = config_prefix + ".executable";
@ -301,7 +301,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
.implicit_key = config.getBool(settings_config_prefix + ".implicit_key", false) .implicit_key = config.getBool(settings_config_prefix + ".implicit_key", false)
}; };
return std::make_unique<ExecutableDictionarySource>(dict_struct, configuration, sample_block, context_local_copy); return std::make_unique<ExecutableDictionarySource>(dict_struct, configuration, sample_block, context);
}; };
factory.registerSource("executable", create_table_source); factory.registerSource("executable", create_table_source);

View File

@ -279,7 +279,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, Block & sample_block,
ContextPtr context, ContextPtr global_context,
const std::string & /* default_database */, const std::string & /* default_database */,
bool created_from_ddl) -> DictionarySourcePtr bool created_from_ddl) -> DictionarySourcePtr
{ {
@ -289,17 +289,15 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory)
/// Executable dictionaries may execute arbitrary commands. /// Executable dictionaries may execute arbitrary commands.
/// It's OK for dictionaries created by administrator from xml-file, but /// It's OK for dictionaries created by administrator from xml-file, but
/// maybe dangerous for dictionaries created from DDL-queries. /// maybe dangerous for dictionaries created from DDL-queries.
if (created_from_ddl && context->getApplicationType() != Context::ApplicationType::LOCAL) if (created_from_ddl && global_context->getApplicationType() != Context::ApplicationType::LOCAL)
throw Exception(ErrorCodes::DICTIONARY_ACCESS_DENIED, "Dictionaries with executable pool dictionary source are not allowed to be created from DDL query"); throw Exception(ErrorCodes::DICTIONARY_ACCESS_DENIED, "Dictionaries with executable pool dictionary source are not allowed to be created from DDL query");
auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config); ContextMutablePtr context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix);
/** Currently parallel parsing input format cannot read exactly max_block_size rows from input, /** Currently parallel parsing input format cannot read exactly max_block_size rows from input,
* so it will be blocked on ReadBufferFromFileDescriptor because this file descriptor represent pipe that does not have eof. * so it will be blocked on ReadBufferFromFileDescriptor because this file descriptor represent pipe that does not have eof.
*/ */
auto settings_no_parallel_parsing = context_local_copy->getSettings(); context->setSetting("input_format_parallel_parsing", Field{false});
settings_no_parallel_parsing.input_format_parallel_parsing = false;
context_local_copy->setSettings(settings_no_parallel_parsing);
String settings_config_prefix = config_prefix + ".executable_pool"; String settings_config_prefix = config_prefix + ".executable_pool";
@ -319,7 +317,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory)
.implicit_key = config.getBool(settings_config_prefix + ".implicit_key", false), .implicit_key = config.getBool(settings_config_prefix + ".implicit_key", false),
}; };
return std::make_unique<ExecutablePoolDictionarySource>(dict_struct, configuration, sample_block, context_local_copy); return std::make_unique<ExecutablePoolDictionarySource>(dict_struct, configuration, sample_block, context);
}; };
factory.registerSource("executable_pool", create_table_source); factory.registerSource("executable_pool", create_table_source);

View File

@ -77,7 +77,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, Block & sample_block,
ContextPtr context, ContextPtr global_context,
const std::string & /* default_database */, const std::string & /* default_database */,
bool created_from_ddl) -> DictionarySourcePtr bool created_from_ddl) -> DictionarySourcePtr
{ {
@ -87,9 +87,9 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory)
const auto filepath = config.getString(config_prefix + ".file.path"); const auto filepath = config.getString(config_prefix + ".file.path");
const auto format = config.getString(config_prefix + ".file.format"); const auto format = config.getString(config_prefix + ".file.format");
auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config); const auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix);
return std::make_unique<FileDictionarySource>(filepath, format, sample_block, context_local_copy, created_from_ddl); return std::make_unique<FileDictionarySource>(filepath, format, sample_block, context, created_from_ddl);
}; };
factory.registerSource("file", create_table_source); factory.registerSource("file", create_table_source);

View File

@ -557,7 +557,7 @@ void registerDictionaryFlat(DictionaryFactory & factory)
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
DictionarySourcePtr source_ptr, DictionarySourcePtr source_ptr,
ContextPtr /* context */, ContextPtr /* global_context */,
bool /* created_from_ddl */) -> DictionaryPtr bool /* created_from_ddl */) -> DictionaryPtr
{ {
if (dict_struct.key) if (dict_struct.key)

View File

@ -213,13 +213,13 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, Block & sample_block,
ContextPtr context, ContextPtr global_context,
const std::string & /* default_database */, const std::string & /* default_database */,
bool created_from_ddl) -> DictionarySourcePtr { bool created_from_ddl) -> DictionarySourcePtr {
if (dict_struct.has_expressions) if (dict_struct.has_expressions)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `http` does not support attribute expressions"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `http` does not support attribute expressions");
auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config); auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix);
const auto & settings_config_prefix = config_prefix + ".http"; const auto & settings_config_prefix = config_prefix + ".http";
const auto & credentials_prefix = settings_config_prefix + ".credentials"; const auto & credentials_prefix = settings_config_prefix + ".credentials";
@ -258,7 +258,7 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
.header_entries = std::move(header_entries) .header_entries = std::move(header_entries)
}; };
return std::make_unique<HTTPDictionarySource>(dict_struct, configuration, credentials, sample_block, context_local_copy, created_from_ddl); return std::make_unique<HTTPDictionarySource>(dict_struct, configuration, credentials, sample_block, context, created_from_ddl);
}; };
factory.registerSource("http", create_table_source); factory.registerSource("http", create_table_source);
} }

View File

@ -756,13 +756,13 @@ void registerDictionaryHashed(DictionaryFactory & factory)
using namespace std::placeholders; using namespace std::placeholders;
factory.registerLayout("hashed", factory.registerLayout("hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ false); }, false); [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ false); }, false);
factory.registerLayout("sparse_hashed", factory.registerLayout("sparse_hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ true); }, false); [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ true); }, false);
factory.registerLayout("complex_key_hashed", factory.registerLayout("complex_key_hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ false); }, true); [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ false); }, true);
factory.registerLayout("complex_key_sparse_hashed", factory.registerLayout("complex_key_sparse_hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ true); }, true); [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ true); }, true);
} }

View File

@ -954,7 +954,7 @@ void registerDictionaryTrie(DictionaryFactory & factory)
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
DictionarySourcePtr source_ptr, DictionarySourcePtr source_ptr,
ContextPtr /* context */, ContextPtr /* global_context */,
bool /*created_from_ddl*/) -> DictionaryPtr bool /*created_from_ddl*/) -> DictionaryPtr
{ {
if (!dict_struct.key || dict_struct.key->size() != 1) if (!dict_struct.key || dict_struct.key->size() != 1)

View File

@ -186,11 +186,11 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, Block & sample_block,
ContextPtr context, ContextPtr global_context,
const std::string & /* default_database */, const std::string & /* default_database */,
bool created_from_ddl) -> DictionarySourcePtr bool created_from_ddl) -> DictionarySourcePtr
{ {
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, context, created_from_ddl); return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, global_context, created_from_ddl);
}; };
factory.registerSource("library", create_table_source); factory.registerSource("library", create_table_source);

View File

@ -31,11 +31,11 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory)
[[maybe_unused]] const Poco::Util::AbstractConfiguration & config, [[maybe_unused]] const Poco::Util::AbstractConfiguration & config,
[[maybe_unused]] const std::string & config_prefix, [[maybe_unused]] const std::string & config_prefix,
[[maybe_unused]] Block & sample_block, [[maybe_unused]] Block & sample_block,
[[maybe_unused]] ContextPtr context, [[maybe_unused]] ContextPtr global_context,
const std::string & /* default_database */, const std::string & /* default_database */,
bool /* created_from_ddl */) -> DictionarySourcePtr { bool /* created_from_ddl */) -> DictionarySourcePtr {
#if USE_MYSQL #if USE_MYSQL
StreamSettings mysql_input_stream_settings(context->getSettingsRef() StreamSettings mysql_input_stream_settings(global_context->getSettingsRef()
, config.getBool(config_prefix + ".mysql.close_connection", false) || config.getBool(config_prefix + ".mysql.share_connection", false) , config.getBool(config_prefix + ".mysql.close_connection", false) || config.getBool(config_prefix + ".mysql.share_connection", false)
, false , false
, config.getBool(config_prefix + ".mysql.fail_on_connection_loss", false) ? 1 : default_num_tries_on_connection_loss); , config.getBool(config_prefix + ".mysql.fail_on_connection_loss", false) ? 1 : default_num_tries_on_connection_loss);

View File

@ -167,7 +167,7 @@ DictionaryPtr createLayout(const std::string & ,
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
DictionarySourcePtr source_ptr, DictionarySourcePtr source_ptr,
ContextPtr /* context */, ContextPtr /* global_context */,
bool /*created_from_ddl*/) bool /*created_from_ddl*/)
{ {
const String database = config.getString(config_prefix + ".database", ""); const String database = config.getString(config_prefix + ".database", "");

View File

@ -182,7 +182,7 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, Block & sample_block,
ContextPtr context, ContextPtr global_context,
const std::string & /* default_database */, const std::string & /* default_database */,
bool /* created_from_ddl */) -> DictionarySourcePtr bool /* created_from_ddl */) -> DictionarySourcePtr
{ {
@ -190,8 +190,8 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory)
const auto settings_config_prefix = config_prefix + ".postgresql"; const auto settings_config_prefix = config_prefix + ".postgresql";
auto pool = std::make_shared<postgres::PoolWithFailover>( auto pool = std::make_shared<postgres::PoolWithFailover>(
config, settings_config_prefix, config, settings_config_prefix,
context->getSettingsRef().postgresql_connection_pool_size, global_context->getSettingsRef().postgresql_connection_pool_size,
context->getSettingsRef().postgresql_connection_pool_wait_timeout); global_context->getSettingsRef().postgresql_connection_pool_wait_timeout);
PostgreSQLDictionarySource::Configuration configuration PostgreSQLDictionarySource::Configuration configuration
{ {
@ -211,7 +211,7 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory)
(void)config; (void)config;
(void)config_prefix; (void)config_prefix;
(void)sample_block; (void)sample_block;
(void)context; (void)global_context;
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
"Dictionary source of type `postgresql` is disabled because ClickHouse was built without postgresql support."); "Dictionary source of type `postgresql` is disabled because ClickHouse was built without postgresql support.");
#endif #endif

View File

@ -688,7 +688,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory)
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
DictionarySourcePtr source_ptr, DictionarySourcePtr source_ptr,
ContextPtr /* context */, ContextPtr /* global_context */,
bool /*created_from_ddl*/) -> DictionaryPtr bool /*created_from_ddl*/) -> DictionaryPtr
{ {
if (dict_struct.key) if (dict_struct.key)

View File

@ -12,7 +12,7 @@ void registerDictionarySourceRedis(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const String & config_prefix, const String & config_prefix,
Block & sample_block, Block & sample_block,
ContextPtr /* context */, ContextPtr /* global_context */,
const std::string & /* default_database */, const std::string & /* default_database */,
bool /* created_from_ddl */) -> DictionarySourcePtr { bool /* created_from_ddl */) -> DictionarySourcePtr {
return std::make_unique<RedisDictionarySource>(dict_struct, config, config_prefix + ".redis", sample_block); return std::make_unique<RedisDictionarySource>(dict_struct, config, config_prefix + ".redis", sample_block);

View File

@ -234,12 +234,12 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, Block & sample_block,
ContextPtr context, ContextPtr global_context,
const std::string & /* default_database */, const std::string & /* default_database */,
bool /* check_config */) -> DictionarySourcePtr { bool /* check_config */) -> DictionarySourcePtr {
#if USE_ODBC #if USE_ODBC
BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<ODBCBridgeMixin>>( BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<ODBCBridgeMixin>>(
context, context->getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string")); global_context, global_context->getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string"));
std::string settings_config_prefix = config_prefix + ".odbc"; std::string settings_config_prefix = config_prefix + ".odbc";
@ -255,13 +255,13 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory)
.update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1) .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1)
}; };
return std::make_unique<XDBCDictionarySource>(dict_struct, configuration, sample_block, context, bridge); return std::make_unique<XDBCDictionarySource>(dict_struct, configuration, sample_block, global_context, bridge);
#else #else
(void)dict_struct; (void)dict_struct;
(void)config; (void)config;
(void)config_prefix; (void)config_prefix;
(void)sample_block; (void)sample_block;
(void)context; (void)global_context;
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
"Dictionary source of type `odbc` is disabled because poco library was built without ODBC support."); "Dictionary source of type `odbc` is disabled because poco library was built without ODBC support.");
#endif #endif
@ -276,7 +276,7 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & /* config */, const Poco::Util::AbstractConfiguration & /* config */,
const std::string & /* config_prefix */, const std::string & /* config_prefix */,
Block & /* sample_block */, Block & /* sample_block */,
ContextPtr /* context */, ContextPtr /* global_context */,
const std::string & /* default_database */, const std::string & /* default_database */,
bool /* created_from_ddl */) -> DictionarySourcePtr { bool /* created_from_ddl */) -> DictionarySourcePtr {
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,

View File

@ -154,7 +154,7 @@ DictionaryPtr createCacheDictionaryLayout(
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
DictionarySourcePtr source_ptr, DictionarySourcePtr source_ptr,
ContextPtr context [[maybe_unused]], ContextPtr global_context [[maybe_unused]],
bool created_from_ddl [[maybe_unused]]) bool created_from_ddl [[maybe_unused]])
{ {
String layout_type; String layout_type;
@ -213,8 +213,8 @@ DictionaryPtr createCacheDictionaryLayout(
else else
{ {
auto storage_configuration = parseSSDCacheStorageConfiguration(config, full_name, layout_type, dictionary_layout_prefix, dict_lifetime); auto storage_configuration = parseSSDCacheStorageConfiguration(config, full_name, layout_type, dictionary_layout_prefix, dict_lifetime);
if (created_from_ddl && !pathStartsWith(storage_configuration.file_path, context->getUserFilesPath())) if (created_from_ddl && !pathStartsWith(storage_configuration.file_path, global_context->getUserFilesPath()))
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", storage_configuration.file_path, context->getUserFilesPath()); throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", storage_configuration.file_path, global_context->getUserFilesPath());
storage = std::make_shared<SSDCacheDictionaryStorage<dictionary_key_type>>(storage_configuration); storage = std::make_shared<SSDCacheDictionaryStorage<dictionary_key_type>>(storage_configuration);
} }
@ -239,10 +239,10 @@ void registerDictionaryCache(DictionaryFactory & factory)
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
DictionarySourcePtr source_ptr, DictionarySourcePtr source_ptr,
ContextPtr context, ContextPtr global_context,
bool created_from_ddl) -> DictionaryPtr bool created_from_ddl) -> DictionaryPtr
{ {
return createCacheDictionaryLayout<DictionaryKeyType::Simple, false/* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); return createCacheDictionaryLayout<DictionaryKeyType::Simple, false/* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl);
}; };
factory.registerLayout("cache", create_simple_cache_layout, false); factory.registerLayout("cache", create_simple_cache_layout, false);
@ -252,10 +252,10 @@ void registerDictionaryCache(DictionaryFactory & factory)
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
DictionarySourcePtr source_ptr, DictionarySourcePtr source_ptr,
ContextPtr context, ContextPtr global_context,
bool created_from_ddl) -> DictionaryPtr bool created_from_ddl) -> DictionaryPtr
{ {
return createCacheDictionaryLayout<DictionaryKeyType::Complex, false /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); return createCacheDictionaryLayout<DictionaryKeyType::Complex, false /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl);
}; };
factory.registerLayout("complex_key_cache", create_complex_key_cache_layout, true); factory.registerLayout("complex_key_cache", create_complex_key_cache_layout, true);
@ -267,10 +267,10 @@ void registerDictionaryCache(DictionaryFactory & factory)
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
DictionarySourcePtr source_ptr, DictionarySourcePtr source_ptr,
ContextPtr context, ContextPtr global_context,
bool created_from_ddl) -> DictionaryPtr bool created_from_ddl) -> DictionaryPtr
{ {
return createCacheDictionaryLayout<DictionaryKeyType::Simple, true /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); return createCacheDictionaryLayout<DictionaryKeyType::Simple, true /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl);
}; };
factory.registerLayout("ssd_cache", create_simple_ssd_cache_layout, false); factory.registerLayout("ssd_cache", create_simple_ssd_cache_layout, false);
@ -280,9 +280,9 @@ void registerDictionaryCache(DictionaryFactory & factory)
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
DictionarySourcePtr source_ptr, DictionarySourcePtr source_ptr,
ContextPtr context, ContextPtr global_context,
bool created_from_ddl) -> DictionaryPtr { bool created_from_ddl) -> DictionaryPtr {
return createCacheDictionaryLayout<DictionaryKeyType::Complex, true /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); return createCacheDictionaryLayout<DictionaryKeyType::Complex, true /* ssd */>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl);
}; };
factory.registerLayout("complex_key_ssd_cache", create_complex_key_ssd_cache_layout, true); factory.registerLayout("complex_key_ssd_cache", create_complex_key_ssd_cache_layout, true);

View File

@ -589,27 +589,6 @@ ConfigurationPtr Context::getUsersConfig()
} }
void Context::authenticate(const String & name, const String & password, const Poco::Net::SocketAddress & address)
{
authenticate(BasicCredentials(name, password), address);
}
void Context::authenticate(const Credentials & credentials, const Poco::Net::SocketAddress & address)
{
auto authenticated_user_id = getAccessControlManager().login(credentials, address.host());
client_info.current_user = credentials.getUserName();
client_info.current_address = address;
#if defined(ARCADIA_BUILD)
/// This is harmful field that is used only in foreign "Arcadia" build.
if (const auto * basic_credentials = dynamic_cast<const BasicCredentials *>(&credentials))
client_info.current_password = basic_credentials->getPassword();
#endif
setUser(authenticated_user_id);
}
void Context::setUser(const UUID & user_id_) void Context::setUser(const UUID & user_id_)
{ {
auto lock = getLock(); auto lock = getLock();

View File

@ -362,13 +362,9 @@ public:
void setUsersConfig(const ConfigurationPtr & config); void setUsersConfig(const ConfigurationPtr & config);
ConfigurationPtr getUsersConfig(); ConfigurationPtr getUsersConfig();
/// Sets the current user, checks the credentials and that the specified address is allowed to connect from.
/// The function throws an exception if there is no such user or password is wrong.
void authenticate(const String & user_name, const String & password, const Poco::Net::SocketAddress & address);
void authenticate(const Credentials & credentials, const Poco::Net::SocketAddress & address);
/// Sets the current user assuming that he/she is already authenticated. /// Sets the current user assuming that he/she is already authenticated.
/// WARNING: This function doesn't check password! Don't use until it's necessary! /// WARNING: This function doesn't check password!
/// Normally you shouldn't call this function. Use the Session class to do authentication instead.
void setUser(const UUID & user_id_); void setUser(const UUID & user_id_);
UserPtr getUser() const; UserPtr getUser() const;

View File

@ -45,12 +45,20 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create(
ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::getDictionary(const std::string & dictionary_name, ContextPtr local_context) const ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::getDictionary(const std::string & dictionary_name, ContextPtr local_context) const
{ {
std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, local_context->getCurrentDatabase()); std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, local_context->getCurrentDatabase());
if (local_context->hasQueryContext() && local_context->getSettingsRef().log_queries)
local_context->addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, resolved_dictionary_name);
return std::static_pointer_cast<const IDictionary>(load(resolved_dictionary_name)); return std::static_pointer_cast<const IDictionary>(load(resolved_dictionary_name));
} }
ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::tryGetDictionary(const std::string & dictionary_name, ContextPtr local_context) const ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::tryGetDictionary(const std::string & dictionary_name, ContextPtr local_context) const
{ {
std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, local_context->getCurrentDatabase()); std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, local_context->getCurrentDatabase());
if (local_context->hasQueryContext() && local_context->getSettingsRef().log_queries)
local_context->addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, resolved_dictionary_name);
return std::static_pointer_cast<const IDictionary>(tryLoad(resolved_dictionary_name)); return std::static_pointer_cast<const IDictionary>(tryLoad(resolved_dictionary_name));
} }