mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Make ContextPtr const by default.
This commit is contained in:
parent
a6ef5778cc
commit
afc1fe7f3d
@ -159,7 +159,7 @@ private:
|
||||
bool print_stacktrace;
|
||||
const Settings & settings;
|
||||
SharedContextHolder shared_context;
|
||||
ContextPtr global_context;
|
||||
ContextMutablePtr global_context;
|
||||
QueryProcessingStage::Enum query_processing_stage;
|
||||
|
||||
/// Don't execute new queries after timelimit or SIGINT or exception
|
||||
|
@ -181,7 +181,7 @@ private:
|
||||
bool has_vertical_output_suffix = false; /// Is \G present at the end of the query string?
|
||||
|
||||
SharedContextHolder shared_context = Context::createShared();
|
||||
ContextPtr context = Context::createGlobal(shared_context.get());
|
||||
ContextMutablePtr context = Context::createGlobal(shared_context.get());
|
||||
|
||||
/// Buffer that reads from stdin in batch mode.
|
||||
ReadBufferFromFileDescriptor std_in{STDIN_FILENO};
|
||||
|
@ -12,14 +12,14 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ClusterCopier : WithContext
|
||||
class ClusterCopier : WithMutableContext
|
||||
{
|
||||
public:
|
||||
ClusterCopier(const String & task_path_,
|
||||
const String & host_id_,
|
||||
const String & proxy_database_name_,
|
||||
ContextPtr context_)
|
||||
: WithContext(context_),
|
||||
ContextMutablePtr context_)
|
||||
: WithMutableContext(context_),
|
||||
task_zookeeper_path(task_path_),
|
||||
host_id(host_id_),
|
||||
working_database_name(proxy_database_name_),
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
return BaseDaemon::logger();
|
||||
}
|
||||
|
||||
ContextPtr context() const override
|
||||
ContextMutablePtr context() const override
|
||||
{
|
||||
return global_context;
|
||||
}
|
||||
@ -58,7 +58,7 @@ protected:
|
||||
std::string getDefaultConfigFileName() const override;
|
||||
|
||||
private:
|
||||
ContextPtr global_context;
|
||||
ContextMutablePtr global_context;
|
||||
|
||||
Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const;
|
||||
|
||||
|
@ -100,7 +100,7 @@ void LocalServer::initialize(Poco::Util::Application & self)
|
||||
}
|
||||
}
|
||||
|
||||
void LocalServer::applyCmdSettings(ContextPtr context)
|
||||
void LocalServer::applyCmdSettings(ContextMutablePtr context)
|
||||
{
|
||||
context->applySettingsChanges(cmd_settings.changes());
|
||||
}
|
||||
@ -645,7 +645,7 @@ void LocalServer::init(int argc, char ** argv)
|
||||
argsToConfig(arguments, config(), 100);
|
||||
}
|
||||
|
||||
void LocalServer::applyCmdOptions(ContextPtr context)
|
||||
void LocalServer::applyCmdOptions(ContextMutablePtr context)
|
||||
{
|
||||
context->setDefaultFormat(config().getString("output-format", config().getString("format", "TSV")));
|
||||
applyCmdSettings(context);
|
||||
|
@ -36,8 +36,8 @@ private:
|
||||
std::string getInitialCreateTableQuery();
|
||||
|
||||
void tryInitPath();
|
||||
void applyCmdOptions(ContextPtr context);
|
||||
void applyCmdSettings(ContextPtr context);
|
||||
void applyCmdOptions(ContextMutablePtr context);
|
||||
void applyCmdSettings(ContextMutablePtr context);
|
||||
void processQueries();
|
||||
void setupUsers();
|
||||
void cleanup();
|
||||
@ -45,7 +45,7 @@ private:
|
||||
|
||||
protected:
|
||||
SharedContextHolder shared_context;
|
||||
ContextPtr global_context;
|
||||
ContextMutablePtr global_context;
|
||||
|
||||
/// Settings specified via command line args
|
||||
Settings cmd_settings;
|
||||
|
@ -1133,7 +1133,7 @@ try
|
||||
}
|
||||
|
||||
SharedContextHolder shared_context = Context::createShared();
|
||||
ContextPtr context = Context::createGlobal(shared_context.get());
|
||||
auto context = Context::createGlobal(shared_context.get());
|
||||
context->makeGlobalContext();
|
||||
|
||||
ReadBufferFromFileDescriptor file_in(STDIN_FILENO);
|
||||
|
@ -40,7 +40,7 @@ public:
|
||||
return BaseDaemon::logger();
|
||||
}
|
||||
|
||||
ContextPtr context() const override
|
||||
ContextMutablePtr context() const override
|
||||
{
|
||||
return global_context;
|
||||
}
|
||||
@ -64,7 +64,7 @@ protected:
|
||||
std::string getDefaultCorePath() const override;
|
||||
|
||||
private:
|
||||
ContextPtr global_context;
|
||||
ContextMutablePtr global_context;
|
||||
Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const;
|
||||
|
||||
using CreateServerFunc = std::function<void(UInt16)>;
|
||||
|
@ -87,7 +87,7 @@ public:
|
||||
/// Initializes query with current thread as master thread in constructor, and detaches it in destructor
|
||||
struct QueryScope
|
||||
{
|
||||
explicit QueryScope(ContextPtr query_context);
|
||||
explicit QueryScope(ContextMutablePtr query_context);
|
||||
~QueryScope();
|
||||
|
||||
void logPeakMemoryUsage();
|
||||
|
@ -80,10 +80,10 @@ public:
|
||||
/// Parsing of external table used when sending tables via http
|
||||
/// The `handlePart` function will be called for each table passed,
|
||||
/// so it's also necessary to call `clean` at the end of the `handlePart`.
|
||||
class ExternalTablesHandler : public HTMLForm::PartHandler, BaseExternalTable, WithContext
|
||||
class ExternalTablesHandler : public HTMLForm::PartHandler, BaseExternalTable, WithMutableContext
|
||||
{
|
||||
public:
|
||||
ExternalTablesHandler(ContextPtr context_, const Poco::Net::NameValueCollection & params_) : WithContext(context_), params(params_) {}
|
||||
ExternalTablesHandler(ContextMutablePtr context_, const Poco::Net::NameValueCollection & params_) : WithMutableContext(context_), params(params_) {}
|
||||
|
||||
void handlePart(const Poco::Net::MessageHeader & header, ReadBuffer & stream) override;
|
||||
|
||||
|
@ -73,7 +73,7 @@ Native41::Native41(const String & password, const String & auth_plugin_data)
|
||||
}
|
||||
|
||||
void Native41::authenticate(
|
||||
const String & user_name, std::optional<String> auth_response, ContextPtr context,
|
||||
const String & user_name, std::optional<String> auth_response, ContextMutablePtr context,
|
||||
std::shared_ptr<PacketEndpoint> packet_endpoint, bool, const Poco::Net::SocketAddress & address)
|
||||
{
|
||||
if (!auth_response)
|
||||
@ -136,7 +136,7 @@ Sha256Password::Sha256Password(RSA & public_key_, RSA & private_key_, Poco::Logg
|
||||
}
|
||||
|
||||
void Sha256Password::authenticate(
|
||||
const String & user_name, std::optional<String> auth_response, ContextPtr context,
|
||||
const String & user_name, std::optional<String> auth_response, ContextMutablePtr context,
|
||||
std::shared_ptr<PacketEndpoint> packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address)
|
||||
{
|
||||
if (!auth_response)
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
virtual String getAuthPluginData() = 0;
|
||||
|
||||
virtual void authenticate(
|
||||
const String & user_name, std::optional<String> auth_response, ContextPtr context,
|
||||
const String & user_name, std::optional<String> auth_response, ContextMutablePtr context,
|
||||
std::shared_ptr<PacketEndpoint> packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address) = 0;
|
||||
};
|
||||
|
||||
@ -49,7 +49,7 @@ public:
|
||||
String getAuthPluginData() override { return scramble; }
|
||||
|
||||
void authenticate(
|
||||
const String & user_name, std::optional<String> auth_response, ContextPtr context,
|
||||
const String & user_name, std::optional<String> auth_response, ContextMutablePtr context,
|
||||
std::shared_ptr<PacketEndpoint> packet_endpoint, bool /* is_secure_connection */, const Poco::Net::SocketAddress & address) override;
|
||||
|
||||
private:
|
||||
@ -69,7 +69,7 @@ public:
|
||||
String getAuthPluginData() override { return scramble; }
|
||||
|
||||
void authenticate(
|
||||
const String & user_name, std::optional<String> auth_response, ContextPtr context,
|
||||
const String & user_name, std::optional<String> auth_response, ContextMutablePtr context,
|
||||
std::shared_ptr<PacketEndpoint> packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address) override;
|
||||
|
||||
private:
|
||||
|
@ -802,7 +802,7 @@ protected:
|
||||
static void setPassword(
|
||||
const String & user_name,
|
||||
const String & password,
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
Messaging::MessageTransport & mt,
|
||||
const Poco::Net::SocketAddress & address)
|
||||
{
|
||||
@ -821,7 +821,7 @@ protected:
|
||||
public:
|
||||
virtual void authenticate(
|
||||
const String & user_name,
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
Messaging::MessageTransport & mt,
|
||||
const Poco::Net::SocketAddress & address) = 0;
|
||||
|
||||
@ -835,7 +835,7 @@ class NoPasswordAuth : public AuthenticationMethod
|
||||
public:
|
||||
void authenticate(
|
||||
const String & user_name,
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
Messaging::MessageTransport & mt,
|
||||
const Poco::Net::SocketAddress & address) override
|
||||
{
|
||||
@ -853,7 +853,7 @@ class CleartextPasswordAuth : public AuthenticationMethod
|
||||
public:
|
||||
void authenticate(
|
||||
const String & user_name,
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
Messaging::MessageTransport & mt,
|
||||
const Poco::Net::SocketAddress & address) override
|
||||
{
|
||||
@ -896,7 +896,7 @@ public:
|
||||
|
||||
void authenticate(
|
||||
const String & user_name,
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
Messaging::MessageTransport & mt,
|
||||
const Poco::Net::SocketAddress & address)
|
||||
{
|
||||
|
@ -54,8 +54,8 @@ private:
|
||||
};
|
||||
|
||||
std::vector<ViewInfo> views;
|
||||
ContextPtr select_context;
|
||||
ContextPtr insert_context;
|
||||
ContextMutablePtr select_context;
|
||||
ContextMutablePtr insert_context;
|
||||
|
||||
void process(const Block & block, ViewInfo & view);
|
||||
};
|
||||
|
@ -416,7 +416,7 @@ UUID DatabaseAtomic::tryGetTableUUID(const String & table_name) const
|
||||
return UUIDHelpers::Nil;
|
||||
}
|
||||
|
||||
void DatabaseAtomic::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool force_attach)
|
||||
void DatabaseAtomic::loadStoredObjects(ContextMutablePtr local_context, bool has_force_restore_data_flag, bool force_attach)
|
||||
{
|
||||
/// Recreate symlinks to table data dirs in case of force restore, because some of them may be broken
|
||||
if (has_force_restore_data_flag)
|
||||
|
@ -47,7 +47,7 @@ public:
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override;
|
||||
|
||||
void loadStoredObjects(ContextPtr context, bool has_force_restore_data_flag, bool force_attach) override;
|
||||
void loadStoredObjects(ContextMutablePtr context, bool has_force_restore_data_flag, bool force_attach) override;
|
||||
|
||||
/// Atomic database cannot be detached if there is detached table which still in use
|
||||
void assertCanBeDetached(bool cleanup) override;
|
||||
|
@ -35,7 +35,7 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_,
|
||||
|
||||
|
||||
void DatabaseLazy::loadStoredObjects(
|
||||
ContextPtr local_context,
|
||||
ContextMutablePtr local_context,
|
||||
bool /* has_force_restore_data_flag */,
|
||||
bool /*force_attach*/)
|
||||
{
|
||||
|
@ -27,7 +27,7 @@ public:
|
||||
bool canContainDistributedTables() const override { return false; }
|
||||
|
||||
void loadStoredObjects(
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
bool has_force_restore_data_flag, bool force_attach) override;
|
||||
|
||||
void createTable(
|
||||
|
@ -45,7 +45,7 @@ std::pair<String, StoragePtr> createTableFromAST(
|
||||
ASTCreateQuery ast_create_query,
|
||||
const String & database_name,
|
||||
const String & table_data_path_relative,
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
bool has_force_restore_data_flag)
|
||||
{
|
||||
ast_create_query.attach = true;
|
||||
|
@ -16,7 +16,7 @@ std::pair<String, StoragePtr> createTableFromAST(
|
||||
ASTCreateQuery ast_create_query,
|
||||
const String & database_name,
|
||||
const String & table_data_path_relative,
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
bool has_force_restore_data_flag);
|
||||
|
||||
/** Get the string with the table definition based on the CREATE query.
|
||||
|
@ -35,7 +35,7 @@ static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
|
||||
namespace
|
||||
{
|
||||
void tryAttachTable(
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
const ASTCreateQuery & query,
|
||||
DatabaseOrdinary & database,
|
||||
const String & database_name,
|
||||
@ -84,7 +84,7 @@ DatabaseOrdinary::DatabaseOrdinary(
|
||||
{
|
||||
}
|
||||
|
||||
void DatabaseOrdinary::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool /*force_attach*/)
|
||||
void DatabaseOrdinary::loadStoredObjects(ContextMutablePtr local_context, bool has_force_restore_data_flag, bool /*force_attach*/)
|
||||
{
|
||||
/** Tables load faster if they are loaded in sorted (by name) order.
|
||||
* Otherwise (for the ext4 filesystem), `DirectoryIterator` iterates through them in some order,
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
|
||||
String getEngineName() const override { return "Ordinary"; }
|
||||
|
||||
void loadStoredObjects(ContextPtr context, bool has_force_restore_data_flag, bool force_attach) override;
|
||||
void loadStoredObjects(ContextMutablePtr context, bool has_force_restore_data_flag, bool force_attach) override;
|
||||
|
||||
void alterTable(
|
||||
ContextPtr context,
|
||||
|
@ -300,7 +300,7 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt
|
||||
current_zookeeper->multi(ops);
|
||||
}
|
||||
|
||||
void DatabaseReplicated::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool force_attach)
|
||||
void DatabaseReplicated::loadStoredObjects(ContextMutablePtr local_context, bool has_force_restore_data_flag, bool force_attach)
|
||||
{
|
||||
tryConnectToZooKeeperAndInitDatabase(force_attach);
|
||||
|
||||
|
@ -57,7 +57,7 @@ public:
|
||||
|
||||
void drop(ContextPtr /*context*/) override;
|
||||
|
||||
void loadStoredObjects(ContextPtr context, bool has_force_restore_data_flag, bool force_attach) override;
|
||||
void loadStoredObjects(ContextMutablePtr context, bool has_force_restore_data_flag, bool force_attach) override;
|
||||
void shutdown() override;
|
||||
|
||||
friend struct DatabaseReplicatedTask;
|
||||
|
@ -122,7 +122,7 @@ public:
|
||||
|
||||
/// Load a set of existing tables.
|
||||
/// You can call only once, right after the object is created.
|
||||
virtual void loadStoredObjects(ContextPtr /*context*/, bool /*has_force_restore_data_flag*/, bool /*force_attach*/ = false) {}
|
||||
virtual void loadStoredObjects(ContextMutablePtr /*context*/, bool /*has_force_restore_data_flag*/, bool /*force_attach*/ = false) {}
|
||||
|
||||
/// Check the existence of the table.
|
||||
virtual bool isTableExist(const String & name, ContextPtr context) const = 0;
|
||||
|
@ -399,7 +399,7 @@ String DatabaseConnectionMySQL::getMetadataPath() const
|
||||
return metadata_path;
|
||||
}
|
||||
|
||||
void DatabaseConnectionMySQL::loadStoredObjects(ContextPtr, bool, bool /*force_attach*/)
|
||||
void DatabaseConnectionMySQL::loadStoredObjects(ContextMutablePtr, bool, bool /*force_attach*/)
|
||||
{
|
||||
|
||||
std::lock_guard<std::mutex> lock{mutex};
|
||||
|
@ -74,7 +74,7 @@ public:
|
||||
|
||||
void createTable(ContextPtr, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) override;
|
||||
|
||||
void loadStoredObjects(ContextPtr, bool, bool force_attach) override;
|
||||
void loadStoredObjects(ContextMutablePtr, bool, bool force_attach) override;
|
||||
|
||||
StoragePtr detachTable(const String & table_name) override;
|
||||
|
||||
|
@ -92,7 +92,7 @@ void DatabaseMaterializeMySQL<Base>::setException(const std::exception_ptr & exc
|
||||
}
|
||||
|
||||
template<typename Base>
|
||||
void DatabaseMaterializeMySQL<Base>::loadStoredObjects(ContextPtr context_, bool has_force_restore_data_flag, bool force_attach)
|
||||
void DatabaseMaterializeMySQL<Base>::loadStoredObjects(ContextMutablePtr context_, bool has_force_restore_data_flag, bool force_attach)
|
||||
{
|
||||
Base::loadStoredObjects(context_, has_force_restore_data_flag, force_attach);
|
||||
if (!force_attach)
|
||||
|
@ -43,7 +43,7 @@ protected:
|
||||
public:
|
||||
String getEngineName() const override { return "MaterializeMySQL"; }
|
||||
|
||||
void loadStoredObjects(ContextPtr context_, bool has_force_restore_data_flag, bool force_attach) override;
|
||||
void loadStoredObjects(ContextMutablePtr context_, bool has_force_restore_data_flag, bool force_attach) override;
|
||||
|
||||
void createTable(ContextPtr context_, const String & name, const StoragePtr & table, const ASTPtr & query) override;
|
||||
|
||||
|
@ -41,7 +41,7 @@ namespace ErrorCodes
|
||||
|
||||
static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync";
|
||||
|
||||
static ContextPtr createQueryContext(ContextPtr context)
|
||||
static ContextMutablePtr createQueryContext(ContextPtr context)
|
||||
{
|
||||
Settings new_query_settings = context->getSettings();
|
||||
new_query_settings.insert_allow_materialized_columns = true;
|
||||
@ -59,7 +59,7 @@ static ContextPtr createQueryContext(ContextPtr context)
|
||||
return query_context;
|
||||
}
|
||||
|
||||
static BlockIO tryToExecuteQuery(const String & query_to_execute, ContextPtr query_context, const String & database, const String & comment)
|
||||
static BlockIO tryToExecuteQuery(const String & query_to_execute, ContextMutablePtr query_context, const String & database, const String & comment)
|
||||
{
|
||||
try
|
||||
{
|
||||
@ -281,7 +281,7 @@ static inline void cleanOutdatedTables(const String & database_name, ContextPtr
|
||||
}
|
||||
|
||||
static inline BlockOutputStreamPtr
|
||||
getTableOutput(const String & database_name, const String & table_name, ContextPtr query_context, bool insert_materialized = false)
|
||||
getTableOutput(const String & database_name, const String & table_name, ContextMutablePtr query_context, bool insert_materialized = false)
|
||||
{
|
||||
const StoragePtr & storage = DatabaseCatalog::instance().getTable(StorageID(database_name, table_name), query_context);
|
||||
|
||||
|
@ -280,7 +280,7 @@ void DatabasePostgreSQL::drop(ContextPtr /*context*/)
|
||||
}
|
||||
|
||||
|
||||
void DatabasePostgreSQL::loadStoredObjects(ContextPtr /* context */, bool, bool /*force_attach*/)
|
||||
void DatabasePostgreSQL::loadStoredObjects(ContextMutablePtr /* context */, bool, bool /*force_attach*/)
|
||||
{
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{mutex};
|
||||
|
@ -47,7 +47,7 @@ public:
|
||||
|
||||
bool empty() const override;
|
||||
|
||||
void loadStoredObjects(ContextPtr, bool, bool force_attach) override;
|
||||
void loadStoredObjects(ContextMutablePtr, bool, bool force_attach) override;
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override;
|
||||
|
||||
|
@ -79,7 +79,7 @@ private:
|
||||
mutable std::string invalidate_query_response;
|
||||
ExternalQueryBuilder query_builder;
|
||||
Block sample_block;
|
||||
ContextPtr context;
|
||||
ContextMutablePtr context;
|
||||
ConnectionPoolWithFailoverPtr pool;
|
||||
const std::string load_all_query;
|
||||
Poco::Logger * log = &Poco::Logger::get("ClickHouseDictionarySource");
|
||||
|
@ -67,7 +67,7 @@ Block blockForKeys(
|
||||
return block;
|
||||
}
|
||||
|
||||
ContextPtr copyContextAndApplySettings(
|
||||
ContextMutablePtr copyContextAndApplySettings(
|
||||
const std::string & config_prefix,
|
||||
ContextConstPtr context,
|
||||
const Poco::Util::AbstractConfiguration & config)
|
||||
|
@ -37,7 +37,7 @@ Block blockForKeys(
|
||||
const std::vector<size_t> & requested_rows);
|
||||
|
||||
/// Used for applying settings to copied context in some register[...]Source functions
|
||||
ContextPtr copyContextAndApplySettings(
|
||||
ContextMutablePtr copyContextAndApplySettings(
|
||||
const std::string & config_prefix,
|
||||
ContextConstPtr context,
|
||||
const Poco::Util::AbstractConfiguration & config);
|
||||
|
@ -24,7 +24,7 @@ namespace ErrorCodes
|
||||
namespace ClusterProxy
|
||||
{
|
||||
|
||||
ContextPtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, Poco::Logger * log)
|
||||
ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, Poco::Logger * log)
|
||||
{
|
||||
Settings new_settings = settings;
|
||||
new_settings.queue_max_wait_ms = Cluster::saturate(new_settings.queue_max_wait_ms, settings.max_execution_time);
|
||||
|
@ -31,7 +31,7 @@ class IStreamFactory;
|
||||
/// - optimize_skip_unused_shards_nesting
|
||||
///
|
||||
/// @return new Context with adjusted settings
|
||||
ContextPtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, Poco::Logger * log = nullptr);
|
||||
ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, Poco::Logger * log = nullptr);
|
||||
|
||||
/// Execute a distributed query, creating a vector of BlockInputStreams, from which the result can be read.
|
||||
/// `stream_factory` object encapsulates the logic of creating streams for a different type of query
|
||||
|
@ -140,7 +140,7 @@ public:
|
||||
/// Find existing session or create a new.
|
||||
std::shared_ptr<NamedSession> acquireSession(
|
||||
const String & session_id,
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
std::chrono::steady_clock::duration timeout,
|
||||
bool throw_if_not_found)
|
||||
{
|
||||
@ -513,7 +513,7 @@ SharedContextHolder::SharedContextHolder(std::unique_ptr<ContextSharedPart> shar
|
||||
|
||||
void SharedContextHolder::reset() { shared.reset(); }
|
||||
|
||||
ContextPtr Context::createGlobal(ContextSharedPart * shared)
|
||||
ContextMutablePtr Context::createGlobal(ContextSharedPart * shared)
|
||||
{
|
||||
auto res = std::shared_ptr<Context>(new Context);
|
||||
res->shared = shared;
|
||||
@ -530,19 +530,19 @@ SharedContextHolder Context::createShared()
|
||||
return SharedContextHolder(std::make_unique<ContextSharedPart>());
|
||||
}
|
||||
|
||||
ContextPtr Context::createCopy(const ContextConstPtr & other)
|
||||
ContextMutablePtr Context::createCopy(const ContextPtr & other)
|
||||
{
|
||||
return std::shared_ptr<Context>(new Context(*other));
|
||||
}
|
||||
|
||||
ContextPtr Context::createCopy(const ContextWeakConstPtr & other)
|
||||
ContextMutablePtr Context::createCopy(const ContextWeakConstPtr & other)
|
||||
{
|
||||
auto ptr = other.lock();
|
||||
if (!ptr) throw Exception("Can't copy an expired context", ErrorCodes::LOGICAL_ERROR);
|
||||
return createCopy(ptr);
|
||||
}
|
||||
|
||||
ContextPtr Context::createCopy(const ContextPtr & other)
|
||||
ContextMutablePtr Context::createCopy(const ContextMutablePtr & other)
|
||||
{
|
||||
return createCopy(std::const_pointer_cast<const Context>(other));
|
||||
}
|
||||
@ -1079,7 +1079,7 @@ void Context::addViewSource(const StoragePtr & storage)
|
||||
}
|
||||
|
||||
|
||||
StoragePtr Context::getViewSource()
|
||||
StoragePtr Context::getViewSource() const
|
||||
{
|
||||
return view_source;
|
||||
}
|
||||
@ -1313,7 +1313,7 @@ void Context::setMacros(std::unique_ptr<Macros> && macros)
|
||||
shared->macros.set(std::move(macros));
|
||||
}
|
||||
|
||||
ContextPtr Context::getQueryContext() const
|
||||
ContextMutablePtr Context::getQueryContext() const
|
||||
{
|
||||
auto ptr = query_context.lock();
|
||||
if (!ptr) throw Exception("There is no query or query context has expired", ErrorCodes::THERE_IS_NO_QUERY);
|
||||
@ -1326,21 +1326,21 @@ bool Context::isInternalSubquery() const
|
||||
return ptr && ptr.get() != this;
|
||||
}
|
||||
|
||||
ContextPtr Context::getSessionContext() const
|
||||
ContextMutablePtr Context::getSessionContext() const
|
||||
{
|
||||
auto ptr = session_context.lock();
|
||||
if (!ptr) throw Exception("There is no session or session context has expired", ErrorCodes::THERE_IS_NO_SESSION);
|
||||
return ptr;
|
||||
}
|
||||
|
||||
ContextPtr Context::getGlobalContext() const
|
||||
ContextMutablePtr Context::getGlobalContext() const
|
||||
{
|
||||
auto ptr = global_context.lock();
|
||||
if (!ptr) throw Exception("There is no global context or global context has expired", ErrorCodes::LOGICAL_ERROR);
|
||||
return ptr;
|
||||
}
|
||||
|
||||
ContextPtr Context::getBufferContext() const
|
||||
ContextMutablePtr Context::getBufferContext() const
|
||||
{
|
||||
if (!buffer_context) throw Exception("There is no buffer context", ErrorCodes::LOGICAL_ERROR);
|
||||
return buffer_context;
|
||||
@ -1871,7 +1871,7 @@ std::shared_ptr<Cluster> Context::tryGetCluster(const std::string & cluster_name
|
||||
}
|
||||
|
||||
|
||||
void Context::reloadClusterConfig()
|
||||
void Context::reloadClusterConfig() const
|
||||
{
|
||||
while (true)
|
||||
{
|
||||
@ -1958,7 +1958,7 @@ bool Context::hasTraceCollector() const
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<QueryLog> Context::getQueryLog()
|
||||
std::shared_ptr<QueryLog> Context::getQueryLog() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
@ -1969,7 +1969,7 @@ std::shared_ptr<QueryLog> Context::getQueryLog()
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<QueryThreadLog> Context::getQueryThreadLog()
|
||||
std::shared_ptr<QueryThreadLog> Context::getQueryThreadLog() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
@ -1980,7 +1980,7 @@ std::shared_ptr<QueryThreadLog> Context::getQueryThreadLog()
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<PartLog> Context::getPartLog(const String & part_database)
|
||||
std::shared_ptr<PartLog> Context::getPartLog(const String & part_database) const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
@ -1998,7 +1998,7 @@ std::shared_ptr<PartLog> Context::getPartLog(const String & part_database)
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<TraceLog> Context::getTraceLog()
|
||||
std::shared_ptr<TraceLog> Context::getTraceLog() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
@ -2009,7 +2009,7 @@ std::shared_ptr<TraceLog> Context::getTraceLog()
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<TextLog> Context::getTextLog()
|
||||
std::shared_ptr<TextLog> Context::getTextLog() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
@ -2020,7 +2020,7 @@ std::shared_ptr<TextLog> Context::getTextLog()
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<MetricLog> Context::getMetricLog()
|
||||
std::shared_ptr<MetricLog> Context::getMetricLog() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
@ -2042,7 +2042,7 @@ std::shared_ptr<AsynchronousMetricLog> Context::getAsynchronousMetricLog() const
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<OpenTelemetrySpanLog> Context::getOpenTelemetrySpanLog()
|
||||
std::shared_ptr<OpenTelemetrySpanLog> Context::getOpenTelemetrySpanLog() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
@ -2640,11 +2640,14 @@ ZooKeeperMetadataTransactionPtr Context::getZooKeeperMetadataTransaction() const
|
||||
return metadata_transaction;
|
||||
}
|
||||
|
||||
PartUUIDsPtr Context::getPartUUIDs()
|
||||
PartUUIDsPtr Context::getPartUUIDs() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
if (!part_uuids)
|
||||
part_uuids = std::make_shared<PartUUIDs>();
|
||||
/// For context itself, only this initialization is not const.
|
||||
/// We could have done in constructor.
|
||||
/// TODO: probably, remove this from Context.
|
||||
const_cast<PartUUIDsPtr &>(part_uuids) = std::make_shared<PartUUIDs>();
|
||||
|
||||
return part_uuids;
|
||||
}
|
||||
@ -2663,11 +2666,11 @@ void Context::setReadTaskCallback(ReadTaskCallback && callback)
|
||||
next_task_callback = callback;
|
||||
}
|
||||
|
||||
PartUUIDsPtr Context::getIgnoredPartUUIDs()
|
||||
PartUUIDsPtr Context::getIgnoredPartUUIDs() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
if (!ignored_part_uuids)
|
||||
ignored_part_uuids = std::make_shared<PartUUIDs>();
|
||||
const_cast<PartUUIDsPtr &>(ignored_part_uuids) = std::make_shared<PartUUIDs>();
|
||||
|
||||
return ignored_part_uuids;
|
||||
}
|
||||
|
@ -252,12 +252,12 @@ private:
|
||||
StoragePtr view_source; /// Temporary StorageValues used to generate alias columns for materialized views
|
||||
Tables table_function_results; /// Temporary tables obtained by execution of table functions. Keyed by AST tree id.
|
||||
|
||||
ContextWeakPtr query_context;
|
||||
ContextWeakPtr session_context; /// Session context or nullptr. Could be equal to this.
|
||||
ContextWeakPtr global_context; /// Global context. Could be equal to this.
|
||||
ContextWeakMutablePtr query_context;
|
||||
ContextWeakMutablePtr session_context; /// Session context or nullptr. Could be equal to this.
|
||||
ContextWeakMutablePtr global_context; /// Global context. Could be equal to this.
|
||||
|
||||
/// XXX: move this stuff to shared part instead.
|
||||
ContextPtr buffer_context; /// Buffer context. Could be equal to this.
|
||||
ContextMutablePtr buffer_context; /// Buffer context. Could be equal to this.
|
||||
|
||||
public:
|
||||
// Top-level OpenTelemetry trace context for the query. Makes sense only for a query context.
|
||||
@ -293,10 +293,10 @@ private:
|
||||
|
||||
public:
|
||||
/// Create initial Context with ContextShared and etc.
|
||||
static ContextPtr createGlobal(ContextSharedPart * shared);
|
||||
static ContextPtr createCopy(const ContextWeakConstPtr & other);
|
||||
static ContextPtr createCopy(const ContextConstPtr & other);
|
||||
static ContextPtr createCopy(const ContextPtr & other);
|
||||
static ContextMutablePtr createGlobal(ContextSharedPart * shared);
|
||||
static ContextMutablePtr createCopy(const ContextWeakConstPtr & other);
|
||||
static ContextMutablePtr createCopy(const ContextMutablePtr & other);
|
||||
static ContextMutablePtr createCopy(const ContextPtr & other);
|
||||
static SharedContextHolder createShared();
|
||||
|
||||
void copyFrom(const ContextPtr & other);
|
||||
@ -459,7 +459,7 @@ public:
|
||||
StoragePtr executeTableFunction(const ASTPtr & table_expression);
|
||||
|
||||
void addViewSource(const StoragePtr & storage);
|
||||
StoragePtr getViewSource();
|
||||
StoragePtr getViewSource() const;
|
||||
|
||||
String getCurrentDatabase() const;
|
||||
String getCurrentQueryId() const { return client_info.current_query_id; }
|
||||
@ -555,14 +555,14 @@ public:
|
||||
|
||||
/// For methods below you may need to acquire the context lock by yourself.
|
||||
|
||||
ContextPtr getQueryContext() const;
|
||||
ContextMutablePtr getQueryContext() const;
|
||||
bool hasQueryContext() const { return !query_context.expired(); }
|
||||
bool isInternalSubquery() const;
|
||||
|
||||
ContextPtr getSessionContext() const;
|
||||
ContextMutablePtr getSessionContext() const;
|
||||
bool hasSessionContext() const { return !session_context.expired(); }
|
||||
|
||||
ContextPtr getGlobalContext() const;
|
||||
ContextMutablePtr getGlobalContext() const;
|
||||
bool hasGlobalContext() const { return !global_context.expired(); }
|
||||
bool isGlobalContext() const
|
||||
{
|
||||
@ -570,10 +570,10 @@ public:
|
||||
return ptr && ptr.get() == this;
|
||||
}
|
||||
|
||||
ContextPtr getBufferContext() const;
|
||||
ContextMutablePtr getBufferContext() const;
|
||||
|
||||
void setQueryContext(ContextPtr context_) { query_context = context_; }
|
||||
void setSessionContext(ContextPtr context_) { session_context = context_; }
|
||||
void setQueryContext(ContextMutablePtr context_) { query_context = context_; }
|
||||
void setSessionContext(ContextMutablePtr context_) { session_context = context_; }
|
||||
|
||||
void makeQueryContext() { query_context = shared_from_this(); }
|
||||
void makeSessionContext() { session_context = shared_from_this(); }
|
||||
@ -668,7 +668,7 @@ public:
|
||||
void setClustersConfig(const ConfigurationPtr & config, const String & config_name = "remote_servers");
|
||||
/// Sets custom cluster, but doesn't update configuration
|
||||
void setCluster(const String & cluster_name, const std::shared_ptr<Cluster> & cluster);
|
||||
void reloadClusterConfig();
|
||||
void reloadClusterConfig() const;
|
||||
|
||||
Compiler & getCompiler();
|
||||
|
||||
@ -681,17 +681,17 @@ public:
|
||||
bool hasTraceCollector() const;
|
||||
|
||||
/// Nullptr if the query log is not ready for this moment.
|
||||
std::shared_ptr<QueryLog> getQueryLog();
|
||||
std::shared_ptr<QueryThreadLog> getQueryThreadLog();
|
||||
std::shared_ptr<TraceLog> getTraceLog();
|
||||
std::shared_ptr<TextLog> getTextLog();
|
||||
std::shared_ptr<MetricLog> getMetricLog();
|
||||
std::shared_ptr<QueryLog> getQueryLog() const;
|
||||
std::shared_ptr<QueryThreadLog> getQueryThreadLog() const;
|
||||
std::shared_ptr<TraceLog> getTraceLog() const;
|
||||
std::shared_ptr<TextLog> getTextLog() const;
|
||||
std::shared_ptr<MetricLog> getMetricLog() const;
|
||||
std::shared_ptr<AsynchronousMetricLog> getAsynchronousMetricLog() const;
|
||||
std::shared_ptr<OpenTelemetrySpanLog> getOpenTelemetrySpanLog();
|
||||
std::shared_ptr<OpenTelemetrySpanLog> getOpenTelemetrySpanLog() const;
|
||||
|
||||
/// Returns an object used to log operations with parts if it possible.
|
||||
/// Provide table name to make required checks.
|
||||
std::shared_ptr<PartLog> getPartLog(const String & part_database);
|
||||
std::shared_ptr<PartLog> getPartLog(const String & part_database) const;
|
||||
|
||||
const MergeTreeSettings & getMergeTreeSettings() const;
|
||||
const MergeTreeSettings & getReplicatedMergeTreeSettings() const;
|
||||
@ -778,8 +778,8 @@ public:
|
||||
|
||||
MySQLWireContext mysql;
|
||||
|
||||
PartUUIDsPtr getPartUUIDs();
|
||||
PartUUIDsPtr getIgnoredPartUUIDs();
|
||||
PartUUIDsPtr getPartUUIDs() const;
|
||||
PartUUIDsPtr getIgnoredPartUUIDs() const;
|
||||
|
||||
ReadTaskCallback getReadTaskCallback() const;
|
||||
void setReadTaskCallback(ReadTaskCallback && callback);
|
||||
@ -820,7 +820,7 @@ struct NamedSession
|
||||
{
|
||||
NamedSessionKey key;
|
||||
UInt64 close_cycle = 0;
|
||||
ContextPtr context;
|
||||
ContextMutablePtr context;
|
||||
std::chrono::steady_clock::duration timeout;
|
||||
NamedSessions & parent;
|
||||
|
||||
|
@ -22,10 +22,12 @@ class Context;
|
||||
|
||||
/// Most used types have shorter names
|
||||
/// TODO: in the first part of refactoring all the context pointers are non-const.
|
||||
using ContextPtr = std::shared_ptr<Context>;
|
||||
using ContextPtr = std::shared_ptr<const Context>;
|
||||
using ContextConstPtr = std::shared_ptr<const Context>;
|
||||
using ContextWeakPtr = std::weak_ptr<Context>;
|
||||
using ContextMutablePtr = std::shared_ptr<Context>;
|
||||
using ContextWeakPtr = std::weak_ptr<const Context>;
|
||||
using ContextWeakConstPtr = std::weak_ptr<const Context>;
|
||||
using ContextWeakMutablePtr = std::weak_ptr<Context>;
|
||||
|
||||
template <class Shared = ContextPtr>
|
||||
struct WithContextImpl
|
||||
@ -50,5 +52,6 @@ protected:
|
||||
|
||||
using WithContext = WithContextImpl<>;
|
||||
using WithConstContext = WithContextImpl<ContextConstPtr>;
|
||||
using WithMutableContext = WithContextImpl<ContextMutablePtr>;
|
||||
|
||||
}
|
||||
|
@ -145,7 +145,7 @@ void DDLTaskBase::parseQueryFromEntry(ContextPtr context)
|
||||
query = parseQuery(parser_query, begin, end, description, 0, context->getSettingsRef().max_parser_depth);
|
||||
}
|
||||
|
||||
ContextPtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & /*zookeeper*/)
|
||||
ContextMutablePtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & /*zookeeper*/)
|
||||
{
|
||||
auto query_context = Context::createCopy(from_context);
|
||||
query_context->makeQueryContext();
|
||||
@ -355,7 +355,7 @@ void DatabaseReplicatedTask::parseQueryFromEntry(ContextPtr context)
|
||||
}
|
||||
}
|
||||
|
||||
ContextPtr DatabaseReplicatedTask::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper)
|
||||
ContextMutablePtr DatabaseReplicatedTask::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper)
|
||||
{
|
||||
auto query_context = DDLTaskBase::makeQueryContext(from_context, zookeeper);
|
||||
query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
|
||||
|
@ -98,7 +98,7 @@ struct DDLTaskBase
|
||||
|
||||
virtual String getShardID() const = 0;
|
||||
|
||||
virtual ContextPtr makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper);
|
||||
virtual ContextMutablePtr makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper);
|
||||
|
||||
inline String getActiveNodePath() const { return entry_path + "/active/" + host_id_str; }
|
||||
inline String getFinishedNodePath() const { return entry_path + "/finished/" + host_id_str; }
|
||||
@ -136,7 +136,7 @@ struct DatabaseReplicatedTask : public DDLTaskBase
|
||||
|
||||
String getShardID() const override;
|
||||
void parseQueryFromEntry(ContextPtr context) override;
|
||||
ContextPtr makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper) override;
|
||||
ContextMutablePtr makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper) override;
|
||||
|
||||
DatabaseReplicated * database;
|
||||
};
|
||||
|
@ -110,7 +110,7 @@ protected:
|
||||
void runMainThread();
|
||||
void runCleanupThread();
|
||||
|
||||
ContextPtr context;
|
||||
ContextMutablePtr context;
|
||||
Poco::Logger * log;
|
||||
|
||||
std::string host_fqdn; /// current host domain name
|
||||
|
@ -528,13 +528,13 @@ void DatabaseCatalog::updateUUIDMapping(const UUID & uuid, DatabasePtr database,
|
||||
|
||||
std::unique_ptr<DatabaseCatalog> DatabaseCatalog::database_catalog;
|
||||
|
||||
DatabaseCatalog::DatabaseCatalog(ContextPtr global_context_)
|
||||
: WithContext(global_context_), log(&Poco::Logger::get("DatabaseCatalog"))
|
||||
DatabaseCatalog::DatabaseCatalog(ContextMutablePtr global_context_)
|
||||
: WithMutableContext(global_context_), log(&Poco::Logger::get("DatabaseCatalog"))
|
||||
{
|
||||
TemporaryLiveViewCleaner::init(global_context_);
|
||||
}
|
||||
|
||||
DatabaseCatalog & DatabaseCatalog::init(ContextPtr global_context_)
|
||||
DatabaseCatalog & DatabaseCatalog::init(ContextMutablePtr global_context_)
|
||||
{
|
||||
if (database_catalog)
|
||||
{
|
||||
|
@ -117,13 +117,13 @@ using TemporaryTablesMapping = std::map<String, std::shared_ptr<TemporaryTableHo
|
||||
class BackgroundSchedulePoolTaskHolder;
|
||||
|
||||
/// For some reason Context is required to get Storage from Database object
|
||||
class DatabaseCatalog : boost::noncopyable, WithContext
|
||||
class DatabaseCatalog : boost::noncopyable, WithMutableContext
|
||||
{
|
||||
public:
|
||||
static constexpr const char * TEMPORARY_DATABASE = "_temporary_and_external_tables";
|
||||
static constexpr const char * SYSTEM_DATABASE = "system";
|
||||
|
||||
static DatabaseCatalog & init(ContextPtr global_context_);
|
||||
static DatabaseCatalog & init(ContextMutablePtr global_context_);
|
||||
static DatabaseCatalog & instance();
|
||||
static void shutdown();
|
||||
|
||||
@ -209,7 +209,7 @@ private:
|
||||
// make emplace(global_context_) compile with private constructor ¯\_(ツ)_/¯.
|
||||
static std::unique_ptr<DatabaseCatalog> database_catalog;
|
||||
|
||||
explicit DatabaseCatalog(ContextPtr global_context_);
|
||||
explicit DatabaseCatalog(ContextMutablePtr global_context_);
|
||||
void assertDatabaseExistsUnlocked(const String & database_name) const;
|
||||
void assertDatabaseDoesntExistUnlocked(const String & database_name) const;
|
||||
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
|
||||
protected:
|
||||
ASTPtr query_ptr;
|
||||
ContextPtr context;
|
||||
ContextMutablePtr context;
|
||||
Block result_header;
|
||||
SelectQueryOptions options;
|
||||
size_t max_streams = 1;
|
||||
|
@ -89,8 +89,8 @@ namespace ErrorCodes
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
InterpreterCreateQuery::InterpreterCreateQuery(const ASTPtr & query_ptr_, ContextPtr context_)
|
||||
: WithContext(context_), query_ptr(query_ptr_)
|
||||
InterpreterCreateQuery::InterpreterCreateQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
|
||||
: WithMutableContext(context_), query_ptr(query_ptr_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -22,10 +22,10 @@ using DatabasePtr = std::shared_ptr<IDatabase>;
|
||||
/** Allows to create new table or database,
|
||||
* or create an object for existing table or database.
|
||||
*/
|
||||
class InterpreterCreateQuery : public IInterpreter, WithContext
|
||||
class InterpreterCreateQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterCreateQuery(const ASTPtr & query_ptr_, ContextPtr context_);
|
||||
InterpreterCreateQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_);
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
|
@ -10,10 +10,10 @@ namespace DB
|
||||
class ASTCreateQuotaQuery;
|
||||
struct Quota;
|
||||
|
||||
class InterpreterCreateQuotaQuery : public IInterpreter, WithContext
|
||||
class InterpreterCreateQuotaQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterCreateQuotaQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {}
|
||||
InterpreterCreateQuotaQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {}
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
|
@ -10,10 +10,10 @@ namespace DB
|
||||
class ASTCreateRoleQuery;
|
||||
struct Role;
|
||||
|
||||
class InterpreterCreateRoleQuery : public IInterpreter, WithContext
|
||||
class InterpreterCreateRoleQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterCreateRoleQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {}
|
||||
InterpreterCreateRoleQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {}
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
|
@ -10,10 +10,10 @@ namespace DB
|
||||
class ASTCreateRowPolicyQuery;
|
||||
struct RowPolicy;
|
||||
|
||||
class InterpreterCreateRowPolicyQuery : public IInterpreter, WithContext
|
||||
class InterpreterCreateRowPolicyQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterCreateRowPolicyQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {}
|
||||
InterpreterCreateRowPolicyQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {}
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
|
@ -10,10 +10,10 @@ namespace DB
|
||||
class ASTCreateSettingsProfileQuery;
|
||||
struct SettingsProfile;
|
||||
|
||||
class InterpreterCreateSettingsProfileQuery : public IInterpreter, WithContext
|
||||
class InterpreterCreateSettingsProfileQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterCreateSettingsProfileQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {}
|
||||
InterpreterCreateSettingsProfileQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {}
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
|
@ -10,10 +10,10 @@ namespace DB
|
||||
class ASTCreateUserQuery;
|
||||
struct User;
|
||||
|
||||
class InterpreterCreateUserQuery : public IInterpreter, WithContext
|
||||
class InterpreterCreateUserQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterCreateUserQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {}
|
||||
InterpreterCreateUserQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {}
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
|
@ -9,10 +9,10 @@ namespace DB
|
||||
|
||||
class AccessRightsElements;
|
||||
|
||||
class InterpreterDropAccessEntityQuery : public IInterpreter, WithContext
|
||||
class InterpreterDropAccessEntityQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterDropAccessEntityQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {}
|
||||
InterpreterDropAccessEntityQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {}
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
|
@ -42,7 +42,7 @@ static DatabasePtr tryGetDatabase(const String & database_name, bool if_exists)
|
||||
}
|
||||
|
||||
|
||||
InterpreterDropQuery::InterpreterDropQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_)
|
||||
InterpreterDropQuery::InterpreterDropQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -16,10 +16,10 @@ class AccessRightsElements;
|
||||
* or remove information about table (just forget) from server (DETACH),
|
||||
* or just clear all data in table (TRUNCATE).
|
||||
*/
|
||||
class InterpreterDropQuery : public IInterpreter, WithContext
|
||||
class InterpreterDropQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterDropQuery(const ASTPtr & query_ptr_, ContextPtr context_);
|
||||
InterpreterDropQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_);
|
||||
|
||||
/// Drop table or database.
|
||||
BlockIO execute() override;
|
||||
|
@ -26,8 +26,8 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
InterpreterExternalDDLQuery::InterpreterExternalDDLQuery(const ASTPtr & query_, ContextPtr context_)
|
||||
: WithContext(context_), query(query_)
|
||||
InterpreterExternalDDLQuery::InterpreterExternalDDLQuery(const ASTPtr & query_, ContextMutablePtr context_)
|
||||
: WithMutableContext(context_), query(query_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -6,10 +6,10 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class InterpreterExternalDDLQuery : public IInterpreter, WithContext
|
||||
class InterpreterExternalDDLQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterExternalDDLQuery(const ASTPtr & query_, ContextPtr context_);
|
||||
InterpreterExternalDDLQuery(const ASTPtr & query_, ContextMutablePtr context_);
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
|
@ -92,7 +92,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextPtr context, const SelectQueryOptions & options)
|
||||
std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextMutablePtr context, const SelectQueryOptions & options)
|
||||
{
|
||||
OpenTelemetrySpanHolder span("InterpreterFactory::get()");
|
||||
|
||||
|
@ -16,7 +16,7 @@ class InterpreterFactory
|
||||
public:
|
||||
static std::unique_ptr<IInterpreter> get(
|
||||
ASTPtr & query,
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
const SelectQueryOptions & options = {});
|
||||
};
|
||||
|
||||
|
@ -12,10 +12,10 @@ class ASTGrantQuery;
|
||||
struct User;
|
||||
struct Role;
|
||||
|
||||
class InterpreterGrantQuery : public IInterpreter, WithContext
|
||||
class InterpreterGrantQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterGrantQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {}
|
||||
InterpreterGrantQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {}
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
|
@ -10,10 +10,10 @@ namespace DB
|
||||
|
||||
class AccessRightsElements;
|
||||
|
||||
class InterpreterKillQueryQuery final : public IInterpreter, WithContext
|
||||
class InterpreterKillQueryQuery final : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterKillQueryQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) { }
|
||||
InterpreterKillQueryQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) { }
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
|
@ -11,10 +11,10 @@ class ASTSetQuery;
|
||||
|
||||
/** Change one or several settings for the session or just for the current context.
|
||||
*/
|
||||
class InterpreterSetQuery : public IInterpreter, WithContext
|
||||
class InterpreterSetQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterSetQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {}
|
||||
InterpreterSetQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {}
|
||||
|
||||
/** Usual SET query. Set setting for the session.
|
||||
*/
|
||||
|
@ -11,10 +11,10 @@ class ASTSetRoleQuery;
|
||||
struct RolesOrUsersSet;
|
||||
struct User;
|
||||
|
||||
class InterpreterSetRoleQuery : public IInterpreter, WithContext
|
||||
class InterpreterSetRoleQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterSetRoleQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {}
|
||||
InterpreterSetRoleQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {}
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
|
@ -17,8 +17,8 @@ namespace ErrorCodes
|
||||
using EntityType = IAccessEntity::Type;
|
||||
|
||||
|
||||
InterpreterShowAccessEntitiesQuery::InterpreterShowAccessEntitiesQuery(const ASTPtr & query_ptr_, ContextPtr context_)
|
||||
: WithContext(context_), query_ptr(query_ptr_)
|
||||
InterpreterShowAccessEntitiesQuery::InterpreterShowAccessEntitiesQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
|
||||
: WithMutableContext(context_), query_ptr(query_ptr_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -7,10 +7,10 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class InterpreterShowAccessEntitiesQuery : public IInterpreter, WithContext
|
||||
class InterpreterShowAccessEntitiesQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterShowAccessEntitiesQuery(const ASTPtr & query_ptr_, ContextPtr context_);
|
||||
InterpreterShowAccessEntitiesQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_);
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
|
@ -4,7 +4,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
InterpreterShowPrivilegesQuery::InterpreterShowPrivilegesQuery(const ASTPtr & query_ptr_, ContextPtr context_)
|
||||
InterpreterShowPrivilegesQuery::InterpreterShowPrivilegesQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
|
||||
: query_ptr(query_ptr_), context(context_)
|
||||
{
|
||||
}
|
||||
|
@ -11,7 +11,7 @@ class Context;
|
||||
class InterpreterShowPrivilegesQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterShowPrivilegesQuery(const ASTPtr & query_ptr_, ContextPtr context_);
|
||||
InterpreterShowPrivilegesQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_);
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
@ -20,7 +20,7 @@ public:
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
ContextPtr context;
|
||||
ContextMutablePtr context;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -9,11 +9,11 @@ namespace DB
|
||||
|
||||
/** Return list of currently executing queries.
|
||||
*/
|
||||
class InterpreterShowProcesslistQuery : public IInterpreter, WithContext
|
||||
class InterpreterShowProcesslistQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterShowProcesslistQuery(const ASTPtr & query_ptr_, ContextPtr context_)
|
||||
: WithContext(context_), query_ptr(query_ptr_) {}
|
||||
InterpreterShowProcesslistQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
|
||||
: WithMutableContext(context_), query_ptr(query_ptr_) {}
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
|
@ -18,8 +18,8 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
InterpreterShowTablesQuery::InterpreterShowTablesQuery(const ASTPtr & query_ptr_, ContextPtr context_)
|
||||
: WithContext(context_), query_ptr(query_ptr_)
|
||||
InterpreterShowTablesQuery::InterpreterShowTablesQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
|
||||
: WithMutableContext(context_), query_ptr(query_ptr_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -13,10 +13,10 @@ class Context;
|
||||
/** Return a list of tables or databases meets specified conditions.
|
||||
* Interprets a query through replacing it to SELECT query from system.tables or system.databases.
|
||||
*/
|
||||
class InterpreterShowTablesQuery : public IInterpreter, WithContext
|
||||
class InterpreterShowTablesQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterShowTablesQuery(const ASTPtr & query_ptr_, ContextPtr context_);
|
||||
InterpreterShowTablesQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_);
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
|
@ -192,8 +192,8 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type,
|
||||
}
|
||||
|
||||
|
||||
InterpreterSystemQuery::InterpreterSystemQuery(const ASTPtr & query_ptr_, ContextPtr context_)
|
||||
: WithContext(context_), query_ptr(query_ptr_->clone()), log(&Poco::Logger::get("InterpreterSystemQuery"))
|
||||
InterpreterSystemQuery::InterpreterSystemQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
|
||||
: WithMutableContext(context_), query_ptr(query_ptr_->clone()), log(&Poco::Logger::get("InterpreterSystemQuery"))
|
||||
{
|
||||
}
|
||||
|
||||
@ -424,7 +424,7 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
}
|
||||
|
||||
|
||||
StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, ContextPtr system_context, bool need_ddl_guard)
|
||||
StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, ContextMutablePtr system_context, bool need_ddl_guard)
|
||||
{
|
||||
getContext()->checkAccess(AccessType::SYSTEM_RESTART_REPLICA, replica);
|
||||
|
||||
@ -469,7 +469,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica,
|
||||
return table;
|
||||
}
|
||||
|
||||
void InterpreterSystemQuery::restartReplicas(ContextPtr system_context)
|
||||
void InterpreterSystemQuery::restartReplicas(ContextMutablePtr system_context)
|
||||
{
|
||||
std::vector<StorageID> replica_names;
|
||||
auto & catalog = DatabaseCatalog::instance();
|
||||
|
@ -30,10 +30,10 @@ class ASTSystemQuery;
|
||||
* - start/stop actions for all existing tables.
|
||||
* Note that the actions for tables that will be created after this query will not be affected.
|
||||
*/
|
||||
class InterpreterSystemQuery : public IInterpreter, WithContext
|
||||
class InterpreterSystemQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterSystemQuery(const ASTPtr & query_ptr_, ContextPtr context_);
|
||||
InterpreterSystemQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_);
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
@ -45,9 +45,9 @@ private:
|
||||
|
||||
/// Tries to get a replicated table and restart it
|
||||
/// Returns pointer to a newly created table if the restart was successful
|
||||
StoragePtr tryRestartReplica(const StorageID & replica, ContextPtr context, bool need_ddl_guard = true);
|
||||
StoragePtr tryRestartReplica(const StorageID & replica, ContextMutablePtr context, bool need_ddl_guard = true);
|
||||
|
||||
void restartReplicas(ContextPtr system_context);
|
||||
void restartReplicas(ContextMutablePtr system_context);
|
||||
void syncReplica(ASTSystemQuery & query);
|
||||
void dropReplica(ASTSystemQuery & query);
|
||||
bool dropReplicaImpl(ASTSystemQuery & query, const StoragePtr & table);
|
||||
|
@ -179,7 +179,7 @@ bool isStorageTouchedByMutations(
|
||||
const StoragePtr & storage,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const std::vector<MutationCommand> & commands,
|
||||
ContextPtr context_copy)
|
||||
ContextMutablePtr context_copy)
|
||||
{
|
||||
if (commands.empty())
|
||||
return false;
|
||||
|
@ -23,7 +23,7 @@ bool isStorageTouchedByMutations(
|
||||
const StoragePtr & storage,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
const std::vector<MutationCommand> & commands,
|
||||
ContextPtr context_copy
|
||||
ContextMutablePtr context_copy
|
||||
);
|
||||
|
||||
ASTPtr getPartitionAndPredicateExpressionForMutationCommand(
|
||||
|
@ -55,12 +55,12 @@ namespace MySQLInterpreter
|
||||
};
|
||||
|
||||
template <typename InterpreterImpl>
|
||||
class InterpreterMySQLDDLQuery : public IInterpreter, WithContext
|
||||
class InterpreterMySQLDDLQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
InterpreterMySQLDDLQuery(
|
||||
const ASTPtr & query_ptr_, ContextPtr context_, const String & mapped_to_database_, const String & mysql_database_)
|
||||
: WithContext(context_), query_ptr(query_ptr_), mapped_to_database(mapped_to_database_), mysql_database(mysql_database_)
|
||||
const ASTPtr & query_ptr_, ContextMutablePtr context_, const String & mapped_to_database_, const String & mysql_database_)
|
||||
: WithMutableContext(context_), query_ptr(query_ptr_), mapped_to_database(mapped_to_database_), mysql_database(mysql_database_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -508,7 +508,7 @@ void CurrentThread::detachQueryIfNotDetached()
|
||||
}
|
||||
|
||||
|
||||
CurrentThread::QueryScope::QueryScope(ContextPtr query_context)
|
||||
CurrentThread::QueryScope::QueryScope(ContextMutablePtr query_context)
|
||||
{
|
||||
CurrentThread::initializeQuery();
|
||||
CurrentThread::attachQueryContext(query_context);
|
||||
|
@ -335,7 +335,7 @@ static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr
|
||||
}
|
||||
}
|
||||
|
||||
static void setQuerySpecificSettings(ASTPtr & ast, ContextPtr context)
|
||||
static void setQuerySpecificSettings(ASTPtr & ast, ContextMutablePtr context)
|
||||
{
|
||||
if (auto * ast_insert_into = dynamic_cast<ASTInsertQuery *>(ast.get()))
|
||||
{
|
||||
@ -347,7 +347,7 @@ static void setQuerySpecificSettings(ASTPtr & ast, ContextPtr context)
|
||||
static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
const char * begin,
|
||||
const char * end,
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
bool internal,
|
||||
QueryProcessingStage::Enum stage,
|
||||
bool has_query_tail,
|
||||
@ -910,7 +910,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
|
||||
BlockIO executeQuery(
|
||||
const String & query,
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
bool internal,
|
||||
QueryProcessingStage::Enum stage,
|
||||
bool may_have_embedded_data)
|
||||
@ -935,7 +935,7 @@ BlockIO executeQuery(
|
||||
|
||||
BlockIO executeQuery(
|
||||
const String & query,
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
bool internal,
|
||||
QueryProcessingStage::Enum stage,
|
||||
bool may_have_embedded_data,
|
||||
@ -954,7 +954,7 @@ void executeQuery(
|
||||
ReadBuffer & istr,
|
||||
WriteBuffer & ostr,
|
||||
bool allow_into_outfile,
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
std::function<void(const String &, const String &, const String &, const String &)> set_result_details)
|
||||
{
|
||||
PODArray<char> parse_buf;
|
||||
|
@ -16,7 +16,7 @@ void executeQuery(
|
||||
ReadBuffer & istr, /// Where to read query from (and data for INSERT, if present).
|
||||
WriteBuffer & ostr, /// Where to write query output to.
|
||||
bool allow_into_outfile, /// If true and the query contains INTO OUTFILE section, redirect output to that file.
|
||||
ContextPtr context, /// DB, tables, data types, storage engines, functions, aggregate functions...
|
||||
ContextMutablePtr context, /// DB, tables, data types, storage engines, functions, aggregate functions...
|
||||
std::function<void(const String &, const String &, const String &, const String &)> set_result_details /// If a non-empty callback is passed, it will be called with the query id, the content-type, the format, and the timezone.
|
||||
);
|
||||
|
||||
@ -37,7 +37,7 @@ void executeQuery(
|
||||
/// must be done separately.
|
||||
BlockIO executeQuery(
|
||||
const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out.
|
||||
ContextPtr context, /// DB, tables, data types, storage engines, functions, aggregate functions...
|
||||
ContextMutablePtr context, /// DB, tables, data types, storage engines, functions, aggregate functions...
|
||||
bool internal = false, /// If true, this query is caused by another query and thus needn't be registered in the ProcessList.
|
||||
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete, /// To which stage the query must be executed.
|
||||
bool may_have_embedded_data = false /// If insert query may have embedded data
|
||||
@ -46,7 +46,7 @@ BlockIO executeQuery(
|
||||
/// Old interface with allow_processors flag. For compatibility.
|
||||
BlockIO executeQuery(
|
||||
const String & query,
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
bool internal,
|
||||
QueryProcessingStage::Enum stage,
|
||||
bool may_have_embedded_data,
|
||||
|
@ -25,7 +25,7 @@ namespace DB
|
||||
|
||||
static void executeCreateQuery(
|
||||
const String & query,
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
const String & database,
|
||||
const String & file_name,
|
||||
bool has_force_restore_data_flag)
|
||||
@ -46,7 +46,7 @@ static void executeCreateQuery(
|
||||
|
||||
|
||||
static void loadDatabase(
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
const String & database,
|
||||
const String & database_path,
|
||||
bool force_restore_data)
|
||||
@ -84,7 +84,7 @@ static void loadDatabase(
|
||||
}
|
||||
|
||||
|
||||
void loadMetadata(ContextPtr context, const String & default_database_name)
|
||||
void loadMetadata(ContextMutablePtr context, const String & default_database_name)
|
||||
{
|
||||
Poco::Logger * log = &Poco::Logger::get("loadMetadata");
|
||||
|
||||
@ -168,7 +168,7 @@ void loadMetadata(ContextPtr context, const String & default_database_name)
|
||||
}
|
||||
|
||||
|
||||
void loadMetadataSystem(ContextPtr context)
|
||||
void loadMetadataSystem(ContextMutablePtr context)
|
||||
{
|
||||
String path = context->getPath() + "metadata/" + DatabaseCatalog::SYSTEM_DATABASE;
|
||||
String metadata_file = path + ".sql";
|
||||
|
@ -8,9 +8,9 @@ namespace DB
|
||||
|
||||
/// Load tables from system database. Only real tables like query_log, part_log.
|
||||
/// You should first load system database, then attach system tables that you need into it, then load other databases.
|
||||
void loadMetadataSystem(ContextPtr context);
|
||||
void loadMetadataSystem(ContextMutablePtr context);
|
||||
|
||||
/// Load tables from databases and add them to context. Database 'system' is ignored. Use separate function to load system tables.
|
||||
void loadMetadata(ContextPtr context, const String & default_database_name = {});
|
||||
void loadMetadata(ContextMutablePtr context, const String & default_database_name = {});
|
||||
|
||||
}
|
||||
|
@ -110,7 +110,7 @@ public:
|
||||
/// Do not allow to change the table while the processors of pipe are alive.
|
||||
void addTableLock(TableLockHolder lock) { holder.table_locks.emplace_back(std::move(lock)); }
|
||||
/// This methods are from QueryPipeline. Needed to make conversion from pipeline to pipe possible.
|
||||
void addInterpreterContext(std::shared_ptr<Context> context) { holder.interpreter_context.emplace_back(std::move(context)); }
|
||||
void addInterpreterContext(std::shared_ptr<const Context> context) { holder.interpreter_context.emplace_back(std::move(context)); }
|
||||
void addStorageHolder(StoragePtr storage) { holder.storage_holders.emplace_back(std::move(storage)); }
|
||||
void addQueryIdHolder(std::shared_ptr<QueryIdHolder> query_id_holder) { holder.query_id_holder = std::move(query_id_holder); }
|
||||
/// For queries with nested interpreters (i.e. StorageDistributed)
|
||||
@ -129,7 +129,7 @@ private:
|
||||
/// Some processors may implicitly use Context or temporary Storage created by Interpreter.
|
||||
/// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here,
|
||||
/// because QueryPipeline is alive until query is finished.
|
||||
std::vector<std::shared_ptr<Context>> interpreter_context;
|
||||
std::vector<std::shared_ptr<const Context>> interpreter_context;
|
||||
std::vector<StoragePtr> storage_holders;
|
||||
std::vector<TableLockHolder> table_locks;
|
||||
std::vector<std::unique_ptr<QueryPlan>> query_plans;
|
||||
|
@ -119,7 +119,7 @@ public:
|
||||
const Block & getHeader() const { return pipe.getHeader(); }
|
||||
|
||||
void addTableLock(TableLockHolder lock) { pipe.addTableLock(std::move(lock)); }
|
||||
void addInterpreterContext(std::shared_ptr<Context> context) { pipe.addInterpreterContext(std::move(context)); }
|
||||
void addInterpreterContext(std::shared_ptr<const Context> context) { pipe.addInterpreterContext(std::move(context)); }
|
||||
void addStorageHolder(StoragePtr storage) { pipe.addStorageHolder(std::move(storage)); }
|
||||
void addQueryPlan(std::unique_ptr<QueryPlan> plan) { pipe.addQueryPlan(std::move(plan)); }
|
||||
void setLimits(const StreamLocalLimits & limits) { pipe.setLimits(limits); }
|
||||
|
@ -4,7 +4,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, std::shared_ptr<Context> context_)
|
||||
ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, std::shared_ptr<const Context> context_)
|
||||
: ISourceStep(DataStream{.header = pipe_.getHeader()})
|
||||
, pipe(std::move(pipe_))
|
||||
, context(std::move(context_))
|
||||
|
@ -9,7 +9,7 @@ namespace DB
|
||||
class ReadFromPreparedSource : public ISourceStep
|
||||
{
|
||||
public:
|
||||
explicit ReadFromPreparedSource(Pipe pipe_, std::shared_ptr<Context> context_ = nullptr);
|
||||
explicit ReadFromPreparedSource(Pipe pipe_, std::shared_ptr<const Context> context_ = nullptr);
|
||||
|
||||
String getName() const override { return "ReadFromPreparedSource"; }
|
||||
|
||||
@ -17,7 +17,7 @@ public:
|
||||
|
||||
private:
|
||||
Pipe pipe;
|
||||
std::shared_ptr<Context> context;
|
||||
std::shared_ptr<const Context> context;
|
||||
};
|
||||
|
||||
class ReadFromStorageStep : public ReadFromPreparedSource
|
||||
|
@ -521,7 +521,7 @@ namespace
|
||||
Poco::Logger * log = nullptr;
|
||||
|
||||
std::shared_ptr<NamedSession> session;
|
||||
ContextPtr query_context;
|
||||
ContextMutablePtr query_context;
|
||||
std::optional<CurrentThread::QueryScope> query_scope;
|
||||
String query_text;
|
||||
ASTPtr ast;
|
||||
@ -932,8 +932,8 @@ namespace
|
||||
String format = external_table.format();
|
||||
if (format.empty())
|
||||
format = "TabSeparated";
|
||||
ContextPtr external_table_context = query_context;
|
||||
ContextPtr temp_context;
|
||||
ContextMutablePtr external_table_context = query_context;
|
||||
ContextMutablePtr temp_context;
|
||||
if (!external_table.settings().empty())
|
||||
{
|
||||
temp_context = Context::createCopy(query_context);
|
||||
|
@ -277,7 +277,7 @@ HTTPHandler::~HTTPHandler()
|
||||
|
||||
|
||||
bool HTTPHandler::authenticateUser(
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
HTTPServerRequest & request,
|
||||
HTMLForm & params,
|
||||
HTTPServerResponse & response)
|
||||
@ -441,7 +441,7 @@ bool HTTPHandler::authenticateUser(
|
||||
|
||||
|
||||
void HTTPHandler::processQuery(
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
HTTPServerRequest & request,
|
||||
HTMLForm & params,
|
||||
HTTPServerResponse & response,
|
||||
@ -937,7 +937,7 @@ DynamicQueryHandler::DynamicQueryHandler(IServer & server_, const std::string &
|
||||
{
|
||||
}
|
||||
|
||||
bool DynamicQueryHandler::customizeQueryParam(ContextPtr context, const std::string & key, const std::string & value)
|
||||
bool DynamicQueryHandler::customizeQueryParam(ContextMutablePtr context, const std::string & key, const std::string & value)
|
||||
{
|
||||
if (key == param_name)
|
||||
return true; /// do nothing
|
||||
@ -953,7 +953,7 @@ bool DynamicQueryHandler::customizeQueryParam(ContextPtr context, const std::str
|
||||
return false;
|
||||
}
|
||||
|
||||
std::string DynamicQueryHandler::getQuery(HTTPServerRequest & request, HTMLForm & params, ContextPtr context)
|
||||
std::string DynamicQueryHandler::getQuery(HTTPServerRequest & request, HTMLForm & params, ContextMutablePtr context)
|
||||
{
|
||||
if (likely(!startsWith(request.getContentType(), "multipart/form-data")))
|
||||
{
|
||||
@ -992,7 +992,7 @@ PredefinedQueryHandler::PredefinedQueryHandler(
|
||||
{
|
||||
}
|
||||
|
||||
bool PredefinedQueryHandler::customizeQueryParam(ContextPtr context, const std::string & key, const std::string & value)
|
||||
bool PredefinedQueryHandler::customizeQueryParam(ContextMutablePtr context, const std::string & key, const std::string & value)
|
||||
{
|
||||
if (receive_params.count(key))
|
||||
{
|
||||
@ -1003,7 +1003,7 @@ bool PredefinedQueryHandler::customizeQueryParam(ContextPtr context, const std::
|
||||
return false;
|
||||
}
|
||||
|
||||
void PredefinedQueryHandler::customizeContext(HTTPServerRequest & request, ContextPtr context)
|
||||
void PredefinedQueryHandler::customizeContext(HTTPServerRequest & request, ContextMutablePtr context)
|
||||
{
|
||||
/// If in the configuration file, the handler's header is regex and contains named capture group
|
||||
/// We will extract regex named capture groups as query parameters
|
||||
@ -1039,7 +1039,7 @@ void PredefinedQueryHandler::customizeContext(HTTPServerRequest & request, Conte
|
||||
}
|
||||
}
|
||||
|
||||
std::string PredefinedQueryHandler::getQuery(HTTPServerRequest & request, HTMLForm & params, ContextPtr context)
|
||||
std::string PredefinedQueryHandler::getQuery(HTTPServerRequest & request, HTMLForm & params, ContextMutablePtr context)
|
||||
{
|
||||
if (unlikely(startsWith(request.getContentType(), "multipart/form-data")))
|
||||
{
|
||||
|
@ -33,11 +33,11 @@ public:
|
||||
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override;
|
||||
|
||||
/// This method is called right before the query execution.
|
||||
virtual void customizeContext(HTTPServerRequest & /* request */, ContextPtr /* context */) {}
|
||||
virtual void customizeContext(HTTPServerRequest & /* request */, ContextMutablePtr /* context */) {}
|
||||
|
||||
virtual bool customizeQueryParam(ContextPtr context, const std::string & key, const std::string & value) = 0;
|
||||
virtual bool customizeQueryParam(ContextMutablePtr context, const std::string & key, const std::string & value) = 0;
|
||||
|
||||
virtual std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextPtr context) = 0;
|
||||
virtual std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextMutablePtr context) = 0;
|
||||
|
||||
private:
|
||||
struct Output
|
||||
@ -73,7 +73,7 @@ private:
|
||||
|
||||
// The request_context and the request_credentials instances may outlive a single request/response loop.
|
||||
// This happens only when the authentication mechanism requires more than a single request/response exchange (e.g., SPNEGO).
|
||||
ContextPtr request_context;
|
||||
ContextMutablePtr request_context;
|
||||
std::unique_ptr<Credentials> request_credentials;
|
||||
|
||||
// Returns true when the user successfully authenticated,
|
||||
@ -82,14 +82,14 @@ private:
|
||||
// the request_context and request_credentials instances are preserved.
|
||||
// Throws an exception if authentication failed.
|
||||
bool authenticateUser(
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
HTTPServerRequest & request,
|
||||
HTMLForm & params,
|
||||
HTTPServerResponse & response);
|
||||
|
||||
/// Also initializes 'used_output'.
|
||||
void processQuery(
|
||||
ContextPtr context,
|
||||
ContextMutablePtr context,
|
||||
HTTPServerRequest & request,
|
||||
HTMLForm & params,
|
||||
HTTPServerResponse & response,
|
||||
@ -113,9 +113,9 @@ private:
|
||||
public:
|
||||
explicit DynamicQueryHandler(IServer & server_, const std::string & param_name_ = "query");
|
||||
|
||||
std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextPtr context) override;
|
||||
std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextMutablePtr context) override;
|
||||
|
||||
bool customizeQueryParam(ContextPtr context, const std::string &key, const std::string &value) override;
|
||||
bool customizeQueryParam(ContextMutablePtr context, const std::string &key, const std::string &value) override;
|
||||
};
|
||||
|
||||
class PredefinedQueryHandler : public HTTPHandler
|
||||
@ -130,11 +130,11 @@ public:
|
||||
IServer & server_, const NameSet & receive_params_, const std::string & predefined_query_
|
||||
, const CompiledRegexPtr & url_regex_, const std::unordered_map<String, CompiledRegexPtr> & header_name_with_regex_);
|
||||
|
||||
virtual void customizeContext(HTTPServerRequest & request, ContextPtr context) override;
|
||||
virtual void customizeContext(HTTPServerRequest & request, ContextMutablePtr context) override;
|
||||
|
||||
std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextPtr context) override;
|
||||
std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextMutablePtr context) override;
|
||||
|
||||
bool customizeQueryParam(ContextPtr context, const std::string & key, const std::string & value) override;
|
||||
bool customizeQueryParam(ContextMutablePtr context, const std::string & key, const std::string & value) override;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -28,7 +28,7 @@ public:
|
||||
virtual Poco::Logger & logger() const = 0;
|
||||
|
||||
/// Returns global application's context.
|
||||
virtual ContextPtr context() const = 0;
|
||||
virtual ContextMutablePtr context() const = 0;
|
||||
|
||||
/// Returns true if shutdown signaled.
|
||||
virtual bool isCancelled() const = 0;
|
||||
|
@ -56,7 +56,7 @@ private:
|
||||
protected:
|
||||
Poco::Logger * log;
|
||||
|
||||
ContextPtr connection_context;
|
||||
ContextMutablePtr connection_context;
|
||||
|
||||
std::shared_ptr<MySQLProtocol::PacketEndpoint> packet_endpoint;
|
||||
|
||||
|
@ -37,7 +37,7 @@ private:
|
||||
Poco::Logger * log = &Poco::Logger::get("PostgreSQLHandler");
|
||||
|
||||
IServer & server;
|
||||
ContextPtr connection_context;
|
||||
ContextMutablePtr connection_context;
|
||||
bool ssl_enabled = false;
|
||||
Int32 connection_id = 0;
|
||||
Int32 secret_key = 0;
|
||||
|
@ -132,8 +132,8 @@ private:
|
||||
UInt64 client_version_patch = 0;
|
||||
UInt64 client_tcp_protocol_version = 0;
|
||||
|
||||
ContextPtr connection_context;
|
||||
ContextPtr query_context;
|
||||
ContextMutablePtr connection_context;
|
||||
ContextMutablePtr query_context;
|
||||
|
||||
size_t unknown_packet_in_send_data = 0;
|
||||
|
||||
|
@ -84,7 +84,7 @@ private:
|
||||
/// Returns the number of blocks was written for each cluster node. Uses during exception handling.
|
||||
std::string getCurrentStateDescription();
|
||||
|
||||
ContextPtr context;
|
||||
ContextMutablePtr context;
|
||||
StorageDistributed & storage;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
ASTPtr query_ast;
|
||||
|
@ -12,7 +12,7 @@ public:
|
||||
explicit KafkaBlockOutputStream(
|
||||
StorageKafka & storage_,
|
||||
const StorageMetadataPtr & metadata_snapshot_,
|
||||
const std::shared_ptr<Context> & context_);
|
||||
const std::shared_ptr<const Context> & context_);
|
||||
|
||||
Block getHeader() const override;
|
||||
|
||||
@ -25,7 +25,7 @@ public:
|
||||
private:
|
||||
StorageKafka & storage;
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
const std::shared_ptr<Context> context;
|
||||
const std::shared_ptr<const Context> context;
|
||||
ProducerBufferPtr buffer;
|
||||
BlockOutputStreamPtr child;
|
||||
};
|
||||
|
@ -191,7 +191,7 @@ private:
|
||||
ASTPtr inner_query; /// stored query : SELECT * FROM ( SELECT a FROM A)
|
||||
ASTPtr inner_subquery; /// stored query's innermost subquery if any
|
||||
ASTPtr inner_blocks_query; /// query over the mergeable blocks to produce final result
|
||||
ContextPtr live_view_context;
|
||||
ContextMutablePtr live_view_context;
|
||||
|
||||
Poco::Logger * log;
|
||||
|
||||
|
@ -16,7 +16,7 @@ namespace ErrorCodes
|
||||
|
||||
namespace
|
||||
{
|
||||
void executeDropQuery(const StorageID & storage_id, ContextPtr context)
|
||||
void executeDropQuery(const StorageID & storage_id, ContextMutablePtr context)
|
||||
{
|
||||
if (!DatabaseCatalog::instance().isTableExist(storage_id, context))
|
||||
return;
|
||||
@ -42,7 +42,7 @@ namespace
|
||||
std::unique_ptr<TemporaryLiveViewCleaner> TemporaryLiveViewCleaner::the_instance;
|
||||
|
||||
|
||||
void TemporaryLiveViewCleaner::init(ContextPtr global_context_)
|
||||
void TemporaryLiveViewCleaner::init(ContextMutablePtr global_context_)
|
||||
{
|
||||
if (the_instance)
|
||||
throw Exception("TemporaryLiveViewCleaner already initialized", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -63,7 +63,7 @@ void TemporaryLiveViewCleaner::shutdown()
|
||||
the_instance.reset();
|
||||
}
|
||||
|
||||
TemporaryLiveViewCleaner::TemporaryLiveViewCleaner(ContextPtr global_context_) : WithContext(global_context_)
|
||||
TemporaryLiveViewCleaner::TemporaryLiveViewCleaner(ContextMutablePtr global_context_) : WithMutableContext(global_context_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -14,7 +14,7 @@ struct StorageID;
|
||||
|
||||
/// This class removes temporary live views in the background thread when it's possible.
|
||||
/// There should only a single instance of this class.
|
||||
class TemporaryLiveViewCleaner : WithContext
|
||||
class TemporaryLiveViewCleaner : WithMutableContext
|
||||
{
|
||||
public:
|
||||
static TemporaryLiveViewCleaner & instance() { return *the_instance; }
|
||||
@ -23,7 +23,7 @@ public:
|
||||
void addView(const std::shared_ptr<StorageLiveView> & view);
|
||||
|
||||
/// Should be called once.
|
||||
static void init(ContextPtr global_context_);
|
||||
static void init(ContextMutablePtr global_context_);
|
||||
static void shutdown();
|
||||
|
||||
void startup();
|
||||
@ -31,7 +31,7 @@ public:
|
||||
private:
|
||||
friend std::unique_ptr<TemporaryLiveViewCleaner>::deleter_type;
|
||||
|
||||
TemporaryLiveViewCleaner(ContextPtr global_context_);
|
||||
TemporaryLiveViewCleaner(ContextMutablePtr global_context_);
|
||||
~TemporaryLiveViewCleaner();
|
||||
|
||||
void backgroundThreadFunc();
|
||||
|
@ -136,7 +136,7 @@ MergeTreeData::MergeTreeData(
|
||||
const StorageID & table_id_,
|
||||
const String & relative_data_path_,
|
||||
const StorageInMemoryMetadata & metadata_,
|
||||
ContextPtr context_,
|
||||
ContextMutablePtr context_,
|
||||
const String & date_column_name,
|
||||
const MergingParams & merging_params_,
|
||||
std::unique_ptr<MergeTreeSettings> storage_settings_,
|
||||
@ -144,7 +144,7 @@ MergeTreeData::MergeTreeData(
|
||||
bool attach,
|
||||
BrokenPartCallback broken_part_callback_)
|
||||
: IStorage(table_id_)
|
||||
, WithContext(context_->getGlobalContext())
|
||||
, WithMutableContext(context_->getGlobalContext())
|
||||
, merging_params(merging_params_)
|
||||
, require_part_metadata(require_part_metadata_)
|
||||
, relative_data_path(relative_data_path_)
|
||||
|
@ -113,7 +113,7 @@ namespace ErrorCodes
|
||||
/// - MergeTreeDataWriter
|
||||
/// - MergeTreeDataMergerMutator
|
||||
|
||||
class MergeTreeData : public IStorage, public WithContext
|
||||
class MergeTreeData : public IStorage, public WithMutableContext
|
||||
{
|
||||
public:
|
||||
/// Function to call if the part is suspected to contain corrupt data.
|
||||
@ -353,7 +353,7 @@ public:
|
||||
MergeTreeData(const StorageID & table_id_,
|
||||
const String & relative_data_path_,
|
||||
const StorageInMemoryMetadata & metadata_,
|
||||
ContextPtr context_,
|
||||
ContextMutablePtr context_,
|
||||
const String & date_column_name,
|
||||
const MergingParams & merging_params_,
|
||||
std::unique_ptr<MergeTreeSettings> settings_,
|
||||
|
@ -79,7 +79,7 @@ protected:
|
||||
std::unique_ptr<RabbitMQSettings> rabbitmq_settings_);
|
||||
|
||||
private:
|
||||
ContextPtr rabbitmq_context;
|
||||
ContextMutablePtr rabbitmq_context;
|
||||
std::unique_ptr<RabbitMQSettings> rabbitmq_settings;
|
||||
|
||||
const String exchange_name;
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user