mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #51724 from arenadata/ADQM-939
Added max_sessions_for_user setting
This commit is contained in:
commit
7359a8108b
@ -327,3 +327,39 @@ The maximum amount of data consumed by temporary files on disk in bytes for all
|
||||
Zero means unlimited.
|
||||
|
||||
Default value: 0.
|
||||
|
||||
## max_sessions_for_user {#max-sessions-per-user}
|
||||
|
||||
Maximum number of simultaneous sessions per authenticated user to the ClickHouse server.
|
||||
|
||||
Example:
|
||||
|
||||
``` xml
|
||||
<profiles>
|
||||
<single_session_profile>
|
||||
<max_sessions_for_user>1</max_sessions_for_user>
|
||||
</single_session_profile>
|
||||
<two_sessions_profile>
|
||||
<max_sessions_for_user>2</max_sessions_for_user>
|
||||
</two_sessions_profile>
|
||||
<unlimited_sessions_profile>
|
||||
<max_sessions_for_user>0</max_sessions_for_user>
|
||||
</unlimited_sessions_profile>
|
||||
</profiles>
|
||||
<users>
|
||||
<!-- User Alice can connect to a ClickHouse server no more than once at a time. -->
|
||||
<Alice>
|
||||
<profile>single_session_user</profile>
|
||||
</Alice>
|
||||
<!-- User Bob can use 2 simultaneous sessions. -->
|
||||
<Bob>
|
||||
<profile>two_sessions_profile</profile>
|
||||
</Bob>
|
||||
<!-- User Charles can use arbitrarily many of simultaneous sessions. -->
|
||||
<Charles>
|
||||
<profile>unlimited_sessions_profile</profile>
|
||||
</Charles>
|
||||
</users>
|
||||
```
|
||||
|
||||
Default value: 0 (Infinite count of simultaneous sessions).
|
||||
|
@ -39,7 +39,7 @@ Example:
|
||||
<max_threads>8</max_threads>
|
||||
</default>
|
||||
|
||||
<!-- Settings for quries from the user interface -->
|
||||
<!-- Settings for queries from the user interface -->
|
||||
<web>
|
||||
<max_rows_to_read>1000000000</max_rows_to_read>
|
||||
<max_bytes_to_read>100000000000</max_bytes_to_read>
|
||||
@ -67,6 +67,8 @@ Example:
|
||||
<max_ast_depth>50</max_ast_depth>
|
||||
<max_ast_elements>100</max_ast_elements>
|
||||
|
||||
<max_sessions_for_user>4</max_sessions_for_user>
|
||||
|
||||
<readonly>1</readonly>
|
||||
</web>
|
||||
</profiles>
|
||||
|
@ -314,3 +314,40 @@ FORMAT Null;
|
||||
При вставке данных, ClickHouse вычисляет количество партиций во вставленном блоке. Если число партиций больше, чем `max_partitions_per_insert_block`, ClickHouse генерирует исключение со следующим текстом:
|
||||
|
||||
> «Too many partitions for single INSERT block (more than» + toString(max_parts) + «). The limit is controlled by ‘max_partitions_per_insert_block’ setting. Large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).»
|
||||
|
||||
## max_sessions_for_user {#max-sessions-per-user}
|
||||
|
||||
Максимальное количество одновременных сессий на одного аутентифицированного пользователя.
|
||||
|
||||
Пример:
|
||||
|
||||
``` xml
|
||||
<profiles>
|
||||
<single_session_profile>
|
||||
<max_sessions_for_user>1</max_sessions_for_user>
|
||||
</single_session_profile>
|
||||
<two_sessions_profile>
|
||||
<max_sessions_for_user>2</max_sessions_for_user>
|
||||
</two_sessions_profile>
|
||||
<unlimited_sessions_profile>
|
||||
<max_sessions_for_user>0</max_sessions_for_user>
|
||||
</unlimited_sessions_profile>
|
||||
</profiles>
|
||||
<users>
|
||||
<!-- Пользователь Alice может одновременно подключаться не
|
||||
более одного раза к серверу ClickHouse. -->
|
||||
<Alice>
|
||||
<profile>single_session_profile</profile>
|
||||
</Alice>
|
||||
<!-- Пользователь Bob может использовать 2 одновременных сессии. -->
|
||||
<Bob>
|
||||
<profile>two_sessions_profile</profile>
|
||||
</Bob>
|
||||
<!-- Пользователь Charles может иметь любое количество одновременных сессий. -->
|
||||
<Charles>
|
||||
<profile>unlimited_sessions_profile</profile>
|
||||
</Charles>
|
||||
</users>
|
||||
```
|
||||
|
||||
Значение по умолчанию: 0 (неограниченное количество сессий).
|
||||
|
@ -39,7 +39,7 @@ SET profile = 'web'
|
||||
<max_threads>8</max_threads>
|
||||
</default>
|
||||
|
||||
<!-- Settings for quries from the user interface -->
|
||||
<!-- Settings for queries from the user interface -->
|
||||
<web>
|
||||
<max_rows_to_read>1000000000</max_rows_to_read>
|
||||
<max_bytes_to_read>100000000000</max_bytes_to_read>
|
||||
@ -67,6 +67,7 @@ SET profile = 'web'
|
||||
<max_ast_depth>50</max_ast_depth>
|
||||
<max_ast_elements>100</max_ast_elements>
|
||||
|
||||
<max_sessions_for_user>4</max_sessions_for_user>
|
||||
<readonly>1</readonly>
|
||||
</web>
|
||||
</profiles>
|
||||
|
@ -328,9 +328,6 @@ void ContextAccess::setRolesInfo(const std::shared_ptr<const EnabledRolesInfo> &
|
||||
|
||||
enabled_row_policies = access_control->getEnabledRowPolicies(*params.user_id, roles_info->enabled_roles);
|
||||
|
||||
enabled_quota = access_control->getEnabledQuota(
|
||||
*params.user_id, user_name, roles_info->enabled_roles, params.address, params.forwarded_address, params.quota_key);
|
||||
|
||||
enabled_settings = access_control->getEnabledSettings(
|
||||
*params.user_id, user->settings, roles_info->enabled_roles, roles_info->settings_from_enabled_roles);
|
||||
|
||||
@ -416,19 +413,32 @@ RowPolicyFilterPtr ContextAccess::getRowPolicyFilter(const String & database, co
|
||||
std::shared_ptr<const EnabledQuota> ContextAccess::getQuota() const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
if (enabled_quota)
|
||||
return enabled_quota;
|
||||
static const auto unlimited_quota = EnabledQuota::getUnlimitedQuota();
|
||||
return unlimited_quota;
|
||||
|
||||
if (!enabled_quota)
|
||||
{
|
||||
if (roles_info)
|
||||
{
|
||||
enabled_quota = access_control->getEnabledQuota(*params.user_id,
|
||||
user_name,
|
||||
roles_info->enabled_roles,
|
||||
params.address,
|
||||
params.forwarded_address,
|
||||
params.quota_key);
|
||||
}
|
||||
else
|
||||
{
|
||||
static const auto unlimited_quota = EnabledQuota::getUnlimitedQuota();
|
||||
return unlimited_quota;
|
||||
}
|
||||
}
|
||||
|
||||
return enabled_quota;
|
||||
}
|
||||
|
||||
|
||||
std::optional<QuotaUsage> ContextAccess::getQuotaUsage() const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
if (enabled_quota)
|
||||
return enabled_quota->getUsage();
|
||||
return {};
|
||||
return getQuota()->getUsage();
|
||||
}
|
||||
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <string_view>
|
||||
#include <unordered_map>
|
||||
#include <Access/SettingsConstraints.h>
|
||||
#include <Access/resolveSetting.h>
|
||||
#include <Access/AccessControl.h>
|
||||
@ -6,6 +7,7 @@
|
||||
#include <Storages/MergeTree/MergeTreeSettings.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
#include <Common/SettingSource.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <boost/range/algorithm_ext/erase.hpp>
|
||||
@ -20,6 +22,39 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_SETTING;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
struct SettingSourceRestrictions
|
||||
{
|
||||
constexpr SettingSourceRestrictions() { allowed_sources.set(); }
|
||||
|
||||
constexpr SettingSourceRestrictions(std::initializer_list<SettingSource> allowed_sources_)
|
||||
{
|
||||
for (auto allowed_source : allowed_sources_)
|
||||
setSourceAllowed(allowed_source, true);
|
||||
}
|
||||
|
||||
constexpr bool isSourceAllowed(SettingSource source) { return allowed_sources[source]; }
|
||||
constexpr void setSourceAllowed(SettingSource source, bool allowed) { allowed_sources[source] = allowed; }
|
||||
|
||||
std::bitset<SettingSource::COUNT> allowed_sources;
|
||||
};
|
||||
|
||||
const std::unordered_map<std::string_view, SettingSourceRestrictions> SETTINGS_SOURCE_RESTRICTIONS = {
|
||||
{"max_sessions_for_user", {SettingSource::PROFILE}},
|
||||
};
|
||||
|
||||
SettingSourceRestrictions getSettingSourceRestrictions(std::string_view name)
|
||||
{
|
||||
auto settingConstraintIter = SETTINGS_SOURCE_RESTRICTIONS.find(name);
|
||||
if (settingConstraintIter != SETTINGS_SOURCE_RESTRICTIONS.end())
|
||||
return settingConstraintIter->second;
|
||||
else
|
||||
return SettingSourceRestrictions(); // allows everything
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
SettingsConstraints::SettingsConstraints(const AccessControl & access_control_) : access_control(&access_control_)
|
||||
{
|
||||
}
|
||||
@ -98,7 +133,7 @@ void SettingsConstraints::merge(const SettingsConstraints & other)
|
||||
}
|
||||
|
||||
|
||||
void SettingsConstraints::check(const Settings & current_settings, const SettingsProfileElements & profile_elements) const
|
||||
void SettingsConstraints::check(const Settings & current_settings, const SettingsProfileElements & profile_elements, SettingSource source) const
|
||||
{
|
||||
for (const auto & element : profile_elements)
|
||||
{
|
||||
@ -108,19 +143,19 @@ void SettingsConstraints::check(const Settings & current_settings, const Setting
|
||||
if (element.value)
|
||||
{
|
||||
SettingChange value(element.setting_name, *element.value);
|
||||
check(current_settings, value);
|
||||
check(current_settings, value, source);
|
||||
}
|
||||
|
||||
if (element.min_value)
|
||||
{
|
||||
SettingChange value(element.setting_name, *element.min_value);
|
||||
check(current_settings, value);
|
||||
check(current_settings, value, source);
|
||||
}
|
||||
|
||||
if (element.max_value)
|
||||
{
|
||||
SettingChange value(element.setting_name, *element.max_value);
|
||||
check(current_settings, value);
|
||||
check(current_settings, value, source);
|
||||
}
|
||||
|
||||
SettingConstraintWritability new_value = SettingConstraintWritability::WRITABLE;
|
||||
@ -142,24 +177,24 @@ void SettingsConstraints::check(const Settings & current_settings, const Setting
|
||||
}
|
||||
}
|
||||
|
||||
void SettingsConstraints::check(const Settings & current_settings, const SettingChange & change) const
|
||||
void SettingsConstraints::check(const Settings & current_settings, const SettingChange & change, SettingSource source) const
|
||||
{
|
||||
checkImpl(current_settings, const_cast<SettingChange &>(change), THROW_ON_VIOLATION);
|
||||
checkImpl(current_settings, const_cast<SettingChange &>(change), THROW_ON_VIOLATION, source);
|
||||
}
|
||||
|
||||
void SettingsConstraints::check(const Settings & current_settings, const SettingsChanges & changes) const
|
||||
void SettingsConstraints::check(const Settings & current_settings, const SettingsChanges & changes, SettingSource source) const
|
||||
{
|
||||
for (const auto & change : changes)
|
||||
check(current_settings, change);
|
||||
check(current_settings, change, source);
|
||||
}
|
||||
|
||||
void SettingsConstraints::check(const Settings & current_settings, SettingsChanges & changes) const
|
||||
void SettingsConstraints::check(const Settings & current_settings, SettingsChanges & changes, SettingSource source) const
|
||||
{
|
||||
boost::range::remove_erase_if(
|
||||
changes,
|
||||
[&](SettingChange & change) -> bool
|
||||
{
|
||||
return !checkImpl(current_settings, const_cast<SettingChange &>(change), THROW_ON_VIOLATION);
|
||||
return !checkImpl(current_settings, const_cast<SettingChange &>(change), THROW_ON_VIOLATION, source);
|
||||
});
|
||||
}
|
||||
|
||||
@ -174,13 +209,13 @@ void SettingsConstraints::check(const MergeTreeSettings & current_settings, cons
|
||||
check(current_settings, change);
|
||||
}
|
||||
|
||||
void SettingsConstraints::clamp(const Settings & current_settings, SettingsChanges & changes) const
|
||||
void SettingsConstraints::clamp(const Settings & current_settings, SettingsChanges & changes, SettingSource source) const
|
||||
{
|
||||
boost::range::remove_erase_if(
|
||||
changes,
|
||||
[&](SettingChange & change) -> bool
|
||||
{
|
||||
return !checkImpl(current_settings, change, CLAMP_ON_VIOLATION);
|
||||
return !checkImpl(current_settings, change, CLAMP_ON_VIOLATION, source);
|
||||
});
|
||||
}
|
||||
|
||||
@ -215,7 +250,10 @@ bool getNewValueToCheck(const T & current_settings, SettingChange & change, Fiel
|
||||
return true;
|
||||
}
|
||||
|
||||
bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const
|
||||
bool SettingsConstraints::checkImpl(const Settings & current_settings,
|
||||
SettingChange & change,
|
||||
ReactionOnViolation reaction,
|
||||
SettingSource source) const
|
||||
{
|
||||
std::string_view setting_name = Settings::Traits::resolveName(change.name);
|
||||
|
||||
@ -247,7 +285,7 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh
|
||||
if (!getNewValueToCheck(current_settings, change, new_value, reaction == THROW_ON_VIOLATION))
|
||||
return false;
|
||||
|
||||
return getChecker(current_settings, setting_name).check(change, new_value, reaction);
|
||||
return getChecker(current_settings, setting_name).check(change, new_value, reaction, source);
|
||||
}
|
||||
|
||||
bool SettingsConstraints::checkImpl(const MergeTreeSettings & current_settings, SettingChange & change, ReactionOnViolation reaction) const
|
||||
@ -255,10 +293,13 @@ bool SettingsConstraints::checkImpl(const MergeTreeSettings & current_settings,
|
||||
Field new_value;
|
||||
if (!getNewValueToCheck(current_settings, change, new_value, reaction == THROW_ON_VIOLATION))
|
||||
return false;
|
||||
return getMergeTreeChecker(change.name).check(change, new_value, reaction);
|
||||
return getMergeTreeChecker(change.name).check(change, new_value, reaction, SettingSource::QUERY);
|
||||
}
|
||||
|
||||
bool SettingsConstraints::Checker::check(SettingChange & change, const Field & new_value, ReactionOnViolation reaction) const
|
||||
bool SettingsConstraints::Checker::check(SettingChange & change,
|
||||
const Field & new_value,
|
||||
ReactionOnViolation reaction,
|
||||
SettingSource source) const
|
||||
{
|
||||
if (!explain.empty())
|
||||
{
|
||||
@ -326,6 +367,14 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n
|
||||
change.value = max_value;
|
||||
}
|
||||
|
||||
if (!getSettingSourceRestrictions(setting_name).isSourceAllowed(source))
|
||||
{
|
||||
if (reaction == THROW_ON_VIOLATION)
|
||||
throw Exception(ErrorCodes::READONLY, "Setting {} is not allowed to be set by {}", setting_name, toString(source));
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Access/SettingsProfileElement.h>
|
||||
#include <Common/SettingsChanges.h>
|
||||
#include <Common/SettingSource.h>
|
||||
#include <unordered_map>
|
||||
|
||||
namespace Poco::Util
|
||||
@ -73,17 +74,18 @@ public:
|
||||
void merge(const SettingsConstraints & other);
|
||||
|
||||
/// Checks whether `change` violates these constraints and throws an exception if so.
|
||||
void check(const Settings & current_settings, const SettingsProfileElements & profile_elements) const;
|
||||
void check(const Settings & current_settings, const SettingChange & change) const;
|
||||
void check(const Settings & current_settings, const SettingsChanges & changes) const;
|
||||
void check(const Settings & current_settings, SettingsChanges & changes) const;
|
||||
void check(const Settings & current_settings, const SettingsProfileElements & profile_elements, SettingSource source) const;
|
||||
void check(const Settings & current_settings, const SettingChange & change, SettingSource source) const;
|
||||
void check(const Settings & current_settings, const SettingsChanges & changes, SettingSource source) const;
|
||||
void check(const Settings & current_settings, SettingsChanges & changes, SettingSource source) const;
|
||||
|
||||
/// Checks whether `change` violates these constraints and throws an exception if so. (setting short name is expected inside `changes`)
|
||||
void check(const MergeTreeSettings & current_settings, const SettingChange & change) const;
|
||||
void check(const MergeTreeSettings & current_settings, const SettingsChanges & changes) const;
|
||||
|
||||
/// Checks whether `change` violates these and clamps the `change` if so.
|
||||
void clamp(const Settings & current_settings, SettingsChanges & changes) const;
|
||||
void clamp(const Settings & current_settings, SettingsChanges & changes, SettingSource source) const;
|
||||
|
||||
|
||||
friend bool operator ==(const SettingsConstraints & left, const SettingsConstraints & right);
|
||||
friend bool operator !=(const SettingsConstraints & left, const SettingsConstraints & right) { return !(left == right); }
|
||||
@ -133,7 +135,10 @@ private:
|
||||
{}
|
||||
|
||||
// Perform checking
|
||||
bool check(SettingChange & change, const Field & new_value, ReactionOnViolation reaction) const;
|
||||
bool check(SettingChange & change,
|
||||
const Field & new_value,
|
||||
ReactionOnViolation reaction,
|
||||
SettingSource source) const;
|
||||
};
|
||||
|
||||
struct StringHash
|
||||
@ -145,7 +150,11 @@ private:
|
||||
}
|
||||
};
|
||||
|
||||
bool checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const;
|
||||
bool checkImpl(const Settings & current_settings,
|
||||
SettingChange & change,
|
||||
ReactionOnViolation reaction,
|
||||
SettingSource source) const;
|
||||
|
||||
bool checkImpl(const MergeTreeSettings & current_settings, SettingChange & change, ReactionOnViolation reaction) const;
|
||||
|
||||
Checker getChecker(const Settings & current_settings, std::string_view setting_name) const;
|
||||
|
@ -124,6 +124,9 @@ void Suggest::load(ContextPtr context, const ConnectionParameters & connection_p
|
||||
if (e.code() == ErrorCodes::DEADLOCK_AVOIDED)
|
||||
continue;
|
||||
|
||||
/// Client can successfully connect to the server and
|
||||
/// get ErrorCodes::USER_SESSION_LIMIT_EXCEEDED for suggestion connection.
|
||||
|
||||
/// We should not use std::cerr here, because this method works concurrently with the main thread.
|
||||
/// WriteBufferFromFileDescriptor will write directly to the file descriptor, avoiding data race on std::cerr.
|
||||
|
||||
|
@ -582,6 +582,7 @@
|
||||
M(697, CANNOT_RESTORE_TO_NONENCRYPTED_DISK) \
|
||||
M(698, INVALID_REDIS_STORAGE_TYPE) \
|
||||
M(699, INVALID_REDIS_TABLE_STRUCTURE) \
|
||||
M(700, USER_SESSION_LIMIT_EXCEEDED) \
|
||||
\
|
||||
M(999, KEEPER_EXCEPTION) \
|
||||
M(1000, POCO_EXCEPTION) \
|
||||
|
43
src/Common/SettingSource.h
Normal file
43
src/Common/SettingSource.h
Normal file
@ -0,0 +1,43 @@
|
||||
#pragma once
|
||||
|
||||
#include <string_view>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
enum SettingSource
|
||||
{
|
||||
/// Query or session change:
|
||||
/// SET <setting> = <value>
|
||||
/// SELECT ... SETTINGS [<setting> = <value]
|
||||
QUERY,
|
||||
|
||||
/// Profile creation or altering:
|
||||
/// CREATE SETTINGS PROFILE ... SETTINGS [<setting> = <value]
|
||||
/// ALTER SETTINGS PROFILE ... SETTINGS [<setting> = <value]
|
||||
PROFILE,
|
||||
|
||||
/// Role creation or altering:
|
||||
/// CREATE ROLE ... SETTINGS [<setting> = <value>]
|
||||
/// ALTER ROLE ... SETTINGS [<setting> = <value]
|
||||
ROLE,
|
||||
|
||||
/// User creation or altering:
|
||||
/// CREATE USER ... SETTINGS [<setting> = <value>]
|
||||
/// ALTER USER ... SETTINGS [<setting> = <value]
|
||||
USER,
|
||||
|
||||
COUNT,
|
||||
};
|
||||
|
||||
constexpr std::string_view toString(SettingSource source)
|
||||
{
|
||||
switch (source)
|
||||
{
|
||||
case SettingSource::QUERY: return "query";
|
||||
case SettingSource::PROFILE: return "profile";
|
||||
case SettingSource::USER: return "user";
|
||||
case SettingSource::ROLE: return "role";
|
||||
default: return "unknown";
|
||||
}
|
||||
}
|
||||
}
|
@ -386,6 +386,8 @@ class IColumn;
|
||||
M(UInt64, max_temporary_columns, 0, "If a query generates more than the specified number of temporary columns in memory as a result of intermediate calculation, exception is thrown. Zero value means unlimited. This setting is useful to prevent too complex queries.", 0) \
|
||||
M(UInt64, max_temporary_non_const_columns, 0, "Similar to the 'max_temporary_columns' setting but applies only to non-constant columns. This makes sense, because constant columns are cheap and it is reasonable to allow more of them.", 0) \
|
||||
\
|
||||
M(UInt64, max_sessions_for_user, 0, "Maximum number of simultaneous sessions for a user.", 0) \
|
||||
\
|
||||
M(UInt64, max_subquery_depth, 100, "If a query has more than specified number of nested subqueries, throw an exception. This allows you to have a sanity check to protect the users of your cluster from going insane with their queries.", 0) \
|
||||
M(UInt64, max_analyze_depth, 5000, "Maximum number of analyses performed by interpreter.", 0) \
|
||||
M(UInt64, max_ast_depth, 1000, "Maximum depth of query syntax tree. Checked after parsing.", 0) \
|
||||
|
@ -46,7 +46,7 @@ BlockIO InterpreterCreateRoleQuery::execute()
|
||||
settings_from_query = SettingsProfileElements{*query.settings, access_control};
|
||||
|
||||
if (!query.attach)
|
||||
getContext()->checkSettingsConstraints(*settings_from_query);
|
||||
getContext()->checkSettingsConstraints(*settings_from_query, SettingSource::ROLE);
|
||||
}
|
||||
|
||||
if (!query.cluster.empty())
|
||||
|
@ -54,7 +54,7 @@ BlockIO InterpreterCreateSettingsProfileQuery::execute()
|
||||
settings_from_query = SettingsProfileElements{*query.settings, access_control};
|
||||
|
||||
if (!query.attach)
|
||||
getContext()->checkSettingsConstraints(*settings_from_query);
|
||||
getContext()->checkSettingsConstraints(*settings_from_query, SettingSource::PROFILE);
|
||||
}
|
||||
|
||||
if (!query.cluster.empty())
|
||||
|
@ -133,7 +133,7 @@ BlockIO InterpreterCreateUserQuery::execute()
|
||||
settings_from_query = SettingsProfileElements{*query.settings, access_control};
|
||||
|
||||
if (!query.attach)
|
||||
getContext()->checkSettingsConstraints(*settings_from_query);
|
||||
getContext()->checkSettingsConstraints(*settings_from_query, SettingSource::USER);
|
||||
}
|
||||
|
||||
if (!query.cluster.empty())
|
||||
|
@ -45,6 +45,7 @@
|
||||
#include <Interpreters/Cache/QueryCache.h>
|
||||
#include <Interpreters/Cache/FileCacheFactory.h>
|
||||
#include <Interpreters/Cache/FileCache.h>
|
||||
#include <Interpreters/SessionTracker.h>
|
||||
#include <Core/ServerSettings.h>
|
||||
#include <Interpreters/PreparedSets.h>
|
||||
#include <Core/Settings.h>
|
||||
@ -158,6 +159,7 @@ namespace CurrentMetrics
|
||||
extern const Metric IOWriterThreadsActive;
|
||||
}
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -276,6 +278,7 @@ struct ContextSharedPart : boost::noncopyable
|
||||
mutable QueryCachePtr query_cache; /// Cache of query results.
|
||||
mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads.
|
||||
ProcessList process_list; /// Executing queries at the moment.
|
||||
SessionTracker session_tracker;
|
||||
GlobalOvercommitTracker global_overcommit_tracker;
|
||||
MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree)
|
||||
MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree)
|
||||
@ -739,6 +742,9 @@ std::unique_lock<std::recursive_mutex> Context::getLock() const
|
||||
ProcessList & Context::getProcessList() { return shared->process_list; }
|
||||
const ProcessList & Context::getProcessList() const { return shared->process_list; }
|
||||
OvercommitTracker * Context::getGlobalOvercommitTracker() const { return &shared->global_overcommit_tracker; }
|
||||
|
||||
SessionTracker & Context::getSessionTracker() { return shared->session_tracker; }
|
||||
|
||||
MergeList & Context::getMergeList() { return shared->merge_list; }
|
||||
const MergeList & Context::getMergeList() const { return shared->merge_list; }
|
||||
MovesList & Context::getMovesList() { return shared->moves_list; }
|
||||
@ -1094,7 +1100,7 @@ void Context::setUser(const UUID & user_id_, bool set_current_profiles_, bool se
|
||||
std::optional<ContextAccessParams> params;
|
||||
{
|
||||
auto lock = getLock();
|
||||
params.emplace(ContextAccessParams{user_id_, /* full_access= */ false, /* use_default_roles = */ true, {}, settings, current_database, client_info});
|
||||
params.emplace(ContextAccessParams{user_id_, /* full_access= */ false, /* use_default_roles = */ true, {}, settings, current_database, client_info });
|
||||
}
|
||||
/// `temp_access` is used here only to extract information about the user, not to actually check access.
|
||||
/// NOTE: AccessControl::getContextAccess() may require some IO work, so Context::getLock() must be unlocked while we're doing this.
|
||||
@ -1157,13 +1163,6 @@ std::optional<UUID> Context::getUserID() const
|
||||
}
|
||||
|
||||
|
||||
void Context::setQuotaKey(String quota_key_)
|
||||
{
|
||||
auto lock = getLock();
|
||||
client_info.quota_key = std::move(quota_key_);
|
||||
}
|
||||
|
||||
|
||||
void Context::setCurrentRoles(const std::vector<UUID> & current_roles_)
|
||||
{
|
||||
auto lock = getLock();
|
||||
@ -1303,7 +1302,7 @@ void Context::setCurrentProfiles(const SettingsProfilesInfo & profiles_info, boo
|
||||
{
|
||||
auto lock = getLock();
|
||||
if (check_constraints)
|
||||
checkSettingsConstraints(profiles_info.settings);
|
||||
checkSettingsConstraints(profiles_info.settings, SettingSource::PROFILE);
|
||||
applySettingsChanges(profiles_info.settings);
|
||||
settings_constraints_and_current_profiles = profiles_info.getConstraintsAndProfileIDs(settings_constraints_and_current_profiles);
|
||||
}
|
||||
@ -1857,29 +1856,29 @@ void Context::applySettingsChanges(const SettingsChanges & changes)
|
||||
}
|
||||
|
||||
|
||||
void Context::checkSettingsConstraints(const SettingsProfileElements & profile_elements) const
|
||||
void Context::checkSettingsConstraints(const SettingsProfileElements & profile_elements, SettingSource source) const
|
||||
{
|
||||
getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, profile_elements);
|
||||
getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, profile_elements, source);
|
||||
}
|
||||
|
||||
void Context::checkSettingsConstraints(const SettingChange & change) const
|
||||
void Context::checkSettingsConstraints(const SettingChange & change, SettingSource source) const
|
||||
{
|
||||
getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, change);
|
||||
getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, change, source);
|
||||
}
|
||||
|
||||
void Context::checkSettingsConstraints(const SettingsChanges & changes) const
|
||||
void Context::checkSettingsConstraints(const SettingsChanges & changes, SettingSource source) const
|
||||
{
|
||||
getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, changes);
|
||||
getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, changes, source);
|
||||
}
|
||||
|
||||
void Context::checkSettingsConstraints(SettingsChanges & changes) const
|
||||
void Context::checkSettingsConstraints(SettingsChanges & changes, SettingSource source) const
|
||||
{
|
||||
getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, changes);
|
||||
getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, changes, source);
|
||||
}
|
||||
|
||||
void Context::clampToSettingsConstraints(SettingsChanges & changes) const
|
||||
void Context::clampToSettingsConstraints(SettingsChanges & changes, SettingSource source) const
|
||||
{
|
||||
getSettingsConstraintsAndCurrentProfiles()->constraints.clamp(settings, changes);
|
||||
getSettingsConstraintsAndCurrentProfiles()->constraints.clamp(settings, changes, source);
|
||||
}
|
||||
|
||||
void Context::checkMergeTreeSettingsConstraints(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Common/HTTPHeaderFilter.h>
|
||||
#include <Common/ThreadPool_fwd.h>
|
||||
#include <Common/Throttler_fwd.h>
|
||||
#include <Common/SettingSource.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Core/UUID.h>
|
||||
@ -202,6 +203,8 @@ using MergeTreeMetadataCachePtr = std::shared_ptr<MergeTreeMetadataCache>;
|
||||
class PreparedSetsCache;
|
||||
using PreparedSetsCachePtr = std::shared_ptr<PreparedSetsCache>;
|
||||
|
||||
class SessionTracker;
|
||||
|
||||
/// An empty interface for an arbitrary object that may be attached by a shared pointer
|
||||
/// to query context, when using ClickHouse as a library.
|
||||
struct IHostContext
|
||||
@ -539,8 +542,6 @@ public:
|
||||
|
||||
String getUserName() const;
|
||||
|
||||
void setQuotaKey(String quota_key_);
|
||||
|
||||
void setCurrentRoles(const std::vector<UUID> & current_roles_);
|
||||
void setCurrentRolesDefault();
|
||||
boost::container::flat_set<UUID> getCurrentRoles() const;
|
||||
@ -735,11 +736,11 @@ public:
|
||||
void applySettingsChanges(const SettingsChanges & changes);
|
||||
|
||||
/// Checks the constraints.
|
||||
void checkSettingsConstraints(const SettingsProfileElements & profile_elements) const;
|
||||
void checkSettingsConstraints(const SettingChange & change) const;
|
||||
void checkSettingsConstraints(const SettingsChanges & changes) const;
|
||||
void checkSettingsConstraints(SettingsChanges & changes) const;
|
||||
void clampToSettingsConstraints(SettingsChanges & changes) const;
|
||||
void checkSettingsConstraints(const SettingsProfileElements & profile_elements, SettingSource source) const;
|
||||
void checkSettingsConstraints(const SettingChange & change, SettingSource source) const;
|
||||
void checkSettingsConstraints(const SettingsChanges & changes, SettingSource source) const;
|
||||
void checkSettingsConstraints(SettingsChanges & changes, SettingSource source) const;
|
||||
void clampToSettingsConstraints(SettingsChanges & changes, SettingSource source) const;
|
||||
void checkMergeTreeSettingsConstraints(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const;
|
||||
|
||||
/// Reset settings to default value
|
||||
@ -861,6 +862,8 @@ public:
|
||||
|
||||
OvercommitTracker * getGlobalOvercommitTracker() const;
|
||||
|
||||
SessionTracker & getSessionTracker();
|
||||
|
||||
MergeList & getMergeList();
|
||||
const MergeList & getMergeList() const;
|
||||
|
||||
|
@ -15,7 +15,7 @@ namespace DB
|
||||
BlockIO InterpreterSetQuery::execute()
|
||||
{
|
||||
const auto & ast = query_ptr->as<ASTSetQuery &>();
|
||||
getContext()->checkSettingsConstraints(ast.changes);
|
||||
getContext()->checkSettingsConstraints(ast.changes, SettingSource::QUERY);
|
||||
auto session_context = getContext()->getSessionContext();
|
||||
session_context->applySettingsChanges(ast.changes);
|
||||
session_context->addQueryParameters(ast.query_parameters);
|
||||
@ -28,7 +28,7 @@ void InterpreterSetQuery::executeForCurrentContext(bool ignore_setting_constrain
|
||||
{
|
||||
const auto & ast = query_ptr->as<ASTSetQuery &>();
|
||||
if (!ignore_setting_constraints)
|
||||
getContext()->checkSettingsConstraints(ast.changes);
|
||||
getContext()->checkSettingsConstraints(ast.changes, SettingSource::QUERY);
|
||||
getContext()->applySettingsChanges(ast.changes);
|
||||
getContext()->resetSettingsToDefaultValue(ast.default_settings);
|
||||
}
|
||||
|
@ -3,11 +3,13 @@
|
||||
#include <Access/AccessControl.h>
|
||||
#include <Access/Credentials.h>
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Access/SettingsProfilesInfo.h>
|
||||
#include <Access/User.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Interpreters/SessionTracker.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/SessionLog.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
@ -200,7 +202,6 @@ private:
|
||||
|
||||
LOG_TEST(log, "Schedule closing session with session_id: {}, user_id: {}",
|
||||
session.key.second, session.key.first);
|
||||
|
||||
}
|
||||
|
||||
void cleanThread()
|
||||
@ -336,6 +337,9 @@ void Session::authenticate(const Credentials & credentials_, const Poco::Net::So
|
||||
if (session_context)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "If there is a session context it must be created after authentication");
|
||||
|
||||
if (session_tracker_handle)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Session tracker handle was created before authentication finish");
|
||||
|
||||
auto address = address_;
|
||||
if ((address == Poco::Net::SocketAddress{}) && (prepared_client_info->interface == ClientInfo::Interface::LOCAL))
|
||||
address = Poco::Net::SocketAddress{"127.0.0.1", 0};
|
||||
@ -490,6 +494,8 @@ ContextMutablePtr Session::makeSessionContext()
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Session context must be created before any query context");
|
||||
if (!user_id)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Session context must be created after authentication");
|
||||
if (session_tracker_handle)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Session tracker handle was created before making session");
|
||||
|
||||
LOG_DEBUG(log, "{} Creating session context with user_id: {}",
|
||||
toString(auth_id), toString(*user_id));
|
||||
@ -503,13 +509,17 @@ ContextMutablePtr Session::makeSessionContext()
|
||||
prepared_client_info.reset();
|
||||
|
||||
/// Set user information for the new context: current profiles, roles, access rights.
|
||||
if (user_id)
|
||||
new_session_context->setUser(*user_id);
|
||||
new_session_context->setUser(*user_id);
|
||||
|
||||
/// Session context is ready.
|
||||
session_context = new_session_context;
|
||||
user = session_context->getUser();
|
||||
|
||||
session_tracker_handle = session_context->getSessionTracker().trackSession(
|
||||
*user_id,
|
||||
{},
|
||||
session_context->getSettingsRef().max_sessions_for_user);
|
||||
|
||||
return session_context;
|
||||
}
|
||||
|
||||
@ -521,6 +531,8 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std:
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Session context must be created before any query context");
|
||||
if (!user_id)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Session context must be created after authentication");
|
||||
if (session_tracker_handle)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Session tracker handle was created before making session");
|
||||
|
||||
LOG_DEBUG(log, "{} Creating named session context with name: {}, user_id: {}",
|
||||
toString(auth_id), session_name_, toString(*user_id));
|
||||
@ -541,9 +553,23 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std:
|
||||
new_session_context->setClientInfo(*prepared_client_info);
|
||||
prepared_client_info.reset();
|
||||
|
||||
auto access = new_session_context->getAccess();
|
||||
UInt64 max_sessions_for_user = 0;
|
||||
/// Set user information for the new context: current profiles, roles, access rights.
|
||||
if (user_id && !new_session_context->getAccess()->tryGetUser())
|
||||
if (!access->tryGetUser())
|
||||
{
|
||||
new_session_context->setUser(*user_id);
|
||||
max_sessions_for_user = new_session_context->getSettingsRef().max_sessions_for_user;
|
||||
}
|
||||
else
|
||||
{
|
||||
// Always get setting from profile
|
||||
// profile can be changed by ALTER PROFILE during single session
|
||||
auto settings = access->getDefaultSettings();
|
||||
const Field * max_session_for_user_field = settings.tryGet("max_sessions_for_user");
|
||||
if (max_session_for_user_field)
|
||||
max_sessions_for_user = max_session_for_user_field->safeGet<UInt64>();
|
||||
}
|
||||
|
||||
/// Session context is ready.
|
||||
session_context = std::move(new_session_context);
|
||||
@ -551,6 +577,11 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std:
|
||||
named_session_created = new_named_session_created;
|
||||
user = session_context->getUser();
|
||||
|
||||
session_tracker_handle = session_context->getSessionTracker().trackSession(
|
||||
*user_id,
|
||||
{ session_name_ },
|
||||
max_sessions_for_user);
|
||||
|
||||
return session_context;
|
||||
}
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Access/AuthenticationData.h>
|
||||
#include <Interpreters/ClientInfo.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Interpreters/SessionTracker.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <memory>
|
||||
@ -113,6 +114,8 @@ private:
|
||||
std::shared_ptr<NamedSessionData> named_session;
|
||||
bool named_session_created = false;
|
||||
|
||||
SessionTracker::SessionTrackerHandle session_tracker_handle;
|
||||
|
||||
Poco::Logger * log = nullptr;
|
||||
};
|
||||
|
||||
|
62
src/Interpreters/SessionTracker.cpp
Normal file
62
src/Interpreters/SessionTracker.cpp
Normal file
@ -0,0 +1,62 @@
|
||||
#include "SessionTracker.h"
|
||||
|
||||
#include <Common/Exception.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int USER_SESSION_LIMIT_EXCEEDED;
|
||||
}
|
||||
|
||||
SessionTracker::Session::Session(SessionTracker & tracker_,
|
||||
const UUID& user_id_,
|
||||
SessionInfos::const_iterator session_info_iter_) noexcept
|
||||
: tracker(tracker_), user_id(user_id_), session_info_iter(session_info_iter_)
|
||||
{
|
||||
}
|
||||
|
||||
SessionTracker::Session::~Session()
|
||||
{
|
||||
tracker.stopTracking(user_id, session_info_iter);
|
||||
}
|
||||
|
||||
SessionTracker::SessionTrackerHandle
|
||||
SessionTracker::trackSession(const UUID & user_id,
|
||||
const SessionInfo & session_info,
|
||||
size_t max_sessions_for_user)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
auto sessions_for_user_iter = sessions_for_user.find(user_id);
|
||||
if (sessions_for_user_iter == sessions_for_user.end())
|
||||
sessions_for_user_iter = sessions_for_user.emplace(user_id, SessionInfos()).first;
|
||||
|
||||
SessionInfos & session_infos = sessions_for_user_iter->second;
|
||||
if (max_sessions_for_user && session_infos.size() >= max_sessions_for_user)
|
||||
{
|
||||
throw Exception(ErrorCodes::USER_SESSION_LIMIT_EXCEEDED,
|
||||
"User {} has overflown session count {}",
|
||||
toString(user_id),
|
||||
max_sessions_for_user);
|
||||
}
|
||||
|
||||
session_infos.emplace_front(session_info);
|
||||
|
||||
return std::make_unique<SessionTracker::Session>(*this, user_id, session_infos.begin());
|
||||
}
|
||||
|
||||
void SessionTracker::stopTracking(const UUID& user_id, SessionInfos::const_iterator session_info_iter)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
auto sessions_for_user_iter = sessions_for_user.find(user_id);
|
||||
chassert(sessions_for_user_iter != sessions_for_user.end());
|
||||
|
||||
sessions_for_user_iter->second.erase(session_info_iter);
|
||||
if (sessions_for_user_iter->second.empty())
|
||||
sessions_for_user.erase(sessions_for_user_iter);
|
||||
}
|
||||
|
||||
}
|
60
src/Interpreters/SessionTracker.h
Normal file
60
src/Interpreters/SessionTracker.h
Normal file
@ -0,0 +1,60 @@
|
||||
#pragma once
|
||||
|
||||
#include "ClientInfo.h"
|
||||
|
||||
#include <list>
|
||||
#include <map>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct SessionInfo
|
||||
{
|
||||
const String session_id;
|
||||
};
|
||||
|
||||
using SessionInfos = std::list<SessionInfo>;
|
||||
|
||||
using SessionsForUser = std::unordered_map<UUID, SessionInfos>;
|
||||
|
||||
class SessionTracker;
|
||||
|
||||
class SessionTracker
|
||||
{
|
||||
public:
|
||||
class Session : boost::noncopyable
|
||||
{
|
||||
public:
|
||||
explicit Session(SessionTracker & tracker_,
|
||||
const UUID & user_id_,
|
||||
SessionInfos::const_iterator session_info_iter_) noexcept;
|
||||
|
||||
~Session();
|
||||
|
||||
private:
|
||||
friend class SessionTracker;
|
||||
|
||||
SessionTracker & tracker;
|
||||
const UUID user_id;
|
||||
const SessionInfos::const_iterator session_info_iter;
|
||||
};
|
||||
|
||||
using SessionTrackerHandle = std::unique_ptr<SessionTracker::Session>;
|
||||
|
||||
SessionTrackerHandle trackSession(const UUID & user_id,
|
||||
const SessionInfo & session_info,
|
||||
size_t max_sessions_for_user);
|
||||
|
||||
private:
|
||||
/// disallow manual messing with session tracking
|
||||
friend class Session;
|
||||
|
||||
std::mutex mutex;
|
||||
SessionsForUser sessions_for_user TSA_GUARDED_BY(mutex);
|
||||
|
||||
void stopTracking(const UUID& user_id, SessionInfos::const_iterator session_info_iter);
|
||||
};
|
||||
|
||||
}
|
@ -833,7 +833,7 @@ namespace
|
||||
{
|
||||
settings_changes.push_back({key, value});
|
||||
}
|
||||
query_context->checkSettingsConstraints(settings_changes);
|
||||
query_context->checkSettingsConstraints(settings_changes, SettingSource::QUERY);
|
||||
query_context->applySettingsChanges(settings_changes);
|
||||
|
||||
query_context->setCurrentQueryId(query_info.query_id());
|
||||
@ -1118,7 +1118,7 @@ namespace
|
||||
SettingsChanges settings_changes;
|
||||
for (const auto & [key, value] : external_table.settings())
|
||||
settings_changes.push_back({key, value});
|
||||
external_table_context->checkSettingsConstraints(settings_changes);
|
||||
external_table_context->checkSettingsConstraints(settings_changes, SettingSource::QUERY);
|
||||
external_table_context->applySettingsChanges(settings_changes);
|
||||
}
|
||||
auto in = external_table_context->getInputFormat(
|
||||
|
@ -764,7 +764,7 @@ void HTTPHandler::processQuery(
|
||||
context->setDefaultFormat(default_format);
|
||||
|
||||
/// For external data we also want settings
|
||||
context->checkSettingsConstraints(settings_changes);
|
||||
context->checkSettingsConstraints(settings_changes, SettingSource::QUERY);
|
||||
context->applySettingsChanges(settings_changes);
|
||||
|
||||
/// Set the query id supplied by the user, if any, and also update the OpenTelemetry fields.
|
||||
|
@ -184,14 +184,17 @@ void TCPHandler::runImpl()
|
||||
try
|
||||
{
|
||||
receiveHello();
|
||||
|
||||
/// In interserver mode queries are executed without a session context.
|
||||
if (!is_interserver_mode)
|
||||
session->makeSessionContext();
|
||||
|
||||
sendHello();
|
||||
if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM)
|
||||
receiveAddendum();
|
||||
|
||||
if (!is_interserver_mode) /// In interserver mode queries are executed without a session context.
|
||||
if (!is_interserver_mode)
|
||||
{
|
||||
session->makeSessionContext();
|
||||
|
||||
/// If session created, then settings in session context has been updated.
|
||||
/// So it's better to update the connection settings for flexibility.
|
||||
extractConnectionSettingsFromContext(session->sessionContext());
|
||||
@ -1181,7 +1184,6 @@ std::unique_ptr<Session> TCPHandler::makeSession()
|
||||
res->setClientName(client_name);
|
||||
res->setClientVersion(client_version_major, client_version_minor, client_version_patch, client_tcp_protocol_version);
|
||||
res->setConnectionClientVersion(client_version_major, client_version_minor, client_version_patch, client_tcp_protocol_version);
|
||||
res->setQuotaClientKey(quota_key);
|
||||
res->setClientInterface(interface);
|
||||
|
||||
return res;
|
||||
@ -1274,11 +1276,10 @@ void TCPHandler::receiveHello()
|
||||
void TCPHandler::receiveAddendum()
|
||||
{
|
||||
if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_QUOTA_KEY)
|
||||
{
|
||||
readStringBinary(quota_key, *in);
|
||||
if (!is_interserver_mode)
|
||||
session->setQuotaClientKey(quota_key);
|
||||
}
|
||||
|
||||
if (!is_interserver_mode)
|
||||
session->setQuotaClientKey(quota_key);
|
||||
}
|
||||
|
||||
|
||||
@ -1591,12 +1592,12 @@ void TCPHandler::receiveQuery()
|
||||
if (query_kind == ClientInfo::QueryKind::INITIAL_QUERY)
|
||||
{
|
||||
/// Throw an exception if the passed settings violate the constraints.
|
||||
query_context->checkSettingsConstraints(settings_changes);
|
||||
query_context->checkSettingsConstraints(settings_changes, SettingSource::QUERY);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Quietly clamp to the constraints if it's not an initial query.
|
||||
query_context->clampToSettingsConstraints(settings_changes);
|
||||
query_context->clampToSettingsConstraints(settings_changes, SettingSource::QUERY);
|
||||
}
|
||||
query_context->applySettingsChanges(settings_changes);
|
||||
|
||||
|
@ -82,5 +82,15 @@
|
||||
"test_system_flush_logs/test.py::test_log_buffer_size_rows_flush_threshold",
|
||||
"test_system_flush_logs/test.py::test_log_max_size",
|
||||
"test_crash_log/test.py::test_pkill_query_log",
|
||||
"test_crash_log/test.py::test_pkill"
|
||||
"test_crash_log/test.py::test_pkill",
|
||||
|
||||
"test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_tcp",
|
||||
"test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_postgres",
|
||||
"test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_mysql",
|
||||
"test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_http",
|
||||
"test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_http_named_session",
|
||||
"test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_grpc",
|
||||
"test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_tcp_and_others",
|
||||
"test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_tcp",
|
||||
"test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_end_session"
|
||||
]
|
||||
|
0
tests/integration/test_profile_max_sessions_for_user/__init__.py
Executable file
0
tests/integration/test_profile_max_sessions_for_user/__init__.py
Executable file
8
tests/integration/test_profile_max_sessions_for_user/configs/dhparam.pem
Executable file
8
tests/integration/test_profile_max_sessions_for_user/configs/dhparam.pem
Executable file
@ -0,0 +1,8 @@
|
||||
-----BEGIN DH PARAMETERS-----
|
||||
MIIBCAKCAQEAua92DDli13gJ+//ZXyGaggjIuidqB0crXfhUlsrBk9BV1hH3i7fR
|
||||
XGP9rUdk2ubnB3k2ejBStL5oBrkHm9SzUFSQHqfDjLZjKoUpOEmuDc4cHvX1XTR5
|
||||
Pr1vf5cd0yEncJWG5W4zyUB8k++SUdL2qaeslSs+f491HBLDYn/h8zCgRbBvxhxb
|
||||
9qeho1xcbnWeqkN6Kc9bgGozA16P9NLuuLttNnOblkH+lMBf42BSne/TWt3AlGZf
|
||||
slKmmZcySUhF8aKfJnLKbkBCFqOtFRh8zBA9a7g+BT/lSANATCDPaAk1YVih2EKb
|
||||
dpc3briTDbRsiqg2JKMI7+VdULY9bh3EawIBAg==
|
||||
-----END DH PARAMETERS-----
|
@ -0,0 +1,9 @@
|
||||
<clickhouse>
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
|
||||
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
|
||||
<size>1000M</size>
|
||||
<count>10</count>
|
||||
</logger>
|
||||
</clickhouse>
|
@ -0,0 +1,9 @@
|
||||
<clickhouse>
|
||||
<postgresql_port>5433</postgresql_port>
|
||||
<mysql_port>9001</mysql_port>
|
||||
<grpc_port>9100</grpc_port>
|
||||
<grpc replace="replace">
|
||||
<!-- Enable if you want very detailed logs -->
|
||||
<verbose_logs>false</verbose_logs>
|
||||
</grpc>
|
||||
</clickhouse>
|
18
tests/integration/test_profile_max_sessions_for_user/configs/server.crt
Executable file
18
tests/integration/test_profile_max_sessions_for_user/configs/server.crt
Executable file
@ -0,0 +1,18 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIIC+zCCAeOgAwIBAgIJANhP897Se2gmMA0GCSqGSIb3DQEBCwUAMBQxEjAQBgNV
|
||||
BAMMCWxvY2FsaG9zdDAeFw0yMDA0MTgyMTE2NDBaFw0yMTA0MTgyMTE2NDBaMBQx
|
||||
EjAQBgNVBAMMCWxvY2FsaG9zdDCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoC
|
||||
ggEBAM92kcojQoMsjZ9YGhPMY6h/fDUsZeSKHLxgqE6wbmfU1oZKCPWqnvl+4n0J
|
||||
pnT5h1ETxxYZLepimKq0DEVPUTmCl0xmcKbtUNiaTUKYKsdita6b2vZCX9wUPN9p
|
||||
2Kjnm41l+aZNqIEBhIgHNWg9qowi20y0EIXR79jQLwwaInHAaJLZxVsqY2zjQ/D7
|
||||
1Zh82MXud7iqxBQiEfw9Cz35UFA239R8QTlPkVQfsN1gfLxnLk24QUX3o+hbUI1g
|
||||
nlSpyYDHYQlOmwz8doDs6THHAZNJ4bPE9xHNFpw6dGZdbtH+IKQ/qRZIiOaiNuzJ
|
||||
IOHl6XQDRDkW2LMTiCQ6fjC7Pz8CAwEAAaNQME4wHQYDVR0OBBYEFFvhaA/Eguyf
|
||||
BXkMj8BkNLBqMnz2MB8GA1UdIwQYMBaAFFvhaA/EguyfBXkMj8BkNLBqMnz2MAwG
|
||||
A1UdEwQFMAMBAf8wDQYJKoZIhvcNAQELBQADggEBACeU/oL48eVAKH7NQntHhRaJ
|
||||
ZGeQzKIjrSBjFo8BGXD1nJZhUeFsylLrhCkC8/5/3grE3BNVX9bxcGjO81C9Mn4U
|
||||
t0z13d6ovJjCZSQArtLwgeJGlpH7gNdD3DyT8DQmrqYVnmnB7UmBu45XH1LWGQZr
|
||||
FAOhGRVs6s6mNj8QlLMgdmsOeOQnsGCMdoss8zV9vO2dc4A5SDSSL2mqGGY4Yjtt
|
||||
X+XlEhXXnksGyx8NGVOZX4wcj8WeCAj/lihQ7Zh6XYwZH9i+E46ompUwoziZnNPu
|
||||
2RH63tLNCxkOY2HF5VMlbMmzer3FkhlM6TAZZRPcvSphKPwXK4A33yqc6wnWvpc=
|
||||
-----END CERTIFICATE-----
|
28
tests/integration/test_profile_max_sessions_for_user/configs/server.key
Executable file
28
tests/integration/test_profile_max_sessions_for_user/configs/server.key
Executable file
@ -0,0 +1,28 @@
|
||||
-----BEGIN PRIVATE KEY-----
|
||||
MIIEvgIBADANBgkqhkiG9w0BAQEFAASCBKgwggSkAgEAAoIBAQDPdpHKI0KDLI2f
|
||||
WBoTzGOof3w1LGXkihy8YKhOsG5n1NaGSgj1qp75fuJ9CaZ0+YdRE8cWGS3qYpiq
|
||||
tAxFT1E5gpdMZnCm7VDYmk1CmCrHYrWum9r2Ql/cFDzfadio55uNZfmmTaiBAYSI
|
||||
BzVoPaqMIttMtBCF0e/Y0C8MGiJxwGiS2cVbKmNs40Pw+9WYfNjF7ne4qsQUIhH8
|
||||
PQs9+VBQNt/UfEE5T5FUH7DdYHy8Zy5NuEFF96PoW1CNYJ5UqcmAx2EJTpsM/HaA
|
||||
7OkxxwGTSeGzxPcRzRacOnRmXW7R/iCkP6kWSIjmojbsySDh5el0A0Q5FtizE4gk
|
||||
On4wuz8/AgMBAAECggEAJ54J2yL+mZQRe2NUn4FBarTloDXZQ1pIgISov1Ybz0Iq
|
||||
sTxEF728XAKp95y3J9Fa0NXJB+RJC2BGrRpy2W17IlNY1yMc0hOxg5t7s4LhcG/e
|
||||
J/jlSG+GZL2MnlFVKXQJFWhq0yIzUmdayqstvLlB7z7cx/n+yb88YRfoVBRNjZEL
|
||||
Tdrsw+087igDjrIxZJ3eMN5Wi434n9s4yAoRQC1bP5wcWx0gD4MzdmL8ip6suiRc
|
||||
LRuBAhV/Op812xlxUhrF5dInUM9OLlGTXpUzexAS8Cyy7S4bfkW2BaCxTF7I7TFw
|
||||
Whx28CKn/G49tIuU0m6AlxWbXpLVePTFyMb7RJz5cQKBgQD7VQd2u3HM6eE3PcXD
|
||||
p6ObdLTUk8OAJ5BMmADFc71W0Epyo26/e8KXKGYGxE2W3fr13y+9b0fl5fxZPuhS
|
||||
MgvXEO7rItAVsLcp0IzaqY0WUee2b4XWPAU0XuPqvjYMpx8H5OEHqFK6lhZysAqM
|
||||
X7Ot3/Hux9X0MC4v5a/HNbDUOQKBgQDTUPaP3ADRrmpmE2sWuzWEnCSEz5f0tCLO
|
||||
wTqhV/UraWUNlAbgK5NB790IjH/gotBSqqNPLJwJh0LUfClKM4LiaHsEag0OArOF
|
||||
GhPMK1Ohps8c2RRsiG8+hxX2HEHeAVbkouEDPDiHdIW/92pBViDoETXL6qxDKbm9
|
||||
LkOcVeDfNwKBgQChh1xsqrvQ/t+IKWNZA/zahH9TwEP9sW/ESkz0mhYuHWA7nV4o
|
||||
ItpFW+l2n+Nd+vy32OFN1p9W2iD9GrklWpTRfEiRRqaFyjVt4mMkhaPvnGRXlAVo
|
||||
Utrldbb1v5ntN9txr2ARE9VXpe53dzzQSxGnxi4vUK/paK3GitAWMCOdwQKBgQCi
|
||||
hmGsUXQb0P6qVYMGr6PAw2re7t8baLRguoMCdqjs45nCMLh9D2apzvb8TTtJJU/+
|
||||
VJlYGqJEPdDrpjcHh8jBo8QBqCM0RGWYGG9jl2syKB6hPGCV/PU6bSE58Y/DVNpk
|
||||
7NUM7PM5UyhPddY2PC0A78Ole29UFLJzSzLa+b4DTwKBgH9Wh2k4YPnPcRrX89UL
|
||||
eSwWa1CGq6HWX8Kd5qyz256aeHWuG5nv15+rBt+D7nwajUsqeVkAXz5H/dHuG1xz
|
||||
jb7RW+pEjx0GVAmIbkM9vOLqEUfHHHPuk4AXCGGZ5sarPiKg4BHKBBsY1dpoO5UH
|
||||
0j71fRA6zurHnTXDaCLWlUpZ
|
||||
-----END PRIVATE KEY-----
|
@ -0,0 +1,17 @@
|
||||
<clickhouse>
|
||||
<!-- Used with https_port and tcp_port_secure. Full ssl options list: https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h#L71 -->
|
||||
<openSSL>
|
||||
<server> <!-- Used for https server AND secure tcp port -->
|
||||
<!-- openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout /etc/clickhouse-server/server.key -out /etc/clickhouse-server/server.crt -->
|
||||
<certificateFile>/etc/clickhouse-server/config.d/server.crt</certificateFile>
|
||||
<privateKeyFile>/etc/clickhouse-server/config.d/server.key</privateKeyFile>
|
||||
<!-- openssl dhparam -out /etc/clickhouse-server/dhparam.pem 4096 -->
|
||||
<dhParamsFile>/etc/clickhouse-server/config.d/dhparam.pem</dhParamsFile>
|
||||
<verificationMode>none</verificationMode>
|
||||
<loadDefaultCAFile>true</loadDefaultCAFile>
|
||||
<cacheSessions>true</cacheSessions>
|
||||
<disableProtocols>sslv2,sslv3</disableProtocols>
|
||||
<preferServerCiphers>true</preferServerCiphers>
|
||||
</server>
|
||||
</openSSL>
|
||||
</clickhouse>
|
@ -0,0 +1,16 @@
|
||||
<clickhouse>
|
||||
<profiles>
|
||||
<default>
|
||||
<max_sessions_for_user>2</max_sessions_for_user>
|
||||
<function_sleep_max_microseconds_per_block>0</function_sleep_max_microseconds_per_block>
|
||||
</default>
|
||||
</profiles>
|
||||
|
||||
<users>
|
||||
<default>
|
||||
</default>
|
||||
<test_user>
|
||||
<password>123</password>
|
||||
</test_user>
|
||||
</users>
|
||||
</clickhouse>
|
@ -0,0 +1 @@
|
||||
../../../../src/Server/grpc_protos/clickhouse_grpc.proto
|
222
tests/integration/test_profile_max_sessions_for_user/test.py
Executable file
222
tests/integration/test_profile_max_sessions_for_user/test.py
Executable file
@ -0,0 +1,222 @@
|
||||
import os
|
||||
|
||||
import grpc
|
||||
import pymysql.connections
|
||||
import psycopg2 as py_psql
|
||||
import pytest
|
||||
import sys
|
||||
import threading
|
||||
|
||||
from helpers.cluster import ClickHouseCluster, run_and_check
|
||||
|
||||
MAX_SESSIONS_FOR_USER = 2
|
||||
POSTGRES_SERVER_PORT = 5433
|
||||
MYSQL_SERVER_PORT = 9001
|
||||
GRPC_PORT = 9100
|
||||
|
||||
TEST_USER = "test_user"
|
||||
TEST_PASSWORD = "123"
|
||||
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
DEFAULT_ENCODING = "utf-8"
|
||||
|
||||
# Use grpcio-tools to generate *pb2.py files from *.proto.
|
||||
proto_dir = os.path.join(SCRIPT_DIR, "./protos")
|
||||
gen_dir = os.path.join(SCRIPT_DIR, "./_gen")
|
||||
os.makedirs(gen_dir, exist_ok=True)
|
||||
run_and_check(
|
||||
"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} \
|
||||
{proto_dir}/clickhouse_grpc.proto".format(
|
||||
proto_dir=proto_dir, gen_dir=gen_dir
|
||||
),
|
||||
shell=True,
|
||||
)
|
||||
|
||||
sys.path.append(gen_dir)
|
||||
|
||||
import clickhouse_grpc_pb2
|
||||
import clickhouse_grpc_pb2_grpc
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance(
|
||||
"node",
|
||||
main_configs=[
|
||||
"configs/ports.xml",
|
||||
"configs/log.xml",
|
||||
"configs/ssl_conf.xml",
|
||||
"configs/dhparam.pem",
|
||||
"configs/server.crt",
|
||||
"configs/server.key",
|
||||
],
|
||||
user_configs=["configs/users.xml"],
|
||||
env_variables={"UBSAN_OPTIONS": "print_stacktrace=1"},
|
||||
)
|
||||
|
||||
|
||||
def get_query(name, id):
|
||||
return f"SElECT '{name}', {id}, sleep(1)"
|
||||
|
||||
|
||||
def grpc_get_url():
|
||||
return f"{instance.ip_address}:{GRPC_PORT}"
|
||||
|
||||
|
||||
def grpc_create_insecure_channel():
|
||||
channel = grpc.insecure_channel(grpc_get_url())
|
||||
grpc.channel_ready_future(channel).result(timeout=2)
|
||||
return channel
|
||||
|
||||
|
||||
def grpc_query(query_text, channel, session_id_):
|
||||
query_info = clickhouse_grpc_pb2.QueryInfo(
|
||||
query=query_text,
|
||||
session_id=session_id_,
|
||||
user_name=TEST_USER,
|
||||
password=TEST_PASSWORD,
|
||||
)
|
||||
|
||||
stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel)
|
||||
result = stub.ExecuteQuery(query_info)
|
||||
if result and result.HasField("exception"):
|
||||
raise Exception(result.exception.display_text)
|
||||
return result.output.decode(DEFAULT_ENCODING)
|
||||
|
||||
|
||||
def threaded_run_test(sessions):
|
||||
thread_list = []
|
||||
for i in range(len(sessions)):
|
||||
thread = ThreadWithException(target=sessions[i], args=(i,))
|
||||
thread_list.append(thread)
|
||||
thread.start()
|
||||
|
||||
for thread in thread_list:
|
||||
thread.join()
|
||||
|
||||
exception_count = 0
|
||||
for i in range(len(sessions)):
|
||||
if thread_list[i].run_exception != None:
|
||||
exception_count += 1
|
||||
|
||||
assert exception_count == 1
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
class ThreadWithException(threading.Thread):
|
||||
run_exception = None
|
||||
|
||||
def run(self):
|
||||
try:
|
||||
super().run()
|
||||
except:
|
||||
self.run_exception = sys.exc_info()
|
||||
|
||||
def join(self):
|
||||
super().join()
|
||||
|
||||
|
||||
def postgres_session(id):
|
||||
ch = py_psql.connect(
|
||||
host=instance.ip_address,
|
||||
port=POSTGRES_SERVER_PORT,
|
||||
user=TEST_USER,
|
||||
password=TEST_PASSWORD,
|
||||
database="default",
|
||||
)
|
||||
cur = ch.cursor()
|
||||
cur.execute(get_query("postgres_session", id))
|
||||
cur.fetchall()
|
||||
|
||||
|
||||
def mysql_session(id):
|
||||
client = pymysql.connections.Connection(
|
||||
host=instance.ip_address,
|
||||
user=TEST_USER,
|
||||
password=TEST_PASSWORD,
|
||||
database="default",
|
||||
port=MYSQL_SERVER_PORT,
|
||||
)
|
||||
cursor = client.cursor(pymysql.cursors.DictCursor)
|
||||
cursor.execute(get_query("mysql_session", id))
|
||||
cursor.fetchall()
|
||||
|
||||
|
||||
def tcp_session(id):
|
||||
instance.query(get_query("tcp_session", id), user=TEST_USER, password=TEST_PASSWORD)
|
||||
|
||||
|
||||
def http_session(id):
|
||||
instance.http_query(
|
||||
get_query("http_session", id), user=TEST_USER, password=TEST_PASSWORD
|
||||
)
|
||||
|
||||
|
||||
def http_named_session(id):
|
||||
instance.http_query(
|
||||
get_query("http_named_session", id),
|
||||
user=TEST_USER,
|
||||
password=TEST_PASSWORD,
|
||||
params={"session_id": id},
|
||||
)
|
||||
|
||||
|
||||
def grpc_session(id):
|
||||
grpc_query(
|
||||
get_query("grpc_session", id), grpc_create_insecure_channel(), f"session_{id}"
|
||||
)
|
||||
|
||||
|
||||
def test_profile_max_sessions_for_user_tcp(started_cluster):
|
||||
threaded_run_test([tcp_session] * 3)
|
||||
|
||||
|
||||
def test_profile_max_sessions_for_user_postgres(started_cluster):
|
||||
threaded_run_test([postgres_session] * 3)
|
||||
|
||||
|
||||
def test_profile_max_sessions_for_user_mysql(started_cluster):
|
||||
threaded_run_test([mysql_session] * 3)
|
||||
|
||||
|
||||
def test_profile_max_sessions_for_user_http(started_cluster):
|
||||
threaded_run_test([http_session] * 3)
|
||||
|
||||
|
||||
def test_profile_max_sessions_for_user_http_named_session(started_cluster):
|
||||
threaded_run_test([http_named_session] * 3)
|
||||
|
||||
|
||||
def test_profile_max_sessions_for_user_grpc(started_cluster):
|
||||
threaded_run_test([grpc_session] * 3)
|
||||
|
||||
|
||||
def test_profile_max_sessions_for_user_tcp_and_others(started_cluster):
|
||||
threaded_run_test([tcp_session, grpc_session, grpc_session])
|
||||
threaded_run_test([tcp_session, http_session, http_session])
|
||||
threaded_run_test([tcp_session, mysql_session, mysql_session])
|
||||
threaded_run_test([tcp_session, postgres_session, postgres_session])
|
||||
threaded_run_test([tcp_session, http_session, postgres_session])
|
||||
threaded_run_test([tcp_session, postgres_session, http_session])
|
||||
|
||||
|
||||
def test_profile_max_sessions_for_user_end_session(started_cluster):
|
||||
for conection_func in [
|
||||
tcp_session,
|
||||
http_session,
|
||||
grpc_session,
|
||||
mysql_session,
|
||||
postgres_session,
|
||||
]:
|
||||
threaded_run_test([conection_func] * MAX_SESSIONS_FOR_USER)
|
||||
threaded_run_test([conection_func] * MAX_SESSIONS_FOR_USER)
|
||||
|
||||
|
||||
def test_profile_max_sessions_for_user_end_session(started_cluster):
|
||||
instance.query_and_get_error("SET max_sessions_for_user = 10")
|
@ -0,0 +1,12 @@
|
||||
test_alter_profile case: max_session_count 1 alter_sessions_count 1
|
||||
test_alter_profile case: max_session_count 2 alter_sessions_count 1
|
||||
USER_SESSION_LIMIT_EXCEEDED
|
||||
test_alter_profile case: max_session_count 1 alter_sessions_count 2
|
||||
test_alter_profile case: max_session_count 2 alter_sessions_count 2
|
||||
READONLY
|
||||
READONLY
|
||||
READONLY
|
||||
READONLY
|
||||
READONLY
|
||||
READONLY
|
||||
READONLY
|
64
tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh
Executable file
64
tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh
Executable file
@ -0,0 +1,64 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
SESSION_ID_PREFIX="02832_alter_max_sessions_session_$$"
|
||||
PROFILE="02832_alter_max_sessions_profile_$$"
|
||||
USER="02832_alter_max_sessions_user_$$"
|
||||
USER2="02832_alter_max_sessions_user_two_$$"
|
||||
ROLE="02832_alter_max_sessions_role_$$"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q $"DROP USER IF EXISTS '${USER}'"
|
||||
${CLICKHOUSE_CLIENT} -q $"DROP PROFILE IF EXISTS ${PROFILE}"
|
||||
${CLICKHOUSE_CLIENT} -q $"CREATE SETTINGS PROFILE ${PROFILE}"
|
||||
${CLICKHOUSE_CLIENT} -q $"CREATE USER '${USER}' SETTINGS PROFILE '${PROFILE}'"
|
||||
|
||||
function test_alter_profile()
|
||||
{
|
||||
local max_session_count="$1"
|
||||
local alter_sessions_count="$2"
|
||||
echo $"test_alter_profile case: max_session_count ${max_session_count} alter_sessions_count ${alter_sessions_count}"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q $"ALTER SETTINGS PROFILE ${PROFILE} SETTINGS max_sessions_for_user = ${max_session_count}"
|
||||
|
||||
# Create sesssions with $max_session_count resriction
|
||||
for ((i = 1 ; i <= ${max_session_count} ; i++)); do
|
||||
local session_id="${SESSION_ID_PREFIX}_${i}"
|
||||
# Skip output from this query
|
||||
${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${USER}&session_id=${session_id}&session_check=0" --data-binary "SELECT 1" > /dev/null
|
||||
done
|
||||
|
||||
# Update resriction to $alter_sessions_count
|
||||
${CLICKHOUSE_CLIENT} -q $"ALTER SETTINGS PROFILE ${PROFILE} SETTINGS max_sessions_for_user = ${alter_sessions_count}"
|
||||
|
||||
# Simultaneous sessions should use max settings from profile ($alter_sessions_count)
|
||||
for ((i = 1 ; i <= ${max_session_count} ; i++)); do
|
||||
local session_id="${SESSION_ID_PREFIX}_${i}"
|
||||
# ignore select 1, we need only errors
|
||||
${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${USER}&session_id=${session_id}&session_check=1" --data-binary "select sleep(0.3)" | grep -o -m 1 'USER_SESSION_LIMIT_EXCEEDED' &
|
||||
done
|
||||
|
||||
wait
|
||||
}
|
||||
|
||||
test_alter_profile 1 1
|
||||
test_alter_profile 2 1
|
||||
test_alter_profile 1 2
|
||||
test_alter_profile 2 2
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT 1 SETTINGS max_sessions_for_user = 1" 2>&1 | grep -m 1 -o 'READONLY' | head -1
|
||||
${CLICKHOUSE_CLIENT} -q $"SET max_sessions_for_user = 1 " 2>&1 | grep -o -m 1 'READONLY' | head -1
|
||||
${CLICKHOUSE_CLIENT} --max_sessions_for_user=1 -q $"SELECT 1 " 2>&1 | grep -o -m 1 'READONLY' | head -1
|
||||
# max_sessions_for_user is profile setting
|
||||
${CLICKHOUSE_CLIENT} -q $"CREATE USER ${USER2} SETTINGS max_sessions_for_user = 1 " 2>&1 | grep -o -m 1 'READONLY' | head -1
|
||||
${CLICKHOUSE_CLIENT} -q $"ALTER USER ${USER} SETTINGS max_sessions_for_user = 1" 2>&1 | grep -o -m 1 'READONLY' | head -1
|
||||
${CLICKHOUSE_CLIENT} -q $"CREATE ROLE ${ROLE} SETTINGS max_sessions_for_user = 1" 2>&1 | grep -o -m 1 'READONLY' | head -1
|
||||
${CLICKHOUSE_CLIENT} -q $"CREATE ROLE ${ROLE}"
|
||||
${CLICKHOUSE_CLIENT} -q $"ALTER ROLE ${ROLE} SETTINGS max_sessions_for_user = 1 " 2>&1 | grep -o -m 1 'READONLY' | head -1
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q $"DROP USER IF EXISTS '${USER}'"
|
||||
${CLICKHOUSE_CLIENT} -q $"DROP USER IF EXISTS '${USER2}'"
|
||||
${CLICKHOUSE_CLIENT} -q $"DROP PROFILE IF EXISTS ${PROFILE}"
|
||||
${CLICKHOUSE_CLIENT} -q $"DROP ROLE IF EXISTS ${ROLE}"
|
Loading…
Reference in New Issue
Block a user