ClickHouse/dbms/src/Interpreters/Context.cpp

2286 lines
67 KiB
C++
Raw Normal View History

2015-04-16 06:12:35 +00:00
#include <map>
#include <set>
2019-02-04 23:18:04 +00:00
#include <optional>
#include <memory>
2015-04-16 06:12:35 +00:00
#include <Poco/Mutex.h>
#include <Poco/UUID.h>
#include <Poco/Net/IPAddress.h>
2019-11-04 19:17:27 +00:00
#include <Poco/Util/Application.h>
#include <Common/Macros.h>
#include <Common/escapeForFileName.h>
#include <Common/setThreadName.h>
#include <Common/Stopwatch.h>
#include <Common/formatReadable.h>
2019-07-28 15:30:38 +00:00
#include <Common/thread_local_rng.h>
2018-12-21 12:17:30 +00:00
#include <Compression/ICompressionCodec.h>
2018-08-20 15:34:37 +00:00
#include <Core/BackgroundSchedulePool.h>
#include <Formats/FormatFactory.h>
#include <Databases/IDatabase.h>
#include <Storages/IStorage.h>
#include <Storages/MarkCache.h>
#include <Storages/MergeTree/BackgroundProcessingPool.h>
#include <Storages/MergeTree/MergeList.h>
#include <Storages/MergeTree/MergeTreeSettings.h>
2018-12-21 12:17:30 +00:00
#include <Storages/CompressionCodecSelector.h>
#include <Disks/DiskLocal.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Interpreters/ActionLocksManager.h>
#include <Core/Settings.h>
#include <Access/AccessControlManager.h>
2020-02-12 03:03:33 +00:00
#include <Access/AccessRightsContext.h>
#include <Access/RowPolicyContext.h>
#include <Access/User.h>
#include <Access/SettingsConstraints.h>
#include <Interpreters/ExpressionJIT.h>
2019-09-26 16:12:15 +00:00
#include <Dictionaries/Embedded/GeoDictionariesLoader.h>
#include <Interpreters/EmbeddedDictionaries.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/ExternalModelsLoader.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/Cluster.h>
#include <Interpreters/InterserverIOHandler.h>
#include <Interpreters/SystemLog.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
2018-04-19 13:56:14 +00:00
#include <Common/DNSResolver.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/UncompressedCache.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseQuery.h>
2019-07-29 22:26:44 +00:00
#include <Common/StackTrace.h>
#include <Common/Config/ConfigProcessor.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ShellCommand.h>
2019-07-10 20:47:39 +00:00
#include <Common/TraceCollector.h>
#include <common/logger_useful.h>
#include <Common/RemoteHostFilter.h>
2020-01-22 15:20:19 +00:00
#include <ext/singleton.h>
namespace ProfileEvents
{
extern const Event ContextLock;
extern const Event CompiledCacheSizeBytes;
}
namespace CurrentMetrics
{
extern const Metric ContextLockWait;
extern const Metric MemoryTrackingForMerges;
extern const Metric BackgroundMovePoolTask;
extern const Metric MemoryTrackingInBackgroundMoveProcessingPool;
}
2012-08-02 17:33:31 +00:00
namespace DB
{
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int BAD_ARGUMENTS;
extern const int BAD_GET;
extern const int UNKNOWN_DATABASE;
extern const int UNKNOWN_TABLE;
extern const int TABLE_ALREADY_EXISTS;
extern const int DATABASE_ALREADY_EXISTS;
extern const int THERE_IS_NO_SESSION;
extern const int THERE_IS_NO_QUERY;
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT;
extern const int SESSION_NOT_FOUND;
extern const int SESSION_IS_LOCKED;
2019-09-03 19:53:59 +00:00
extern const int LOGICAL_ERROR;
2019-10-19 20:36:35 +00:00
extern const int UNKNOWN_SCALAR;
extern const int AUTHENTICATION_FAILED;
}
2015-04-16 06:12:35 +00:00
/** Set of known objects (environment), that could be used in query.
* Shared (global) part. Order of members (especially, order of destruction) is very important.
2015-04-16 06:12:35 +00:00
*/
struct ContextShared
{
Logger * log = &Logger::get("Context");
/// For access of most of shared objects. Recursive mutex.
mutable std::recursive_mutex mutex;
/// Separate mutex for access of dictionaries. Separate mutex to avoid locks when server doing request to itself.
mutable std::mutex embedded_dictionaries_mutex;
mutable std::mutex external_dictionaries_mutex;
mutable std::mutex external_models_mutex;
2020-01-11 09:50:41 +00:00
/// Separate mutex for re-initialization of zookeeper session. This operation could take a long time and must not interfere with another operations.
mutable std::mutex zookeeper_mutex;
2017-04-02 17:37:49 +00:00
mutable zkutil::ZooKeeperPtr zookeeper; /// Client for ZooKeeper.
2017-04-02 17:37:49 +00:00
String interserver_io_host; /// The host name by which this server is available for other servers.
UInt16 interserver_io_port = 0; /// and port.
String interserver_io_user;
String interserver_io_password;
String interserver_scheme; /// http or https
2017-04-02 17:37:49 +00:00
String path; /// Path to the data directory, with a slash at the end.
String flags_path; /// Path to the directory with some control flags for server maintenance.
String user_files_path; /// Path to the directory with user provided files, usable by 'file' table function.
String dictionaries_lib_path; /// Path to the directory with user provided binaries and libraries for external dictionaries.
ConfigurationPtr config; /// Global configuration settings.
String tmp_path; /// Path to the temporary files that occur when processing the request.
mutable VolumePtr tmp_volume; /// Volume for the the temporary files that occur when processing the request.
2017-04-02 17:37:49 +00:00
Databases databases; /// List of databases and tables in them.
2019-02-04 23:18:04 +00:00
mutable std::optional<EmbeddedDictionaries> embedded_dictionaries; /// Metrica's dictionaries. Have lazy initialization.
mutable std::optional<ExternalDictionariesLoader> external_dictionaries_loader;
mutable std::optional<ExternalModelsLoader> external_models_loader;
String default_profile_name; /// Default profile name used for default values.
String system_profile_name; /// Profile used by system processes
AccessControlManager access_control_manager;
2017-04-02 17:37:49 +00:00
mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks.
mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files.
ProcessList process_list; /// Executing queries at the moment.
MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree)
ViewDependencies view_dependencies; /// Current dependencies
ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections.
InterserverIOHandler interserver_io_handler; /// Handler for interserver communication.
2019-02-04 23:18:04 +00:00
std::optional<BackgroundProcessingPool> background_pool; /// The thread pool for the background work performed by the tables.
std::optional<BackgroundProcessingPool> background_move_pool; /// The thread pool for the background moves performed by the tables.
2019-02-04 23:18:04 +00:00
std::optional<BackgroundSchedulePool> schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables)
2018-03-13 23:44:23 +00:00
MultiVersion<Macros> macros; /// Substitutions extracted from config.
std::unique_ptr<DDLWorker> ddl_worker; /// Process ddl commands from zk.
/// Rules for selecting the compression settings, depending on the size of the part.
2018-12-21 12:17:30 +00:00
mutable std::unique_ptr<CompressionCodecSelector> compression_codec_selector;
/// Storage disk chooser for MergeTree engines
2019-11-27 09:39:44 +00:00
mutable std::unique_ptr<DiskSelector> merge_tree_disk_selector;
/// Storage policy chooser for MergeTree engines
2019-11-27 09:39:44 +00:00
mutable std::unique_ptr<StoragePolicySelector> merge_tree_storage_policy_selector;
2019-09-04 12:44:12 +00:00
std::optional<MergeTreeSettings> merge_tree_settings; /// Settings of MergeTree* engines.
std::atomic_size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default)
std::atomic_size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default)
String format_schema_path; /// Path to a directory that contains schema files used by input formats.
ActionLocksManagerPtr action_locks_manager; /// Set of storages' action lockers
std::optional<SystemLogs> system_logs; /// Used to log queries and operations on parts
RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml
2019-02-03 09:57:12 +00:00
/// Named sessions. The user could specify session identifier to reuse settings and temporary tables in subsequent requests.
class SessionKeyHash
{
public:
size_t operator()(const Context::SessionKey & key) const
{
SipHash hash;
hash.update(key.first);
hash.update(key.second);
return hash.get64();
}
};
using Sessions = std::unordered_map<Context::SessionKey, std::shared_ptr<Context>, SessionKeyHash>;
using CloseTimes = std::deque<std::vector<Context::SessionKey>>;
mutable Sessions sessions;
mutable CloseTimes close_times;
std::chrono::steady_clock::duration close_interval = std::chrono::seconds(1);
std::chrono::steady_clock::time_point close_cycle_time = std::chrono::steady_clock::now();
UInt64 close_cycle = 0;
/// Clusters for distributed tables
/// Initialized on demand (on distributed storages initialization) since Settings should be initialized
std::unique_ptr<Clusters> clusters;
2020-01-11 09:50:41 +00:00
ConfigurationPtr clusters_config; /// Stores updated configs
mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config
#if USE_EMBEDDED_COMPILER
std::shared_ptr<CompiledExpressionCache> compiled_expression_cache;
#endif
bool shutdown_called = false;
2017-04-02 17:37:49 +00:00
/// Do not allow simultaneous execution of DDL requests on the same table.
2019-10-22 10:47:43 +00:00
/// database -> object -> (mutex, counter), counter: how many threads are running a query on the table at the same time
2018-09-24 18:02:25 +00:00
/// For the duration of the operation, an element is placed here, and an object is returned,
2018-09-19 09:34:07 +00:00
/// which deletes the element in the destructor when counter becomes zero.
/// In case the element already exists, waits, when query will be executed in other thread. See class DDLGuard below.
using DDLGuards = std::unordered_map<String, DDLGuard::Map>;
DDLGuards ddl_guards;
2017-04-02 17:37:49 +00:00
/// If you capture mutex and ddl_guards_mutex, then you need to grab them strictly in this order.
mutable std::mutex ddl_guards_mutex;
Stopwatch uptime_watch;
Context::ApplicationType application_type = Context::ApplicationType::SERVER;
/// vector of xdbc-bridge commands, they will be killed when Context will be destroyed
std::vector<std::unique_ptr<ShellCommand>> bridge_commands;
Context::ConfigReloadCallback config_reload_callback;
2019-09-26 16:12:15 +00:00
ContextShared()
: macros(std::make_unique<Macros>())
2017-06-19 20:35:53 +00:00
{
/// TODO: make it singleton (?)
2017-06-19 20:35:53 +00:00
static std::atomic<size_t> num_calls{0};
if (++num_calls > 1)
{
std::cerr << "Attempting to create multiple ContextShared instances. Stack trace:\n" << StackTrace().toString();
std::cerr.flush();
std::terminate();
}
}
~ContextShared()
{
try
{
shutdown();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
2017-04-02 17:37:49 +00:00
/** Perform a complex job of destroying objects in advance.
*/
void shutdown()
{
if (shutdown_called)
return;
shutdown_called = true;
2019-06-21 17:25:47 +00:00
/** After system_logs have been shut down it is guaranteed that no system table gets created or written to.
* Note that part changes at shutdown won't be logged to part log.
*/
2019-03-21 19:22:38 +00:00
if (system_logs)
system_logs->shutdown();
2017-04-02 17:37:49 +00:00
/** At this point, some tables may have threads that block our mutex.
2019-03-21 19:22:38 +00:00
* To shutdown them correctly, we will copy the current list of tables,
2017-04-02 17:37:49 +00:00
* and ask them all to finish their work.
* Then delete all objects with tables.
*/
Databases current_databases;
{
std::lock_guard lock(mutex);
current_databases = databases;
}
2019-03-21 19:22:38 +00:00
/// We still hold "databases" in Context (instead of std::move) for Buffer tables to flush data correctly.
for (auto & database : current_databases)
database.second->shutdown();
{
std::lock_guard lock(mutex);
databases.clear();
}
2019-02-04 23:18:04 +00:00
/// Preemptive destruction is important, because these objects may have a refcount to ContextShared (cyclic reference).
/// TODO: Get rid of this.
2019-06-30 01:56:16 +00:00
system_logs.reset();
2019-02-04 23:18:04 +00:00
embedded_dictionaries.reset();
external_dictionaries_loader.reset();
external_models_loader.reset();
2019-02-04 23:18:04 +00:00
background_pool.reset();
background_move_pool.reset();
2019-02-04 23:18:04 +00:00
schedule_pool.reset();
ddl_worker.reset();
2019-02-09 22:26:51 +00:00
ext::Singleton<TraceCollector>::reset();
}
2019-05-19 20:22:44 +00:00
void initializeTraceCollector(std::shared_ptr<TraceLog> trace_log)
2019-02-09 22:40:47 +00:00
{
if (trace_log == nullptr)
return;
2020-01-22 15:20:19 +00:00
ext::Singleton<TraceCollector>()->setTraceLog(trace_log);
}
2015-04-16 06:12:35 +00:00
};
Context::Context() = default;
Context::Context(const Context &) = default;
Context & Context::operator=(const Context &) = default;
2019-09-26 16:12:15 +00:00
Context Context::createGlobal()
2015-04-16 06:12:35 +00:00
{
Context res;
2020-01-12 21:00:55 +00:00
res.access_rights = std::make_shared<AccessRightsContext>();
2020-02-12 03:03:33 +00:00
res.initial_row_policy = std::make_shared<RowPolicyContext>();
2019-09-27 09:02:06 +00:00
res.shared = std::make_shared<ContextShared>();
return res;
2015-04-16 06:12:35 +00:00
}
Context::~Context() = default;
2015-04-16 06:12:35 +00:00
InterserverIOHandler & Context::getInterserverIOHandler() { return shared->interserver_io_handler; }
std::unique_lock<std::recursive_mutex> Context::getLock() const
{
ProfileEvents::increment(ProfileEvents::ContextLock);
CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait};
return std::unique_lock(shared->mutex);
}
ProcessList & Context::getProcessList() { return shared->process_list; }
const ProcessList & Context::getProcessList() const { return shared->process_list; }
MergeList & Context::getMergeList() { return shared->merge_list; }
const MergeList & Context::getMergeList() const { return shared->merge_list; }
2015-04-16 06:12:35 +00:00
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
const Databases Context::getDatabases() const
{
auto lock = getLock();
return shared->databases;
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
}
Databases Context::getDatabases()
{
auto lock = getLock();
return shared->databases;
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
}
Context::SessionKey Context::getSessionKey(const String & session_id) const
{
auto & user_name = client_info.current_user;
if (user_name.empty())
throw Exception("Empty user name.", ErrorCodes::LOGICAL_ERROR);
return SessionKey(user_name, session_id);
}
void Context::scheduleCloseSession(const Context::SessionKey & key, std::chrono::steady_clock::duration timeout)
{
const UInt64 close_index = timeout / shared->close_interval + 1;
const auto new_close_cycle = shared->close_cycle + close_index;
if (session_close_cycle != new_close_cycle)
{
session_close_cycle = new_close_cycle;
if (shared->close_times.size() < close_index + 1)
shared->close_times.resize(close_index + 1);
shared->close_times[close_index].emplace_back(key);
}
}
std::shared_ptr<Context> Context::acquireSession(const String & session_id, std::chrono::steady_clock::duration timeout, bool session_check) const
{
auto lock = getLock();
const auto & key = getSessionKey(session_id);
auto it = shared->sessions.find(key);
if (it == shared->sessions.end())
{
if (session_check)
throw Exception("Session not found.", ErrorCodes::SESSION_NOT_FOUND);
auto new_session = std::make_shared<Context>(*this);
new_session->scheduleCloseSession(key, timeout);
it = shared->sessions.insert(std::make_pair(key, std::move(new_session))).first;
}
else if (it->second->client_info.current_user != client_info.current_user)
{
throw Exception("Session belongs to a different user", ErrorCodes::LOGICAL_ERROR);
}
const auto & session = it->second;
if (session->session_is_used)
throw Exception("Session is locked by a concurrent client.", ErrorCodes::SESSION_IS_LOCKED);
session->session_is_used = true;
session->client_info = client_info;
return session;
}
void Context::releaseSession(const String & session_id, std::chrono::steady_clock::duration timeout)
{
auto lock = getLock();
session_is_used = false;
scheduleCloseSession(getSessionKey(session_id), timeout);
}
std::chrono::steady_clock::duration Context::closeSessions() const
{
auto lock = getLock();
const auto now = std::chrono::steady_clock::now();
if (now < shared->close_cycle_time)
return shared->close_cycle_time - now;
const auto current_cycle = shared->close_cycle;
++shared->close_cycle;
shared->close_cycle_time = now + shared->close_interval;
if (shared->close_times.empty())
return shared->close_interval;
auto & sessions_to_close = shared->close_times.front();
for (const auto & key : sessions_to_close)
{
const auto session = shared->sessions.find(key);
if (session != shared->sessions.end() && session->second->session_close_cycle <= current_cycle)
{
if (session->second->session_is_used)
session->second->scheduleCloseSession(key, std::chrono::seconds(0));
else
shared->sessions.erase(session);
}
}
shared->close_times.pop_front();
return shared->close_interval;
}
static String resolveDatabase(const String & database_name, const String & current_database)
{
String res = database_name.empty() ? current_database : database_name;
if (res.empty())
throw Exception("Default database is not selected", ErrorCodes::UNKNOWN_DATABASE);
return res;
}
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
const DatabasePtr Context::getDatabase(const String & database_name) const
{
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
assertDatabaseExists(db);
return shared->databases[db];
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
}
DatabasePtr Context::getDatabase(const String & database_name)
{
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
assertDatabaseExists(db);
return shared->databases[db];
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
}
const DatabasePtr Context::tryGetDatabase(const String & database_name) const
{
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
auto it = shared->databases.find(db);
if (it == shared->databases.end())
return {};
return it->second;
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
}
DatabasePtr Context::tryGetDatabase(const String & database_name)
{
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
auto it = shared->databases.find(db);
if (it == shared->databases.end())
return {};
return it->second;
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
}
2012-08-02 17:33:31 +00:00
String Context::getPath() const
{
auto lock = getLock();
return shared->path;
2012-08-02 17:33:31 +00:00
}
String Context::getFlagsPath() const
{
auto lock = getLock();
return shared->flags_path;
}
String Context::getUserFilesPath() const
{
auto lock = getLock();
return shared->user_files_path;
}
2012-08-02 17:33:31 +00:00
String Context::getDictionariesLibPath() const
{
auto lock = getLock();
return shared->dictionaries_lib_path;
}
VolumePtr Context::getTemporaryVolume() const
{
auto lock = getLock();
return shared->tmp_volume;
}
2012-08-02 17:33:31 +00:00
void Context::setPath(const String & path)
{
auto lock = getLock();
shared->path = path;
if (shared->tmp_path.empty() && !shared->tmp_volume)
shared->tmp_path = shared->path + "tmp/";
if (shared->flags_path.empty())
shared->flags_path = shared->path + "flags/";
if (shared->user_files_path.empty())
shared->user_files_path = shared->path + "user_files/";
if (shared->dictionaries_lib_path.empty())
shared->dictionaries_lib_path = shared->path + "dictionaries_lib/";
2012-08-02 17:33:31 +00:00
}
VolumePtr Context::setTemporaryStorage(const String & path, const String & policy_name)
{
auto lock = getLock();
if (policy_name.empty())
{
shared->tmp_path = path;
if (!shared->tmp_path.ends_with('/'))
shared->tmp_path += '/';
auto disk = std::make_shared<DiskLocal>("_tmp_default", shared->tmp_path, 0);
shared->tmp_volume = std::make_shared<Volume>("_tmp_default", std::vector<DiskPtr>{disk}, 0);
}
else
{
StoragePolicyPtr tmp_policy = getStoragePolicySelector()[policy_name];
if (tmp_policy->getVolumes().size() != 1)
throw Exception("Policy " + policy_name + " is used temporary files, such policy should have exactly one volume", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
shared->tmp_volume = tmp_policy->getVolume(0);
}
if (!shared->tmp_volume->disks.size())
throw Exception("No disks volume for temporary files", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
return shared->tmp_volume;
}
void Context::setFlagsPath(const String & path)
{
auto lock = getLock();
shared->flags_path = path;
}
void Context::setUserFilesPath(const String & path)
{
auto lock = getLock();
shared->user_files_path = path;
}
void Context::setDictionariesLibPath(const String & path)
{
auto lock = getLock();
shared->dictionaries_lib_path = path;
}
void Context::setConfig(const ConfigurationPtr & config)
{
auto lock = getLock();
shared->config = config;
}
const Poco::Util::AbstractConfiguration & Context::getConfigRef() const
{
auto lock = getLock();
return shared->config ? *shared->config : Poco::Util::Application::instance().config();
}
2012-08-02 17:33:31 +00:00
2020-02-12 03:03:33 +00:00
AccessControlManager & Context::getAccessControlManager()
{
return shared->access_control_manager;
}
const AccessControlManager & Context::getAccessControlManager() const
{
return shared->access_control_manager;
}
2020-02-12 03:03:33 +00:00
void Context::setUsersConfig(const ConfigurationPtr & config)
{
auto lock = getLock();
shared->users_config = config;
shared->access_control_manager.setUsersConfig(*shared->users_config);
2020-02-12 03:03:33 +00:00
}
ConfigurationPtr Context::getUsersConfig()
{
auto lock = getLock();
return shared->users_config;
}
void Context::setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address, const String & quota_key)
{
auto lock = getLock();
client_info.current_user = name;
client_info.current_password = password;
client_info.current_address = address;
if (!quota_key.empty())
client_info.quota_key = quota_key;
auto new_user_id = getAccessControlManager().find<User>(name);
AccessRightsContextPtr new_access_rights;
if (new_user_id)
{
new_access_rights = getAccessControlManager().getAccessRightsContext(*new_user_id, {}, true, settings, current_database, client_info);
if (!new_access_rights->isClientHostAllowed() || !new_access_rights->isCorrectPassword(password))
{
new_user_id = {};
new_access_rights = nullptr;
}
}
if (!new_user_id || !new_access_rights)
throw Exception(name + ": Authentication failed: password is incorrect or there is no user with such name", ErrorCodes::AUTHENTICATION_FAILED);
2020-02-12 03:03:33 +00:00
user_id = new_user_id;
access_rights = std::move(new_access_rights);
2020-02-21 00:17:07 +00:00
current_roles.clear();
use_default_roles = true;
2020-02-12 03:03:33 +00:00
calculateUserSettings();
}
std::shared_ptr<const User> Context::getUser() const
{
auto lock = getLock();
return access_rights->getUser();
}
String Context::getUserName() const
{
auto lock = getLock();
return access_rights->getUserName();
}
UUID Context::getUserID() const
{
auto lock = getLock();
if (!user_id)
throw Exception("No current user", ErrorCodes::LOGICAL_ERROR);
return *user_id;
}
2020-02-21 00:17:07 +00:00
void Context::setCurrentRoles(const std::vector<UUID> & current_roles_)
{
auto lock = getLock();
if (current_roles == current_roles_ && !use_default_roles)
return;
current_roles = current_roles_;
use_default_roles = false;
calculateAccessRights();
}
void Context::setCurrentRolesDefault()
{
auto lock = getLock();
if (use_default_roles)
return;
current_roles.clear();
use_default_roles = true;
calculateAccessRights();
}
std::vector<UUID> Context::getCurrentRoles() const
{
return getAccessRights()->getCurrentRoles();
}
Strings Context::getCurrentRolesNames() const
{
return getAccessRights()->getCurrentRolesNames();
}
std::vector<UUID> Context::getEnabledRoles() const
{
return getAccessRights()->getEnabledRoles();
}
Strings Context::getEnabledRolesNames() const
{
return getAccessRights()->getEnabledRolesNames();
}
2020-02-12 03:03:33 +00:00
void Context::calculateAccessRights()
{
auto lock = getLock();
if (user_id)
2020-02-21 00:17:07 +00:00
access_rights = getAccessControlManager().getAccessRightsContext(*user_id, current_roles, use_default_roles, settings, current_database, client_info);
2020-02-12 03:03:33 +00:00
}
2020-01-12 21:00:55 +00:00
template <typename... Args>
void Context::checkAccessImpl(const Args &... args) const
{
2020-02-12 03:03:33 +00:00
getAccessRights()->checkAccess(args...);
2020-01-12 21:00:55 +00:00
}
void Context::checkAccess(const AccessFlags & access) const { return checkAccessImpl(access); }
void Context::checkAccess(const AccessFlags & access, const std::string_view & database) const { return checkAccessImpl(access, database); }
void Context::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl(access, database, table); }
void Context::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl(access, database, table, column); }
void Context::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const { return checkAccessImpl(access, database, table, columns); }
void Context::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl(access, database, table, columns); }
void Context::checkAccess(const AccessRightsElement & access) const { return checkAccessImpl(access); }
void Context::checkAccess(const AccessRightsElements & access) const { return checkAccessImpl(access); }
2020-02-12 03:03:33 +00:00
AccessRightsContextPtr Context::getAccessRights() const
{
2020-02-12 03:03:33 +00:00
auto lock = getLock();
return access_rights;
}
2020-01-12 21:00:55 +00:00
2020-02-12 03:03:33 +00:00
RowPolicyContextPtr Context::getRowPolicy() const
{
return getAccessRights()->getRowPolicy();
}
void Context::setInitialRowPolicy()
{
auto lock = getLock();
auto initial_user_id = getAccessControlManager().find<User>(client_info.initial_user);
if (initial_user_id)
2020-02-21 00:17:07 +00:00
initial_row_policy = getAccessControlManager().getRowPolicyContext(*initial_user_id, {});
}
2020-02-12 03:03:33 +00:00
RowPolicyContextPtr Context::getInitialRowPolicy() const
{
auto lock = getLock();
2020-02-12 03:03:33 +00:00
return initial_row_policy;
}
2020-02-12 03:03:33 +00:00
QuotaContextPtr Context::getQuota() const
{
2020-02-12 03:03:33 +00:00
return getAccessRights()->getQuota();
}
2020-02-12 03:03:33 +00:00
void Context::calculateUserSettings()
{
auto lock = getLock();
2020-02-12 03:03:33 +00:00
String profile = getUser()->profile;
bool old_readonly = settings.readonly;
bool old_allow_ddl = settings.allow_ddl;
bool old_allow_introspection_functions = settings.allow_introspection_functions;
/// 1) Set default settings (hardcoded values)
/// NOTE: we ignore global_context settings (from which it is usually copied)
/// NOTE: global_context settings are immutable and not auto updated
settings = Settings();
settings_constraints = nullptr;
/// 2) Apply settings from default profile
auto default_profile_name = getDefaultProfileName();
if (profile != default_profile_name)
setProfile(default_profile_name);
/// 3) Apply settings from current user
setProfile(profile);
2020-02-12 03:03:33 +00:00
/// 4) Recalculate access rights if it's necessary.
if ((settings.readonly != old_readonly) || (settings.allow_ddl != old_allow_ddl) || (settings.allow_introspection_functions != old_allow_introspection_functions))
calculateAccessRights();
2020-01-12 21:00:55 +00:00
}
void Context::setProfile(const String & profile)
{
settings.setProfile(profile, *shared->users_config);
auto new_constraints
= settings_constraints ? std::make_shared<SettingsConstraints>(*settings_constraints) : std::make_shared<SettingsConstraints>();
new_constraints->setProfile(profile, *shared->users_config);
settings_constraints = std::move(new_constraints);
}
2019-12-05 11:42:13 +00:00
void Context::addDependencyUnsafe(const StorageID & from, const StorageID & where)
{
shared->view_dependencies[from].insert(where);
// Notify table of dependencies change
2019-12-10 19:48:16 +00:00
auto table = tryGetTable(from);
if (table != nullptr)
table->updateDependencies();
}
2019-12-05 11:42:13 +00:00
void Context::addDependency(const StorageID & from, const StorageID & where)
{
auto lock = getLock();
addDependencyUnsafe(from, where);
}
2019-12-05 11:42:13 +00:00
void Context::removeDependencyUnsafe(const StorageID & from, const StorageID & where)
{
shared->view_dependencies[from].erase(where);
// Notify table of dependencies change
2019-12-10 19:48:16 +00:00
auto table = tryGetTable(from);
if (table != nullptr)
table->updateDependencies();
}
2019-12-05 11:42:13 +00:00
void Context::removeDependency(const StorageID & from, const StorageID & where)
{
auto lock = getLock();
removeDependencyUnsafe(from, where);
}
2019-12-05 11:42:13 +00:00
Dependencies Context::getDependencies(const StorageID & from) const
{
auto lock = getLock();
2019-12-05 11:42:13 +00:00
String db = resolveDatabase(from.database_name, current_database);
2019-12-27 19:30:22 +00:00
ViewDependencies::const_iterator iter = shared->view_dependencies.find(StorageID(db, from.table_name, from.uuid));
if (iter == shared->view_dependencies.end())
return {};
return Dependencies(iter->second.begin(), iter->second.end());
}
2012-08-02 17:33:31 +00:00
bool Context::isTableExist(const String & database_name, const String & table_name) const
{
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
Databases::const_iterator it = shared->databases.find(db);
return shared->databases.end() != it
&& it->second->isTableExist(*this, table_name);
2012-08-02 17:33:31 +00:00
}
bool Context::isDictionaryExists(const String & database_name, const String & dictionary_name) const
2012-08-02 17:33:31 +00:00
{
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
Databases::const_iterator it = shared->databases.find(db);
return shared->databases.end() != it && it->second->isDictionaryExist(*this, dictionary_name);
}
bool Context::isDatabaseExist(const String & database_name) const
2012-08-02 17:33:31 +00:00
{
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
return shared->databases.end() != shared->databases.find(db);
}
bool Context::isExternalTableExist(const String & table_name) const
{
return external_tables.end() != external_tables.find(table_name);
2012-08-02 17:33:31 +00:00
}
void Context::assertTableDoesntExist(const String & database_name, const String & table_name) const
2012-08-02 17:33:31 +00:00
{
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
Databases::const_iterator it = shared->databases.find(db);
if (shared->databases.end() != it && it->second->isTableExist(*this, table_name))
throw Exception("Table " + backQuoteIfNeed(db) + "." + backQuoteIfNeed(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
2012-08-02 17:33:31 +00:00
}
void Context::assertDatabaseExists(const String & database_name) const
2012-08-02 17:33:31 +00:00
{
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
if (shared->databases.end() == shared->databases.find(db))
throw Exception("Database " + backQuoteIfNeed(db) + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
2012-08-02 17:33:31 +00:00
}
void Context::assertDatabaseDoesntExist(const String & database_name) const
{
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
if (shared->databases.end() != shared->databases.find(db))
throw Exception("Database " + backQuoteIfNeed(db) + " already exists.", ErrorCodes::DATABASE_ALREADY_EXISTS);
2012-08-02 17:33:31 +00:00
}
2019-10-19 20:36:35 +00:00
const Scalars & Context::getScalars() const
{
return scalars;
}
const Block & Context::getScalar(const String & name) const
{
auto it = scalars.find(name);
if (scalars.end() == it)
throw Exception("Scalar " + backQuoteIfNeed(name) + " doesn't exist (internal bug)", ErrorCodes::UNKNOWN_SCALAR);
return it->second;
}
Tables Context::getExternalTables() const
{
auto lock = getLock();
Tables res;
2018-02-23 01:23:57 +00:00
for (auto & table : external_tables)
res[table.first] = table.second.first;
if (session_context && session_context != this)
{
Tables buf = session_context->getExternalTables();
res.insert(buf.begin(), buf.end());
}
else if (global_context && global_context != this)
{
Tables buf = global_context->getExternalTables();
res.insert(buf.begin(), buf.end());
}
return res;
}
StoragePtr Context::tryGetExternalTable(const String & table_name) const
{
TableAndCreateASTs::const_iterator jt = external_tables.find(table_name);
if (external_tables.end() == jt)
return StoragePtr();
return jt->second.first;
}
2012-08-02 17:33:31 +00:00
StoragePtr Context::getTable(const String & database_name, const String & table_name) const
2019-12-10 19:48:16 +00:00
{
return getTable(StorageID(database_name, table_name));
}
StoragePtr Context::getTable(const StorageID & table_id) const
{
std::optional<Exception> exc;
2019-12-10 19:48:16 +00:00
auto res = getTableImpl(table_id, &exc);
if (!res)
2020-01-10 12:47:59 +00:00
throw *exc;
return res;
}
StoragePtr Context::tryGetTable(const String & database_name, const String & table_name) const
{
return getTableImpl(StorageID(database_name, table_name), {});
2019-12-10 19:48:16 +00:00
}
StoragePtr Context::tryGetTable(const StorageID & table_id) const
{
return getTableImpl(table_id, {});
}
StoragePtr Context::getTableImpl(const StorageID & table_id, std::optional<Exception> * exception) const
2012-08-02 17:33:31 +00:00
{
String db;
DatabasePtr database;
2012-08-02 17:33:31 +00:00
{
auto lock = getLock();
2019-12-10 19:48:16 +00:00
if (table_id.database_name.empty())
{
2019-12-10 19:48:16 +00:00
StoragePtr res = tryGetExternalTable(table_id.table_name);
if (res)
return res;
}
2019-12-10 19:48:16 +00:00
db = resolveDatabase(table_id.database_name, current_database);
Databases::const_iterator it = shared->databases.find(db);
if (shared->databases.end() == it)
{
if (exception)
exception->emplace("Database " + backQuoteIfNeed(db) + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
return {};
}
database = it->second;
}
2019-12-10 19:48:16 +00:00
auto table = database->tryGetTable(*this, table_id.table_name);
if (!table)
{
if (exception)
exception->emplace("Table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
return {};
}
return table;
}
2012-08-02 17:33:31 +00:00
2018-02-23 01:23:57 +00:00
void Context::addExternalTable(const String & table_name, const StoragePtr & storage, const ASTPtr & ast)
{
if (external_tables.end() != external_tables.find(table_name))
throw Exception("Temporary table " + backQuoteIfNeed(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
external_tables[table_name] = std::pair(storage, ast);
}
2019-10-19 20:36:35 +00:00
void Context::addScalar(const String & name, const Block & block)
{
scalars[name] = block;
}
bool Context::hasScalar(const String & name) const
{
return scalars.count(name);
}
StoragePtr Context::tryRemoveExternalTable(const String & table_name)
{
TableAndCreateASTs::const_iterator it = external_tables.find(table_name);
if (external_tables.end() == it)
return StoragePtr();
auto storage = it->second.first;
external_tables.erase(it);
return storage;
}
StoragePtr Context::executeTableFunction(const ASTPtr & table_expression)
{
/// Slightly suboptimal.
auto hash = table_expression->getTreeHash();
String key = toString(hash.first) + '_' + toString(hash.second);
StoragePtr & res = table_function_results[key];
if (!res)
{
2019-03-11 13:22:51 +00:00
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression->as<ASTFunction>()->name, *this);
/// Run it and remember the result
2019-07-18 18:29:49 +00:00
res = table_function_ptr->execute(table_expression, *this, table_function_ptr->getName());
}
return res;
}
void Context::addViewSource(const StoragePtr & storage)
{
if (view_source)
throw Exception(
"Temporary view source storage " + backQuoteIfNeed(view_source->getName()) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
view_source = storage;
}
StoragePtr Context::getViewSource()
{
return view_source;
}
2018-09-24 19:08:11 +00:00
DDLGuard::DDLGuard(Map & map_, std::unique_lock<std::mutex> guards_lock_, const String & elem)
: map(map_), guards_lock(std::move(guards_lock_))
{
2018-09-18 18:33:15 +00:00
it = map.emplace(elem, Entry{std::make_unique<std::mutex>(), 0}).first;
++it->second.counter;
2018-09-24 19:08:11 +00:00
guards_lock.unlock();
2019-01-02 06:44:36 +00:00
table_lock = std::unique_lock(*it->second.mutex);
}
DDLGuard::~DDLGuard()
{
2018-09-24 19:08:11 +00:00
guards_lock.lock();
2018-09-18 18:33:15 +00:00
--it->second.counter;
if (!it->second.counter)
{
table_lock.unlock();
map.erase(it);
}
}
2018-09-18 18:33:15 +00:00
std::unique_ptr<DDLGuard> Context::getDDLGuard(const String & database, const String & table) const
{
2019-01-02 06:44:36 +00:00
std::unique_lock lock(shared->ddl_guards_mutex);
2018-09-24 19:08:11 +00:00
return std::make_unique<DDLGuard>(shared->ddl_guards[database], std::move(lock), table);
}
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
void Context::addDatabase(const String & database_name, const DatabasePtr & database)
2012-08-02 17:33:31 +00:00
{
auto lock = getLock();
2012-08-02 17:33:31 +00:00
assertDatabaseDoesntExist(database_name);
shared->databases[database_name] = database;
2012-08-02 17:33:31 +00:00
}
DatabasePtr Context::detachDatabase(const String & database_name)
2012-08-02 17:33:31 +00:00
{
auto lock = getLock();
auto res = getDatabase(database_name);
shared->databases.erase(database_name);
2019-10-17 17:18:54 +00:00
return res;
2012-08-02 17:33:31 +00:00
}
ASTPtr Context::getCreateExternalTableQuery(const String & table_name) const
{
TableAndCreateASTs::const_iterator jt = external_tables.find(table_name);
if (external_tables.end() == jt)
throw Exception("Temporary table " + backQuoteIfNeed(table_name) + " doesn't exist", ErrorCodes::UNKNOWN_TABLE);
return jt->second.second;
}
2012-08-02 17:33:31 +00:00
Settings Context::getSettings() const
{
return settings;
2012-08-02 17:33:31 +00:00
}
void Context::setSettings(const Settings & settings_)
{
2020-01-12 21:00:55 +00:00
auto lock = getLock();
bool old_readonly = settings.readonly;
bool old_allow_ddl = settings.allow_ddl;
bool old_allow_introspection_functions = settings.allow_introspection_functions;
settings = settings_;
2020-01-12 21:00:55 +00:00
if ((settings.readonly != old_readonly) || (settings.allow_ddl != old_allow_ddl) || (settings.allow_introspection_functions != old_allow_introspection_functions))
calculateAccessRights();
2012-08-02 17:33:31 +00:00
}
void Context::setSetting(const String & name, const String & value)
2012-08-02 19:03:32 +00:00
{
auto lock = getLock();
if (name == "profile")
{
setProfile(value);
return;
}
settings.set(name, value);
2020-01-12 21:00:55 +00:00
if (name == "readonly" || name == "allow_ddl" || name == "allow_introspection_functions")
calculateAccessRights();
}
void Context::setSetting(const String & name, const Field & value)
{
auto lock = getLock();
if (name == "profile")
{
setProfile(value.safeGet<String>());
return;
}
settings.set(name, value);
2020-01-12 21:00:55 +00:00
if (name == "readonly" || name == "allow_ddl" || name == "allow_introspection_functions")
calculateAccessRights();
}
void Context::applySettingChange(const SettingChange & change)
{
setSetting(change.name, change.value);
}
void Context::applySettingsChanges(const SettingsChanges & changes)
{
auto lock = getLock();
for (const SettingChange & change : changes)
applySettingChange(change);
}
void Context::checkSettingsConstraints(const SettingChange & change) const
{
if (settings_constraints)
settings_constraints->check(settings, change);
}
void Context::checkSettingsConstraints(const SettingsChanges & changes) const
{
if (settings_constraints)
settings_constraints->check(settings, changes);
2012-08-02 19:03:32 +00:00
}
void Context::clampToSettingsConstraints(SettingChange & change) const
{
if (settings_constraints)
settings_constraints->clamp(settings, change);
}
void Context::clampToSettingsConstraints(SettingsChanges & changes) const
{
if (settings_constraints)
settings_constraints->clamp(settings, changes);
}
2012-08-02 17:33:31 +00:00
String Context::getCurrentDatabase() const
{
return current_database;
2012-08-02 17:33:31 +00:00
}
String Context::getCurrentQueryId() const
{
return client_info.current_query_id;
}
String Context::getInitialQueryId() const
{
return client_info.initial_query_id;
}
2012-08-02 17:33:31 +00:00
void Context::setCurrentDatabase(const String & name)
{
auto lock = getLock();
assertDatabaseExists(name);
current_database = name;
2020-01-12 21:00:55 +00:00
calculateAccessRights();
2012-08-02 17:33:31 +00:00
}
void Context::setCurrentQueryId(const String & query_id)
{
if (!client_info.current_query_id.empty())
throw Exception("Logical error: attempt to set query_id twice", ErrorCodes::LOGICAL_ERROR);
String query_id_to_set = query_id;
2017-04-02 17:37:49 +00:00
if (query_id_to_set.empty()) /// If the user did not submit his query_id, then we generate it ourselves.
{
/// Generate random UUID, but using lower quality RNG,
/// because Poco::UUIDGenerator::generateRandom method is using /dev/random, that is very expensive.
/// NOTE: Actually we don't need to use UUIDs for query identifiers.
/// We could use any suitable string instead.
union
{
char bytes[16];
struct
{
UInt64 a;
UInt64 b;
} words;
} random;
2019-09-06 12:46:42 +00:00
random.words.a = thread_local_rng(); //-V656
random.words.b = thread_local_rng(); //-V656
/// Use protected constructor.
2019-08-03 11:02:40 +00:00
struct qUUID : Poco::UUID
{
2019-08-03 11:02:40 +00:00
qUUID(const char * bytes, Poco::UUID::Version version)
: Poco::UUID(bytes, version) {}
};
2019-08-03 11:02:40 +00:00
query_id_to_set = qUUID(random.bytes, Poco::UUID::UUID_RANDOM).toString();
}
client_info.current_query_id = query_id_to_set;
}
void Context::killCurrentQuery()
{
if (process_list_elem)
{
process_list_elem->cancelQuery(true);
}
};
String Context::getDefaultFormat() const
{
return default_format.empty() ? "TabSeparated" : default_format;
}
void Context::setDefaultFormat(const String & name)
{
default_format = name;
}
2018-03-13 23:44:23 +00:00
MultiVersion<Macros>::Version Context::getMacros() const
2014-08-11 15:59:01 +00:00
{
2018-03-13 23:44:23 +00:00
return shared->macros.get();
2014-08-11 15:59:01 +00:00
}
2018-03-13 23:44:23 +00:00
void Context::setMacros(std::unique_ptr<Macros> && macros)
2014-08-11 15:59:01 +00:00
{
2018-03-13 23:44:23 +00:00
shared->macros.set(std::move(macros));
2014-08-11 15:59:01 +00:00
}
const Context & Context::getQueryContext() const
{
if (!query_context)
throw Exception("There is no query", ErrorCodes::THERE_IS_NO_QUERY);
return *query_context;
}
Context & Context::getQueryContext()
{
if (!query_context)
throw Exception("There is no query", ErrorCodes::THERE_IS_NO_QUERY);
return *query_context;
}
2016-11-16 11:29:51 +00:00
const Context & Context::getSessionContext() const
{
if (!session_context)
throw Exception("There is no session", ErrorCodes::THERE_IS_NO_SESSION);
return *session_context;
2016-11-16 11:29:51 +00:00
}
2012-08-02 17:33:31 +00:00
Context & Context::getSessionContext()
{
if (!session_context)
throw Exception("There is no session", ErrorCodes::THERE_IS_NO_SESSION);
return *session_context;
2012-08-02 17:33:31 +00:00
}
const Context & Context::getGlobalContext() const
{
if (!global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
return *global_context;
}
2012-08-02 17:33:31 +00:00
Context & Context::getGlobalContext()
{
if (!global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
return *global_context;
2012-08-02 17:33:31 +00:00
}
2012-12-19 20:15:15 +00:00
const EmbeddedDictionaries & Context::getEmbeddedDictionaries() const
2012-12-19 20:15:15 +00:00
{
return getEmbeddedDictionariesImpl(false);
}
EmbeddedDictionaries & Context::getEmbeddedDictionaries()
{
return getEmbeddedDictionariesImpl(false);
}
const ExternalDictionariesLoader & Context::getExternalDictionariesLoader() const
{
2019-01-02 06:44:36 +00:00
std::lock_guard lock(shared->external_dictionaries_mutex);
if (!shared->external_dictionaries_loader)
{
if (!this->global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
shared->external_dictionaries_loader.emplace(*this->global_context);
}
return *shared->external_dictionaries_loader;
}
ExternalDictionariesLoader & Context::getExternalDictionariesLoader()
{
return const_cast<ExternalDictionariesLoader &>(const_cast<const Context *>(this)->getExternalDictionariesLoader());
}
const ExternalModelsLoader & Context::getExternalModelsLoader() const
{
2019-01-02 06:44:36 +00:00
std::lock_guard lock(shared->external_models_mutex);
if (!shared->external_models_loader)
{
if (!this->global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
shared->external_models_loader.emplace(*this->global_context);
}
return *shared->external_models_loader;
}
ExternalModelsLoader & Context::getExternalModelsLoader()
{
return const_cast<ExternalModelsLoader &>(const_cast<const Context *>(this)->getExternalModelsLoader());
}
EmbeddedDictionaries & Context::getEmbeddedDictionariesImpl(const bool throw_on_error) const
{
std::lock_guard lock(shared->embedded_dictionaries_mutex);
if (!shared->embedded_dictionaries)
{
2019-09-26 16:12:15 +00:00
auto geo_dictionaries_loader = std::make_unique<GeoDictionariesLoader>();
shared->embedded_dictionaries.emplace(
std::move(geo_dictionaries_loader),
*this->global_context,
throw_on_error);
}
return *shared->embedded_dictionaries;
2012-12-19 20:15:15 +00:00
}
void Context::tryCreateEmbeddedDictionaries() const
{
static_cast<void>(getEmbeddedDictionariesImpl(true));
}
void Context::setProgressCallback(ProgressCallback callback)
{
2017-04-02 17:37:49 +00:00
/// Callback is set to a session or to a query. In the session, only one query is processed at a time. Therefore, the lock is not needed.
progress_callback = callback;
}
ProgressCallback Context::getProgressCallback() const
{
return progress_callback;
}
void Context::setProcessListElement(ProcessList::Element * elem)
{
2017-04-02 17:37:49 +00:00
/// Set to a session or query. In the session, only one query is processed at a time. Therefore, the lock is not needed.
process_list_elem = elem;
}
ProcessList::Element * Context::getProcessListElement() const
{
return process_list_elem;
}
2014-03-28 14:36:24 +00:00
void Context::setUncompressedCache(size_t max_size_in_bytes)
{
auto lock = getLock();
if (shared->uncompressed_cache)
throw Exception("Uncompressed cache has been already created.", ErrorCodes::LOGICAL_ERROR);
shared->uncompressed_cache = std::make_shared<UncompressedCache>(max_size_in_bytes);
}
UncompressedCachePtr Context::getUncompressedCache() const
{
auto lock = getLock();
return shared->uncompressed_cache;
}
void Context::dropUncompressedCache() const
{
auto lock = getLock();
if (shared->uncompressed_cache)
shared->uncompressed_cache->reset();
}
2014-02-11 13:30:42 +00:00
void Context::setMarkCache(size_t cache_size_in_bytes)
{
auto lock = getLock();
2014-02-11 13:30:42 +00:00
if (shared->mark_cache)
throw Exception("Mark cache has been already created.", ErrorCodes::LOGICAL_ERROR);
2014-02-11 13:30:42 +00:00
shared->mark_cache = std::make_shared<MarkCache>(cache_size_in_bytes);
2014-02-11 13:30:42 +00:00
}
2014-02-11 13:30:42 +00:00
MarkCachePtr Context::getMarkCache() const
{
auto lock = getLock();
return shared->mark_cache;
2014-02-11 13:30:42 +00:00
}
void Context::dropMarkCache() const
{
auto lock = getLock();
if (shared->mark_cache)
shared->mark_cache->reset();
}
void Context::dropCaches() const
{
auto lock = getLock();
if (shared->uncompressed_cache)
shared->uncompressed_cache->reset();
if (shared->mark_cache)
shared->mark_cache->reset();
}
2014-07-02 12:30:38 +00:00
BackgroundProcessingPool & Context::getBackgroundPool()
{
auto lock = getLock();
if (!shared->background_pool)
2019-11-26 10:41:52 +00:00
shared->background_pool.emplace(settings.background_pool_size);
return *shared->background_pool;
2014-07-02 12:30:38 +00:00
}
BackgroundProcessingPool & Context::getBackgroundMovePool()
2019-11-07 08:54:28 +00:00
{
auto lock = getLock();
if (!shared->background_move_pool)
2019-12-05 08:05:07 +00:00
{
BackgroundProcessingPool::PoolSettings pool_settings;
auto & config = getConfigRef();
pool_settings.thread_sleep_seconds = config.getDouble("background_move_processing_pool_thread_sleep_seconds", 10);
pool_settings.thread_sleep_seconds_random_part = config.getDouble("background_move_processing_pool_thread_sleep_seconds_random_part", 1.0);
pool_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_move_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1);
pool_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 10);
pool_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 600);
pool_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1);
pool_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0);
pool_settings.tasks_metric = CurrentMetrics::BackgroundMovePoolTask;
pool_settings.memory_metric = CurrentMetrics::MemoryTrackingInBackgroundMoveProcessingPool;
2019-12-05 08:05:07 +00:00
shared->background_move_pool.emplace(settings.background_move_pool_size, pool_settings, "BackgroundMovePool", "BgMoveProcPool");
}
return *shared->background_move_pool;
2019-11-07 08:54:28 +00:00
}
BackgroundSchedulePool & Context::getSchedulePool()
{
auto lock = getLock();
if (!shared->schedule_pool)
2019-02-04 23:18:04 +00:00
shared->schedule_pool.emplace(settings.background_schedule_pool_size);
return *shared->schedule_pool;
}
void Context::setDDLWorker(std::unique_ptr<DDLWorker> ddl_worker)
{
auto lock = getLock();
if (shared->ddl_worker)
throw Exception("DDL background thread has already been initialized.", ErrorCodes::LOGICAL_ERROR);
shared->ddl_worker = std::move(ddl_worker);
}
DDLWorker & Context::getDDLWorker() const
{
auto lock = getLock();
if (!shared->ddl_worker)
throw Exception("DDL background thread is not initialized.", ErrorCodes::LOGICAL_ERROR);
return *shared->ddl_worker;
}
2014-05-13 10:10:26 +00:00
zkutil::ZooKeeperPtr Context::getZooKeeper() const
2014-03-21 19:17:59 +00:00
{
2019-01-02 06:44:36 +00:00
std::lock_guard lock(shared->zookeeper_mutex);
2014-04-25 13:55:15 +00:00
if (!shared->zookeeper)
shared->zookeeper = std::make_shared<zkutil::ZooKeeper>(getConfigRef(), "zookeeper");
else if (shared->zookeeper->expired())
shared->zookeeper = shared->zookeeper->startNewSession();
2014-04-25 13:55:15 +00:00
return shared->zookeeper;
2014-03-21 19:17:59 +00:00
}
void Context::resetZooKeeper() const
{
std::lock_guard lock(shared->zookeeper_mutex);
shared->zookeeper.reset();
}
bool Context::hasZooKeeper() const
{
return getConfigRef().has("zookeeper");
}
2014-03-21 19:17:59 +00:00
void Context::setInterserverIOAddress(const String & host, UInt16 port)
{
shared->interserver_io_host = host;
shared->interserver_io_port = port;
}
std::pair<String, UInt16> Context::getInterserverIOAddress() const
{
if (shared->interserver_io_host.empty() || shared->interserver_io_port == 0)
throw Exception("Parameter 'interserver_http(s)_port' required for replication is not specified in configuration file.",
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
return { shared->interserver_io_host, shared->interserver_io_port };
}
2020-01-12 21:00:55 +00:00
void Context::setInterserverCredentials(const String & user_, const String & password)
{
2020-01-12 21:00:55 +00:00
shared->interserver_io_user = user_;
shared->interserver_io_password = password;
}
std::pair<String, String> Context::getInterserverCredentials() const
{
return { shared->interserver_io_user, shared->interserver_io_password };
}
void Context::setInterserverScheme(const String & scheme)
{
shared->interserver_scheme = scheme;
}
String Context::getInterserverScheme() const
{
return shared->interserver_scheme;
}
void Context::setRemoteHostFilter(const Poco::Util::AbstractConfiguration & config)
{
shared->remote_host_filter.setValuesFromConfig(config);
}
2019-10-21 14:36:24 +00:00
const RemoteHostFilter & Context::getRemoteHostFilter() const
{
return shared->remote_host_filter;
}
UInt16 Context::getTCPPort() const
{
auto lock = getLock();
auto & config = getConfigRef();
return config.getInt("tcp_port", DBMS_DEFAULT_PORT);
}
std::optional<UInt16> Context::getTCPPortSecure() const
{
auto lock = getLock();
auto & config = getConfigRef();
if (config.has("tcp_port_secure"))
return config.getInt("tcp_port_secure");
return {};
}
std::shared_ptr<Cluster> Context::getCluster(const std::string & cluster_name) const
2013-12-07 16:51:29 +00:00
{
auto res = getClusters().getCluster(cluster_name);
if (!res)
throw Exception("Requested cluster '" + cluster_name + "' not found", ErrorCodes::BAD_GET);
return res;
2013-12-07 16:51:29 +00:00
}
std::shared_ptr<Cluster> Context::tryGetCluster(const std::string & cluster_name) const
{
return getClusters().getCluster(cluster_name);
2013-12-07 16:51:29 +00:00
}
void Context::reloadClusterConfig()
{
while (true)
{
ConfigurationPtr cluster_config;
{
2019-01-02 06:44:36 +00:00
std::lock_guard lock(shared->clusters_mutex);
cluster_config = shared->clusters_config;
}
auto & config = cluster_config ? *cluster_config : getConfigRef();
auto new_clusters = std::make_unique<Clusters>(config, settings);
{
2019-01-02 06:44:36 +00:00
std::lock_guard lock(shared->clusters_mutex);
if (shared->clusters_config.get() == cluster_config.get())
{
shared->clusters = std::move(new_clusters);
return;
}
/// Clusters config has been suddenly changed, recompute clusters
}
}
}
Clusters & Context::getClusters() const
2015-04-30 12:43:16 +00:00
{
2019-01-02 06:44:36 +00:00
std::lock_guard lock(shared->clusters_mutex);
if (!shared->clusters)
{
auto & config = shared->clusters_config ? *shared->clusters_config : getConfigRef();
shared->clusters = std::make_unique<Clusters>(config, settings);
}
2016-03-10 03:08:09 +00:00
return *shared->clusters;
2015-04-30 12:43:16 +00:00
}
/// On repeating calls updates existing clusters and adds new clusters, doesn't delete old clusters
void Context::setClustersConfig(const ConfigurationPtr & config, const String & config_name)
{
2019-01-02 06:44:36 +00:00
std::lock_guard lock(shared->clusters_mutex);
shared->clusters_config = config;
if (!shared->clusters)
shared->clusters = std::make_unique<Clusters>(*shared->clusters_config, settings, config_name);
else
shared->clusters->updateClusters(*shared->clusters_config, settings, config_name);
2015-04-30 12:43:16 +00:00
}
void Context::setCluster(const String & cluster_name, const std::shared_ptr<Cluster> & cluster)
{
2019-01-02 06:44:36 +00:00
std::lock_guard lock(shared->clusters_mutex);
if (!shared->clusters)
throw Exception("Clusters are not set", ErrorCodes::LOGICAL_ERROR);
shared->clusters->setCluster(cluster_name, cluster);
2015-04-30 12:43:16 +00:00
}
2019-08-04 15:51:04 +00:00
void Context::initializeSystemLogs()
{
auto lock = getLock();
2019-02-06 18:33:29 +00:00
shared->system_logs.emplace(*global_context, getConfigRef());
}
2019-02-03 21:30:45 +00:00
void Context::initializeTraceCollector()
{
2019-02-03 21:30:45 +00:00
shared->initializeTraceCollector(getTraceLog());
}
2019-03-21 19:22:38 +00:00
std::shared_ptr<QueryLog> Context::getQueryLog()
{
auto lock = getLock();
if (!shared->system_logs || !shared->system_logs->query_log)
2019-03-21 19:22:38 +00:00
return {};
2019-03-21 19:22:38 +00:00
return shared->system_logs->query_log;
}
2019-03-21 19:22:38 +00:00
std::shared_ptr<QueryThreadLog> Context::getQueryThreadLog()
{
auto lock = getLock();
if (!shared->system_logs || !shared->system_logs->query_thread_log)
2019-03-21 19:22:38 +00:00
return {};
2019-03-21 19:22:38 +00:00
return shared->system_logs->query_thread_log;
}
2019-03-21 19:22:38 +00:00
std::shared_ptr<PartLog> Context::getPartLog(const String & part_database)
{
auto lock = getLock();
/// No part log or system logs are shutting down.
if (!shared->system_logs || !shared->system_logs->part_log)
2019-03-21 19:22:38 +00:00
return {};
/// Will not log operations on system tables (including part_log itself).
/// It doesn't make sense and not allow to destruct PartLog correctly due to infinite logging and flushing,
/// and also make troubles on startup.
if (part_database == shared->system_logs->part_log_database)
2019-03-21 19:22:38 +00:00
return {};
2019-03-21 19:22:38 +00:00
return shared->system_logs->part_log;
}
2019-05-19 20:22:44 +00:00
std::shared_ptr<TraceLog> Context::getTraceLog()
2019-02-03 21:30:45 +00:00
{
auto lock = getLock();
2019-02-09 22:26:51 +00:00
if (!shared->system_logs || !shared->system_logs->trace_log)
2019-07-23 14:50:38 +00:00
return {};
2019-02-03 21:30:45 +00:00
2019-05-19 20:22:44 +00:00
return shared->system_logs->trace_log;
2019-02-03 21:30:45 +00:00
}
2019-08-13 14:31:46 +00:00
std::shared_ptr<TextLog> Context::getTextLog()
{
auto lock = getLock();
2019-08-04 15:51:04 +00:00
if (!shared->system_logs || !shared->system_logs->text_log)
return {};
2019-07-31 14:18:59 +00:00
2019-08-04 15:51:04 +00:00
return shared->system_logs->text_log;
}
2019-08-13 14:31:46 +00:00
std::shared_ptr<MetricLog> Context::getMetricLog()
{
auto lock = getLock();
if (!shared->system_logs || !shared->system_logs->metric_log)
return {};
return shared->system_logs->metric_log;
}
2018-12-21 12:17:30 +00:00
CompressionCodecPtr Context::chooseCompressionCodec(size_t part_size, double part_size_ratio) const
{
auto lock = getLock();
2018-12-21 12:17:30 +00:00
if (!shared->compression_codec_selector)
{
constexpr auto config_name = "compression";
auto & config = getConfigRef();
if (config.has(config_name))
2018-12-21 12:17:30 +00:00
shared->compression_codec_selector = std::make_unique<CompressionCodecSelector>(config, "compression");
else
2018-12-21 12:17:30 +00:00
shared->compression_codec_selector = std::make_unique<CompressionCodecSelector>();
}
2018-12-21 12:17:30 +00:00
return shared->compression_codec_selector->choose(part_size, part_size_ratio);
}
2019-11-27 09:39:44 +00:00
const DiskPtr & Context::getDisk(const String & name) const
{
auto lock = getLock();
const auto & disk_selector = getDiskSelector();
return disk_selector[name];
}
2019-11-27 09:39:44 +00:00
DiskSelector & Context::getDiskSelector() const
{
auto lock = getLock();
if (!shared->merge_tree_disk_selector)
{
constexpr auto config_name = "storage_configuration.disks";
auto & config = getConfigRef();
2019-11-27 09:39:44 +00:00
shared->merge_tree_disk_selector = std::make_unique<DiskSelector>(config, config_name, *this);
}
return *shared->merge_tree_disk_selector;
}
2019-11-27 09:39:44 +00:00
const StoragePolicyPtr & Context::getStoragePolicy(const String & name) const
{
auto lock = getLock();
2019-08-01 10:29:14 +00:00
auto & policy_selector = getStoragePolicySelector();
2019-08-01 10:29:14 +00:00
return policy_selector[name];
}
2019-11-27 09:39:44 +00:00
StoragePolicySelector & Context::getStoragePolicySelector() const
{
auto lock = getLock();
if (!shared->merge_tree_storage_policy_selector)
{
constexpr auto config_name = "storage_configuration.policies";
auto & config = getConfigRef();
2019-11-27 09:39:44 +00:00
shared->merge_tree_storage_policy_selector = std::make_unique<StoragePolicySelector>(config, config_name, getDiskSelector());
}
return *shared->merge_tree_storage_policy_selector;
}
const MergeTreeSettings & Context::getMergeTreeSettings() const
2015-07-16 21:32:51 +00:00
{
auto lock = getLock();
2015-07-16 21:32:51 +00:00
if (!shared->merge_tree_settings)
{
auto & config = getConfigRef();
2019-08-26 14:24:29 +00:00
MergeTreeSettings mt_settings;
mt_settings.loadFromConfig("merge_tree", config);
shared->merge_tree_settings.emplace(mt_settings);
}
2015-07-16 21:32:51 +00:00
return *shared->merge_tree_settings;
2015-07-16 21:32:51 +00:00
}
void Context::checkCanBeDropped(const String & database, const String & table, const size_t & size, const size_t & max_size_to_drop) const
{
if (!max_size_to_drop || size <= max_size_to_drop)
return;
Poco::File force_file(getFlagsPath() + "force_drop_table");
bool force_file_exists = force_file.exists();
if (force_file_exists)
{
try
{
force_file.remove();
return;
}
catch (...)
{
/// User should recreate force file on each drop, it shouldn't be protected
tryLogCurrentException("Drop table check", "Can't remove force file to enable table or partition drop");
}
}
String size_str = formatReadableSizeWithDecimalSuffix(size);
String max_size_to_drop_str = formatReadableSizeWithDecimalSuffix(max_size_to_drop);
std::stringstream ostr;
ostr << "Table or Partition in " << backQuoteIfNeed(database) << "." << backQuoteIfNeed(table) << " was not dropped.\n"
<< "Reason:\n"
2018-08-30 09:33:31 +00:00
<< "1. Size (" << size_str << ") is greater than max_[table/partition]_size_to_drop (" << max_size_to_drop_str << ")\n"
2017-08-16 10:24:06 +00:00
<< "2. File '" << force_file.path() << "' intended to force DROP "
<< (force_file_exists ? "exists but not writeable (could not be removed)" : "doesn't exist") << "\n";
ostr << "How to fix this:\n"
2018-08-30 09:33:31 +00:00
<< "1. Either increase (or set to zero) max_[table/partition]_size_to_drop in server config and restart ClickHouse\n"
<< "2. Either create forcing file " << force_file.path() << " and make sure that ClickHouse has write permission for it.\n"
<< "Example:\nsudo touch '" << force_file.path() << "' && sudo chmod 666 '" << force_file.path() << "'";
throw Exception(ostr.str(), ErrorCodes::TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT);
}
void Context::setMaxTableSizeToDrop(size_t max_size)
{
2019-11-22 13:56:16 +00:00
// Is initialized at server startup and updated at config reload
shared->max_table_size_to_drop.store(max_size, std::memory_order_relaxed);
}
void Context::checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const
{
size_t max_table_size_to_drop = shared->max_table_size_to_drop.load(std::memory_order_relaxed);
2018-08-06 08:43:34 +00:00
checkCanBeDropped(database, table, table_size, max_table_size_to_drop);
}
void Context::setMaxPartitionSizeToDrop(size_t max_size)
{
2019-11-22 13:56:16 +00:00
// Is initialized at server startup and updated at config reload
shared->max_partition_size_to_drop.store(max_size, std::memory_order_relaxed);
}
void Context::checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const
{
size_t max_partition_size_to_drop = shared->max_partition_size_to_drop.load(std::memory_order_relaxed);
2018-08-06 08:43:34 +00:00
checkCanBeDropped(database, table, partition_size, max_partition_size_to_drop);
}
2019-02-10 16:55:12 +00:00
BlockInputStreamPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size) const
{
return FormatFactory::instance().getInput(name, buf, sample, *this, max_block_size);
}
BlockOutputStreamPtr Context::getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample) const
{
return FormatFactory::instance().getOutput(name, buf, sample, *this);
}
2019-03-26 18:28:37 +00:00
OutputFormatPtr Context::getOutputFormatProcessor(const String & name, WriteBuffer & buf, const Block & sample) const
{
return FormatFactory::instance().getOutputFormat(name, buf, sample, *this);
}
2016-04-09 07:47:08 +00:00
time_t Context::getUptimeSeconds() const
{
auto lock = getLock();
return shared->uptime_watch.elapsedSeconds();
2016-04-09 07:47:08 +00:00
}
void Context::setConfigReloadCallback(ConfigReloadCallback && callback)
{
/// Is initialized at server startup, so lock isn't required. Otherwise use mutex.
shared->config_reload_callback = std::move(callback);
}
void Context::reloadConfig() const
{
/// Use mutex if callback may be changed after startup.
if (!shared->config_reload_callback)
throw Exception("Can't reload config beacuse config_reload_callback is not set.", ErrorCodes::LOGICAL_ERROR);
shared->config_reload_callback();
}
2015-04-16 06:12:35 +00:00
void Context::shutdown()
{
shared->shutdown();
2015-04-16 06:12:35 +00:00
}
Context::ApplicationType Context::getApplicationType() const
{
return shared->application_type;
}
void Context::setApplicationType(ApplicationType type)
{
/// Lock isn't required, you should set it at start
shared->application_type = type;
}
void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & config)
{
shared->default_profile_name = config.getString("default_profile", "default");
shared->system_profile_name = config.getString("system_profile", shared->default_profile_name);
setSetting("profile", shared->system_profile_name);
}
String Context::getDefaultProfileName() const
{
return shared->default_profile_name;
}
String Context::getSystemProfileName() const
{
return shared->system_profile_name;
}
String Context::getFormatSchemaPath() const
{
return shared->format_schema_path;
}
void Context::setFormatSchemaPath(const String & path)
{
shared->format_schema_path = path;
}
2018-07-05 23:36:09 +00:00
Context::SampleBlockCache & Context::getSampleBlockCache() const
{
2018-07-05 23:36:09 +00:00
return getQueryContext().sample_block_cache;
}
2019-05-25 13:43:52 +00:00
bool Context::hasQueryParameters() const
2019-05-18 21:07:23 +00:00
{
2019-06-15 17:52:53 +00:00
return !query_parameters.empty();
2019-05-18 21:07:23 +00:00
}
2019-06-15 17:52:53 +00:00
const NameToNameMap & Context::getQueryParameters() const
2019-05-18 21:07:23 +00:00
{
2019-06-15 17:52:53 +00:00
return query_parameters;
2019-05-18 21:07:23 +00:00
}
2019-06-15 17:52:53 +00:00
void Context::setQueryParameter(const String & name, const String & value)
2019-05-18 21:07:23 +00:00
{
2019-06-15 17:52:53 +00:00
if (!query_parameters.emplace(name, value).second)
throw Exception("Duplicate name " + backQuote(name) + " of query parameter", ErrorCodes::BAD_ARGUMENTS);
2019-05-18 21:07:23 +00:00
}
#if USE_EMBEDDED_COMPILER
std::shared_ptr<CompiledExpressionCache> Context::getCompiledExpressionCache() const
{
auto lock = getLock();
return shared->compiled_expression_cache;
}
2018-09-05 12:42:37 +00:00
void Context::setCompiledExpressionCache(size_t cache_size)
{
auto lock = getLock();
if (shared->compiled_expression_cache)
throw Exception("Compiled expressions cache has been already created.", ErrorCodes::LOGICAL_ERROR);
shared->compiled_expression_cache = std::make_shared<CompiledExpressionCache>(cache_size);
}
void Context::dropCompiledExpressionCache() const
{
auto lock = getLock();
if (shared->compiled_expression_cache)
shared->compiled_expression_cache->reset();
}
#endif
2019-10-10 20:47:47 +00:00
void Context::addXDBCBridgeCommand(std::unique_ptr<ShellCommand> cmd) const
{
auto lock = getLock();
shared->bridge_commands.emplace_back(std::move(cmd));
}
IHostContextPtr & Context::getHostContext()
{
return host_context;
}
const IHostContextPtr & Context::getHostContext() const
{
return host_context;
}
std::shared_ptr<ActionLocksManager> Context::getActionLocksManager()
{
auto lock = getLock();
if (!shared->action_locks_manager)
shared->action_locks_manager = std::make_shared<ActionLocksManager>(getGlobalContext());
return shared->action_locks_manager;
}
void Context::setExternalTablesInitializer(ExternalTablesInitializer && initializer)
{
if (external_tables_initializer_callback)
throw Exception("External tables initializer is already set", ErrorCodes::LOGICAL_ERROR);
external_tables_initializer_callback = std::move(initializer);
}
void Context::initializeExternalTablesIfSet()
{
if (external_tables_initializer_callback)
{
external_tables_initializer_callback(*this);
/// Reset callback
external_tables_initializer_callback = {};
}
}
2019-05-28 18:30:10 +00:00
void Context::setInputInitializer(InputInitializer && initializer)
{
if (input_initializer_callback)
throw Exception("Input initializer is already set", ErrorCodes::LOGICAL_ERROR);
input_initializer_callback = std::move(initializer);
}
void Context::initializeInput(const StoragePtr & input_storage)
{
if (!input_initializer_callback)
throw Exception("Input initializer is not set", ErrorCodes::LOGICAL_ERROR);
input_initializer_callback(*this, input_storage);
/// Reset callback
input_initializer_callback = {};
}
void Context::setInputBlocksReaderCallback(InputBlocksReader && reader)
{
if (input_blocks_reader)
throw Exception("Input blocks reader is already set", ErrorCodes::LOGICAL_ERROR);
input_blocks_reader = std::move(reader);
}
InputBlocksReader Context::getInputBlocksReaderCallback() const
{
return input_blocks_reader;
}
void Context::resetInputCallbacks()
{
if (input_initializer_callback)
input_initializer_callback = {};
if (input_blocks_reader)
input_blocks_reader = {};
}
SessionCleaner::~SessionCleaner()
{
try
{
{
2019-01-02 06:44:36 +00:00
std::lock_guard lock{mutex};
quit = true;
}
cond.notify_one();
thread.join();
}
catch (...)
{
DB::tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void SessionCleaner::run()
{
2019-02-10 21:15:14 +00:00
setThreadName("SessionCleaner");
2019-01-02 06:44:36 +00:00
std::unique_lock lock{mutex};
while (true)
{
auto interval = context.closeSessions();
if (cond.wait_for(lock, interval, [this]() -> bool { return quit; }))
break;
}
}
2012-08-02 17:33:31 +00:00
}