diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 44e9880fabb..957bda4d75c 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -374,14 +375,13 @@ void LocalServer::processQueries() if (!parse_res.second) 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) - /// so we can't reuse it safely as a query context and need a copy here - auto context = Context::createCopy(global_context); + /// Authenticate and create a context to execute queries. + Session session{global_context, ClientInfo::Interface::TCP}; + session.authenticate("default", "", Poco::Net::SocketAddress{}); - context->makeSessionContext(); - context->makeQueryContext(); - - context->authenticate("default", "", Poco::Net::SocketAddress{}); + /// Use the same context for all queries. + auto context = session.makeQueryContext(); + context->makeSessionContext(); /// initial_create_query requires a session context to be set. context->setCurrentQueryId(""); applyCmdSettings(context); diff --git a/src/Dictionaries/CassandraDictionarySource.cpp b/src/Dictionaries/CassandraDictionarySource.cpp index aa8d6107508..d9a4dd0fd22 100644 --- a/src/Dictionaries/CassandraDictionarySource.cpp +++ b/src/Dictionaries/CassandraDictionarySource.cpp @@ -17,7 +17,7 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory) [[maybe_unused]] const Poco::Util::AbstractConfiguration & config, [[maybe_unused]] const std::string & config_prefix, [[maybe_unused]] Block & sample_block, - ContextPtr /* context */, + ContextPtr /* global_context */, const std::string & /* default_database */, bool /*created_from_ddl*/) -> DictionarySourcePtr { diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index d4f01dee8b2..b09a7b14cc4 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -63,19 +64,18 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( const DictionaryStructure & dict_struct_, const Configuration & configuration_, const Block & sample_block_, - ContextPtr context_) + ContextMutablePtr context_, + std::shared_ptr local_session_) : update_time{std::chrono::system_clock::from_time_t(0)} , dict_struct{dict_struct_} , configuration{configuration_} , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks} , sample_block{sample_block_} - , context(Context::createCopy(context_)) + , local_session(local_session_) + , context(context_) , pool{createPool(configuration)} , 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) @@ -85,11 +85,11 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionar , invalidate_query_response{other.invalidate_query_response} , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks} , sample_block{other.sample_block} + , local_session(other.local_session) , context(Context::createCopy(other.context)) , pool{createPool(configuration)} , load_all_query{other.load_all_query} { - context->makeQueryContext(); } std::string ClickHouseDictionarySource::getUpdateFieldAndDate() @@ -222,14 +222,13 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & default_database [[maybe_unused]], bool /* created_from_ddl */) -> DictionarySourcePtr { 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 host = config.getString(settings_config_prefix + ".host", "localhost"); @@ -252,12 +251,18 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .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 local_session; if (configuration.is_local) { - context_copy->authenticate(configuration.user, configuration.password, Poco::Net::SocketAddress("127.0.0.1", 0)); - context_copy = copyContextAndApplySettings(config_prefix, context_copy, config); + /// Start local session in case when the dictionary is loaded in-process (without TCP communication). + local_session = std::make_shared(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_database = config.getString(".dictionary.database", ""); @@ -265,7 +270,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) if (dictionary_name == configuration.table && dictionary_database == configuration.db) throw Exception(ErrorCodes::BAD_ARGUMENTS, "ClickHouseDictionarySource table cannot be dictionary table"); - return std::make_unique(dict_struct, configuration, sample_block, context_copy); + return std::make_unique(dict_struct, configuration, sample_block, context, local_session); }; factory.registerSource("clickhouse", create_table_source); diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index 2daa296af3e..58243e43b15 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -39,7 +39,8 @@ public: const DictionaryStructure & dict_struct_, const Configuration & configuration_, const Block & sample_block_, - ContextPtr context); + ContextMutablePtr context_, + std::shared_ptr local_session_); /// copy-constructor is provided in order to support cloneability ClickHouseDictionarySource(const ClickHouseDictionarySource & other); @@ -81,6 +82,7 @@ private: mutable std::string invalidate_query_response; ExternalQueryBuilder query_builder; Block sample_block; + std::shared_ptr local_session; ContextMutablePtr context; ConnectionPoolWithFailoverPtr pool; const std::string load_all_query; diff --git a/src/Dictionaries/DictionaryFactory.cpp b/src/Dictionaries/DictionaryFactory.cpp index 62b28ed7d14..4cab42c9445 100644 --- a/src/Dictionaries/DictionaryFactory.cpp +++ b/src/Dictionaries/DictionaryFactory.cpp @@ -31,7 +31,7 @@ DictionaryPtr DictionaryFactory::create( const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - ContextPtr context, + ContextPtr global_context, bool created_from_ddl) const { Poco::Util::AbstractConfiguration::Keys keys; @@ -45,12 +45,9 @@ DictionaryPtr DictionaryFactory::create( const DictionaryStructure dict_struct{config, config_prefix}; 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); - if (context->hasQueryContext() && context->getSettingsRef().log_queries) - context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, name); - const auto & layout_type = keys.front(); { @@ -58,7 +55,7 @@ DictionaryPtr DictionaryFactory::create( if (found != registered_layouts.end()) { 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); } -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); - return DictionaryFactory::create(name, *configuration, "dictionary", context, true); + auto configuration = getDictionaryConfigurationFromAST(ast, global_context); + return DictionaryFactory::create(name, *configuration, "dictionary", global_context, true); } bool DictionaryFactory::isComplex(const std::string & layout_type) const diff --git a/src/Dictionaries/DictionaryFactory.h b/src/Dictionaries/DictionaryFactory.h index b717009aa8a..b1dad340f4b 100644 --- a/src/Dictionaries/DictionaryFactory.h +++ b/src/Dictionaries/DictionaryFactory.h @@ -36,13 +36,13 @@ public: const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - ContextPtr context, + ContextPtr global_context, bool created_from_ddl) const; /// Create dictionary from DDL-query DictionaryPtr create(const std::string & name, const ASTCreateQuery & ast, - ContextPtr context) const; + ContextPtr global_context) const; using LayoutCreateFunction = std::function; bool isComplex(const std::string & layout_type) const; diff --git a/src/Dictionaries/DictionarySourceFactory.cpp b/src/Dictionaries/DictionarySourceFactory.cpp index 1992c87d31f..80b60f29e37 100644 --- a/src/Dictionaries/DictionarySourceFactory.cpp +++ b/src/Dictionaries/DictionarySourceFactory.cpp @@ -80,7 +80,7 @@ DictionarySourcePtr DictionarySourceFactory::create( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const DictionaryStructure & dict_struct, - ContextPtr context, + ContextPtr global_context, const std::string & default_database, bool check_config) const { @@ -99,7 +99,7 @@ DictionarySourcePtr DictionarySourceFactory::create( { const auto & create_source = found->second; 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, diff --git a/src/Dictionaries/DictionarySourceFactory.h b/src/Dictionaries/DictionarySourceFactory.h index bb583927ac4..f4c3fa12163 100644 --- a/src/Dictionaries/DictionarySourceFactory.h +++ b/src/Dictionaries/DictionarySourceFactory.h @@ -35,7 +35,7 @@ public: const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & default_database, bool check_config)>; @@ -48,7 +48,7 @@ public: const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const DictionaryStructure & dict_struct, - ContextPtr context, + ContextPtr global_context, const std::string & default_database, bool check_config) const; diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index 092e7187e8f..cf003dceb8e 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -59,30 +59,36 @@ Block blockForKeys( return block; } -ContextMutablePtr copyContextAndApplySettings( - const std::string & config_prefix, - ContextPtr context, - const Poco::Util::AbstractConfiguration & config) + +SettingsChanges readSettingsFromDictionaryConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) { - 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"; - - 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); + const auto value = config.getString(prefix + "." + key); + changes.emplace_back(key, value); } - 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) diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index b955b6ffb66..5470321745a 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -14,6 +14,7 @@ namespace DB { struct DictionaryStructure; +class SettingsChanges; /// For simple key @@ -29,10 +30,8 @@ Block blockForKeys( const std::vector & requested_rows); /// Used for applying settings to copied context in some register[...]Source functions -ContextMutablePtr copyContextAndApplySettings( - const std::string & config_prefix, - ContextPtr context, - const Poco::Util::AbstractConfiguration & config); +SettingsChanges readSettingsFromDictionaryConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); +ContextMutablePtr copyContextAndApplySettingsFromDictionaryConfig(const ContextPtr & context, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); /** A stream, adds additional columns to each block that it will read from inner stream. * diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index e12100a556d..10e7414b42f 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -307,7 +307,7 @@ namespace const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr /* context */, + ContextPtr /* global_context */, bool /* created_from_ddl */) { const auto * layout_name = dictionary_key_type == DictionaryKeyType::Simple ? "direct" : "complex_key_direct"; diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 8802d04ff30..5c6add34f1f 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -275,7 +275,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { @@ -285,10 +285,10 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) /// Executable dictionaries may execute arbitrary commands. /// It's OK for dictionaries created by administrator from xml-file, but /// 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"); - 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"; @@ -301,7 +301,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) .implicit_key = config.getBool(settings_config_prefix + ".implicit_key", false) }; - return std::make_unique(dict_struct, configuration, sample_block, context_local_copy); + return std::make_unique(dict_struct, configuration, sample_block, context); }; factory.registerSource("executable", create_table_source); diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index e97b4253407..e18664e477c 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -279,7 +279,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { @@ -289,17 +289,15 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) /// Executable dictionaries may execute arbitrary commands. /// It's OK for dictionaries created by administrator from xml-file, but /// 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"); - 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, * 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(); - settings_no_parallel_parsing.input_format_parallel_parsing = false; - context_local_copy->setSettings(settings_no_parallel_parsing); + context->setSetting("input_format_parallel_parsing", Field{false}); 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), }; - return std::make_unique(dict_struct, configuration, sample_block, context_local_copy); + return std::make_unique(dict_struct, configuration, sample_block, context); }; factory.registerSource("executable_pool", create_table_source); diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index bea14d88d1e..54ce5e4a448 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -77,7 +77,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { @@ -87,9 +87,9 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory) const auto filepath = config.getString(config_prefix + ".file.path"); 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(filepath, format, sample_block, context_local_copy, created_from_ddl); + return std::make_unique(filepath, format, sample_block, context, created_from_ddl); }; factory.registerSource("file", create_table_source); diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 26667db1081..5ecf3299ea6 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -557,7 +557,7 @@ void registerDictionaryFlat(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr /* context */, + ContextPtr /* global_context */, bool /* created_from_ddl */) -> DictionaryPtr { if (dict_struct.key) diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 4a80ebdf975..b5cf59b4474 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -213,13 +213,13 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { if (dict_struct.has_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 & credentials_prefix = settings_config_prefix + ".credentials"; @@ -258,7 +258,7 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) .header_entries = std::move(header_entries) }; - return std::make_unique(dict_struct, configuration, credentials, sample_block, context_local_copy, created_from_ddl); + return std::make_unique(dict_struct, configuration, credentials, sample_block, context, created_from_ddl); }; factory.registerSource("http", create_table_source); } diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index d462631fba8..fd5865e24c0 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -756,13 +756,13 @@ void registerDictionaryHashed(DictionaryFactory & factory) using namespace std::placeholders; 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", - [=](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", - [=](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", - [=](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); } diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index fbe911c1d49..b688362d048 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -954,7 +954,7 @@ void registerDictionaryTrie(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr /* context */, + ContextPtr /* global_context */, bool /*created_from_ddl*/) -> DictionaryPtr { if (!dict_struct.key || dict_struct.key->size() != 1) diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index f2c5cefa543..2a47d4c9172 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -186,11 +186,11 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { - return std::make_unique(dict_struct, config, config_prefix + ".library", sample_block, context, created_from_ddl); + return std::make_unique(dict_struct, config, config_prefix + ".library", sample_block, global_context, created_from_ddl); }; factory.registerSource("library", create_table_source); diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 2eebb6970d0..bd53c1e60a7 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -31,11 +31,11 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) [[maybe_unused]] const Poco::Util::AbstractConfiguration & config, [[maybe_unused]] const std::string & config_prefix, [[maybe_unused]] Block & sample_block, - [[maybe_unused]] ContextPtr context, + [[maybe_unused]] ContextPtr global_context, const std::string & /* default_database */, bool /* created_from_ddl */) -> DictionarySourcePtr { #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) , false , config.getBool(config_prefix + ".mysql.fail_on_connection_loss", false) ? 1 : default_num_tries_on_connection_loss); diff --git a/src/Dictionaries/PolygonDictionaryImplementations.cpp b/src/Dictionaries/PolygonDictionaryImplementations.cpp index 7c3eb421a4a..72869ad57ba 100644 --- a/src/Dictionaries/PolygonDictionaryImplementations.cpp +++ b/src/Dictionaries/PolygonDictionaryImplementations.cpp @@ -167,7 +167,7 @@ DictionaryPtr createLayout(const std::string & , const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr /* context */, + ContextPtr /* global_context */, bool /*created_from_ddl*/) { const String database = config.getString(config_prefix + ".database", ""); diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 5a546820959..3fe9e899cd9 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -182,7 +182,7 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & /* default_database */, bool /* created_from_ddl */) -> DictionarySourcePtr { @@ -190,8 +190,8 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) const auto settings_config_prefix = config_prefix + ".postgresql"; auto pool = std::make_shared( config, settings_config_prefix, - context->getSettingsRef().postgresql_connection_pool_size, - context->getSettingsRef().postgresql_connection_pool_wait_timeout); + global_context->getSettingsRef().postgresql_connection_pool_size, + global_context->getSettingsRef().postgresql_connection_pool_wait_timeout); PostgreSQLDictionarySource::Configuration configuration { @@ -211,7 +211,7 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) (void)config; (void)config_prefix; (void)sample_block; - (void)context; + (void)global_context; throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Dictionary source of type `postgresql` is disabled because ClickHouse was built without postgresql support."); #endif diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index ea0af493bdf..390871661c7 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -688,7 +688,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr /* context */, + ContextPtr /* global_context */, bool /*created_from_ddl*/) -> DictionaryPtr { if (dict_struct.key) diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 6561a122e9d..24a14d8cc80 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -12,7 +12,7 @@ void registerDictionarySourceRedis(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const String & config_prefix, Block & sample_block, - ContextPtr /* context */, + ContextPtr /* global_context */, const std::string & /* default_database */, bool /* created_from_ddl */) -> DictionarySourcePtr { return std::make_unique(dict_struct, config, config_prefix + ".redis", sample_block); diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index e79e55910b7..9fc7e92634b 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -234,12 +234,12 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & /* default_database */, bool /* check_config */) -> DictionarySourcePtr { #if USE_ODBC BridgeHelperPtr bridge = std::make_shared>( - 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"; @@ -255,13 +255,13 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory) .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1) }; - return std::make_unique(dict_struct, configuration, sample_block, context, bridge); + return std::make_unique(dict_struct, configuration, sample_block, global_context, bridge); #else (void)dict_struct; (void)config; (void)config_prefix; (void)sample_block; - (void)context; + (void)global_context; throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Dictionary source of type `odbc` is disabled because poco library was built without ODBC support."); #endif @@ -276,7 +276,7 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & /* config */, const std::string & /* config_prefix */, Block & /* sample_block */, - ContextPtr /* context */, + ContextPtr /* global_context */, const std::string & /* default_database */, bool /* created_from_ddl */) -> DictionarySourcePtr { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, diff --git a/src/Dictionaries/registerCacheDictionaries.cpp b/src/Dictionaries/registerCacheDictionaries.cpp index 64c1c55e0ba..69197f992f0 100644 --- a/src/Dictionaries/registerCacheDictionaries.cpp +++ b/src/Dictionaries/registerCacheDictionaries.cpp @@ -154,7 +154,7 @@ DictionaryPtr createCacheDictionaryLayout( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr context [[maybe_unused]], + ContextPtr global_context [[maybe_unused]], bool created_from_ddl [[maybe_unused]]) { String layout_type; @@ -213,8 +213,8 @@ DictionaryPtr createCacheDictionaryLayout( else { 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())) - throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", 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, global_context->getUserFilesPath()); storage = std::make_shared>(storage_configuration); } @@ -239,10 +239,10 @@ void registerDictionaryCache(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr context, + ContextPtr global_context, bool created_from_ddl) -> DictionaryPtr { - return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); + return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl); }; factory.registerLayout("cache", create_simple_cache_layout, false); @@ -252,10 +252,10 @@ void registerDictionaryCache(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr context, + ContextPtr global_context, bool created_from_ddl) -> DictionaryPtr { - return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); + return createCacheDictionaryLayout(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); @@ -267,10 +267,10 @@ void registerDictionaryCache(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr context, + ContextPtr global_context, bool created_from_ddl) -> DictionaryPtr { - return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); + return createCacheDictionaryLayout(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); @@ -280,9 +280,9 @@ void registerDictionaryCache(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr context, + ContextPtr global_context, bool created_from_ddl) -> DictionaryPtr { - return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); + return createCacheDictionaryLayout(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); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a634c19dcd6..84a858d8603 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -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(&credentials)) - client_info.current_password = basic_credentials->getPassword(); -#endif - - setUser(authenticated_user_id); -} - void Context::setUser(const UUID & user_id_) { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 4e378dacf01..1b636deb532 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -362,13 +362,9 @@ public: void setUsersConfig(const ConfigurationPtr & config); 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. - /// 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_); UserPtr getUser() const; diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 83931649443..cbb0e52b91b 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -45,12 +45,20 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create( ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::getDictionary(const std::string & dictionary_name, ContextPtr local_context) const { 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(load(resolved_dictionary_name)); } ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::tryGetDictionary(const std::string & dictionary_name, ContextPtr local_context) const { 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(tryLoad(resolved_dictionary_name)); }