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>
|
2019-05-17 14:34:25 +00:00
|
|
|
#include <Core/Settings.h>
|
2020-09-15 09:55:57 +00:00
|
|
|
#include <common/types.h>
|
2020-02-04 22:22:27 +00:00
|
|
|
#include <Core/UUID.h>
|
2019-05-17 14:34:25 +00:00
|
|
|
#include <DataStreams/IBlockStream_fwd.h>
|
2019-03-11 14:01:45 +00:00
|
|
|
#include <Interpreters/ClientInfo.h>
|
2020-02-10 13:10:17 +00:00
|
|
|
#include <Interpreters/DatabaseCatalog.h>
|
2019-03-11 14:01:45 +00:00
|
|
|
#include <Parsers/IAST_fwd.h>
|
2020-03-07 17:37:38 +00:00
|
|
|
#include <Access/RowPolicy.h>
|
2019-03-11 14:01:45 +00:00
|
|
|
#include <Common/MultiVersion.h>
|
|
|
|
#include <Common/ThreadPool.h>
|
2020-11-18 17:43:18 +00:00
|
|
|
#include <Common/OpenTelemetryTraceContext.h>
|
2019-05-17 14:34:25 +00:00
|
|
|
#include <Storages/IStorage_fwd.h>
|
2019-03-11 14:01:45 +00:00
|
|
|
#include <atomic>
|
2017-06-02 18:48:33 +00:00
|
|
|
#include <chrono>
|
|
|
|
#include <condition_variable>
|
2015-04-16 06:12:35 +00:00
|
|
|
#include <functional>
|
|
|
|
#include <memory>
|
2017-06-02 18:48:33 +00:00
|
|
|
#include <mutex>
|
2018-11-07 11:46:39 +00:00
|
|
|
#include <optional>
|
2019-03-11 14:01:45 +00:00
|
|
|
#include <thread>
|
2019-10-10 12:58:06 +00:00
|
|
|
#include <Common/RemoteHostFilter.h>
|
2015-04-16 06:12:35 +00:00
|
|
|
|
2020-04-16 12:31:57 +00:00
|
|
|
#if !defined(ARCADIA_BUILD)
|
|
|
|
# include "config_core.h"
|
|
|
|
#endif
|
|
|
|
|
2015-04-16 06:12:35 +00:00
|
|
|
|
2017-01-21 04:24:28 +00:00
|
|
|
namespace Poco
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
namespace Net
|
|
|
|
{
|
|
|
|
class IPAddress;
|
|
|
|
}
|
2017-01-21 04:24:28 +00:00
|
|
|
}
|
2011-08-09 19:19:25 +00:00
|
|
|
|
2015-04-16 07:22:29 +00:00
|
|
|
namespace zkutil
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
class ZooKeeper;
|
2015-04-16 07:22:29 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2011-08-09 19:19:25 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-03-07 05:05:42 +00:00
|
|
|
struct ContextShared;
|
2018-06-01 15:32:27 +00:00
|
|
|
class Context;
|
2020-03-07 17:37:38 +00:00
|
|
|
class ContextAccess;
|
2020-02-12 03:03:33 +00:00
|
|
|
struct User;
|
|
|
|
using UserPtr = std::shared_ptr<const User>;
|
2020-04-29 19:35:56 +00:00
|
|
|
struct EnabledRolesInfo;
|
2020-03-07 17:37:38 +00:00
|
|
|
class EnabledRowPolicies;
|
|
|
|
class EnabledQuota;
|
2020-05-08 12:50:45 +00:00
|
|
|
struct QuotaUsage;
|
2020-01-26 09:49:53 +00:00
|
|
|
class AccessFlags;
|
|
|
|
struct AccessRightsElement;
|
|
|
|
class AccessRightsElements;
|
2017-01-21 04:24:28 +00:00
|
|
|
class EmbeddedDictionaries;
|
2019-09-26 10:41:33 +00:00
|
|
|
class ExternalDictionariesLoader;
|
2019-09-26 10:23:14 +00:00
|
|
|
class ExternalModelsLoader;
|
2015-04-16 06:12:35 +00:00
|
|
|
class InterserverIOHandler;
|
2017-12-29 22:32:04 +00:00
|
|
|
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;
|
|
|
|
class UncompressedCache;
|
|
|
|
class ProcessList;
|
2018-05-14 16:09:00 +00:00
|
|
|
class QueryStatus;
|
2015-04-16 06:12:35 +00:00
|
|
|
class Macros;
|
2016-03-07 04:31:10 +00:00
|
|
|
struct Progress;
|
2015-04-30 12:43:16 +00:00
|
|
|
class Clusters;
|
2015-06-26 20:48:10 +00:00
|
|
|
class QueryLog;
|
2018-05-31 15:54:08 +00:00
|
|
|
class QueryThreadLog;
|
2017-03-07 17:13:54 +00:00
|
|
|
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;
|
2015-07-16 21:32:51 +00:00
|
|
|
struct MergeTreeSettings;
|
2020-06-01 17:16:09 +00:00
|
|
|
class StorageS3Settings;
|
2016-03-19 01:18:49 +00:00
|
|
|
class IDatabase;
|
2017-04-13 13:42:29 +00:00
|
|
|
class DDLWorker;
|
2018-03-01 01:49:36 +00:00
|
|
|
class ITableFunction;
|
2017-01-21 04:24:28 +00:00
|
|
|
class Block;
|
2018-05-21 13:49:54 +00:00
|
|
|
class ActionLocksManager;
|
|
|
|
using ActionLocksManagerPtr = std::shared_ptr<ActionLocksManager>;
|
2018-11-22 15:59:00 +00:00
|
|
|
class ShellCommand;
|
2018-12-21 12:17:30 +00:00
|
|
|
class ICompressionCodec;
|
2019-11-09 15:33:07 +00:00
|
|
|
class AccessControlManager;
|
2019-04-24 22:52:08 +00:00
|
|
|
class SettingsConstraints;
|
2019-10-10 12:58:06 +00:00
|
|
|
class RemoteHostFilter;
|
2019-12-05 11:42:13 +00:00
|
|
|
struct StorageID;
|
2019-12-12 08:57:25 +00:00
|
|
|
class IDisk;
|
|
|
|
using DiskPtr = std::shared_ptr<IDisk>;
|
|
|
|
class DiskSelector;
|
2020-01-09 14:50:34 +00:00
|
|
|
using DiskSelectorPtr = std::shared_ptr<const DiskSelector>;
|
2020-05-22 10:33:57 +00:00
|
|
|
using DisksMap = std::map<String, DiskPtr>;
|
2019-12-12 08:57:25 +00:00
|
|
|
class StoragePolicy;
|
|
|
|
using StoragePolicyPtr = std::shared_ptr<const StoragePolicy>;
|
2020-05-22 10:33:57 +00:00
|
|
|
using StoragePoliciesMap = std::map<String, StoragePolicyPtr>;
|
2019-12-12 08:57:25 +00:00
|
|
|
class StoragePolicySelector;
|
2020-01-09 14:50:34 +00:00
|
|
|
using StoragePolicySelectorPtr = std::shared_ptr<const StoragePolicySelector>;
|
2021-01-21 15:09:48 +00:00
|
|
|
class TestKeeperStorageDispatcher;
|
2019-04-24 22:52:08 +00:00
|
|
|
|
2019-04-05 10:52:07 +00:00
|
|
|
class IOutputFormat;
|
|
|
|
using OutputFormatPtr = std::shared_ptr<IOutputFormat>;
|
2020-07-08 14:25:23 +00:00
|
|
|
class IVolume;
|
|
|
|
using VolumePtr = std::shared_ptr<IVolume>;
|
2020-03-05 19:23:39 +00:00
|
|
|
struct NamedSession;
|
2020-10-16 10:12:31 +00:00
|
|
|
struct BackgroundTaskSchedulingSettings;
|
2020-01-19 14:26:28 +00:00
|
|
|
|
2018-08-30 16:31:20 +00:00
|
|
|
|
|
|
|
#if USE_EMBEDDED_COMPILER
|
2018-09-03 10:14:05 +00:00
|
|
|
class CompiledExpressionCache;
|
2018-08-30 16:31:20 +00:00
|
|
|
#endif
|
2011-08-19 18:31:14 +00:00
|
|
|
|
2018-06-01 15:32:27 +00:00
|
|
|
/// Callback for external tables initializer
|
|
|
|
using ExternalTablesInitializer = std::function<void(Context &)>;
|
|
|
|
|
2019-05-28 18:30:10 +00:00
|
|
|
/// Callback for initialize input()
|
|
|
|
using InputInitializer = std::function<void(Context &, const StoragePtr &)>;
|
|
|
|
/// Callback for reading blocks of data from client for function input()
|
|
|
|
using InputBlocksReader = std::function<Block(Context &)>;
|
|
|
|
|
2019-10-19 20:36:35 +00:00
|
|
|
/// Scalar results of sub queries
|
|
|
|
using Scalars = std::map<String, Block>;
|
|
|
|
|
2019-03-06 16:41:35 +00:00
|
|
|
/// 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>;
|
|
|
|
|
2020-04-17 09:47:40 +00:00
|
|
|
/// 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();
|
2020-04-17 09:47:40 +00:00
|
|
|
SharedContextHolder(std::unique_ptr<ContextShared> shared_context);
|
|
|
|
SharedContextHolder(SharedContextHolder &&) noexcept;
|
|
|
|
|
2020-04-19 15:46:12 +00:00
|
|
|
SharedContextHolder & operator=(SharedContextHolder &&);
|
|
|
|
|
2020-04-17 09:47:40 +00:00
|
|
|
ContextShared * get() const { return shared.get(); }
|
2020-04-17 12:58:52 +00:00
|
|
|
void reset();
|
2020-04-17 09:47:40 +00:00
|
|
|
private:
|
|
|
|
std::unique_ptr<ContextShared> 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
|
|
|
|
{
|
|
|
|
private:
|
2020-04-16 14:37:38 +00:00
|
|
|
ContextShared * shared;
|
2012-06-18 07:49:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
ClientInfo client_info;
|
2018-06-01 15:32:27 +00:00
|
|
|
ExternalTablesInitializer external_tables_initializer_callback;
|
2015-06-26 20:48:10 +00:00
|
|
|
|
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;
|
2020-04-29 19:35:56 +00:00
|
|
|
boost::container::flat_set<UUID> current_roles;
|
2020-02-21 00:17:07 +00:00
|
|
|
bool use_default_roles = false;
|
2020-03-07 17:37:38 +00:00
|
|
|
std::shared_ptr<const ContextAccess> access;
|
|
|
|
std::shared_ptr<const EnabledRowPolicies> initial_row_policy;
|
2017-04-01 07:20:54 +00:00
|
|
|
String current_database;
|
2017-06-05 13:59:38 +00:00
|
|
|
Settings settings; /// Setting for query execution.
|
2017-04-01 07:20:54 +00:00
|
|
|
using ProgressCallback = std::function<void(const Progress & progress)>;
|
2017-06-05 13:59:38 +00:00
|
|
|
ProgressCallback progress_callback; /// Callback for tracking progress of query execution.
|
2018-02-01 17:55:08 +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
|
2016-11-16 05:09:41 +00:00
|
|
|
|
2017-06-02 21:01:17 +00:00
|
|
|
String default_format; /// Format, used when server formats data by itself and if query does not have FORMAT specification.
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Thus, used in HTTP interface. If not specified - then some globally default format is used.
|
2020-01-30 19:00:51 +00:00
|
|
|
TemporaryTablesMapping external_tables_mapping;
|
2019-10-19 20:36:35 +00:00
|
|
|
Scalars scalars;
|
2020-02-21 15:22:28 +00:00
|
|
|
|
2020-12-13 22:36:04 +00:00
|
|
|
/// Record entities accessed by current query, and store this information in system.query_log.
|
2020-12-02 12:08:03 +00:00
|
|
|
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;
|
|
|
|
}
|
|
|
|
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
|
|
|
/// 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{};
|
2020-12-02 12:08:03 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
QueryAccessInfo query_access_info;
|
|
|
|
|
2021-01-21 21:15:11 +00:00
|
|
|
/// Record names of created objects of factories (for testing, etc)
|
2021-01-20 10:54:11 +00:00
|
|
|
struct QueryFactoriesInfo
|
|
|
|
{
|
2021-01-21 21:15:11 +00:00
|
|
|
std::unordered_set<std::string> aggregate_functions;
|
2021-01-23 12:07:37 +00:00
|
|
|
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-20 10:54:11 +00:00
|
|
|
};
|
|
|
|
|
2021-01-21 21:15:11 +00:00
|
|
|
/// Needs to be chandged while having const context in factories methods
|
2021-01-20 10:54:11 +00:00
|
|
|
mutable QueryFactoriesInfo query_factories_info;
|
|
|
|
|
2020-02-21 15:22:28 +00:00
|
|
|
//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
|
2018-03-01 01:49:36 +00:00
|
|
|
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
|
|
|
|
2018-03-02 05:44:17 +00:00
|
|
|
Context * query_context = nullptr;
|
2017-04-01 07:20:54 +00:00
|
|
|
Context * session_context = nullptr; /// Session context or nullptr. Could be equal to this.
|
2019-07-08 00:51:43 +00:00
|
|
|
Context * global_context = nullptr; /// Global context. Could be equal to this.
|
2012-06-18 07:49:19 +00:00
|
|
|
|
2020-11-18 17:43:18 +00:00
|
|
|
public:
|
|
|
|
// Top-level OpenTelemetry trace context for the query. Makes sense only for
|
|
|
|
// a query context.
|
|
|
|
OpenTelemetryTraceContext query_trace_context;
|
|
|
|
|
|
|
|
private:
|
2020-03-05 19:23:39 +00:00
|
|
|
friend class NamedSessions;
|
2012-06-18 07:49:19 +00:00
|
|
|
|
2018-07-05 23:36:09 +00:00
|
|
|
using SampleBlockCache = std::unordered_map<std::string, Block>;
|
|
|
|
mutable SampleBlockCache sample_block_cache;
|
|
|
|
|
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)
|
|
|
|
|
2019-03-06 16:41:35 +00:00
|
|
|
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::Context&)
|
|
|
|
/// methods.
|
|
|
|
|
2017-06-19 20:31:23 +00:00
|
|
|
/// Use copy constructor or createGlobal() instead
|
2017-04-01 07:20:54 +00:00
|
|
|
Context();
|
2017-06-19 20:31:23 +00:00
|
|
|
|
|
|
|
public:
|
|
|
|
/// Create initial Context with ContextShared and etc.
|
2020-04-16 14:37:38 +00:00
|
|
|
static Context createGlobal(ContextShared * shared);
|
2020-04-16 14:51:33 +00:00
|
|
|
static SharedContextHolder createShared();
|
2017-06-19 20:31:23 +00:00
|
|
|
|
2019-04-24 22:52:08 +00:00
|
|
|
Context(const Context &);
|
|
|
|
Context & operator=(const Context &);
|
2017-04-01 07:20:54 +00:00
|
|
|
~Context();
|
2015-04-16 06:12:35 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String getPath() const;
|
|
|
|
String getFlagsPath() const;
|
2018-04-19 05:32:09 +00:00
|
|
|
String getUserFilesPath() const;
|
2019-12-10 17:27:29 +00:00
|
|
|
String getDictionariesLibPath() const;
|
2018-04-19 05:32:09 +00:00
|
|
|
|
2020-07-08 14:25:23 +00:00
|
|
|
VolumePtr getTemporaryVolume() const;
|
2020-01-19 14:26:28 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void setPath(const String & path);
|
|
|
|
void setFlagsPath(const String & path);
|
2018-04-19 05:32:09 +00:00
|
|
|
void setUserFilesPath(const String & path);
|
2019-12-10 17:27:29 +00:00
|
|
|
void setDictionariesLibPath(const String & path);
|
2013-05-05 20:07:11 +00:00
|
|
|
|
2020-07-08 14:25:23 +00:00
|
|
|
VolumePtr setTemporaryStorage(const String & path, const String & policy_name = "");
|
2020-01-19 14:26:28 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
|
2015-04-16 06:12:35 +00:00
|
|
|
|
2017-08-24 14:51:13 +00:00
|
|
|
/// Global application configuration settings.
|
|
|
|
void setConfig(const ConfigurationPtr & config);
|
2018-07-08 04:54:37 +00:00
|
|
|
const Poco::Util::AbstractConfiguration & getConfigRef() const;
|
2017-08-24 14:51:13 +00:00
|
|
|
|
2019-11-09 15:33:07 +00:00
|
|
|
AccessControlManager & getAccessControlManager();
|
|
|
|
const AccessControlManager & getAccessControlManager() const;
|
2020-02-12 03:03:33 +00:00
|
|
|
|
2020-06-10 22:48:15 +00:00
|
|
|
/// Sets external authenticators config (LDAP).
|
|
|
|
void setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config);
|
|
|
|
|
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();
|
|
|
|
|
|
|
|
/// Sets the current user, checks the password and that the specified host is allowed.
|
|
|
|
/// Must be called before getClientInfo.
|
2020-04-15 01:58:10 +00:00
|
|
|
void setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address);
|
2020-09-17 09:58:34 +00:00
|
|
|
|
|
|
|
/// Sets the current user, *do not checks the password and that the specified host is allowed*.
|
2020-09-14 21:55:43 +00:00
|
|
|
/// Must be called before getClientInfo.
|
|
|
|
///
|
|
|
|
/// (Used only internally in cluster, if the secret matches)
|
|
|
|
void setUserWithoutCheckingPassword(const String & name, const Poco::Net::SocketAddress & address);
|
2020-09-17 09:58:34 +00:00
|
|
|
|
2020-04-15 01:58:10 +00:00
|
|
|
void setQuotaKey(String quota_key_);
|
2020-02-12 03:03:33 +00:00
|
|
|
|
|
|
|
UserPtr getUser() const;
|
|
|
|
String getUserName() const;
|
2020-04-05 23:03:20 +00:00
|
|
|
std::optional<UUID> getUserID() const;
|
2020-01-12 21:00:55 +00:00
|
|
|
|
2020-04-29 19:35:56 +00:00
|
|
|
void setCurrentRoles(const boost::container::flat_set<UUID> & current_roles_);
|
2020-02-21 00:17:07 +00:00
|
|
|
void setCurrentRolesDefault();
|
2020-04-29 19:35:56 +00:00
|
|
|
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
|
|
|
|
|
|
|
/// Checks access rights.
|
|
|
|
/// Empty database means the current database.
|
2020-03-07 17:37:38 +00:00
|
|
|
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 getRowPolicyCondition(const String & database, const String & table_name, RowPolicy::ConditionType type) const;
|
2020-02-07 19:31:09 +00:00
|
|
|
|
2020-02-12 20:47:37 +00:00
|
|
|
/// Sets an extra row policy based on `client_info.initial_user`, if it exists.
|
|
|
|
/// TODO: we need a better solution here. It seems we should pass the initial row policy
|
|
|
|
/// because a shard is allowed to don't have the initial user or it may be another user with the same name.
|
|
|
|
void setInitialRowPolicy();
|
2014-02-13 07:17:22 +00:00
|
|
|
|
2020-03-07 17:37:38 +00:00
|
|
|
std::shared_ptr<const EnabledQuota> getQuota() const;
|
2020-05-08 12:50:45 +00:00
|
|
|
std::optional<QuotaUsage> getQuotaUsage() const;
|
2019-07-28 13:12:26 +00:00
|
|
|
|
2018-06-14 13:03:23 +00:00
|
|
|
/// We have to copy external tables inside executeQuery() to track limits. Therefore, set callback for it. Must set once.
|
2018-06-01 15:32:27 +00:00
|
|
|
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();
|
|
|
|
|
2018-06-03 17:43:56 +00:00
|
|
|
ClientInfo & getClientInfo() { return client_info; }
|
|
|
|
const ClientInfo & getClientInfo() const { return client_info; }
|
2015-06-26 20:48:10 +00:00
|
|
|
|
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;
|
2014-08-22 00:57:20 +00:00
|
|
|
|
2020-12-02 12:08:03 +00:00
|
|
|
const QueryAccessInfo & getQueryAccessInfo() const { return query_access_info; }
|
2020-12-14 03:30:39 +00:00
|
|
|
void addQueryAccessInfo(const String & quoted_database_name, const String & full_quoted_table_name, const Names & column_names);
|
2020-12-02 12:08:03 +00:00
|
|
|
|
2021-01-21 21:15:11 +00:00
|
|
|
/// Supported factories for records in query_log
|
|
|
|
enum class QueryLogFactories
|
|
|
|
{
|
|
|
|
AggregateFunction,
|
2021-01-23 12:07:37 +00:00
|
|
|
AggregateFunctionCombinator,
|
2021-01-21 21:15:11 +00:00
|
|
|
Database,
|
|
|
|
DataType,
|
|
|
|
Dictionary,
|
|
|
|
Format,
|
|
|
|
Function,
|
|
|
|
Storage,
|
|
|
|
TableFunction
|
|
|
|
};
|
|
|
|
|
2021-01-20 10:54:11 +00:00
|
|
|
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;
|
2021-01-20 10:54:11 +00:00
|
|
|
|
2018-03-01 01:49:36 +00:00
|
|
|
StoragePtr executeTableFunction(const ASTPtr & table_expression);
|
|
|
|
|
2018-12-09 17:50:35 +00:00
|
|
|
void addViewSource(const StoragePtr & storage);
|
|
|
|
StoragePtr getViewSource();
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String getCurrentDatabase() const;
|
|
|
|
String getCurrentQueryId() const;
|
2019-08-31 12:18:14 +00:00
|
|
|
|
|
|
|
/// Id of initiating query for distributed queries; or current query id if it's not a distributed query.
|
|
|
|
String getInitialQueryId() const;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
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);
|
2017-04-01 07:20:54 +00:00
|
|
|
void setCurrentQueryId(const String & query_id);
|
2012-08-02 17:33:31 +00:00
|
|
|
|
2019-02-01 01:48:25 +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; }
|
2018-11-15 15:03:13 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
String getDefaultFormat() const; /// If default_format is not specified, some global default format is returned.
|
2017-04-01 07:20:54 +00:00
|
|
|
void setDefaultFormat(const String & name);
|
2013-06-29 18:03:57 +00:00
|
|
|
|
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
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Settings getSettings() const;
|
|
|
|
void setSettings(const Settings & settings_);
|
2012-08-02 19:03:32 +00:00
|
|
|
|
2019-04-18 23:29:32 +00:00
|
|
|
/// Set settings by name.
|
2020-03-15 13:52:46 +00:00
|
|
|
void setSetting(const StringRef & name, const String & value);
|
|
|
|
void setSetting(const StringRef & name, const Field & value);
|
2019-04-18 23:29:32 +00:00
|
|
|
void applySettingChange(const SettingChange & change);
|
|
|
|
void applySettingsChanges(const SettingsChanges & changes);
|
2014-02-14 17:38:33 +00:00
|
|
|
|
2019-04-18 23:29:32 +00:00
|
|
|
/// Checks the constraints.
|
2020-02-28 18:55:21 +00:00
|
|
|
void checkSettingsConstraints(const SettingChange & change) const;
|
|
|
|
void checkSettingsConstraints(const SettingsChanges & changes) const;
|
2020-07-19 14:59:07 +00:00
|
|
|
void checkSettingsConstraints(SettingsChanges & changes) const;
|
2020-02-28 18:55:21 +00:00
|
|
|
void clampToSettingsConstraints(SettingsChanges & changes) const;
|
2014-02-13 07:17:22 +00:00
|
|
|
|
2019-10-31 08:45:14 +00:00
|
|
|
/// Returns the current constraints (can return null).
|
2020-03-04 22:27:03 +00:00
|
|
|
std::shared_ptr<const SettingsConstraints> getSettingsConstraints() const;
|
2019-10-30 21:37:28 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const EmbeddedDictionaries & getEmbeddedDictionaries() const;
|
2019-09-26 10:41:33 +00:00
|
|
|
const ExternalDictionariesLoader & getExternalDictionariesLoader() const;
|
2019-09-26 10:23:14 +00:00
|
|
|
const ExternalModelsLoader & getExternalModelsLoader() const;
|
2017-08-24 18:19:06 +00:00
|
|
|
EmbeddedDictionaries & getEmbeddedDictionaries();
|
2019-09-26 10:41:33 +00:00
|
|
|
ExternalDictionariesLoader & getExternalDictionariesLoader();
|
2019-09-26 10:23:14 +00:00
|
|
|
ExternalModelsLoader & getExternalModelsLoader();
|
2017-04-01 07:20:54 +00:00
|
|
|
void tryCreateEmbeddedDictionaries() const;
|
2015-03-27 13:11:22 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// I/O formats.
|
2019-02-10 16:55:12 +00:00
|
|
|
BlockInputStreamPtr getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size) const;
|
2020-12-09 23:22:53 +00:00
|
|
|
|
2020-12-30 03:07:30 +00:00
|
|
|
/// Don't use streams. Better look at getOutputFormat...
|
|
|
|
BlockOutputStreamPtr getOutputStreamParallelIfPossible(const String & name, WriteBuffer & buf, const Block & sample) const;
|
|
|
|
BlockOutputStreamPtr getOutputStream(const String & name, WriteBuffer & buf, const Block & sample) const;
|
2016-02-13 06:37:19 +00:00
|
|
|
|
2020-12-30 03:07:30 +00:00
|
|
|
OutputFormatPtr getOutputFormatParallelIfPossible(const String & name, WriteBuffer & buf, const Block & sample) const;
|
|
|
|
OutputFormatPtr getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample) const;
|
2016-02-13 06:37:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
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.
|
2017-04-01 07:20:54 +00:00
|
|
|
void setInterserverIOAddress(const String & host, UInt16 port);
|
|
|
|
std::pair<String, UInt16> getInterserverIOAddress() const;
|
2018-07-26 15:10:57 +00:00
|
|
|
|
2018-07-30 18:32:21 +00:00
|
|
|
/// Credentials which server will use to communicate with others
|
|
|
|
void setInterserverCredentials(const String & user, const String & password);
|
2018-07-26 15:10:57 +00:00
|
|
|
std::pair<String, String> getInterserverCredentials() const;
|
|
|
|
|
2018-07-30 18:32:21 +00:00
|
|
|
/// Interserver requests scheme (http or https)
|
|
|
|
void setInterserverScheme(const String & scheme);
|
|
|
|
String getInterserverScheme() const;
|
|
|
|
|
2019-10-09 20:29:41 +00:00
|
|
|
/// Storage of allowed hosts from config.xml
|
2019-10-10 12:58:06 +00:00
|
|
|
void setRemoteHostFilter(const Poco::Util::AbstractConfiguration & config);
|
2019-10-21 14:36:24 +00:00
|
|
|
const RemoteHostFilter & getRemoteHostFilter() const;
|
2019-10-09 20:29:41 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// The port that the server listens for executing SQL queries.
|
2017-04-01 07:20:54 +00:00
|
|
|
UInt16 getTCPPort() const;
|
2014-03-21 19:49:27 +00:00
|
|
|
|
2018-11-06 14:42:30 +00:00
|
|
|
std::optional<UInt16> getTCPPortSecure() const;
|
2014-03-21 19:49:27 +00:00
|
|
|
|
2020-03-05 19:23:39 +00:00
|
|
|
/// Allow to use named sessions. The thread will be run to cleanup sessions after timeout has expired.
|
|
|
|
/// The method must be called at the server startup.
|
|
|
|
void enableNamedSessions();
|
2017-06-02 18:48:33 +00:00
|
|
|
|
2020-03-06 18:14:33 +00:00
|
|
|
std::shared_ptr<NamedSession> acquireNamedSession(const String & session_id, std::chrono::steady_clock::duration timeout, bool session_check);
|
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
|
|
|
|
2018-03-02 05:44:17 +00:00
|
|
|
const Context & getQueryContext() const;
|
|
|
|
Context & getQueryContext();
|
|
|
|
bool hasQueryContext() const { return query_context != nullptr; }
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const Context & getSessionContext() const;
|
|
|
|
Context & getSessionContext();
|
2017-08-15 12:34:28 +00:00
|
|
|
bool hasSessionContext() const { return session_context != nullptr; }
|
2016-11-16 11:29:51 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const Context & getGlobalContext() const;
|
|
|
|
Context & getGlobalContext();
|
2017-08-15 12:34:28 +00:00
|
|
|
bool hasGlobalContext() const { return global_context != nullptr; }
|
2012-08-02 17:33:31 +00:00
|
|
|
|
2018-03-02 05:44:17 +00:00
|
|
|
void setQueryContext(Context & context_) { query_context = &context_; }
|
2018-03-01 01:49:36 +00:00
|
|
|
void setSessionContext(Context & context_) { session_context = &context_; }
|
2019-07-08 00:51:43 +00:00
|
|
|
|
|
|
|
void makeQueryContext() { query_context = this; }
|
|
|
|
void makeSessionContext() { session_context = this; }
|
2020-09-07 23:08:17 +00:00
|
|
|
void makeGlobalContext() { initGlobal(); global_context = this; }
|
2012-08-02 17:33:31 +00:00
|
|
|
|
2018-06-03 17:43:56 +00:00
|
|
|
const Settings & getSettingsRef() const { return settings; }
|
2013-02-16 14:55:14 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void setProgressCallback(ProgressCallback callback);
|
2019-01-23 14:48:50 +00:00
|
|
|
/// Used in InterpreterSelectQuery to pass it to the IBlockInputStream.
|
2017-04-01 07:20:54 +00:00
|
|
|
ProgressCallback getProgressCallback() const;
|
2013-09-03 20:21:28 +00:00
|
|
|
|
2019-01-23 14:48:50 +00:00
|
|
|
/** Set in executeQuery and InterpreterSelectQuery. Then it is used in IBlockInputStream,
|
2017-06-02 21:37:28 +00:00
|
|
|
* to update and monitor information about the total number of resources spent for the query.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2018-02-01 17:55:08 +00:00
|
|
|
void setProcessListElement(QueryStatus * elem);
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Can return nullptr if the query was not inserted into the ProcessList.
|
2018-02-01 17:55:08 +00:00
|
|
|
QueryStatus * getProcessListElement() const;
|
2013-11-03 05:32:42 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// List all queries.
|
2017-04-01 07:20:54 +00:00
|
|
|
ProcessList & getProcessList();
|
|
|
|
const ProcessList & getProcessList() const;
|
2013-09-08 05:53:10 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
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.
|
2018-04-21 18:41:06 +00:00
|
|
|
/// If no ZooKeeper configured, throws an exception.
|
2017-04-01 07:20:54 +00:00
|
|
|
std::shared_ptr<zkutil::ZooKeeper> getZooKeeper() const;
|
2020-08-27 14:19:18 +00:00
|
|
|
/// Same as above but return a zookeeper connection from auxiliary_zookeepers configuration entry.
|
|
|
|
std::shared_ptr<zkutil::ZooKeeper> getAuxiliaryZooKeeper(const String & name) const;
|
2020-10-30 14:16:47 +00:00
|
|
|
|
2020-11-11 13:57:24 +00:00
|
|
|
|
2021-01-25 12:29:12 +00:00
|
|
|
void initializeTestKeeperStorageDispatcher() const;
|
2021-01-21 15:09:48 +00:00
|
|
|
std::shared_ptr<TestKeeperStorageDispatcher> & getTestKeeperStorageDispatcher() const;
|
2021-01-27 17:54:25 +00:00
|
|
|
void shutdownTestKeeperStorageDispatcher() const;
|
2020-11-11 13:57:24 +00:00
|
|
|
|
2020-11-03 07:52:31 +00:00
|
|
|
/// Set auxiliary zookeepers configuration at server starting or configuration reloading.
|
|
|
|
void reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config);
|
2017-06-15 20:08:26 +00:00
|
|
|
/// 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;
|
2019-07-04 14:48:40 +00:00
|
|
|
/// Reset current zookeeper session. Do not create a new one.
|
|
|
|
void resetZooKeeper() const;
|
2020-09-10 04:00:33 +00:00
|
|
|
// Reload Zookeeper
|
|
|
|
void reloadZooKeeperIfChanged(const ConfigurationPtr & config) const;
|
2013-12-07 16:51:29 +00:00
|
|
|
|
2017-08-07 17:01:04 +00:00
|
|
|
/// 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.
|
2017-04-01 07:20:54 +00:00
|
|
|
void setMarkCache(size_t cache_size_in_bytes);
|
|
|
|
std::shared_ptr<MarkCache> getMarkCache() const;
|
2017-08-07 17:01:04 +00:00
|
|
|
void dropMarkCache() 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;
|
2021-01-27 18:08:06 +00:00
|
|
|
BackgroundSchedulePool & getMessageBrokerSchedulePool() const;
|
2020-11-06 14:07:56 +00:00
|
|
|
BackgroundSchedulePool & getDistributedSchedulePool() const;
|
2014-07-02 12:30:38 +00:00
|
|
|
|
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;
|
2019-02-21 16:41:27 +00:00
|
|
|
void setDDLWorker(std::unique_ptr<DDLWorker> ddl_worker);
|
2017-09-04 17:49:39 +00:00
|
|
|
DDLWorker & getDDLWorker() const;
|
2017-04-13 13:42:29 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
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;
|
2017-10-13 19:13:41 +00:00
|
|
|
void setClustersConfig(const ConfigurationPtr & config, const String & config_name = "remote_servers");
|
2017-11-03 19:53:10 +00:00
|
|
|
/// Sets custom cluster, but doesn't update configuration
|
|
|
|
void setCluster(const String & cluster_name, const std::shared_ptr<Cluster> & cluster);
|
2018-01-15 14:13:19 +00:00
|
|
|
void reloadClusterConfig();
|
2014-03-03 20:11:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Compiler & getCompiler();
|
2018-03-10 19:57:13 +00:00
|
|
|
|
|
|
|
/// Call after initialization before using system logs. Call for global context.
|
|
|
|
void initializeSystemLogs();
|
|
|
|
|
2020-03-03 00:24:44 +00:00
|
|
|
/// Call after initialization before using trace collector.
|
2019-02-03 21:30:45 +00:00
|
|
|
void initializeTraceCollector();
|
2020-03-03 00:24:44 +00:00
|
|
|
|
|
|
|
bool hasTraceCollector() const;
|
2019-02-03 21:30:45 +00:00
|
|
|
|
2018-03-10 19:57:13 +00:00
|
|
|
/// Nullptr if the query log is not ready for this moment.
|
2019-03-21 19:22:38 +00:00
|
|
|
std::shared_ptr<QueryLog> getQueryLog();
|
|
|
|
std::shared_ptr<QueryThreadLog> getQueryThreadLog();
|
2019-05-19 20:22:44 +00:00
|
|
|
std::shared_ptr<TraceLog> getTraceLog();
|
2019-07-31 14:03:23 +00:00
|
|
|
std::shared_ptr<TextLog> getTextLog();
|
2019-08-13 14:31:46 +00:00
|
|
|
std::shared_ptr<MetricLog> getMetricLog();
|
2020-06-10 19:17:30 +00:00
|
|
|
std::shared_ptr<AsynchronousMetricLog> getAsynchronousMetricLog();
|
2020-10-22 16:47:20 +00:00
|
|
|
std::shared_ptr<OpenTelemetrySpanLog> getOpenTelemetrySpanLog();
|
2017-06-05 13:59:38 +00:00
|
|
|
|
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.
|
2019-03-21 19:22:38 +00:00
|
|
|
std::shared_ptr<PartLog> getPartLog(const String & part_database);
|
2018-03-10 19:57:13 +00:00
|
|
|
|
2018-08-10 17:42:12 +00:00
|
|
|
const MergeTreeSettings & getMergeTreeSettings() const;
|
2020-08-10 11:23:58 +00:00
|
|
|
const MergeTreeSettings & getReplicatedMergeTreeSettings() const;
|
2020-06-01 17:16:09 +00:00
|
|
|
const StorageS3Settings & getStorageS3Settings() const;
|
2015-06-26 20:48:10 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// 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);
|
2019-01-04 12:10:00 +00:00
|
|
|
void checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const;
|
2017-01-19 19:11:12 +00:00
|
|
|
|
2018-08-01 17:41:18 +00:00
|
|
|
/// Prevents DROP PARTITION if its size is greater than max_size (50GB by default, max_size=0 turn off this check)
|
2018-08-03 08:33:57 +00:00
|
|
|
void setMaxPartitionSizeToDrop(size_t max_size);
|
2019-01-04 12:10:00 +00:00
|
|
|
void checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const;
|
2017-01-19 19:11:12 +00:00
|
|
|
|
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;
|
2015-04-17 05:35:53 +00:00
|
|
|
|
2019-04-21 18:38:44 +00:00
|
|
|
|
|
|
|
/// Provides storage disks
|
2020-01-09 14:50:34 +00:00
|
|
|
DiskPtr getDisk(const String & name) const;
|
2019-04-21 18:38:44 +00:00
|
|
|
|
2020-05-22 10:33:57 +00:00
|
|
|
StoragePoliciesMap getPoliciesMap() const;
|
|
|
|
DisksMap getDisksMap() const;
|
2020-01-09 14:50:34 +00:00
|
|
|
void updateStorageConfiguration(const Poco::Util::AbstractConfiguration & config);
|
2019-05-24 19:03:07 +00:00
|
|
|
|
2019-04-05 17:37:27 +00:00
|
|
|
/// Provides storage politics schemes
|
2020-01-09 14:50:34 +00:00
|
|
|
StoragePolicyPtr getStoragePolicy(const String & name) const;
|
2019-04-04 17:19:11 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Get the server uptime in seconds.
|
2017-04-01 07:20:54 +00:00
|
|
|
time_t getUptimeSeconds() const;
|
2016-04-09 07:47:08 +00:00
|
|
|
|
2018-03-13 10:41:47 +00:00
|
|
|
using ConfigReloadCallback = std::function<void()>;
|
|
|
|
void setConfigReloadCallback(ConfigReloadCallback && callback);
|
|
|
|
void reloadConfig() const;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void shutdown();
|
2015-04-02 16:30:18 +00:00
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
ActionLocksManagerPtr getActionLocksManager();
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
enum class ApplicationType
|
|
|
|
{
|
2017-06-05 13:59:38 +00:00
|
|
|
SERVER, /// The program is run as clickhouse-server daemon (default behavior)
|
|
|
|
CLIENT, /// clickhouse-client
|
2017-07-11 20:12:15 +00:00
|
|
|
LOCAL /// clickhouse-local
|
2017-04-01 07:20:54 +00:00
|
|
|
};
|
2016-11-11 17:01:02 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
ApplicationType getApplicationType() const;
|
|
|
|
void setApplicationType(ApplicationType type);
|
2016-11-11 17:01:02 +00:00
|
|
|
|
2018-02-01 13:52:29 +00:00
|
|
|
/// Sets default_profile and system_profile, must be called once during the initialization
|
|
|
|
void setDefaultProfiles(const Poco::Util::AbstractConfiguration & config);
|
2017-04-01 07:20:54 +00:00
|
|
|
String getDefaultProfileName() const;
|
2018-02-01 13:52:29 +00:00
|
|
|
String getSystemProfileName() const;
|
2017-03-23 14:14:56 +00:00
|
|
|
|
2017-11-10 06:48:28 +00:00
|
|
|
/// Base path for format schemas
|
|
|
|
String getFormatSchemaPath() const;
|
|
|
|
void setFormatSchemaPath(const String & path);
|
|
|
|
|
2018-07-05 23:36:09 +00:00
|
|
|
SampleBlockCache & getSampleBlockCache() const;
|
2018-05-30 19:23:15 +00:00
|
|
|
|
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
|
|
|
|
2018-08-30 16:31:20 +00:00
|
|
|
#if USE_EMBEDDED_COMPILER
|
2018-09-05 11:37:41 +00:00
|
|
|
std::shared_ptr<CompiledExpressionCache> getCompiledExpressionCache() const;
|
2018-09-05 12:42:37 +00:00
|
|
|
void setCompiledExpressionCache(size_t cache_size);
|
2018-09-05 11:37:41 +00:00
|
|
|
void dropCompiledExpressionCache() const;
|
2018-08-30 16:31:20 +00:00
|
|
|
#endif
|
|
|
|
|
2018-11-22 15:59:00 +00:00
|
|
|
/// Add started bridge command. It will be killed after context destruction
|
2019-10-10 20:47:47 +00:00
|
|
|
void addXDBCBridgeCommand(std::unique_ptr<ShellCommand> cmd) const;
|
2018-11-22 15:59:00 +00:00
|
|
|
|
2019-03-06 16:41:35 +00:00
|
|
|
IHostContextPtr & getHostContext();
|
|
|
|
const IHostContextPtr & getHostContext() const;
|
|
|
|
|
2019-07-19 18:43:52 +00:00
|
|
|
struct MySQLWireContext
|
2019-07-16 06:39:18 +00:00
|
|
|
{
|
|
|
|
uint8_t sequence_id = 0;
|
|
|
|
uint32_t client_capabilities = 0;
|
|
|
|
size_t max_packet_size = 0;
|
2019-07-19 18:43:52 +00:00
|
|
|
};
|
2019-07-16 06:39:18 +00:00
|
|
|
|
2019-07-16 07:28:53 +00:00
|
|
|
MySQLWireContext mysql;
|
2015-04-02 16:30:18 +00:00
|
|
|
private:
|
2020-02-13 21:00:03 +00:00
|
|
|
std::unique_lock<std::recursive_mutex> getLock() const;
|
|
|
|
|
2020-09-07 23:08:17 +00:00
|
|
|
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;
|
|
|
|
|
2019-04-19 00:45:15 +00:00
|
|
|
void setProfile(const String & profile);
|
|
|
|
|
2017-08-24 18:19:06 +00:00
|
|
|
EmbeddedDictionaries & getEmbeddedDictionariesImpl(bool throw_on_error) const;
|
2015-08-19 21:15:27 +00:00
|
|
|
|
2019-01-04 12:10:00 +00:00
|
|
|
void checkCanBeDropped(const String & database, const String & table, const size_t & size, const size_t & max_size_to_drop) const;
|
2020-05-21 14:11:56 +00:00
|
|
|
|
|
|
|
StoragePolicySelectorPtr getStoragePolicySelector(std::lock_guard<std::mutex> & lock) const;
|
|
|
|
|
|
|
|
DiskSelectorPtr getDiskSelector(std::lock_guard<std::mutex> & /* lock */) const;
|
2020-09-14 21:55:43 +00:00
|
|
|
|
|
|
|
/// If the password is not set, the password will not be checked
|
|
|
|
void setUserImpl(const String & name, const std::optional<String> & password, const Poco::Net::SocketAddress & address);
|
2011-08-09 19:19:25 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2020-03-05 19:23:39 +00:00
|
|
|
class NamedSessions;
|
2020-03-05 04:10:48 +00:00
|
|
|
|
|
|
|
/// User name and session identifier. Named sessions are local to users.
|
2020-03-05 19:23:39 +00:00
|
|
|
using NamedSessionKey = std::pair<String, String>;
|
2020-03-05 04:10:48 +00:00
|
|
|
|
|
|
|
/// Named sessions. The user could specify session identifier to reuse settings and temporary tables in subsequent requests.
|
2020-03-05 19:23:39 +00:00
|
|
|
struct NamedSession
|
2017-06-02 18:48:33 +00:00
|
|
|
{
|
2020-03-05 19:23:39 +00:00
|
|
|
NamedSessionKey key;
|
2020-03-06 18:14:33 +00:00
|
|
|
UInt64 close_cycle = 0;
|
2020-03-05 04:10:48 +00:00
|
|
|
Context context;
|
2020-03-06 18:14:33 +00:00
|
|
|
std::chrono::steady_clock::duration timeout;
|
2020-03-05 19:23:39 +00:00
|
|
|
NamedSessions & parent;
|
2020-03-05 04:10:48 +00:00
|
|
|
|
2020-03-06 18:14:33 +00:00
|
|
|
NamedSession(NamedSessionKey key_, Context & context_, std::chrono::steady_clock::duration timeout_, NamedSessions & parent_)
|
2020-03-05 04:10:48 +00:00
|
|
|
: key(key_), context(context_), timeout(timeout_), parent(parent_)
|
2017-06-02 18:48:33 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2020-03-05 04:10:48 +00:00
|
|
|
void release();
|
2017-06-02 18:48:33 +00:00
|
|
|
};
|
|
|
|
|
2011-08-09 19:19:25 +00:00
|
|
|
}
|