ClickHouse/src/Interpreters/Context.h

913 lines
39 KiB
C++
Raw Normal View History

2011-08-09 19:19:25 +00:00
#pragma once
2019-03-11 14:01:45 +00:00
#include <Core/Block.h>
#include <Core/NamesAndTypes.h>
#include <Core/Settings.h>
2020-02-04 22:22:27 +00:00
#include <Core/UUID.h>
2019-03-11 14:01:45 +00:00
#include <Interpreters/ClientInfo.h>
#include <Interpreters/Context_fwd.h>
#include <Interpreters/DatabaseCatalog.h>
2019-03-11 14:01:45 +00:00
#include <Parsers/IAST_fwd.h>
#include <Storages/IStorage_fwd.h>
2019-03-11 14:01:45 +00:00
#include <Common/MultiVersion.h>
2020-11-18 17:43:18 +00:00
#include <Common/OpenTelemetryTraceContext.h>
#include <Common/RemoteHostFilter.h>
2021-10-18 09:13:24 +00:00
#include <Common/isLocalAddress.h>
2021-10-02 07:13:14 +00:00
#include <base/types.h>
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
2021-10-27 23:10:39 +00:00
#include "config_core.h"
#include <boost/container/flat_set.hpp>
2015-04-16 06:12:35 +00:00
#include <functional>
#include <memory>
#include <mutex>
2018-11-07 11:46:39 +00:00
#include <optional>
2021-08-27 21:29:10 +00:00
#include <exception>
2015-04-16 06:12:35 +00:00
namespace Poco::Net { class IPAddress; }
namespace zkutil { class ZooKeeper; }
2011-08-09 19:19:25 +00:00
namespace DB
{
struct ContextSharedPart;
class ContextAccess;
2020-02-12 03:03:33 +00:00
struct User;
using UserPtr = std::shared_ptr<const User>;
struct EnabledRolesInfo;
class EnabledRowPolicies;
class EnabledQuota;
struct QuotaUsage;
class AccessFlags;
struct AccessRightsElement;
class AccessRightsElements;
enum class RowPolicyFilterType;
class EmbeddedDictionaries;
class ExternalDictionariesLoader;
class ExternalModelsLoader;
class ExternalUserDefinedExecutableFunctionsLoader;
2021-04-07 13:52:11 +00:00
class InterserverCredentials;
using InterserverCredentialsPtr = std::shared_ptr<const InterserverCredentials>;
2015-04-16 06:12:35 +00:00
class InterserverIOHandler;
class BackgroundSchedulePool;
2015-04-16 06:12:35 +00:00
class MergeList;
2020-10-26 16:38:35 +00:00
class ReplicatedFetchList;
2015-04-16 06:12:35 +00:00
class Cluster;
class Compiler;
class MarkCache;
2021-03-28 19:24:28 +00:00
class MMappedFileCache;
2015-04-16 06:12:35 +00:00
class UncompressedCache;
class ProcessList;
class QueryStatus;
2015-04-16 06:12:35 +00:00
class Macros;
2016-03-07 04:31:10 +00:00
struct Progress;
2021-08-01 14:12:34 +00:00
struct FileProgress;
2015-04-30 12:43:16 +00:00
class Clusters;
class QueryLog;
class QueryThreadLog;
2021-06-18 13:44:08 +00:00
class QueryViewsLog;
class PartLog;
2019-07-22 13:54:08 +00:00
class TextLog;
2019-02-03 21:30:45 +00:00
class TraceLog;
2019-08-13 14:31:46 +00:00
class MetricLog;
2020-06-10 19:17:30 +00:00
class AsynchronousMetricLog;
2020-08-20 20:59:40 +00:00
class OpenTelemetrySpanLog;
2021-07-09 14:05:35 +00:00
class ZooKeeperLog;
class SessionLog;
2015-07-16 21:32:51 +00:00
struct MergeTreeSettings;
class StorageS3Settings;
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
class IDatabase;
class DDLWorker;
class ITableFunction;
class Block;
class ActionLocksManager;
using ActionLocksManagerPtr = std::shared_ptr<ActionLocksManager>;
class ShellCommand;
2018-12-21 12:17:30 +00:00
class ICompressionCodec;
class AccessControl;
class Credentials;
class GSSAcceptorContext;
struct SettingsConstraintsAndProfileIDs;
class RemoteHostFilter;
2019-12-05 11:42:13 +00:00
struct StorageID;
class IDisk;
using DiskPtr = std::shared_ptr<IDisk>;
class DiskSelector;
using DiskSelectorPtr = std::shared_ptr<const DiskSelector>;
2020-05-22 10:33:57 +00:00
using DisksMap = std::map<String, DiskPtr>;
2021-01-23 15:20:15 +00:00
class IStoragePolicy;
using StoragePolicyPtr = std::shared_ptr<const IStoragePolicy>;
2020-05-22 10:33:57 +00:00
using StoragePoliciesMap = std::map<String, StoragePolicyPtr>;
class StoragePolicySelector;
using StoragePolicySelectorPtr = std::shared_ptr<const StoragePolicySelector>;
template <class Queue>
2021-08-30 19:37:03 +00:00
class MergeTreeBackgroundExecutor;
class MergeMutateRuntimeQueue;
class OrdinaryRuntimeQueue;
using MergeMutateBackgroundExecutor = MergeTreeBackgroundExecutor<MergeMutateRuntimeQueue>;
using MergeMutateBackgroundExecutorPtr = std::shared_ptr<MergeMutateBackgroundExecutor>;
using OrdinaryBackgroundExecutor = MergeTreeBackgroundExecutor<OrdinaryRuntimeQueue>;
using OrdinaryBackgroundExecutorPtr = std::shared_ptr<OrdinaryBackgroundExecutor>;
struct PartUUIDs;
using PartUUIDsPtr = std::shared_ptr<PartUUIDs>;
class KeeperDispatcher;
class Session;
2021-10-11 16:11:50 +00:00
class IInputFormat;
2019-04-05 10:52:07 +00:00
class IOutputFormat;
2021-10-11 16:11:50 +00:00
using InputFormatPtr = std::shared_ptr<IInputFormat>;
2019-04-05 10:52:07 +00:00
using OutputFormatPtr = std::shared_ptr<IOutputFormat>;
2020-07-08 14:25:23 +00:00
class IVolume;
using VolumePtr = std::shared_ptr<IVolume>;
struct NamedSession;
2020-10-16 10:12:31 +00:00
struct BackgroundTaskSchedulingSettings;
#if USE_NLP
class SynonymsExtensions;
class Lemmatizers;
#endif
2021-05-26 20:37:44 +00:00
class Throttler;
using ThrottlerPtr = std::shared_ptr<Throttler>;
2021-02-19 23:41:58 +00:00
class ZooKeeperMetadataTransaction;
using ZooKeeperMetadataTransactionPtr = std::shared_ptr<ZooKeeperMetadataTransaction>;
2021-06-02 15:39:12 +00:00
class AsynchronousInsertQueue;
/// Callback for external tables initializer
using ExternalTablesInitializer = std::function<void(ContextPtr)>;
2019-05-28 18:30:10 +00:00
/// Callback for initialize input()
using InputInitializer = std::function<void(ContextPtr, const StoragePtr &)>;
2019-05-28 18:30:10 +00:00
/// Callback for reading blocks of data from client for function input()
using InputBlocksReader = std::function<Block(ContextPtr)>;
2019-10-19 20:36:35 +00:00
2021-04-10 02:21:18 +00:00
/// Used in distributed task processing
2021-04-13 10:59:02 +00:00
using ReadTaskCallback = std::function<String()>;
using MergeTreeReadTaskCallback = std::function<std::optional<PartitionReadResponse>(PartitionReadRequest)>;
/// An empty interface for an arbitrary object that may be attached by a shared pointer
/// to query context, when using ClickHouse as a library.
struct IHostContext
{
virtual ~IHostContext() = default;
};
using IHostContextPtr = std::shared_ptr<IHostContext>;
/// A small class which owns ContextShared.
/// We don't use something like unique_ptr directly to allow ContextShared type to be incomplete.
2020-04-16 14:51:33 +00:00
struct SharedContextHolder
{
~SharedContextHolder();
2020-04-19 15:41:35 +00:00
SharedContextHolder();
explicit SharedContextHolder(std::unique_ptr<ContextSharedPart> shared_context);
SharedContextHolder(SharedContextHolder &&) noexcept;
2020-04-19 15:46:12 +00:00
SharedContextHolder & operator=(SharedContextHolder &&);
ContextSharedPart * get() const { return shared.get(); }
2020-04-17 12:58:52 +00:00
void reset();
private:
std::unique_ptr<ContextSharedPart> shared;
2020-04-16 14:51:33 +00:00
};
2017-06-02 21:37:28 +00:00
/** A set of known objects that can be used in the query.
* Consists of a shared part (always common to all sessions and queries)
* and copied part (which can be its own for each session or query).
2012-08-02 17:33:31 +00:00
*
2017-06-02 21:37:28 +00:00
* Everything is encapsulated for all sorts of checks and locks.
2012-08-02 17:33:31 +00:00
*/
class Context: public std::enable_shared_from_this<Context>
2012-08-02 17:33:31 +00:00
{
private:
ContextSharedPart * shared;
2012-06-18 07:49:19 +00:00
ClientInfo client_info;
ExternalTablesInitializer external_tables_initializer_callback;
2019-05-28 18:30:10 +00:00
InputInitializer input_initializer_callback;
InputBlocksReader input_blocks_reader;
2020-02-12 03:03:33 +00:00
std::optional<UUID> user_id;
std::shared_ptr<std::vector<UUID>> current_roles;
std::shared_ptr<const SettingsConstraintsAndProfileIDs> settings_constraints_and_current_profiles;
std::shared_ptr<const ContextAccess> access;
std::shared_ptr<const EnabledRowPolicies> row_policies_of_initial_user;
String current_database;
2021-04-26 13:34:44 +00:00
Settings settings; /// Setting for query execution.
using ProgressCallback = std::function<void(const Progress & progress)>;
2021-04-26 13:34:44 +00:00
ProgressCallback progress_callback; /// Callback for tracking progress of query execution.
using FileProgressCallback = std::function<void(const FileProgress & progress)>;
FileProgressCallback file_progress_callback; /// Callback for tracking progress of file loading.
2021-04-26 13:34:44 +00:00
QueryStatus * process_list_elem = nullptr; /// For tracking total resource usage for query.
2020-03-02 20:23:58 +00:00
StorageID insertion_table = StorageID::createEmpty(); /// Saved insertion table in query context
2021-07-30 09:23:49 +00:00
bool is_distributed = false; /// Whether the current context it used for distributed query
String default_format; /// Format, used when server formats data by itself and if query does not have FORMAT specification.
/// Thus, used in HTTP interface. If not specified - then some globally default format is used.
TemporaryTablesMapping external_tables_mapping;
2019-10-19 20:36:35 +00:00
Scalars scalars;
Scalars local_scalars;
2020-02-21 15:22:28 +00:00
/// Used in s3Cluster table function. With this callback, a worker node could ask an initiator
/// about next file to read from s3.
2021-04-08 19:00:39 +00:00
std::optional<ReadTaskCallback> next_task_callback;
/// Used in parallel reading from replicas. A replica tells about its intentions to read
/// some ranges from some part and initiator will tell the replica about whether it is accepted or denied.
std::optional<MergeTreeReadTaskCallback> merge_tree_read_task_callback;
2020-12-13 22:36:04 +00:00
/// Record entities accessed by current query, and store this information in system.query_log.
struct QueryAccessInfo
{
2021-01-20 12:30:41 +00:00
QueryAccessInfo() = default;
QueryAccessInfo(const QueryAccessInfo & rhs)
{
std::lock_guard<std::mutex> lock(rhs.mutex);
databases = rhs.databases;
tables = rhs.tables;
columns = rhs.columns;
projections = rhs.projections;
views = rhs.views;
2021-01-20 12:30:41 +00:00
}
QueryAccessInfo(QueryAccessInfo && rhs) = delete;
QueryAccessInfo & operator=(QueryAccessInfo rhs)
{
swap(rhs);
return *this;
}
2021-01-20 12:37:53 +00:00
void swap(QueryAccessInfo & rhs)
2021-01-20 12:30:41 +00:00
{
std::swap(databases, rhs.databases);
std::swap(tables, rhs.tables);
std::swap(columns, rhs.columns);
std::swap(projections, rhs.projections);
std::swap(views, rhs.views);
2021-01-20 12:30:41 +00:00
}
/// To prevent a race between copy-constructor and other uses of this structure.
mutable std::mutex mutex{};
std::set<std::string> databases{};
std::set<std::string> tables{};
std::set<std::string> columns{};
std::set<std::string> projections{};
std::set<std::string> views{};
};
QueryAccessInfo query_access_info;
2021-01-21 21:15:11 +00:00
/// Record names of created objects of factories (for testing, etc)
struct QueryFactoriesInfo
{
2021-01-21 21:15:11 +00:00
std::unordered_set<std::string> aggregate_functions;
std::unordered_set<std::string> aggregate_function_combinators;
std::unordered_set<std::string> database_engines;
std::unordered_set<std::string> data_type_families;
2021-01-21 21:15:11 +00:00
std::unordered_set<std::string> dictionaries;
std::unordered_set<std::string> formats;
std::unordered_set<std::string> functions;
std::unordered_set<std::string> storages;
std::unordered_set<std::string> table_functions;
};
2021-01-21 21:15:11 +00:00
/// Needs to be chandged while having const context in factories methods
mutable QueryFactoriesInfo query_factories_info;
/// TODO: maybe replace with temporary tables?
2019-08-04 00:29:32 +00:00
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.
2020-02-21 15:22:28 +00:00
2021-05-31 14:49:02 +00:00
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.
2021-05-31 14:49:02 +00:00
ContextMutablePtr buffer_context; /// Buffer context. Could be equal to this.
2012-06-18 07:49:19 +00:00
2021-09-24 23:10:10 +00:00
/// A flag, used to distinguish between user query and internal query to a database engine (MaterializedPostgreSQL).
bool is_internal_query = false;
2020-11-18 17:43:18 +00:00
public:
// Top-level OpenTelemetry trace context for the query. Makes sense only for a query context.
2020-11-18 17:43:18 +00:00
OpenTelemetryTraceContext query_trace_context;
private:
2018-07-05 23:36:09 +00:00
using SampleBlockCache = std::unordered_map<std::string, Block>;
mutable SampleBlockCache sample_block_cache;
2021-02-05 10:31:46 +00:00
PartUUIDsPtr part_uuids; /// set of parts' uuids, is used for query parts deduplication
PartUUIDsPtr ignored_part_uuids; /// set of parts' uuids are meant to be excluded from query processing
2019-06-15 17:52:53 +00:00
NameToNameMap query_parameters; /// Dictionary with query parameters for prepared statements.
2019-05-18 21:07:23 +00:00
/// (key=name, value)
IHostContextPtr host_context; /// Arbitrary object that may used to attach some host specific information to query context,
/// when using ClickHouse as a library in some project. For example, it may contain host
/// logger, some query identification information, profiling guards, etc. This field is
/// to be customized in HTTP and TCP servers by overloading the customizeContext(DB::ContextPtr)
/// methods.
2021-02-19 23:41:58 +00:00
ZooKeeperMetadataTransactionPtr metadata_transaction; /// Distributed DDL context. I'm not sure if it's a suitable place for this,
2020-11-20 16:06:27 +00:00
/// but it's the easiest way to pass this through the whole stack from executeQuery(...)
/// to DatabaseOnDisk::commitCreateTable(...) or IStorage::alter(...) without changing
/// thousands of signatures.
/// And I hope it will be replaced with more common Transaction sometime.
Context();
Context(const Context &);
Context & operator=(const Context &);
public:
/// Create initial Context with ContextShared and etc.
2021-05-31 14:49:02 +00:00
static ContextMutablePtr createGlobal(ContextSharedPart * shared);
2021-06-01 12:20:52 +00:00
static ContextMutablePtr createCopy(const ContextWeakPtr & other);
2021-05-31 14:49:02 +00:00
static ContextMutablePtr createCopy(const ContextMutablePtr & other);
static ContextMutablePtr createCopy(const ContextPtr & other);
2020-04-16 14:51:33 +00:00
static SharedContextHolder createShared();
~Context();
2015-04-16 06:12:35 +00:00
String getPath() const;
String getFlagsPath() const;
String getUserFilesPath() const;
String getDictionariesLibPath() const;
2021-08-25 19:30:22 +00:00
String getUserScriptsPath() const;
2021-07-13 00:59:52 +00:00
/// A list of warnings about server configuration to place in `system.warnings` table.
Strings getWarnings() const;
2020-07-08 14:25:23 +00:00
VolumePtr getTemporaryVolume() const;
void setPath(const String & path);
void setFlagsPath(const String & path);
void setUserFilesPath(const String & path);
void setDictionariesLibPath(const String & path);
2021-08-25 19:30:22 +00:00
void setUserScriptsPath(const String & path);
2021-07-12 12:21:18 +00:00
2021-07-09 14:40:32 +00:00
void addWarningMessage(const String & msg);
2020-07-08 14:25:23 +00:00
VolumePtr setTemporaryStorage(const String & path, const String & policy_name = "");
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
2015-04-16 06:12:35 +00:00
/// Global application configuration settings.
void setConfig(const ConfigurationPtr & config);
const Poco::Util::AbstractConfiguration & getConfigRef() const;
AccessControl & getAccessControl();
const AccessControl & getAccessControl() const;
2020-02-12 03:03:33 +00:00
/// Sets external authenticators config (LDAP, Kerberos).
void setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config);
/// Creates GSSAcceptorContext instance based on external authenticator params.
std::unique_ptr<GSSAcceptorContext> makeGSSAcceptorContext() const;
2020-02-12 03:03:33 +00:00
/** Take the list of users, quotas and configuration profiles from this config.
* The list of users is completely replaced.
* The accumulated quota values are not reset if the quota is not deleted.
*/
void setUsersConfig(const ConfigurationPtr & config);
ConfigurationPtr getUsersConfig();
2021-08-01 14:12:34 +00:00
/// Sets the current user assuming that he/she is already authenticated.
2021-08-12 15:16:55 +00:00
/// WARNING: This function doesn't check password!
2021-08-01 14:12:34 +00:00
void setUser(const UUID & user_id_);
2020-02-12 03:03:33 +00:00
UserPtr getUser() const;
String getUserName() const;
std::optional<UUID> getUserID() const;
2020-01-12 21:00:55 +00:00
2021-08-01 14:12:34 +00:00
void setQuotaKey(String quota_key_);
void setCurrentRoles(const std::vector<UUID> & current_roles_);
2020-02-21 00:17:07 +00:00
void setCurrentRolesDefault();
boost::container::flat_set<UUID> getCurrentRoles() const;
boost::container::flat_set<UUID> getEnabledRoles() const;
std::shared_ptr<const EnabledRolesInfo> getRolesInfo() const;
2020-01-12 21:00:55 +00:00
void setCurrentProfile(const String & profile_name);
void setCurrentProfile(const UUID & profile_id);
std::vector<UUID> getCurrentProfiles() const;
std::vector<UUID> getEnabledProfiles() const;
2020-01-12 21:00:55 +00:00
/// Checks access rights.
/// Empty database means the current database.
void checkAccess(const AccessFlags & flags) const;
void checkAccess(const AccessFlags & flags, const std::string_view & database) const;
void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const;
void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const;
void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const;
void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const;
void checkAccess(const AccessFlags & flags, const StorageID & table_id) const;
void checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::string_view & column) const;
void checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::vector<std::string_view> & columns) const;
void checkAccess(const AccessFlags & flags, const StorageID & table_id, const Strings & columns) const;
void checkAccess(const AccessRightsElement & element) const;
void checkAccess(const AccessRightsElements & elements) const;
std::shared_ptr<const ContextAccess> getAccess() const;
ASTPtr getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const;
/// Finds and sets extra row policies to be used based on `client_info.initial_user`,
/// if the initial user exists.
/// TODO: we need a better solution here. It seems we should pass the initial row policy
/// because a shard is allowed to not have the initial user or it might be another user
/// with the same name.
void enableRowPoliciesOfInitialUser();
std::shared_ptr<const EnabledQuota> getQuota() const;
std::optional<QuotaUsage> getQuotaUsage() const;
/// We have to copy external tables inside executeQuery() to track limits. Therefore, set callback for it. Must set once.
void setExternalTablesInitializer(ExternalTablesInitializer && initializer);
/// This method is called in executeQuery() and will call the external tables initializer.
void initializeExternalTablesIfSet();
2019-05-28 18:30:10 +00:00
/// When input() is present we have to send columns structure to client
void setInputInitializer(InputInitializer && initializer);
/// This method is called in StorageInput::read while executing query
void initializeInput(const StoragePtr & input_storage);
/// Callback for read data blocks from client one by one for function input()
void setInputBlocksReaderCallback(InputBlocksReader && reader);
/// Get callback for reading data for input()
InputBlocksReader getInputBlocksReaderCallback() const;
void resetInputCallbacks();
ClientInfo & getClientInfo() { return client_info; }
const ClientInfo & getClientInfo() const { return client_info; }
2020-02-10 18:19:35 +00:00
enum StorageNamespace
{
2020-02-12 18:14:12 +00:00
ResolveGlobal = 1u, /// Database name must be specified
ResolveCurrentDatabase = 2u, /// Use current database
ResolveOrdinary = ResolveGlobal | ResolveCurrentDatabase, /// If database name is not specified, use current database
ResolveExternal = 4u, /// Try get external table
ResolveAll = ResolveExternal | ResolveOrdinary /// If database name is not specified, try get external table,
/// if external table not found use current database.
2020-02-10 18:19:35 +00:00
};
2020-02-03 12:54:36 +00:00
String resolveDatabase(const String & database_name) const;
2020-02-12 18:14:12 +00:00
StorageID resolveStorageID(StorageID storage_id, StorageNamespace where = StorageNamespace::ResolveAll) const;
StorageID tryResolveStorageID(StorageID storage_id, StorageNamespace where = StorageNamespace::ResolveAll) const;
StorageID resolveStorageIDImpl(StorageID storage_id, StorageNamespace where, std::optional<Exception> * exception) const;
2012-08-02 17:33:31 +00:00
2020-03-10 19:36:17 +00:00
Tables getExternalTables() const;
void addExternalTable(const String & table_name, TemporaryTableHolder && temporary_table);
2020-03-13 15:41:36 +00:00
std::shared_ptr<TemporaryTableHolder> removeExternalTable(const String & table_name);
2020-03-10 19:36:17 +00:00
2019-10-19 20:36:35 +00:00
const Scalars & getScalars() const;
const Block & getScalar(const String & name) const;
void addScalar(const String & name, const Block & block);
bool hasScalar(const String & name) const;
const Block * tryGetLocalScalar(const String & name) const;
void addLocalScalar(const String & name, const Block & block);
const QueryAccessInfo & getQueryAccessInfo() const { return query_access_info; }
void addQueryAccessInfo(
const String & quoted_database_name,
const String & full_quoted_table_name,
const Names & column_names,
const String & projection_name = {},
const String & view_name = {});
2021-08-27 21:29:10 +00:00
2021-01-21 21:15:11 +00:00
/// Supported factories for records in query_log
enum class QueryLogFactories
{
AggregateFunction,
AggregateFunctionCombinator,
2021-01-21 21:15:11 +00:00
Database,
DataType,
Dictionary,
Format,
Function,
Storage,
TableFunction
};
const QueryFactoriesInfo & getQueryFactoriesInfo() const { return query_factories_info; }
2021-01-21 21:15:11 +00:00
void addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const;
StoragePtr executeTableFunction(const ASTPtr & table_expression);
void addViewSource(const StoragePtr & storage);
2021-05-31 14:49:02 +00:00
StoragePtr getViewSource() const;
String getCurrentDatabase() const;
2021-01-29 15:11:44 +00:00
String getCurrentQueryId() const { return client_info.current_query_id; }
/// Id of initiating query for distributed queries; or current query id if it's not a distributed query.
String getInitialQueryId() const;
void setCurrentDatabase(const String & name);
2020-09-25 11:27:00 +00:00
/// Set current_database for global context. We don't validate that database
/// exists because it should be set before databases loading.
void setCurrentDatabaseNameInGlobalContext(const String & name);
void setCurrentQueryId(const String & query_id);
2012-08-02 17:33:31 +00:00
void killCurrentQuery();
2020-03-02 20:23:58 +00:00
void setInsertionTable(StorageID db_and_table) { insertion_table = std::move(db_and_table); }
const StorageID & getInsertionTable() const { return insertion_table; }
2021-07-30 09:23:49 +00:00
void setDistributed(bool is_distributed_) { is_distributed = is_distributed_; }
bool isDistributed() const { return is_distributed; }
2017-06-02 21:37:28 +00:00
String getDefaultFormat() const; /// If default_format is not specified, some global default format is returned.
void setDefaultFormat(const String & name);
2018-03-13 23:44:23 +00:00
MultiVersion<Macros>::Version getMacros() const;
void setMacros(std::unique_ptr<Macros> && macros);
2014-08-11 15:59:01 +00:00
Settings getSettings() const;
void setSettings(const Settings & settings_);
2012-08-02 19:03:32 +00:00
/// Set settings by name.
void setSetting(const StringRef & name, const String & value);
void setSetting(const StringRef & name, const Field & value);
void applySettingChange(const SettingChange & change);
void applySettingsChanges(const SettingsChanges & changes);
2014-02-14 17:38:33 +00:00
/// Checks the constraints.
void checkSettingsConstraints(const SettingChange & change) const;
void checkSettingsConstraints(const SettingsChanges & changes) const;
void checkSettingsConstraints(SettingsChanges & changes) const;
void clampToSettingsConstraints(SettingsChanges & changes) const;
2019-10-31 08:45:14 +00:00
/// Returns the current constraints (can return null).
std::shared_ptr<const SettingsConstraintsAndProfileIDs> getSettingsConstraintsAndCurrentProfiles() const;
const EmbeddedDictionaries & getEmbeddedDictionaries() const;
const ExternalDictionariesLoader & getExternalDictionariesLoader() const;
const ExternalModelsLoader & getExternalModelsLoader() const;
const ExternalUserDefinedExecutableFunctionsLoader & getExternalUserDefinedExecutableFunctionsLoader() const;
EmbeddedDictionaries & getEmbeddedDictionaries();
ExternalDictionariesLoader & getExternalDictionariesLoader();
ExternalDictionariesLoader & getExternalDictionariesLoaderUnlocked();
ExternalUserDefinedExecutableFunctionsLoader & getExternalUserDefinedExecutableFunctionsLoader();
ExternalUserDefinedExecutableFunctionsLoader & getExternalUserDefinedExecutableFunctionsLoaderUnlocked();
ExternalModelsLoader & getExternalModelsLoader();
2021-04-12 09:28:46 +00:00
ExternalModelsLoader & getExternalModelsLoaderUnlocked();
void tryCreateEmbeddedDictionaries(const Poco::Util::AbstractConfiguration & config) const;
void loadOrReloadDictionaries(const Poco::Util::AbstractConfiguration & config);
void loadOrReloadUserDefinedExecutableFunctions(const Poco::Util::AbstractConfiguration & config);
void loadOrReloadModels(const Poco::Util::AbstractConfiguration & config);
#if USE_NLP
2021-06-04 00:33:41 +00:00
SynonymsExtensions & getSynonymsExtensions() const;
2021-06-05 00:52:35 +00:00
Lemmatizers & getLemmatizers() const;
#endif
2017-06-02 21:37:28 +00:00
/// I/O formats.
2021-10-11 16:11:50 +00:00
InputFormatPtr getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional<FormatSettings> & format_settings = std::nullopt) const;
2021-10-11 16:11:50 +00:00
OutputFormatPtr getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample) const;
2020-12-30 03:07:30 +00:00
OutputFormatPtr getOutputFormatParallelIfPossible(const String & name, WriteBuffer & buf, const Block & sample) const;
InterserverIOHandler & getInterserverIOHandler();
2014-03-21 13:42:14 +00:00
2017-06-02 21:37:28 +00:00
/// How other servers can access this for downloading replicated data.
void setInterserverIOAddress(const String & host, UInt16 port);
std::pair<String, UInt16> getInterserverIOAddress() const;
/// Credentials which server will use to communicate with others
void updateInterserverCredentials(const Poco::Util::AbstractConfiguration & config);
2021-04-07 13:52:11 +00:00
InterserverCredentialsPtr getInterserverCredentials();
/// Interserver requests scheme (http or https)
void setInterserverScheme(const String & scheme);
String getInterserverScheme() const;
/// Storage of allowed hosts from config.xml
void setRemoteHostFilter(const Poco::Util::AbstractConfiguration & config);
2019-10-21 14:36:24 +00:00
const RemoteHostFilter & getRemoteHostFilter() const;
2017-06-02 21:37:28 +00:00
/// The port that the server listens for executing SQL queries.
UInt16 getTCPPort() const;
std::optional<UInt16> getTCPPortSecure() const;
2021-08-20 04:11:47 +00:00
/// Register server ports during server starting up. No lock is held.
void registerServerPort(String port_name, UInt16 port);
2021-08-20 04:11:47 +00:00
UInt16 getServerPort(const String & port_name) const;
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
2020-02-13 21:00:03 +00:00
/// For methods below you may need to acquire the context lock by yourself.
2012-08-02 17:33:31 +00:00
2021-05-31 14:49:02 +00:00
ContextMutablePtr getQueryContext() const;
bool hasQueryContext() const { return !query_context.expired(); }
bool isInternalSubquery() const;
2021-05-31 14:49:02 +00:00
ContextMutablePtr getSessionContext() const;
bool hasSessionContext() const { return !session_context.expired(); }
2016-11-16 11:29:51 +00:00
2021-05-31 14:49:02 +00:00
ContextMutablePtr getGlobalContext() const;
bool hasGlobalContext() const { return !global_context.expired(); }
bool isGlobalContext() const
{
auto ptr = global_context.lock();
return ptr && ptr.get() == this;
}
2012-08-02 17:33:31 +00:00
2021-05-31 14:49:02 +00:00
ContextMutablePtr getBufferContext() const;
2021-05-31 14:49:02 +00:00
void setQueryContext(ContextMutablePtr context_) { query_context = context_; }
void setSessionContext(ContextMutablePtr context_) { session_context = context_; }
2019-07-08 00:51:43 +00:00
void makeQueryContext() { query_context = shared_from_this(); }
void makeSessionContext() { session_context = shared_from_this(); }
void makeGlobalContext() { initGlobal(); global_context = shared_from_this(); }
2012-08-02 17:33:31 +00:00
const Settings & getSettingsRef() const { return settings; }
void setProgressCallback(ProgressCallback callback);
2021-10-19 09:58:10 +00:00
/// Used in executeQuery() to pass it to the QueryPipeline.
ProgressCallback getProgressCallback() const;
void setFileProgressCallback(FileProgressCallback && callback) { file_progress_callback = callback; }
FileProgressCallback getFileProgressCallback() const { return file_progress_callback; }
2021-04-26 13:34:44 +00:00
2021-10-19 09:58:10 +00:00
/** Set in executeQuery and InterpreterSelectQuery. Then it is used in QueryPipeline,
2017-06-02 21:37:28 +00:00
* to update and monitor information about the total number of resources spent for the query.
*/
void setProcessListElement(QueryStatus * elem);
2017-06-02 21:37:28 +00:00
/// Can return nullptr if the query was not inserted into the ProcessList.
QueryStatus * getProcessListElement() const;
2017-06-02 21:37:28 +00:00
/// List all queries.
ProcessList & getProcessList();
const ProcessList & getProcessList() const;
MergeList & getMergeList();
const MergeList & getMergeList() const;
2014-09-10 11:34:26 +00:00
2020-10-26 16:38:35 +00:00
ReplicatedFetchList & getReplicatedFetchList();
const ReplicatedFetchList & getReplicatedFetchList() const;
2017-06-02 21:37:28 +00:00
/// If the current session is expired at the time of the call, synchronously creates and returns a new session with the startNewSession() call.
/// If no ZooKeeper configured, throws an exception.
std::shared_ptr<zkutil::ZooKeeper> getZooKeeper() const;
/// Same as above but return a zookeeper connection from auxiliary_zookeepers configuration entry.
std::shared_ptr<zkutil::ZooKeeper> getAuxiliaryZooKeeper(const String & name) const;
2021-10-18 09:13:24 +00:00
/// Try to connect to Keeper using get(Auxiliary)ZooKeeper. Useful for
/// internal Keeper start (check connection to some other node). Return true
/// if connected successfully (without exception) or our zookeeper client
/// connection configured for some other cluster without our node.
bool tryCheckClientConnectionToMyKeeperCluster() const;
2021-09-13 12:19:37 +00:00
UInt32 getZooKeeperSessionUptime() const;
2021-02-01 13:18:17 +00:00
#if USE_NURAFT
std::shared_ptr<KeeperDispatcher> & getKeeperDispatcher() const;
2021-02-01 13:18:17 +00:00
#endif
void initializeKeeperDispatcher(bool start_async) const;
void shutdownKeeperDispatcher() const;
2021-10-18 15:27:51 +00:00
void updateKeeperConfiguration(const Poco::Util::AbstractConfiguration & config);
/// Set auxiliary zookeepers configuration at server starting or configuration reloading.
void reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config);
/// Has ready or expired ZooKeeper
bool hasZooKeeper() const;
2020-11-16 08:27:33 +00:00
/// Has ready or expired auxiliary ZooKeeper
bool hasAuxiliaryZooKeeper(const String & name) const;
/// Reset current zookeeper session. Do not create a new one.
void resetZooKeeper() const;
// Reload Zookeeper
void reloadZooKeeperIfChanged(const ConfigurationPtr & config) const;
2013-12-07 16:51:29 +00:00
void setSystemZooKeeperLogAfterInitializationIfNeeded();
/// Create a cache of uncompressed blocks of specified size. This can be done only once.
void setUncompressedCache(size_t max_size_in_bytes);
std::shared_ptr<UncompressedCache> getUncompressedCache() const;
void dropUncompressedCache() const;
2017-06-02 21:37:28 +00:00
/// Create a cache of marks of specified size. This can be done only once.
void setMarkCache(size_t cache_size_in_bytes);
std::shared_ptr<MarkCache> getMarkCache() const;
void dropMarkCache() const;
/// Create a cache of index uncompressed blocks of specified size. This can be done only once.
void setIndexUncompressedCache(size_t max_size_in_bytes);
std::shared_ptr<UncompressedCache> getIndexUncompressedCache() const;
void dropIndexUncompressedCache() const;
/// Create a cache of index marks of specified size. This can be done only once.
void setIndexMarkCache(size_t cache_size_in_bytes);
std::shared_ptr<MarkCache> getIndexMarkCache() const;
void dropIndexMarkCache() const;
/// Create a cache of mapped files to avoid frequent open/map/unmap/close and to reuse from several threads.
2021-03-28 19:24:28 +00:00
void setMMappedFileCache(size_t cache_size_in_num_entries);
std::shared_ptr<MMappedFileCache> getMMappedFileCache() const;
void dropMMappedFileCache() const;
/** Clear the caches of the uncompressed blocks and marks.
* This is usually done when renaming tables, changing the type of columns, deleting a table.
* - since caches are linked to file names, and become incorrect.
* (when deleting a table - it is necessary, since in its place another can appear)
* const - because the change in the cache is not considered significant.
*/
void dropCaches() const;
2014-02-11 13:30:42 +00:00
2020-10-16 10:12:31 +00:00
/// Settings for MergeTree background tasks stored in config.xml
BackgroundTaskSchedulingSettings getBackgroundProcessingTaskSchedulingSettings() const;
BackgroundTaskSchedulingSettings getBackgroundMoveTaskSchedulingSettings() const;
2020-11-06 14:07:56 +00:00
BackgroundSchedulePool & getBufferFlushSchedulePool() const;
BackgroundSchedulePool & getSchedulePool() const;
BackgroundSchedulePool & getMessageBrokerSchedulePool() const;
2020-11-06 14:07:56 +00:00
BackgroundSchedulePool & getDistributedSchedulePool() const;
2014-07-02 12:30:38 +00:00
2021-05-26 20:37:44 +00:00
ThrottlerPtr getReplicatedFetchesThrottler() const;
ThrottlerPtr getReplicatedSendsThrottler() const;
2020-11-12 20:11:04 +00:00
/// Has distributed_ddl configuration or not.
2020-11-06 18:28:59 +00:00
bool hasDistributedDDL() const;
void setDDLWorker(std::unique_ptr<DDLWorker> ddl_worker);
DDLWorker & getDDLWorker() const;
2021-06-26 14:15:57 +00:00
std::shared_ptr<Clusters> getClusters() const;
std::shared_ptr<Cluster> getCluster(const std::string & cluster_name) const;
std::shared_ptr<Cluster> tryGetCluster(const std::string & cluster_name) const;
void setClustersConfig(const ConfigurationPtr & config, bool enable_discovery = false, const String & config_name = "remote_servers");
2021-11-11 09:03:53 +00:00
void registerNodeForClusterDiscovery();
/// Sets custom cluster, but doesn't update configuration
void setCluster(const String & cluster_name, const std::shared_ptr<Cluster> & cluster);
2021-05-31 14:49:02 +00:00
void reloadClusterConfig() const;
Compiler & getCompiler();
/// Call after initialization before using system logs. Call for global context.
void initializeSystemLogs();
/// Call after initialization before using trace collector.
2019-02-03 21:30:45 +00:00
void initializeTraceCollector();
bool hasTraceCollector() const;
2019-02-03 21:30:45 +00:00
/// Nullptr if the query log is not ready for this moment.
2021-05-31 14:49:02 +00:00
std::shared_ptr<QueryLog> getQueryLog() const;
std::shared_ptr<QueryThreadLog> getQueryThreadLog() const;
2021-06-18 13:44:08 +00:00
std::shared_ptr<QueryViewsLog> getQueryViewsLog() const;
2021-05-31 14:49:02 +00:00
std::shared_ptr<TraceLog> getTraceLog() const;
std::shared_ptr<TextLog> getTextLog() const;
std::shared_ptr<MetricLog> getMetricLog() const;
std::shared_ptr<AsynchronousMetricLog> getAsynchronousMetricLog() const;
2021-05-31 14:49:02 +00:00
std::shared_ptr<OpenTelemetrySpanLog> getOpenTelemetrySpanLog() const;
2021-07-09 14:05:35 +00:00
std::shared_ptr<ZooKeeperLog> getZooKeeperLog() const;
std::shared_ptr<SessionLog> getSessionLog() const;
2020-08-08 01:01:47 +00:00
/// Returns an object used to log operations with parts if it possible.
/// Provide table name to make required checks.
2021-05-31 14:49:02 +00:00
std::shared_ptr<PartLog> getPartLog(const String & part_database) const;
const MergeTreeSettings & getMergeTreeSettings() const;
2020-08-10 11:23:58 +00:00
const MergeTreeSettings & getReplicatedMergeTreeSettings() const;
const StorageS3Settings & getStorageS3Settings() const;
/// Prevents DROP TABLE if its size is greater than max_size (50GB by default, max_size=0 turn off this check)
void setMaxTableSizeToDrop(size_t max_size);
void checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const;
/// Prevents DROP PARTITION if its size is greater than max_size (50GB by default, max_size=0 turn off this check)
void setMaxPartitionSizeToDrop(size_t max_size);
void checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const;
2018-12-21 12:17:30 +00:00
/// Lets you select the compression codec according to the conditions described in the configuration file.
std::shared_ptr<ICompressionCodec> chooseCompressionCodec(size_t part_size, double part_size_ratio) const;
/// Provides storage disks
DiskPtr getDisk(const String & name) const;
2020-05-22 10:33:57 +00:00
StoragePoliciesMap getPoliciesMap() const;
DisksMap getDisksMap() const;
void updateStorageConfiguration(const Poco::Util::AbstractConfiguration & config);
2021-10-18 15:27:51 +00:00
2019-04-05 17:37:27 +00:00
/// Provides storage politics schemes
StoragePolicyPtr getStoragePolicy(const String & name) const;
2017-06-02 21:37:28 +00:00
/// Get the server uptime in seconds.
time_t getUptimeSeconds() const;
2016-04-09 07:47:08 +00:00
using ConfigReloadCallback = std::function<void()>;
void setConfigReloadCallback(ConfigReloadCallback && callback);
void reloadConfig() const;
void shutdown();
bool isInternalQuery() const { return is_internal_query; }
void setInternalQuery(bool internal) { is_internal_query = internal; }
ActionLocksManagerPtr getActionLocksManager();
enum class ApplicationType
{
SERVER, /// The program is run as clickhouse-server daemon (default behavior)
CLIENT, /// clickhouse-client
2021-05-18 14:08:56 +00:00
LOCAL, /// clickhouse-local
KEEPER, /// clickhouse-keeper (also daemon)
};
ApplicationType getApplicationType() const;
void setApplicationType(ApplicationType type);
/// Sets default_profile and system_profile, must be called once during the initialization
void setDefaultProfiles(const Poco::Util::AbstractConfiguration & config);
String getDefaultProfileName() const;
String getSystemProfileName() const;
/// Base path for format schemas
String getFormatSchemaPath() const;
void setFormatSchemaPath(const String & path);
2018-07-05 23:36:09 +00:00
SampleBlockCache & getSampleBlockCache() const;
2019-05-18 21:07:23 +00:00
/// Query parameters for prepared statements.
2019-05-25 13:43:52 +00:00
bool hasQueryParameters() const;
2019-06-15 17:52:53 +00:00
const NameToNameMap & getQueryParameters() const;
void setQueryParameter(const String & name, const String & value);
2019-10-02 19:54:40 +00:00
void setQueryParameters(const NameToNameMap & parameters) { query_parameters = parameters; }
2019-05-18 21:07:23 +00:00
/// Add started bridge command. It will be killed after context destruction
2021-03-17 08:21:04 +00:00
void addBridgeCommand(std::unique_ptr<ShellCommand> cmd) const;
IHostContextPtr & getHostContext();
const IHostContextPtr & getHostContext() const;
2021-02-19 23:41:58 +00:00
/// Initialize context of distributed DDL query with Replicated database.
void initZooKeeperMetadataTransaction(ZooKeeperMetadataTransactionPtr txn, bool attach_existing = false);
/// Returns context of current distributed DDL query or nullptr.
ZooKeeperMetadataTransactionPtr getZooKeeperMetadataTransaction() const;
2021-07-02 16:39:55 +00:00
/// Removes context of current distributed DDL.
void resetZooKeeperMetadataTransaction();
2020-11-20 16:06:27 +00:00
2021-05-31 14:49:02 +00:00
PartUUIDsPtr getPartUUIDs() const;
PartUUIDsPtr getIgnoredPartUUIDs() const;
AsynchronousInsertQueue * getAsynchronousInsertQueue() const;
2021-04-19 19:16:34 +00:00
void setAsynchronousInsertQueue(const std::shared_ptr<AsynchronousInsertQueue> & ptr);
2021-03-17 14:11:47 +00:00
2021-04-08 19:00:39 +00:00
ReadTaskCallback getReadTaskCallback() const;
void setReadTaskCallback(ReadTaskCallback && callback);
MergeTreeReadTaskCallback getMergeTreeReadTaskCallback() const;
void setMergeTreeReadTaskCallback(MergeTreeReadTaskCallback && callback);
2021-08-30 19:37:03 +00:00
/// Background executors related methods
2021-10-19 08:19:43 +00:00
void initializeBackgroundExecutorsIfNeeded();
2021-08-30 19:37:03 +00:00
MergeMutateBackgroundExecutorPtr getMergeMutateExecutor() const;
OrdinaryBackgroundExecutorPtr getMovesExecutor() const;
OrdinaryBackgroundExecutorPtr getFetchesExecutor() const;
OrdinaryBackgroundExecutorPtr getCommonExecutor() const;
2021-08-30 19:37:03 +00:00
2021-08-24 21:45:58 +00:00
/** Get settings for reading from filesystem. */
ReadSettings getReadSettings() const;
private:
2020-02-13 21:00:03 +00:00
std::unique_lock<std::recursive_mutex> getLock() const;
void initGlobal();
2020-01-12 21:00:55 +00:00
/// Compute and set actual user settings, client_info.current_user should be set
void calculateAccessRights();
template <typename... Args>
void checkAccessImpl(const Args &... args) const;
EmbeddedDictionaries & getEmbeddedDictionariesImpl(bool throw_on_error) const;
void checkCanBeDropped(const String & database, const String & table, const size_t & size, const size_t & max_size_to_drop) const;
StoragePolicySelectorPtr getStoragePolicySelector(std::lock_guard<std::mutex> & lock) const;
DiskSelectorPtr getDiskSelector(std::lock_guard<std::mutex> & /* lock */) const;
};
2011-08-09 19:19:25 +00:00
}