mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge pull request #27817 from vitlibar/use-sessions-more
Use sessions more
This commit is contained in:
commit
c257cd61e5
@ -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);
|
||||||
|
|
||||||
|
@ -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
|
||||||
{
|
{
|
||||||
|
@ -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);
|
||||||
|
@ -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;
|
||||||
|
@ -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
|
||||||
|
@ -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;
|
||||||
|
@ -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,
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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)
|
||||||
|
@ -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.
|
||||||
*
|
*
|
||||||
|
@ -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";
|
||||||
|
@ -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);
|
||||||
|
@ -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);
|
||||||
|
@ -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);
|
||||||
|
@ -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)
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
@ -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);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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)
|
||||||
|
@ -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);
|
||||||
|
@ -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);
|
||||||
|
@ -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", "");
|
||||||
|
@ -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
|
||||||
|
@ -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)
|
||||||
|
@ -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);
|
||||||
|
@ -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,
|
||||||
|
@ -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);
|
||||||
|
@ -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();
|
||||||
|
@ -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;
|
||||||
|
@ -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));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user