2011-08-09 19:19:25 +00:00
|
|
|
#pragma once
|
|
|
|
|
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>
|
|
|
|
#include <thread>
|
2012-09-05 19:51:09 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/Types.h>
|
|
|
|
#include <Core/NamesAndTypes.h>
|
|
|
|
#include <Interpreters/Settings.h>
|
|
|
|
#include <Interpreters/ClientInfo.h>
|
|
|
|
#include <IO/CompressedStream.h>
|
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;
|
2015-04-16 06:12:35 +00:00
|
|
|
class QuotaForIntervals;
|
2017-01-21 04:24:28 +00:00
|
|
|
class EmbeddedDictionaries;
|
2015-04-16 06:12:35 +00:00
|
|
|
class ExternalDictionaries;
|
|
|
|
class InterserverIOHandler;
|
|
|
|
class BackgroundProcessingPool;
|
2016-01-28 01:00:27 +00:00
|
|
|
class ReshardingWorker;
|
2015-04-16 06:12:35 +00:00
|
|
|
class MergeList;
|
|
|
|
class Cluster;
|
|
|
|
class Compiler;
|
|
|
|
class MarkCache;
|
|
|
|
class UncompressedCache;
|
|
|
|
class ProcessList;
|
2016-03-07 04:31:10 +00:00
|
|
|
struct ProcessListElement;
|
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;
|
2017-03-07 17:13:54 +00:00
|
|
|
class PartLog;
|
2015-07-16 21:32:51 +00:00
|
|
|
struct MergeTreeSettings;
|
2016-03-19 01:18:49 +00:00
|
|
|
class IDatabase;
|
2016-03-21 12:57:12 +00:00
|
|
|
class DDLGuard;
|
2017-04-13 13:42:29 +00:00
|
|
|
class DDLWorker;
|
2017-01-21 04:24:28 +00:00
|
|
|
class IStorage;
|
|
|
|
using StoragePtr = std::shared_ptr<IStorage>;
|
|
|
|
using Tables = std::map<String, StoragePtr>;
|
|
|
|
class IAST;
|
|
|
|
using ASTPtr = std::shared_ptr<IAST>;
|
|
|
|
class IBlockInputStream;
|
|
|
|
class IBlockOutputStream;
|
|
|
|
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
|
|
|
|
using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>;
|
|
|
|
class Block;
|
2017-06-05 13:59:38 +00:00
|
|
|
struct SystemLogs;
|
|
|
|
using SystemLogsPtr = std::shared_ptr<SystemLogs>;
|
2014-01-28 16:45:10 +00:00
|
|
|
|
2011-08-19 18:31:14 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// (database name, table name)
|
2016-03-21 12:57:12 +00:00
|
|
|
using DatabaseAndTableName = std::pair<String, String>;
|
2013-11-08 17:43:03 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Table -> set of table-views that make SELECT from it.
|
2016-03-21 12:57:12 +00:00
|
|
|
using ViewDependencies = std::map<DatabaseAndTableName, std::set<DatabaseAndTableName>>;
|
|
|
|
using Dependencies = std::vector<DatabaseAndTableName>;
|
2013-11-08 17:43:03 +00:00
|
|
|
|
2014-03-03 20:11:09 +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:
|
2017-04-01 07:20:54 +00:00
|
|
|
using Shared = std::shared_ptr<ContextShared>;
|
|
|
|
Shared shared;
|
2012-06-18 07:49:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
ClientInfo client_info;
|
2015-06-26 20:48:10 +00:00
|
|
|
|
2017-06-05 13:59:38 +00:00
|
|
|
std::shared_ptr<QuotaForIntervals> quota; /// Current quota. By default - empty quota, that have no limits.
|
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.
|
|
|
|
ProcessListElement * process_list_elem = nullptr; /// For tracking total resource usage for query.
|
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.
|
2017-06-05 13:59:38 +00:00
|
|
|
Tables external_tables; /// Temporary tables.
|
2017-04-01 07:20:54 +00:00
|
|
|
Context * session_context = nullptr; /// Session context or nullptr. Could be equal to this.
|
2017-06-05 13:59:38 +00:00
|
|
|
Context * global_context = nullptr; /// Global context or nullptr. Could be equal to this.
|
|
|
|
SystemLogsPtr system_logs; /// Used to log queries and operations on parts
|
2012-06-18 07:49:19 +00:00
|
|
|
|
2017-06-05 14:40:44 +00:00
|
|
|
UInt64 session_close_cycle = 0;
|
|
|
|
bool session_is_used = false;
|
2012-06-18 07:49:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
using DatabasePtr = std::shared_ptr<IDatabase>;
|
|
|
|
using Databases = std::map<String, std::shared_ptr<IDatabase>>;
|
2016-03-19 01:18:49 +00:00
|
|
|
|
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.
|
|
|
|
static Context createGlobal();
|
|
|
|
|
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 getTemporaryPath() const;
|
|
|
|
String getFlagsPath() const;
|
|
|
|
void setPath(const String & path);
|
|
|
|
void setTemporaryPath(const String & path);
|
|
|
|
void setFlagsPath(const String & path);
|
2013-05-05 20:07:11 +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);
|
|
|
|
ConfigurationPtr getConfig() const;
|
|
|
|
Poco::Util::AbstractConfiguration & getConfigRef() const;
|
|
|
|
|
2017-06-02 21:37:28 +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.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
void setUsersConfig(const ConfigurationPtr & config);
|
|
|
|
ConfigurationPtr getUsersConfig();
|
2014-02-13 07:17:22 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Must be called before getClientInfo.
|
|
|
|
void setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address, const String & quota_key);
|
2017-06-07 12:54:35 +00:00
|
|
|
/// Compute and set actual user settings, client_info.current_user should be set
|
|
|
|
void calculateUserSettings();
|
2015-06-26 20:48:10 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
ClientInfo & getClientInfo() { return client_info; };
|
|
|
|
const ClientInfo & getClientInfo() const { return client_info; };
|
2015-06-26 20:48:10 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address);
|
|
|
|
QuotaForIntervals & getQuota();
|
2013-08-10 07:46:45 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void addDependency(const DatabaseAndTableName & from, const DatabaseAndTableName & where);
|
|
|
|
void removeDependency(const DatabaseAndTableName & from, const DatabaseAndTableName & where);
|
|
|
|
Dependencies getDependencies(const String & database_name, const String & table_name) const;
|
2013-11-08 17:43:03 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Checking the existence of the table/database. Database can be empty - in this case the current database is used.
|
2017-04-01 07:20:54 +00:00
|
|
|
bool isTableExist(const String & database_name, const String & table_name) const;
|
|
|
|
bool isDatabaseExist(const String & database_name) const;
|
|
|
|
void assertTableExists(const String & database_name, const String & table_name) const;
|
2015-10-01 15:10:41 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** The parameter check_database_access_rights exists to not check the permissions of the database again,
|
|
|
|
* when assertTableDoesntExist or assertDatabaseExists is called inside another function that already
|
|
|
|
* made this check.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
void assertTableDoesntExist(const String & database_name, const String & table_name, bool check_database_acccess_rights = true) const;
|
|
|
|
void assertDatabaseExists(const String & database_name, bool check_database_acccess_rights = true) const;
|
2015-10-01 15:10:41 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void assertDatabaseDoesntExist(const String & database_name) const;
|
2012-08-02 17:33:31 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Tables getExternalTables() const;
|
|
|
|
StoragePtr tryGetExternalTable(const String & table_name) const;
|
|
|
|
StoragePtr getTable(const String & database_name, const String & table_name) const;
|
|
|
|
StoragePtr tryGetTable(const String & database_name, const String & table_name) const;
|
|
|
|
void addExternalTable(const String & table_name, StoragePtr storage);
|
2017-08-15 17:00:18 +00:00
|
|
|
StoragePtr tryRemoveExternalTable(const String & table_name);
|
2014-08-22 00:57:20 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void addDatabase(const String & database_name, const DatabasePtr & database);
|
|
|
|
DatabasePtr detachDatabase(const String & database_name);
|
2016-03-21 12:57:12 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Get an object that protects the table from concurrently executing multiple DDL operations.
|
|
|
|
/// If such an object already exists, an exception is thrown.
|
2017-04-01 07:20:54 +00:00
|
|
|
std::unique_ptr<DDLGuard> getDDLGuard(const String & database, const String & table, const String & message) const;
|
2017-06-02 21:37:28 +00:00
|
|
|
/// If the table already exists, it returns nullptr, otherwise guard is created.
|
2017-04-01 07:20:54 +00:00
|
|
|
std::unique_ptr<DDLGuard> getDDLGuardIfTableDoesntExist(const String & database, const String & table, const String & message) const;
|
2012-08-02 17:33:31 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String getCurrentDatabase() const;
|
|
|
|
String getCurrentQueryId() const;
|
|
|
|
void setCurrentDatabase(const String & name);
|
|
|
|
void setCurrentQueryId(const String & query_id);
|
2012-08-02 17:33:31 +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
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const Macros & getMacros() const;
|
|
|
|
void setMacros(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
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Limits getLimits() const;
|
2013-05-05 20:07:11 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Set a setting by name.
|
2017-04-01 07:20:54 +00:00
|
|
|
void setSetting(const String & name, const Field & value);
|
2014-02-14 17:38:33 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Set a setting by name. Read the value in text form from a string (for example, from a config, or from a URL parameter).
|
2017-04-01 07:20:54 +00:00
|
|
|
void setSetting(const String & name, const std::string & value);
|
2014-02-13 07:17:22 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const EmbeddedDictionaries & getEmbeddedDictionaries() const;
|
|
|
|
const ExternalDictionaries & getExternalDictionaries() const;
|
2017-08-24 18:19:06 +00:00
|
|
|
EmbeddedDictionaries & getEmbeddedDictionaries();
|
|
|
|
ExternalDictionaries & getExternalDictionaries();
|
2017-04-01 07:20:54 +00:00
|
|
|
void tryCreateEmbeddedDictionaries() const;
|
|
|
|
void tryCreateExternalDictionaries() const;
|
2015-03-27 13:11:22 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// I/O formats.
|
2017-04-01 07:20:54 +00:00
|
|
|
BlockInputStreamPtr getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, size_t max_block_size) const;
|
|
|
|
BlockOutputStreamPtr 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;
|
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
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Get query for the CREATE table.
|
2017-04-01 07:20:54 +00:00
|
|
|
ASTPtr getCreateQuery(const String & database_name, const String & table_name) const;
|
2012-08-17 19:53:11 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const DatabasePtr getDatabase(const String & database_name) const;
|
|
|
|
DatabasePtr getDatabase(const String & database_name);
|
|
|
|
const DatabasePtr tryGetDatabase(const String & database_name) const;
|
|
|
|
DatabasePtr tryGetDatabase(const String & database_name);
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const Databases getDatabases() const;
|
|
|
|
Databases getDatabases();
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2017-06-02 18:48:33 +00:00
|
|
|
std::shared_ptr<Context> acquireSession(const String & session_id, std::chrono::steady_clock::duration timeout, bool session_check) const;
|
|
|
|
void releaseSession(const String & session_id, std::chrono::steady_clock::duration timeout);
|
|
|
|
|
2017-06-02 21:01:17 +00:00
|
|
|
/// Close sessions, that has been expired. Returns how long to wait for next session to be expired, if no new sessions will be added.
|
|
|
|
std::chrono::steady_clock::duration closeSessions() const;
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// For methods below you may need to acquire a lock by yourself.
|
|
|
|
std::unique_lock<Poco::Mutex> getLock() const;
|
2012-08-02 17:33:31 +00:00
|
|
|
|
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
|
|
|
|
2017-08-07 17:01:04 +00:00
|
|
|
void setSessionContext(Context & context_) { session_context = &context_; }
|
|
|
|
void setGlobalContext(Context & context_) { global_context = &context_; }
|
2012-08-02 17:33:31 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const Settings & getSettingsRef() const { return settings; };
|
|
|
|
Settings & getSettingsRef() { return settings; };
|
2013-02-16 14:55:14 +00:00
|
|
|
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void setProgressCallback(ProgressCallback callback);
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Used in InterpreterSelectQuery to pass it to the IProfilingBlockInputStream.
|
2017-04-01 07:20:54 +00:00
|
|
|
ProgressCallback getProgressCallback() const;
|
2013-09-03 20:21:28 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Set in executeQuery and InterpreterSelectQuery. Then it is used in IProfilingBlockInputStream,
|
|
|
|
* to update and monitor information about the total number of resources spent for the query.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
void setProcessListElement(ProcessListElement * elem);
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Can return nullptr if the query was not inserted into the ProcessList.
|
2017-09-04 17:49:39 +00:00
|
|
|
ProcessListElement * 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
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void setZooKeeper(std::shared_ptr<zkutil::ZooKeeper> zookeeper);
|
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.
|
2017-04-01 07:20:54 +00:00
|
|
|
std::shared_ptr<zkutil::ZooKeeper> getZooKeeper() const;
|
2017-06-15 20:08:26 +00:00
|
|
|
/// Has ready or expired ZooKeeper
|
|
|
|
bool hasZooKeeper() 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
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
BackgroundProcessingPool & getBackgroundPool();
|
2014-07-02 12:30:38 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void setReshardingWorker(std::shared_ptr<ReshardingWorker> resharding_worker);
|
2017-09-04 17:49:39 +00:00
|
|
|
ReshardingWorker & getReshardingWorker() const;
|
2016-01-28 01:00:27 +00:00
|
|
|
|
2017-04-13 13:42:29 +00:00
|
|
|
void setDDLWorker(std::shared_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;
|
|
|
|
void setClustersConfig(const ConfigurationPtr & config);
|
2014-03-03 20:11:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Compiler & getCompiler();
|
|
|
|
QueryLog & getQueryLog();
|
2017-06-05 13:59:38 +00:00
|
|
|
|
|
|
|
/// Returns an object used to log opertaions with parts if it possible.
|
|
|
|
/// Provide table name to make required cheks.
|
|
|
|
PartLog * getPartLog(const String & database, const String & table);
|
2017-04-01 07:20:54 +00:00
|
|
|
const MergeTreeSettings & getMergeTreeSettings();
|
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);
|
|
|
|
void checkTableCanBeDropped(const String & database, const String & table, size_t table_size);
|
2017-01-19 19:11:12 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Lets you select the compression method according to the conditions described in the configuration file.
|
2017-04-01 07:20:54 +00:00
|
|
|
CompressionMethod chooseCompressionMethod(size_t part_size, double part_size_ratio) const;
|
2015-04-17 05:35:53 +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
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void shutdown();
|
2015-04-02 16:30:18 +00:00
|
|
|
|
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
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Set once
|
|
|
|
String getDefaultProfileName() const;
|
|
|
|
void setDefaultProfileName(const String & name);
|
2017-03-23 14:14:56 +00:00
|
|
|
|
2017-06-02 21:01:17 +00:00
|
|
|
/// User name and session identifier. Named sessions are local to users.
|
|
|
|
using SessionKey = std::pair<String, String>;
|
|
|
|
|
2015-04-02 16:30:18 +00:00
|
|
|
private:
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Check if the current client has access to the specified database.
|
|
|
|
* If access is denied, throw an exception.
|
|
|
|
* NOTE: This method should always be called when the `shared->mutex` mutex is acquired.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
void checkDatabaseAccessRights(const std::string & database_name) const;
|
2015-10-01 15:10:41 +00:00
|
|
|
|
2017-08-24 18:19:06 +00:00
|
|
|
EmbeddedDictionaries & getEmbeddedDictionariesImpl(bool throw_on_error) const;
|
|
|
|
ExternalDictionaries & getExternalDictionariesImpl(bool throw_on_error) const;
|
2015-08-19 21:15:27 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
StoragePtr getTableImpl(const String & database_name, const String & table_name, Exception * exception) const;
|
2017-06-02 18:48:33 +00:00
|
|
|
|
|
|
|
SessionKey getSessionKey(const String & session_id) const;
|
2017-06-02 21:01:17 +00:00
|
|
|
|
|
|
|
/// Session will be closed after specified timeout.
|
|
|
|
void scheduleCloseSession(const SessionKey & key, std::chrono::steady_clock::duration timeout);
|
2011-08-09 19:19:25 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2017-02-15 19:38:15 +00:00
|
|
|
/// Puts an element into the map, erases it in the destructor.
|
|
|
|
/// If the element already exists in the map, throws an exception containing provided message.
|
2016-03-21 12:57:12 +00:00
|
|
|
class DDLGuard
|
|
|
|
{
|
2017-02-15 19:38:15 +00:00
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Element name -> message.
|
|
|
|
/// NOTE: using std::map here (and not std::unordered_map) to avoid iterator invalidation on insertion.
|
|
|
|
using Map = std::map<String, String>;
|
2017-02-15 19:38:15 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
DDLGuard(Map & map_, std::mutex & mutex_, std::unique_lock<std::mutex> && lock, const String & elem, const String & message);
|
|
|
|
~DDLGuard();
|
2016-03-21 12:57:12 +00:00
|
|
|
|
2017-02-15 19:38:15 +00:00
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
Map & map;
|
|
|
|
Map::iterator it;
|
|
|
|
std::mutex & mutex;
|
2016-03-21 12:57:12 +00:00
|
|
|
};
|
|
|
|
|
2017-06-02 18:48:33 +00:00
|
|
|
|
|
|
|
class SessionCleaner
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
SessionCleaner(Context & context_)
|
|
|
|
: context{context_}
|
|
|
|
{
|
|
|
|
}
|
|
|
|
~SessionCleaner();
|
|
|
|
|
|
|
|
private:
|
|
|
|
void run();
|
|
|
|
|
|
|
|
Context & context;
|
|
|
|
|
|
|
|
std::mutex mutex;
|
|
|
|
std::condition_variable cond;
|
|
|
|
std::atomic<bool> quit{false};
|
2017-06-23 05:53:04 +00:00
|
|
|
std::thread thread{&SessionCleaner::run, this};
|
2017-06-02 18:48:33 +00:00
|
|
|
};
|
|
|
|
|
2011-08-09 19:19:25 +00:00
|
|
|
}
|