未验证 提交 0f04ff07 编写于 作者: V Vitaly Baranov 提交者: GitHub

Merge pull request #9364 from vitlibar/RBAC-6

RBAC-6
......@@ -44,6 +44,7 @@
#include <Interpreters/DNSCacheUpdater.h>
#include <Interpreters/SystemLog.cpp>
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
#include <Access/AccessControlManager.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/System/attachSystemTables.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
......@@ -493,6 +494,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
users_config_reloader->reload();
});
/// Sets a local directory storing information about access control.
std::string access_control_local_path = config().getString("access_control_path", "");
if (!access_control_local_path.empty())
global_context->getAccessControlManager().setLocalDirectory(access_control_local_path);
/// Limit on total number of concurrently executed queries.
global_context->getProcessList().setMaxSize(config().getInt("max_concurrent_queries", 0));
......
......@@ -2,6 +2,7 @@
#include <Access/MultipleAccessStorage.h>
#include <Access/MemoryAccessStorage.h>
#include <Access/UsersConfigAccessStorage.h>
#include <Access/DiskAccessStorage.h>
#include <Access/AccessRightsContextFactory.h>
#include <Access/RoleContextFactory.h>
#include <Access/RowPolicyContextFactory.h>
......@@ -15,10 +16,14 @@ namespace
std::vector<std::unique_ptr<IAccessStorage>> createStorages()
{
std::vector<std::unique_ptr<IAccessStorage>> list;
list.emplace_back(std::make_unique<MemoryAccessStorage>());
list.emplace_back(std::make_unique<DiskAccessStorage>());
list.emplace_back(std::make_unique<UsersConfigAccessStorage>());
list.emplace_back(std::make_unique<MemoryAccessStorage>());
return list;
}
constexpr size_t DISK_ACCESS_STORAGE_INDEX = 0;
constexpr size_t USERS_CONFIG_ACCESS_STORAGE_INDEX = 1;
}
......@@ -37,10 +42,17 @@ AccessControlManager::~AccessControlManager()
}
void AccessControlManager::loadFromConfig(const Poco::Util::AbstractConfiguration & users_config)
void AccessControlManager::setLocalDirectory(const String & directory_path)
{
auto & disk_access_storage = dynamic_cast<DiskAccessStorage &>(getStorageByIndex(DISK_ACCESS_STORAGE_INDEX));
disk_access_storage.setDirectory(directory_path);
}
void AccessControlManager::setUsersConfig(const Poco::Util::AbstractConfiguration & users_config)
{
auto & users_config_access_storage = dynamic_cast<UsersConfigAccessStorage &>(getStorageByIndex(1));
users_config_access_storage.loadFromConfig(users_config);
auto & users_config_access_storage = dynamic_cast<UsersConfigAccessStorage &>(getStorageByIndex(USERS_CONFIG_ACCESS_STORAGE_INDEX));
users_config_access_storage.setConfiguration(users_config);
}
......
......@@ -45,7 +45,8 @@ public:
AccessControlManager();
~AccessControlManager();
void loadFromConfig(const Poco::Util::AbstractConfiguration & users_config);
void setLocalDirectory(const String & directory);
void setUsersConfig(const Poco::Util::AbstractConfiguration & users_config);
AccessRightsContextPtr getAccessRightsContext(
const UUID & user_id,
......
......@@ -9,6 +9,7 @@ namespace DB
namespace ErrorCodes
{
extern const int INVALID_GRANT;
extern const int LOGICAL_ERROR;
}
......@@ -45,6 +46,13 @@ namespace
const AccessFlags create_table_flag = AccessType::CREATE_TABLE;
const AccessFlags create_temporary_table_flag = AccessType::CREATE_TEMPORARY_TABLE;
};
std::string_view checkCurrentDatabase(const std::string_view & current_database)
{
if (current_database.empty())
throw Exception("No current database", ErrorCodes::LOGICAL_ERROR);
return current_database;
}
}
......@@ -520,21 +528,21 @@ void AccessRights::grantImpl(const AccessRightsElement & element, std::string_vi
else if (element.any_table)
{
if (element.database.empty())
grantImpl(element.access_flags, current_database);
grantImpl(element.access_flags, checkCurrentDatabase(current_database));
else
grantImpl(element.access_flags, element.database);
}
else if (element.any_column)
{
if (element.database.empty())
grantImpl(element.access_flags, current_database, element.table);
grantImpl(element.access_flags, checkCurrentDatabase(current_database), element.table);
else
grantImpl(element.access_flags, element.database, element.table);
}
else
{
if (element.database.empty())
grantImpl(element.access_flags, current_database, element.table, element.columns);
grantImpl(element.access_flags, checkCurrentDatabase(current_database), element.table, element.columns);
else
grantImpl(element.access_flags, element.database, element.table, element.columns);
}
......@@ -575,21 +583,21 @@ void AccessRights::revokeImpl(const AccessRightsElement & element, std::string_v
else if (element.any_table)
{
if (element.database.empty())
revokeImpl<mode>(element.access_flags, current_database);
revokeImpl<mode>(element.access_flags, checkCurrentDatabase(current_database));
else
revokeImpl<mode>(element.access_flags, element.database);
}
else if (element.any_column)
{
if (element.database.empty())
revokeImpl<mode>(element.access_flags, current_database, element.table);
revokeImpl<mode>(element.access_flags, checkCurrentDatabase(current_database), element.table);
else
revokeImpl<mode>(element.access_flags, element.database, element.table);
}
else
{
if (element.database.empty())
revokeImpl<mode>(element.access_flags, current_database, element.table, element.columns);
revokeImpl<mode>(element.access_flags, checkCurrentDatabase(current_database), element.table, element.columns);
else
revokeImpl<mode>(element.access_flags, element.database, element.table, element.columns);
}
......@@ -710,21 +718,21 @@ bool AccessRights::isGrantedImpl(const AccessRightsElement & element, std::strin
else if (element.any_table)
{
if (element.database.empty())
return isGrantedImpl(element.access_flags, current_database);
return isGrantedImpl(element.access_flags, checkCurrentDatabase(current_database));
else
return isGrantedImpl(element.access_flags, element.database);
}
else if (element.any_column)
{
if (element.database.empty())
return isGrantedImpl(element.access_flags, current_database, element.table);
return isGrantedImpl(element.access_flags, checkCurrentDatabase(current_database), element.table);
else
return isGrantedImpl(element.access_flags, element.database, element.table);
}
else
{
if (element.database.empty())
return isGrantedImpl(element.access_flags, current_database, element.table, element.columns);
return isGrantedImpl(element.access_flags, checkCurrentDatabase(current_database), element.table, element.columns);
else
return isGrantedImpl(element.access_flags, element.database, element.table, element.columns);
}
......
此差异已折叠。
#pragma once
#include <Access/MemoryAccessStorage.h>
#include <Common/ThreadPool.h>
#include <boost/container/flat_set.hpp>
namespace DB
{
/// Loads and saves access entities on a local disk to a specified directory.
class DiskAccessStorage : public IAccessStorage
{
public:
DiskAccessStorage();
~DiskAccessStorage() override;
void setDirectory(const String & directory_path_);
private:
std::optional<UUID> findImpl(std::type_index type, const String & name) const override;
std::vector<UUID> findAllImpl(std::type_index type) const override;
bool existsImpl(const UUID & id) const override;
AccessEntityPtr readImpl(const UUID & id) const override;
String readNameImpl(const UUID & id) const override;
bool canInsertImpl(const AccessEntityPtr & entity) const override;
UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) override;
void removeImpl(const UUID & id) override;
void updateImpl(const UUID & id, const UpdateFunc & update_func) override;
ext::scope_guard subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const override;
ext::scope_guard subscribeForChangesImpl(std::type_index type, const OnChangedHandler & handler) const override;
bool hasSubscriptionImpl(const UUID & id) const override;
bool hasSubscriptionImpl(std::type_index type) const override;
void initialize(const String & directory_path_, Notifications & notifications);
bool readLists();
void writeLists();
void scheduleWriteLists(std::type_index type);
void rebuildLists();
void startListsWritingThread();
void stopListsWritingThread();
void listsWritingThreadFunc();
void insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, Notifications & notifications);
void removeNoLock(const UUID & id, Notifications & notifications);
void updateNoLock(const UUID & id, const UpdateFunc & update_func, Notifications & notifications);
AccessEntityPtr readAccessEntityFromDisk(const UUID & id) const;
void writeAccessEntityToDisk(const UUID & id, const IAccessEntity & entity) const;
void deleteAccessEntityOnDisk(const UUID & id) const;
using NameToIDMap = std::unordered_map<String, UUID>;
struct Entry
{
Entry(const std::string_view & name_, std::type_index type_) : name(name_), type(type_) {}
std::string_view name; /// view points to a string in `name_to_id_maps`.
std::type_index type;
mutable AccessEntityPtr entity; /// may be nullptr, if the entity hasn't been loaded yet.
mutable std::list<OnChangedHandler> handlers_by_id;
};
void prepareNotifications(const UUID & id, const Entry & entry, bool remove, Notifications & notifications) const;
String directory_path;
bool initialized = false;
std::unordered_map<std::type_index, NameToIDMap> name_to_id_maps;
std::unordered_map<UUID, Entry> id_to_entry_map;
boost::container::flat_set<std::type_index> types_of_lists_to_write;
bool failed_to_write_lists = false; /// Whether writing of the list files has been failed since the recent restart of the server.
ThreadFromGlobalPool lists_writing_thread; /// List files are written in a separate thread.
std::condition_variable lists_writing_thread_should_exit; /// Signals `lists_writing_thread` to exit.
std::atomic<bool> lists_writing_thread_exited = false;
mutable std::unordered_multimap<std::type_index, OnChangedHandler> handlers_by_type;
mutable std::mutex mutex;
};
}
......@@ -4,6 +4,8 @@
#include <Access/Role.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/formatAST.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <boost/range/algorithm/set_algorithm.hpp>
#include <boost/range/algorithm/sort.hpp>
#include <boost/range/algorithm_ext/push_back.hpp>
......@@ -45,26 +47,51 @@ GenericRoleSet::GenericRoleSet(const boost::container::flat_set<UUID> & ids_)
}
GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager, const std::optional<UUID> & current_user_id)
GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast)
{
init(ast, nullptr, nullptr);
}
GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const UUID & current_user_id)
{
init(ast, nullptr, &current_user_id);
}
GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager)
{
init(ast, &manager, nullptr);
}
GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager, const UUID & current_user_id)
{
init(ast, &manager, &current_user_id);
}
void GenericRoleSet::init(const ASTGenericRoleSet & ast, const AccessControlManager * manager, const UUID * current_user_id)
{
all = ast.all;
auto name_to_id = [id_mode{ast.id_mode}, manager](const String & name) -> UUID
{
if (id_mode)
return parse<UUID>(name);
assert(manager);
auto id = manager->find<User>(name);
if (id)
return *id;
return manager->getID<Role>(name);
};
if (!ast.names.empty() && !all)
{
ids.reserve(ast.names.size());
for (const String & name : ast.names)
{
auto id = manager.find<User>(name);
if (!id)
id = manager.getID<Role>(name);
ids.insert(*id);
}
ids.insert(name_to_id(name));
}
if (ast.current_user && !all)
{
if (!current_user_id)
throw Exception("Current user is unknown", ErrorCodes::LOGICAL_ERROR);
assert(current_user_id);
ids.insert(*current_user_id);
}
......@@ -72,18 +99,12 @@ GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const AccessContro
{
except_ids.reserve(ast.except_names.size());
for (const String & except_name : ast.except_names)
{
auto except_id = manager.find<User>(except_name);
if (!except_id)
except_id = manager.getID<Role>(except_name);
except_ids.insert(*except_id);
}
except_ids.insert(name_to_id(except_name));
}
if (ast.except_current_user)
{
if (!current_user_id)
throw Exception("Current user is unknown", ErrorCodes::LOGICAL_ERROR);
assert(current_user_id);
except_ids.insert(*current_user_id);
}
......@@ -91,7 +112,52 @@ GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const AccessContro
ids.erase(except_id);
}
std::shared_ptr<ASTGenericRoleSet> GenericRoleSet::toAST(const AccessControlManager & manager) const
std::shared_ptr<ASTGenericRoleSet> GenericRoleSet::toAST() const
{
auto ast = std::make_shared<ASTGenericRoleSet>();
ast->id_mode = true;
ast->all = all;
if (!ids.empty())
{
ast->names.reserve(ids.size());
for (const UUID & id : ids)
ast->names.emplace_back(::DB::toString(id));
}
if (!except_ids.empty())
{
ast->except_names.reserve(except_ids.size());
for (const UUID & except_id : except_ids)
ast->except_names.emplace_back(::DB::toString(except_id));
}
return ast;
}
String GenericRoleSet::toString() const
{
auto ast = toAST();
return serializeAST(*ast);
}
Strings GenericRoleSet::toStrings() const
{
if (all || !except_ids.empty())
return {toString()};
Strings names;
names.reserve(ids.size());
for (const UUID & id : ids)
names.emplace_back(::DB::toString(id));
return names;
}
std::shared_ptr<ASTGenericRoleSet> GenericRoleSet::toASTWithNames(const AccessControlManager & manager) const
{
auto ast = std::make_shared<ASTGenericRoleSet>();
ast->all = all;
......@@ -124,17 +190,17 @@ std::shared_ptr<ASTGenericRoleSet> GenericRoleSet::toAST(const AccessControlMana
}
String GenericRoleSet::toString(const AccessControlManager & manager) const
String GenericRoleSet::toStringWithNames(const AccessControlManager & manager) const
{
auto ast = toAST(manager);
auto ast = toASTWithNames(manager);
return serializeAST(*ast);
}
Strings GenericRoleSet::toStrings(const AccessControlManager & manager) const
Strings GenericRoleSet::toStringsWithNames(const AccessControlManager & manager) const
{
if (all || !except_ids.empty())
return {toString(manager)};
return {toStringWithNames(manager)};
Strings names;
names.reserve(ids.size());
......
......@@ -30,11 +30,19 @@ struct GenericRoleSet
GenericRoleSet(const std::vector<UUID> & ids_);
GenericRoleSet(const boost::container::flat_set<UUID> & ids_);
GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager, const std::optional<UUID> & current_user_id = {});
std::shared_ptr<ASTGenericRoleSet> toAST(const AccessControlManager & manager) const;
/// The constructor from AST requires the AccessControlManager if `ast.id_mode == false`.
GenericRoleSet(const ASTGenericRoleSet & ast);
GenericRoleSet(const ASTGenericRoleSet & ast, const UUID & current_user_id);
GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager);
GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager, const UUID & current_user_id);
String toString(const AccessControlManager & manager) const;
Strings toStrings(const AccessControlManager & manager) const;
std::shared_ptr<ASTGenericRoleSet> toAST() const;
String toString() const;
Strings toStrings() const;
std::shared_ptr<ASTGenericRoleSet> toASTWithNames(const AccessControlManager & manager) const;
String toStringWithNames(const AccessControlManager & manager) const;
Strings toStringsWithNames(const AccessControlManager & manager) const;
bool empty() const;
void clear();
......@@ -61,6 +69,9 @@ struct GenericRoleSet
boost::container::flat_set<UUID> ids;
bool all = false;
boost::container::flat_set<UUID> except_ids;
private:
void init(const ASTGenericRoleSet & ast, const AccessControlManager * manager = nullptr, const UUID * current_user_id = nullptr);
};
}
......@@ -15,7 +15,7 @@ namespace ErrorCodes
extern const int BAD_CAST;
extern const int ACCESS_ENTITY_NOT_FOUND;
extern const int ACCESS_ENTITY_ALREADY_EXISTS;
extern const int ACCESS_ENTITY_STORAGE_READONLY;
extern const int ACCESS_STORAGE_READONLY;
extern const int UNKNOWN_USER;
extern const int UNKNOWN_ROLE;
}
......@@ -419,7 +419,7 @@ void IAccessStorage::throwReadonlyCannotInsert(std::type_index type, const Strin
{
throw Exception(
"Cannot insert " + getTypeName(type) + " " + backQuote(name) + " to " + getStorageName() + " because this storage is readonly",
ErrorCodes::ACCESS_ENTITY_STORAGE_READONLY);
ErrorCodes::ACCESS_STORAGE_READONLY);
}
......@@ -427,7 +427,7 @@ void IAccessStorage::throwReadonlyCannotUpdate(std::type_index type, const Strin
{
throw Exception(
"Cannot update " + getTypeName(type) + " " + backQuote(name) + " in " + getStorageName() + " because this storage is readonly",
ErrorCodes::ACCESS_ENTITY_STORAGE_READONLY);
ErrorCodes::ACCESS_STORAGE_READONLY);
}
......@@ -435,6 +435,6 @@ void IAccessStorage::throwReadonlyCannotRemove(std::type_index type, const Strin
{
throw Exception(
"Cannot remove " + getTypeName(type) + " " + backQuote(name) + " from " + getStorageName() + " because this storage is readonly",
ErrorCodes::ACCESS_ENTITY_STORAGE_READONLY);
ErrorCodes::ACCESS_STORAGE_READONLY);
}
}
......@@ -74,6 +74,10 @@ public:
String readName(const UUID & id) const;
std::optional<String> tryReadName(const UUID & id) const;
/// Returns true if a specified entity can be inserted into this storage.
/// This function doesn't check whether there are no entities with such name in the storage.
bool canInsert(const AccessEntityPtr & entity) const { return canInsertImpl(entity); }
/// Inserts an entity to the storage. Returns ID of a new entry in the storage.
/// Throws an exception if the specified name already exists.
UUID insert(const AccessEntityPtr & entity);
......@@ -133,6 +137,7 @@ protected:
virtual bool existsImpl(const UUID & id) const = 0;
virtual AccessEntityPtr readImpl(const UUID & id) const = 0;
virtual String readNameImpl(const UUID & id) const = 0;
virtual bool canInsertImpl(const AccessEntityPtr & entity) const = 0;
virtual UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) = 0;
virtual void removeImpl(const UUID & id) = 0;
virtual void updateImpl(const UUID & id, const UpdateFunc & update_func) = 0;
......
......@@ -293,6 +293,7 @@ ext::scope_guard MemoryAccessStorage::subscribeForChangesImpl(const UUID & id, c
bool MemoryAccessStorage::hasSubscriptionImpl(const UUID & id) const
{
std::lock_guard lock{mutex};
auto it = entries.find(id);
if (it != entries.end())
{
......@@ -305,6 +306,7 @@ bool MemoryAccessStorage::hasSubscriptionImpl(const UUID & id) const
bool MemoryAccessStorage::hasSubscriptionImpl(std::type_index type) const
{
std::lock_guard lock{mutex};
auto range = handlers_by_type.equal_range(type);
return range.first != range.second;
}
......
......@@ -26,6 +26,7 @@ private:
bool existsImpl(const UUID & id) const override;
AccessEntityPtr readImpl(const UUID & id) const override;
String readNameImpl(const UUID & id) const override;
bool canInsertImpl(const AccessEntityPtr &) const override { return true; }
UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) override;
void removeImpl(const UUID & id) override;
void updateImpl(const UUID & id, const UpdateFunc & update_func) override;
......
......@@ -8,6 +8,7 @@ namespace DB
namespace ErrorCodes
{
extern const int ACCESS_ENTITY_FOUND_DUPLICATES;
extern const int ACCESS_STORAGE_FOR_INSERTION_NOT_FOUND;
}
......@@ -29,10 +30,9 @@ namespace
MultipleAccessStorage::MultipleAccessStorage(
std::vector<std::unique_ptr<Storage>> nested_storages_, size_t index_of_nested_storage_for_insertion_)
std::vector<std::unique_ptr<Storage>> nested_storages_)
: IAccessStorage(joinStorageNames(nested_storages_))
, nested_storages(std::move(nested_storages_))
, nested_storage_for_insertion(nested_storages[index_of_nested_storage_for_insertion_].get())
, ids_cache(512 /* cache size */)
{
}
......@@ -161,13 +161,39 @@ String MultipleAccessStorage::readNameImpl(const UUID & id) const
}
bool MultipleAccessStorage::canInsertImpl(const AccessEntityPtr & entity) const
{
for (const auto & nested_storage : nested_storages)
{
if (nested_storage->canInsert(entity))
return true;
}
return false;
}
UUID MultipleAccessStorage::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists)
{
auto id = replace_if_exists ? nested_storage_for_insertion->insertOrReplace(entity) : nested_storage_for_insertion->insert(entity);
IAccessStorage * nested_storage_for_insertion = nullptr;
for (const auto & nested_storage : nested_storages)
{
if (nested_storage->canInsert(entity))
{
nested_storage_for_insertion = nested_storage.get();
break;
}
}
if (!nested_storage_for_insertion)
{
throw Exception(
"Not found a storage to insert " + entity->getTypeName() + backQuote(entity->getName()),
ErrorCodes::ACCESS_STORAGE_FOR_INSERTION_NOT_FOUND);
}
auto id = replace_if_exists ? nested_storage_for_insertion->insertOrReplace(entity) : nested_storage_for_insertion->insert(entity);
std::lock_guard lock{ids_cache_mutex};
ids_cache.set(id, std::make_shared<Storage *>(nested_storage_for_insertion));
return id;
}
......
......@@ -13,7 +13,7 @@ class MultipleAccessStorage : public IAccessStorage
public:
using Storage = IAccessStorage;
MultipleAccessStorage(std::vector<std::unique_ptr<Storage>> nested_storages_, size_t index_of_nested_storage_for_insertion_ = 0);
MultipleAccessStorage(std::vector<std::unique_ptr<Storage>> nested_storages_);
~MultipleAccessStorage() override;
std::vector<UUID> findMultiple(std::type_index type, const String & name) const;
......@@ -35,6 +35,7 @@ protected:
bool existsImpl(const UUID & id) const override;
AccessEntityPtr readImpl(const UUID & id) const override;
String readNameImpl(const UUID &id) const override;
bool canInsertImpl(const AccessEntityPtr & entity) const override;
UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) override;
void removeImpl(const UUID & id) override;
void updateImpl(const UUID & id, const UpdateFunc & update_func) override;
......@@ -45,7 +46,6 @@ protected:
private:
std::vector<std::unique_ptr<Storage>> nested_storages;
IAccessStorage * nested_storage_for_insertion;
mutable LRUCache<UUID, Storage *> ids_cache;
mutable std::mutex ids_cache_mutex;
};
......
......@@ -342,7 +342,7 @@ UsersConfigAccessStorage::UsersConfigAccessStorage() : IAccessStorage("users.xml
UsersConfigAccessStorage::~UsersConfigAccessStorage() {}
void UsersConfigAccessStorage::loadFromConfig(const Poco::Util::AbstractConfiguration & config)
void UsersConfigAccessStorage::setConfiguration(const Poco::Util::AbstractConfiguration & config)
{
std::vector<std::pair<UUID, AccessEntityPtr>> all_entities;
for (const auto & entity : parseUsers(config, getLogger()))
......
......@@ -21,7 +21,7 @@ public:
UsersConfigAccessStorage();
~UsersConfigAccessStorage() override;
void loadFromConfig(const Poco::Util::AbstractConfiguration & config);
void setConfiguration(const Poco::Util::AbstractConfiguration & config);
private:
std::optional<UUID> findImpl(std::type_index type, const String & name) const override;
......@@ -29,6 +29,7 @@ private:
bool existsImpl(const UUID & id) const override;
AccessEntityPtr readImpl(const UUID & id) const override;
String readNameImpl(const UUID & id) const override;
bool canInsertImpl(const AccessEntityPtr &) const override { return false; }
UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) override;
void removeImpl(const UUID & id) override;
void updateImpl(const UUID & id, const UpdateFunc & update_func) override;
......
......@@ -467,7 +467,7 @@ namespace ErrorCodes
extern const int ACCESS_ENTITY_NOT_FOUND = 492;
extern const int ACCESS_ENTITY_ALREADY_EXISTS = 493;
extern const int ACCESS_ENTITY_FOUND_DUPLICATES = 494;
extern const int ACCESS_ENTITY_STORAGE_READONLY = 495;
extern const int ACCESS_STORAGE_READONLY = 495;
extern const int QUOTA_REQUIRES_CLIENT_KEY = 496;
extern const int ACCESS_DENIED = 497;
extern const int LIMIT_BY_WITH_TIES_IS_NOT_SUPPORTED = 498;
......@@ -485,6 +485,8 @@ namespace ErrorCodes
extern const int UNKNOWN_ROLE = 511;
extern const int SET_NON_GRANTED_ROLE = 512;
extern const int UNKNOWN_PART_TYPE = 513;
extern const int ACCESS_STORAGE_FOR_INSERTION_NOT_FOUND = 514;
extern const int INCORRECT_ACCESS_ENTITY_DEFINITION = 515;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;
......
......@@ -626,7 +626,7 @@ void Context::setUsersConfig(const ConfigurationPtr & config)
{
auto lock = getLock();
shared->users_config = config;
shared->access_control_manager.loadFromConfig(*shared->users_config);
shared->access_control_manager.setUsersConfig(*shared->users_config);
}
ConfigurationPtr Context::getUsersConfig()
......
......@@ -12,6 +12,68 @@
namespace DB
{
namespace
{
void updateQuotaFromQueryImpl(Quota & quota, const ASTCreateQuotaQuery & query, const std::optional<GenericRoleSet> & roles_from_query = {})
{
if (query.alter)
{
if (!query.new_name.empty())
quota.setName(query.new_name);
}
else
quota.setName(query.name);
if (query.key_type)
quota.key_type = *query.key_type;
auto & quota_all_limits = quota.all_limits;
for (const auto & query_limits : query.all_limits)
{
auto duration = query_limits.duration;
auto it = boost::range::find_if(quota_all_limits, [&](const Quota::Limits & x) { return x.duration == duration; });
if (query_limits.unset_tracking)
{
if (it != quota_all_limits.end())
quota_all_limits.erase(it);
continue;
}
if (it == quota_all_limits.end())
{
/// We keep `all_limits` sorted by duration.
it = quota_all_limits.insert(
boost::range::upper_bound(
quota_all_limits,
duration,
[](const std::chrono::seconds & lhs, const Quota::Limits & rhs) { return lhs < rhs.duration; }),
Quota::Limits{});
it->duration = duration;
}
auto & quota_limits = *it;
quota_limits.randomize_interval = query_limits.randomize_interval;
for (auto resource_type : ext::range(Quota::MAX_RESOURCE_TYPE))
{
if (query_limits.max[resource_type])
quota_limits.max[resource_type] = *query_limits.max[resource_type];
}
}
const GenericRoleSet * roles = nullptr;
std::optional<GenericRoleSet> temp_role_set;
if (roles_from_query)
roles = &*roles_from_query;
else if (query.roles)
roles = &temp_role_set.emplace(*query.roles);
if (roles)
quota.roles = *roles;
}
}
BlockIO InterpreterCreateQuotaQuery::execute()
{
const auto & query = query_ptr->as<const ASTCreateQuotaQuery &>();
......@@ -27,7 +89,7 @@ BlockIO InterpreterCreateQuotaQuery::execute()
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
{
auto updated_quota = typeid_cast<std::shared_ptr<Quota>>(entity->clone());
updateQuotaFromQuery(*updated_quota, query, roles_from_query);
updateQuotaFromQueryImpl(*updated_quota, query, roles_from_query);
return updated_quota;
};
if (query.if_exists)
......@@ -41,7 +103,7 @@ BlockIO InterpreterCreateQuotaQuery::execute()
else
{
auto new_quota = std::make_shared<Quota>();
updateQuotaFromQuery(*new_quota, query, roles_from_query);
updateQuotaFromQueryImpl(*new_quota, query, roles_from_query);
if (query.if_not_exists)
access_control.tryInsert(new_quota);
......@@ -55,54 +117,9 @@ BlockIO InterpreterCreateQuotaQuery::execute()
}
void InterpreterCreateQuotaQuery::updateQuotaFromQuery(Quota & quota, const ASTCreateQuotaQuery & query, const std::optional<GenericRoleSet> & roles_from_query)
void InterpreterCreateQuotaQuery::updateQuotaFromQuery(Quota & quota, const ASTCreateQuotaQuery & query)
{
if (query.alter)
{
if (!query.new_name.empty())
quota.setName(query.new_name);
}
else
quota.setName(query.name);
if (query.key_type)
quota.key_type = *query.key_type;
auto & quota_all_limits = quota.all_limits;
for (const auto & query_limits : query.all_limits)
{
auto duration = query_limits.duration;
auto it = boost::range::find_if(quota_all_limits, [&](const Quota::Limits & x) { return x.duration == duration; });
if (query_limits.unset_tracking)
{
if (it != quota_all_limits.end())
quota_all_limits.erase(it);
continue;
}
if (it == quota_all_limits.end())
{
/// We keep `all_limits` sorted by duration.
it = quota_all_limits.insert(
boost::range::upper_bound(
quota_all_limits,
duration,
[](const std::chrono::seconds & lhs, const Quota::Limits & rhs) { return lhs < rhs.duration; }),
Quota::Limits{});
it->duration = duration;
}
auto & quota_limits = *it;
quota_limits.randomize_interval = query_limits.randomize_interval;
for (auto resource_type : ext::range(Quota::MAX_RESOURCE_TYPE))
{
if (query_limits.max[resource_type])
quota_limits.max[resource_type] = *query_limits.max[resource_type];
}
}
if (roles_from_query)
quota.roles = *roles_from_query;
updateQuotaFromQueryImpl(quota, query);
}
}
......@@ -9,7 +9,6 @@ namespace DB
{
class ASTCreateQuotaQuery;
struct Quota;
struct GenericRoleSet;
class InterpreterCreateQuotaQuery : public IInterpreter
......@@ -22,9 +21,9 @@ public:
bool ignoreQuota() const override { return true; }
bool ignoreLimits() const override { return true; }
private:
void updateQuotaFromQuery(Quota & quota, const ASTCreateQuotaQuery & query, const std::optional<GenericRoleSet> & roles_from_query);
static void updateQuotaFromQuery(Quota & quota, const ASTCreateQuotaQuery & query);
private:
ASTPtr query_ptr;
Context & context;
};
......
......@@ -17,9 +17,9 @@ public:
BlockIO execute() override;
private:
void updateRoleFromQuery(Role & role, const ASTCreateRoleQuery & query);
static void updateRoleFromQuery(Role & role, const ASTCreateRoleQuery & query);
private:
ASTPtr query_ptr;
Context & context;
};
......
......@@ -10,6 +10,57 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace
{
const String & checkCurrentDatabase(const String & current_database)
{
if (current_database.empty())
throw Exception("No current database", ErrorCodes::LOGICAL_ERROR);
return current_database;
}
void updateRowPolicyFromQueryImpl(
RowPolicy & policy,
const ASTCreateRowPolicyQuery & query,
const std::optional<GenericRoleSet> & roles_from_query = {},
const String & current_database = {})
{
if (query.alter)
{
if (!query.new_policy_name.empty())
policy.setName(query.new_policy_name);
}
else
{
policy.setDatabase(!query.name_parts.database.empty() ? query.name_parts.database : checkCurrentDatabase(current_database));
policy.setTableName(query.name_parts.table_name);
policy.setName(query.name_parts.policy_name);
}
if (query.is_restrictive)
policy.setRestrictive(*query.is_restrictive);
for (const auto & [index, condition] : query.conditions)
policy.conditions[index] = condition ? serializeAST(*condition) : String{};
const GenericRoleSet * roles = nullptr;
std::optional<GenericRoleSet> temp_role_set;
if (roles_from_query)
roles = &*roles_from_query;
else if (query.roles)
roles = &temp_role_set.emplace(*query.roles);
if (roles)
policy.roles = *roles;
}
}
BlockIO InterpreterCreateRowPolicyQuery::execute()
{
const auto & query = query_ptr->as<const ASTCreateRowPolicyQuery &>();
......@@ -20,12 +71,14 @@ BlockIO InterpreterCreateRowPolicyQuery::execute()
if (query.roles)
roles_from_query = GenericRoleSet{*query.roles, access_control, context.getUserID()};
const String current_database = context.getCurrentDatabase();
if (query.alter)
{
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
{
auto updated_policy = typeid_cast<std::shared_ptr<RowPolicy>>(entity->clone());
updateRowPolicyFromQuery(*updated_policy, query, roles_from_query);
updateRowPolicyFromQueryImpl(*updated_policy, query, roles_from_query, current_database);
return updated_policy;
};
String full_name = query.name_parts.getFullName(context);
......@@ -40,7 +93,7 @@ BlockIO InterpreterCreateRowPolicyQuery::execute()
else
{
auto new_policy = std::make_shared<RowPolicy>();
updateRowPolicyFromQuery(*new_policy, query, roles_from_query);
updateRowPolicyFromQueryImpl(*new_policy, query, roles_from_query, current_database);
if (query.if_not_exists)
access_control.tryInsert(new_policy);
......@@ -54,27 +107,9 @@ BlockIO InterpreterCreateRowPolicyQuery::execute()
}
void InterpreterCreateRowPolicyQuery::updateRowPolicyFromQuery(RowPolicy & policy, const ASTCreateRowPolicyQuery & query, const std::optional<GenericRoleSet> & roles_from_query)
void InterpreterCreateRowPolicyQuery::updateRowPolicyFromQuery(RowPolicy & policy, const ASTCreateRowPolicyQuery & query)
{
if (query.alter)
{
if (!query.new_policy_name.empty())
policy.setName(query.new_policy_name);
}
else
{
policy.setDatabase(query.name_parts.database.empty() ? context.getCurrentDatabase() : query.name_parts.database);
policy.setTableName(query.name_parts.table_name);
policy.setName(query.name_parts.policy_name);
}
if (query.is_restrictive)
policy.setRestrictive(*query.is_restrictive);
for (const auto & [index, condition] : query.conditions)
policy.conditions[index] = condition ? serializeAST(*condition) : String{};
if (roles_from_query)
policy.roles = *roles_from_query;
updateRowPolicyFromQueryImpl(policy, query);
}
}
......@@ -9,7 +9,6 @@ namespace DB
{
class ASTCreateRowPolicyQuery;
struct RowPolicy;
struct GenericRoleSet;
class InterpreterCreateRowPolicyQuery : public IInterpreter
......@@ -19,9 +18,9 @@ public:
BlockIO execute() override;
private:
void updateRowPolicyFromQuery(RowPolicy & policy, const ASTCreateRowPolicyQuery & query, const std::optional<GenericRoleSet> & roles_from_query);
static void updateRowPolicyFromQuery(RowPolicy & policy, const ASTCreateRowPolicyQuery & query);
private:
ASTPtr query_ptr;
Context & context;
};
......
......@@ -11,8 +11,46 @@
namespace DB
{
namespace ErrorCodes
namespace
{
void updateUserFromQueryImpl(User & user, const ASTCreateUserQuery & query, const std::optional<GenericRoleSet> & default_roles_from_query = {})
{
if (query.alter)
{
if (!query.new_name.empty())
user.setName(query.new_name);
}
else
user.setName(query.name);
if (query.authentication)
user.authentication = *query.authentication;
if (query.hosts)
user.allowed_client_hosts = *query.hosts;
if (query.remove_hosts)
user.allowed_client_hosts.remove(*query.remove_hosts);
if (query.add_hosts)
user.allowed_client_hosts.add(*query.add_hosts);
const GenericRoleSet * default_roles = nullptr;
std::optional<GenericRoleSet> temp_role_set;
if (default_roles_from_query)
default_roles = &*default_roles_from_query;
else if (query.default_roles)
default_roles = &temp_role_set.emplace(*query.default_roles);
if (default_roles)
{
if (!query.alter && !default_roles->all)
boost::range::copy(default_roles->getMatchingIDs(), std::inserter(user.granted_roles, user.granted_roles.end()));
InterpreterSetRoleQuery::updateUserSetDefaultRoles(user, *default_roles);
}
if (query.profile)
user.profile = *query.profile;
}
}
......@@ -22,12 +60,10 @@ BlockIO InterpreterCreateUserQuery::execute()
auto & access_control = context.getAccessControlManager();
context.checkAccess(query.alter ? AccessType::ALTER_USER : AccessType::CREATE_USER);
GenericRoleSet * default_roles_from_query = nullptr;
GenericRoleSet temp_role_set;
std::optional<GenericRoleSet> default_roles_from_query;
if (query.default_roles)
{
default_roles_from_query = &temp_role_set;
*default_roles_from_query = GenericRoleSet{*query.default_roles, access_control};
default_roles_from_query = GenericRoleSet{*query.default_roles, access_control};
if (!query.alter && !default_roles_from_query->all)
{
for (const UUID & role : default_roles_from_query->getMatchingIDs())
......@@ -40,7 +76,7 @@ BlockIO InterpreterCreateUserQuery::execute()
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
{
auto updated_user = typeid_cast<std::shared_ptr<User>>(entity->clone());
updateUserFromQuery(*updated_user, query, default_roles_from_query);
updateUserFromQueryImpl(*updated_user, query, default_roles_from_query);
return updated_user;
};
if (query.if_exists)
......@@ -54,7 +90,7 @@ BlockIO InterpreterCreateUserQuery::execute()
else
{
auto new_user = std::make_shared<User>();
updateUserFromQuery(*new_user, query, default_roles_from_query);
updateUserFromQueryImpl(*new_user, query, default_roles_from_query);
if (query.if_not_exists)
access_control.tryInsert(new_user);
......@@ -68,36 +104,9 @@ BlockIO InterpreterCreateUserQuery::execute()
}
void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreateUserQuery & query, const GenericRoleSet * default_roles_from_query)
void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreateUserQuery & query)
{
if (query.alter)
{
if (!query.new_name.empty())
user.setName(query.new_name);
}
else
user.setName(query.name);
if (query.authentication)
user.authentication = *query.authentication;
if (query.hosts)
user.allowed_client_hosts = *query.hosts;
if (query.remove_hosts)
user.allowed_client_hosts.remove(*query.remove_hosts);
if (query.add_hosts)
user.allowed_client_hosts.add(*query.add_hosts);
if (default_roles_from_query)
{
if (!query.alter && !default_roles_from_query->all)
boost::range::copy(default_roles_from_query->getMatchingIDs(), std::inserter(user.granted_roles, user.granted_roles.end()));
InterpreterSetRoleQuery::updateUserSetDefaultRoles(user, *default_roles_from_query);
}
if (query.profile)
user.profile = *query.profile;
updateUserFromQueryImpl(user, query);
}
}
......@@ -7,7 +7,6 @@
namespace DB
{
class ASTCreateUserQuery;
struct GenericRoleSet;
struct User;
......@@ -18,9 +17,9 @@ public:
BlockIO execute() override;
private:
void updateUserFromQuery(User & user, const ASTCreateUserQuery & query, const GenericRoleSet * default_roles_from_query);
static void updateUserFromQuery(User & user, const ASTCreateUserQuery & query);
private:
ASTPtr query_ptr;
Context & context;
};
......
......@@ -11,96 +11,91 @@
namespace DB
{
BlockIO InterpreterGrantQuery::execute()
namespace
{
const auto & query = query_ptr->as<const ASTGrantQuery &>();
auto & access_control = context.getAccessControlManager();
context.getAccessRights()->checkGrantOption(query.access_rights_elements);
using Kind = ASTGrantQuery::Kind;
std::vector<UUID> roles;
if (query.roles)
{
roles = GenericRoleSet{*query.roles, access_control}.getMatchingRoles(access_control);
for (const UUID & role : roles)
context.getAccessRights()->checkAdminOption(role);
}
std::vector<UUID> to_roles = GenericRoleSet{*query.to_roles, access_control, context.getUserID()}.getMatchingUsersAndRoles(access_control);
String current_database = context.getCurrentDatabase();
using Kind = ASTGrantQuery::Kind;
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
template <typename T>
void updateFromQueryImpl(T & grantee, const ASTGrantQuery & query, const std::vector<UUID> & roles_from_query, const String & current_database, bool partial_revokes)
{
auto clone = entity->clone();
AccessRights * access = nullptr;
AccessRights * access_with_grant_option = nullptr;
boost::container::flat_set<UUID> * granted_roles = nullptr;
boost::container::flat_set<UUID> * granted_roles_with_admin_option = nullptr;
GenericRoleSet * default_roles = nullptr;
if (auto user = typeid_cast<std::shared_ptr<User>>(clone))
{
access = &user->access;
access_with_grant_option = &user->access_with_grant_option;
granted_roles = &user->granted_roles;
granted_roles_with_admin_option = &user->granted_roles_with_admin_option;
default_roles = &user->default_roles;
}
else if (auto role = typeid_cast<std::shared_ptr<Role>>(clone))
{
access = &role->access;
access_with_grant_option = &role->access_with_grant_option;
granted_roles = &role->granted_roles;
granted_roles_with_admin_option = &role->granted_roles_with_admin_option;
}
else
return entity;
using Kind = ASTGrantQuery::Kind;
if (!query.access_rights_elements.empty())
{
if (query.kind == Kind::GRANT)
{
access->grant(query.access_rights_elements, current_database);
grantee.access.grant(query.access_rights_elements, current_database);
if (query.grant_option)
access_with_grant_option->grant(query.access_rights_elements, current_database);
grantee.access_with_grant_option.grant(query.access_rights_elements, current_database);
}
else if (context.getSettingsRef().partial_revokes)
else if (partial_revokes)
{
access_with_grant_option->partialRevoke(query.access_rights_elements, current_database);
grantee.access_with_grant_option.partialRevoke(query.access_rights_elements, current_database);
if (!query.grant_option)
access->partialRevoke(query.access_rights_elements, current_database);
grantee.access.partialRevoke(query.access_rights_elements, current_database);
}
else
{
access_with_grant_option->revoke(query.access_rights_elements, current_database);
grantee.access_with_grant_option.revoke(query.access_rights_elements, current_database);
if (!query.grant_option)
access->revoke(query.access_rights_elements, current_database);
grantee.access.revoke(query.access_rights_elements, current_database);
}
}
if (!roles.empty())
if (!roles_from_query.empty())
{
if (query.kind == Kind::GRANT)
{
boost::range::copy(roles, std::inserter(*granted_roles, granted_roles->end()));
boost::range::copy(roles_from_query, std::inserter(grantee.granted_roles, grantee.granted_roles.end()));
if (query.admin_option)
boost::range::copy(roles, std::inserter(*granted_roles_with_admin_option, granted_roles_with_admin_option->end()));
boost::range::copy(roles_from_query, std::inserter(grantee.granted_roles_with_admin_option, grantee.granted_roles_with_admin_option.end()));
}
else
{
for (const UUID & role : roles)
for (const UUID & role_from_query : roles_from_query)
{
granted_roles_with_admin_option->erase(role);
grantee.granted_roles_with_admin_option.erase(role_from_query);
if (!query.admin_option)
{
granted_roles->erase(role);
if (default_roles)
default_roles->ids.erase(role);
}
grantee.granted_roles.erase(role_from_query);
if constexpr (std::is_same_v<T, User>)
grantee.default_roles.ids.erase(role_from_query);
}
}
}
return clone;
}
}
BlockIO InterpreterGrantQuery::execute()
{
const auto & query = query_ptr->as<const ASTGrantQuery &>();
auto & access_control = context.getAccessControlManager();
context.getAccessRights()->checkGrantOption(query.access_rights_elements);
std::vector<UUID> roles_from_query;
if (query.roles)
{
roles_from_query = GenericRoleSet{*query.roles, access_control}.getMatchingRoles(access_control);
for (const UUID & role_from_query : roles_from_query)
context.getAccessRights()->checkAdminOption(role_from_query);
}
std::vector<UUID> to_roles = GenericRoleSet{*query.to_roles, access_control, context.getUserID()}.getMatchingUsersAndRoles(access_control);
String current_database = context.getCurrentDatabase();
bool partial_revokes = context.getSettingsRef().partial_revokes;
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
{
auto clone = entity->clone();
if (auto user = typeid_cast<std::shared_ptr<User>>(clone))
{
updateFromQueryImpl(*user, query, roles_from_query, current_database, partial_revokes);
return user;
}
else if (auto role = typeid_cast<std::shared_ptr<Role>>(clone))
{
updateFromQueryImpl(*role, query, roles_from_query, current_database, partial_revokes);
return role;
}
else
return entity;
};
access_control.update(to_roles, update_func);
......@@ -108,4 +103,22 @@ BlockIO InterpreterGrantQuery::execute()
return {};
}
void InterpreterGrantQuery::updateUserFromQuery(User & user, const ASTGrantQuery & query)
{
std::vector<UUID> roles_from_query;
if (query.roles)
roles_from_query = GenericRoleSet{*query.roles}.getMatchingIDs();
updateFromQueryImpl(user, query, roles_from_query, {}, true);
}
void InterpreterGrantQuery::updateRoleFromQuery(Role & role, const ASTGrantQuery & query)
{
std::vector<UUID> roles_from_query;
if (query.roles)
roles_from_query = GenericRoleSet{*query.roles}.getMatchingIDs();
updateFromQueryImpl(role, query, roles_from_query, {}, true);
}
}
......@@ -2,10 +2,16 @@
#include <Interpreters/IInterpreter.h>
#include <Parsers/IAST_fwd.h>
#include <Core/UUID.h>
namespace DB
{
class ASTGrantQuery;
struct User;
struct Role;
class InterpreterGrantQuery : public IInterpreter
{
public:
......@@ -13,6 +19,9 @@ public:
BlockIO execute() override;
static void updateUserFromQuery(User & user, const ASTGrantQuery & query);
static void updateRoleFromQuery(Role & role, const ASTGrantQuery & query);
private:
ASTPtr query_ptr;
Context & context;
......
#include <Interpreters/InterpreterShowCreateAccessEntityQuery.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTCreateUserQuery.h>
#include <Parsers/ASTCreateRoleQuery.h>
#include <Parsers/ASTCreateQuotaQuery.h>
#include <Parsers/ASTCreateRowPolicyQuery.h>
#include <Parsers/ASTShowCreateAccessEntityQuery.h>
......@@ -11,6 +12,7 @@
#include <Access/AccessControlManager.h>
#include <Access/QuotaContext.h>
#include <Access/User.h>
#include <Access/Role.h>
#include <Columns/ColumnString.h>
#include <DataStreams/OneBlockInputStream.h>
#include <DataTypes/DataTypeString.h>
......@@ -21,6 +23,141 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace
{
ASTPtr getCreateQueryImpl(
const User & user,
const AccessControlManager * manager /* not used if attach_mode == true */,
bool attach_mode = false)
{
auto query = std::make_shared<ASTCreateUserQuery>();
query->name = user.getName();
query->attach = attach_mode;
if (user.allowed_client_hosts != AllowedClientHosts::AnyHostTag{})
query->hosts = user.allowed_client_hosts;
if (!user.profile.empty())
query->profile = user.profile;
if (user.default_roles != GenericRoleSet::AllTag{})
{
if (attach_mode)
query->default_roles = GenericRoleSet{user.default_roles}.toAST();
else
query->default_roles = GenericRoleSet{user.default_roles}.toASTWithNames(*manager);
}
if (attach_mode && (user.authentication.getType() != Authentication::NO_PASSWORD))
{
/// We don't show password unless it's an ATTACH statement.
query->authentication = user.authentication;
}
return query;
}
ASTPtr getCreateQueryImpl(const Role & role, const AccessControlManager *, bool attach_mode = false)
{
auto query = std::make_shared<ASTCreateRoleQuery>();
query->name = role.getName();
query->attach = attach_mode;
return query;
}
ASTPtr getCreateQueryImpl(
const Quota & quota,
const AccessControlManager * manager /* not used if attach_mode == true */,
bool attach_mode = false)
{
auto query = std::make_shared<ASTCreateQuotaQuery>();
query->name = quota.getName();
query->attach = attach_mode;
query->key_type = quota.key_type;
query->all_limits.reserve(quota.all_limits.size());
for (const auto & limits : quota.all_limits)
{
ASTCreateQuotaQuery::Limits create_query_limits;
create_query_limits.duration = limits.duration;
create_query_limits.randomize_interval = limits.randomize_interval;
for (auto resource_type : ext::range(Quota::MAX_RESOURCE_TYPE))
if (limits.max[resource_type])
create_query_limits.max[resource_type] = limits.max[resource_type];
query->all_limits.push_back(create_query_limits);
}
if (!quota.roles.empty())
{
if (attach_mode)
query->roles = quota.roles.toAST();
else
query->roles = quota.roles.toASTWithNames(*manager);
}
return query;
}
ASTPtr getCreateQueryImpl(
const RowPolicy & policy,
const AccessControlManager * manager /* not used if attach_mode == true */,
bool attach_mode = false)
{
auto query = std::make_shared<ASTCreateRowPolicyQuery>();
query->name_parts = RowPolicy::FullNameParts{policy.getDatabase(), policy.getTableName(), policy.getName()};
query->attach = attach_mode;
if (policy.isRestrictive())
query->is_restrictive = policy.isRestrictive();
for (auto index : ext::range_with_static_cast<RowPolicy::ConditionIndex>(RowPolicy::MAX_CONDITION_INDEX))
{
const auto & condition = policy.conditions[index];
if (!condition.empty())
{
ParserExpression parser;
ASTPtr expr = parseQuery(parser, condition, 0);
query->conditions.push_back(std::pair{index, expr});
}
}
if (!policy.roles.empty())
{
if (attach_mode)
query->roles = policy.roles.toAST();
else
query->roles = policy.roles.toASTWithNames(*manager);
}
return query;
}
ASTPtr getCreateQueryImpl(
const IAccessEntity & entity,
const AccessControlManager * manager /* not used if attach_mode == true */,
bool attach_mode = false)
{
if (const User * user = typeid_cast<const User *>(&entity))
return getCreateQueryImpl(*user, manager, attach_mode);
if (const Role * role = typeid_cast<const Role *>(&entity))
return getCreateQueryImpl(*role, manager, attach_mode);
if (const RowPolicy * policy = typeid_cast<const RowPolicy *>(&entity))
return getCreateQueryImpl(*policy, manager, attach_mode);
if (const Quota * quota = typeid_cast<const Quota *>(&entity))
return getCreateQueryImpl(*quota, manager, attach_mode);
throw Exception("Unexpected type of access entity: " + entity.getTypeName(), ErrorCodes::LOGICAL_ERROR);
}
}
BlockIO InterpreterShowCreateAccessEntityQuery::execute()
{
BlockIO res;
......@@ -57,98 +194,43 @@ BlockInputStreamPtr InterpreterShowCreateAccessEntityQuery::executeImpl()
ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuery(const ASTShowCreateAccessEntityQuery & show_query) const
{
const auto & access_control = context.getAccessControlManager();
using Kind = ASTShowCreateAccessEntityQuery::Kind;
switch (show_query.kind)
{
case Kind::USER: return getCreateUserQuery(show_query);
case Kind::QUOTA: return getCreateQuotaQuery(show_query);
case Kind::ROW_POLICY: return getCreateRowPolicyQuery(show_query);
}
__builtin_unreachable();
}
ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateUserQuery(const ASTShowCreateAccessEntityQuery & show_query) const
{
UserPtr user;
if (show_query.current_user)
user = context.getUser();
else
user = context.getAccessControlManager().read<User>(show_query.name);
auto create_query = std::make_shared<ASTCreateUserQuery>();
create_query->name = user->getName();
if (user->allowed_client_hosts != AllowedClientHosts::AnyHostTag{})
create_query->hosts = user->allowed_client_hosts;
if (!user->profile.empty())
create_query->profile = user->profile;
if (user->default_roles != GenericRoleSet::AllTag{})
create_query->default_roles = GenericRoleSet{user->default_roles}.toAST(context.getAccessControlManager());
return create_query;
}
ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuotaQuery(const ASTShowCreateAccessEntityQuery & show_query) const
{
auto & access_control = context.getAccessControlManager();
QuotaPtr quota;
if (show_query.current_quota)
quota = access_control.read<Quota>(context.getQuota()->getUsageInfo().quota_id);
else
quota = access_control.read<Quota>(show_query.name);
auto create_query = std::make_shared<ASTCreateQuotaQuery>();
create_query->name = quota->getName();
create_query->key_type = quota->key_type;
create_query->all_limits.reserve(quota->all_limits.size());
for (const auto & limits : quota->all_limits)
{
ASTCreateQuotaQuery::Limits create_query_limits;
create_query_limits.duration = limits.duration;
create_query_limits.randomize_interval = limits.randomize_interval;
for (auto resource_type : ext::range(Quota::MAX_RESOURCE_TYPE))
if (limits.max[resource_type])
create_query_limits.max[resource_type] = limits.max[resource_type];
create_query->all_limits.push_back(create_query_limits);
}
if (!quota->roles.empty())
create_query->roles = quota->roles.toAST(access_control);
return create_query;
}
ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateRowPolicyQuery(const ASTShowCreateAccessEntityQuery & show_query) const
{
auto & access_control = context.getAccessControlManager();
RowPolicyPtr policy = access_control.read<RowPolicy>(show_query.row_policy_name.getFullName(context));
case Kind::USER:
{
UserPtr user;
if (show_query.current_user)
user = context.getUser();
else
user = access_control.read<User>(show_query.name);
return getCreateQueryImpl(*user, &access_control);
}
auto create_query = std::make_shared<ASTCreateRowPolicyQuery>();
create_query->name_parts = RowPolicy::FullNameParts{policy->getDatabase(), policy->getTableName(), policy->getName()};
if (policy->isRestrictive())
create_query->is_restrictive = policy->isRestrictive();
case Kind::QUOTA:
{
QuotaPtr quota;
if (show_query.current_quota)
quota = access_control.read<Quota>(context.getQuota()->getUsageInfo().quota_id);
else
quota = access_control.read<Quota>(show_query.name);
return getCreateQueryImpl(*quota, &access_control);
}
for (auto index : ext::range_with_static_cast<RowPolicy::ConditionIndex>(RowPolicy::MAX_CONDITION_INDEX))
{
const auto & condition = policy->conditions[index];
if (!condition.empty())
case Kind::ROW_POLICY:
{
ParserExpression parser;
ASTPtr expr = parseQuery(parser, condition, 0);
create_query->conditions.push_back(std::pair{index, expr});
RowPolicyPtr policy = access_control.read<RowPolicy>(show_query.row_policy_name.getFullName(context));
return getCreateQueryImpl(*policy, &access_control);
}
}
__builtin_unreachable();
}
if (!policy->roles.empty())
create_query->roles = policy->roles.toAST(access_control);
return create_query;
ASTPtr InterpreterShowCreateAccessEntityQuery::getAttachQuery(const IAccessEntity & entity)
{
return getCreateQueryImpl(entity, nullptr, true);
}
}
......@@ -2,12 +2,14 @@
#include <Interpreters/IInterpreter.h>
#include <Parsers/IAST_fwd.h>
#include <Core/UUID.h>
namespace DB
{
class Context;
class ASTShowCreateAccessEntityQuery;
struct IAccessEntity;
/** Returns a single item containing a statement which could be used to create a specified role.
......@@ -23,15 +25,14 @@ public:
bool ignoreQuota() const override { return true; }
bool ignoreLimits() const override { return true; }
private:
ASTPtr query_ptr;
const Context & context;
static ASTPtr getAttachQuery(const IAccessEntity & entity);
private:
BlockInputStreamPtr executeImpl();
ASTPtr getCreateQuery(const ASTShowCreateAccessEntityQuery & show_query) const;
ASTPtr getCreateUserQuery(const ASTShowCreateAccessEntityQuery & show_query) const;
ASTPtr getCreateQuotaQuery(const ASTShowCreateAccessEntityQuery & show_query) const;
ASTPtr getCreateRowPolicyQuery(const ASTShowCreateAccessEntityQuery & show_query) const;
ASTPtr query_ptr;
const Context & context;
};
......
......@@ -16,6 +16,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace
{
std::vector<AccessRightsElements> groupByTable(AccessRightsElements && elements)
......@@ -47,6 +52,79 @@ namespace
res.partial_revokes = groupByTable(std::move(elements.partial_revokes));
return res;
}
template <typename T>
ASTs getGrantQueriesImpl(
const T & grantee,
const AccessControlManager * manager /* not used if attach_mode == true */,
bool attach_mode = false)
{
ASTs res;
std::shared_ptr<ASTGenericRoleSet> to_roles = std::make_shared<ASTGenericRoleSet>();
to_roles->names.push_back(grantee.getName());
for (bool grant_option : {true, false})
{
if (!grant_option && (grantee.access == grantee.access_with_grant_option))
continue;
const auto & access_rights = grant_option ? grantee.access_with_grant_option : grantee.access;
const auto grouped_elements = groupByTable(access_rights.getElements());
using Kind = ASTGrantQuery::Kind;
for (Kind kind : {Kind::GRANT, Kind::REVOKE})
{
for (const auto & elements : (kind == Kind::GRANT ? grouped_elements.grants : grouped_elements.partial_revokes))
{
auto grant_query = std::make_shared<ASTGrantQuery>();
grant_query->kind = kind;
grant_query->attach = attach_mode;
grant_query->grant_option = grant_option;
grant_query->to_roles = to_roles;
grant_query->access_rights_elements = elements;
res.push_back(std::move(grant_query));
}
}
}
for (bool admin_option : {true, false})
{
if (!admin_option && (grantee.granted_roles == grantee.granted_roles_with_admin_option))
continue;
const auto & roles = admin_option ? grantee.granted_roles_with_admin_option : grantee.granted_roles;
if (roles.empty())
continue;
auto grant_query = std::make_shared<ASTGrantQuery>();
using Kind = ASTGrantQuery::Kind;
grant_query->kind = Kind::GRANT;
grant_query->attach = attach_mode;
grant_query->admin_option = admin_option;
grant_query->to_roles = to_roles;
if (attach_mode)
grant_query->roles = GenericRoleSet{roles}.toAST();
else
grant_query->roles = GenericRoleSet{roles}.toASTWithNames(*manager);
res.push_back(std::move(grant_query));
}
return res;
}
ASTs getGrantQueriesImpl(
const IAccessEntity & entity,
const AccessControlManager * manager /* not used if attach_mode == true */,
bool attach_mode = false)
{
if (const User * user = typeid_cast<const User *>(&entity))
return getGrantQueriesImpl(*user, manager, attach_mode);
if (const Role * role = typeid_cast<const Role *>(&entity))
return getGrantQueriesImpl(*role, manager, attach_mode);
throw Exception("Unexpected type of access entity: " + entity.getTypeName(), ErrorCodes::LOGICAL_ERROR);
}
}
......@@ -90,80 +168,24 @@ BlockInputStreamPtr InterpreterShowGrantsQuery::executeImpl()
ASTs InterpreterShowGrantsQuery::getGrantQueries(const ASTShowGrantsQuery & show_query) const
{
const auto & access_control = context.getAccessControlManager();
UserPtr user;
RolePtr role;
if (show_query.current_user)
user = context.getUser();
else
{
user = access_control.tryRead<User>(show_query.name);
if (!user)
role = access_control.read<Role>(show_query.name);
}
const AccessRights * access = nullptr;
const AccessRights * access_with_grant_option = nullptr;
const boost::container::flat_set<UUID> * granted_roles = nullptr;
const boost::container::flat_set<UUID> * granted_roles_with_admin_option = nullptr;
if (user)
{
access = &user->access;
access_with_grant_option = &user->access_with_grant_option;
granted_roles = &user->granted_roles;
granted_roles_with_admin_option = &user->granted_roles_with_admin_option;
}
AccessEntityPtr user_or_role;
if (show_query.current_user)
user_or_role = context.getUser();
else
{
access = &role->access;
access_with_grant_option = &role->access_with_grant_option;
granted_roles = &role->granted_roles;
granted_roles_with_admin_option = &role->granted_roles_with_admin_option;
user_or_role = access_control.tryRead<User>(show_query.name);
if (!user_or_role)
user_or_role = access_control.read<Role>(show_query.name);
}
ASTs res;
for (bool grant_option : {true, false})
{
if (!grant_option && (*access == *access_with_grant_option))
continue;
const auto & access_rights = grant_option ? *access_with_grant_option : *access;
const auto grouped_elements = groupByTable(access_rights.getElements());
using Kind = ASTGrantQuery::Kind;
for (Kind kind : {Kind::GRANT, Kind::REVOKE})
{
for (const auto & elements : (kind == Kind::GRANT ? grouped_elements.grants : grouped_elements.partial_revokes))
{
auto grant_query = std::make_shared<ASTGrantQuery>();
grant_query->kind = kind;
grant_query->grant_option = grant_option;
grant_query->to_roles = std::make_shared<ASTGenericRoleSet>();
grant_query->to_roles->names.push_back(show_query.name);
grant_query->access_rights_elements = elements;
res.push_back(std::move(grant_query));
}
}
}
return getGrantQueriesImpl(*user_or_role, &access_control);
}
for (bool admin_option : {true, false})
{
if (!admin_option && (*granted_roles == *granted_roles_with_admin_option))
continue;
const auto & roles = admin_option ? *granted_roles_with_admin_option : *granted_roles;
if (roles.empty())
continue;
auto grant_query = std::make_shared<ASTGrantQuery>();
using Kind = ASTGrantQuery::Kind;
grant_query->kind = Kind::GRANT;
grant_query->admin_option = admin_option;
grant_query->to_roles = std::make_shared<ASTGenericRoleSet>();
grant_query->to_roles->names.push_back(show_query.name);
grant_query->roles = GenericRoleSet{roles}.toAST(access_control);
res.push_back(std::move(grant_query));
}
return res;
ASTs InterpreterShowGrantsQuery::getAttachGrantQueries(const IAccessEntity & user_or_role)
{
return getGrantQueriesImpl(user_or_role, nullptr, true);
}
}
......@@ -2,11 +2,13 @@
#include <Interpreters/IInterpreter.h>
#include <Parsers/IAST_fwd.h>
#include <Core/UUID.h>
namespace DB
{
class ASTShowGrantsQuery;
struct IAccessEntity;
class InterpreterShowGrantsQuery : public IInterpreter
......@@ -16,6 +18,8 @@ public:
BlockIO execute() override;
static ASTs getAttachGrantQueries(const IAccessEntity & user_or_role);
private:
BlockInputStreamPtr executeImpl();
ASTs getGrantQueries(const ASTShowGrantsQuery & show_query) const;
......
......@@ -203,7 +203,7 @@ void runOneTest(const TestDescriptor & test_descriptor)
try
{
acl_manager.loadFromConfig(*config);
acl_manager.setUsersConfig(*config);
}
catch (const Poco::Exception & ex)
{
......
......@@ -116,8 +116,15 @@ ASTPtr ASTCreateQuotaQuery::clone() const
void ASTCreateQuotaQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER QUOTA" : "CREATE QUOTA")
<< (settings.hilite ? hilite_none : "");
if (attach)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ATTACH QUOTA" << (settings.hilite ? hilite_none : "");
}
else
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER QUOTA" : "CREATE QUOTA")
<< (settings.hilite ? hilite_none : "");
}
if (if_exists)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF EXISTS" << (settings.hilite ? hilite_none : "");
......
......@@ -29,6 +29,7 @@ class ASTCreateQuotaQuery : public IAST
{
public:
bool alter = false;
bool attach = false;
bool if_exists = false;
bool if_not_exists = false;
......@@ -36,7 +37,6 @@ public:
String name;
String new_name;
using KeyType = Quota::KeyType;
std::optional<KeyType> key_type;
......
......@@ -28,8 +28,15 @@ ASTPtr ASTCreateRoleQuery::clone() const
void ASTCreateRoleQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER ROLE" : "CREATE ROLE")
<< (settings.hilite ? hilite_none : "");
if (attach)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ATTACH ROLE" << (settings.hilite ? hilite_none : "");
}
else
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER ROLE" : "CREATE ROLE")
<< (settings.hilite ? hilite_none : "");
}
if (if_exists)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF EXISTS" << (settings.hilite ? hilite_none : "");
......
......@@ -14,6 +14,7 @@ class ASTCreateRoleQuery : public IAST
{
public:
bool alter = false;
bool attach = false;
bool if_exists = false;
bool if_not_exists = false;
......
......@@ -134,8 +134,15 @@ ASTPtr ASTCreateRowPolicyQuery::clone() const
void ASTCreateRowPolicyQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER POLICY" : "CREATE POLICY")
<< (settings.hilite ? hilite_none : "");
if (attach)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ATTACH POLICY";
}
else
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER POLICY" : "CREATE POLICY")
<< (settings.hilite ? hilite_none : "");
}
if (if_exists)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF EXISTS" << (settings.hilite ? hilite_none : "");
......
......@@ -29,6 +29,7 @@ class ASTCreateRowPolicyQuery : public IAST
{
public:
bool alter = false;
bool attach = false;
bool if_exists = false;
bool if_not_exists = false;
......
......@@ -164,8 +164,15 @@ ASTPtr ASTCreateUserQuery::clone() const
void ASTCreateUserQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER USER" : "CREATE USER")
<< (settings.hilite ? hilite_none : "");
if (attach)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ATTACH USER" << (settings.hilite ? hilite_none : "");
}
else
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER USER" : "CREATE USER")
<< (settings.hilite ? hilite_none : "");
}
if (if_exists)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF EXISTS" << (settings.hilite ? hilite_none : "");
......
......@@ -26,6 +26,7 @@ class ASTCreateUserQuery : public IAST
{
public:
bool alter = false;
bool attach = false;
bool if_exists = false;
bool if_not_exists = false;
......
......@@ -4,6 +4,22 @@
namespace DB
{
namespace
{
void formatRoleNameOrID(const String & str, bool is_id, const IAST::FormatSettings & settings)
{
if (is_id)
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "ID" << (settings.hilite ? IAST::hilite_none : "") << "("
<< quoteString(str) << ")";
}
else
{
settings.ostr << backQuoteIfNeed(str);
}
}
}
void ASTGenericRoleSet::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
if (empty())
......@@ -25,7 +41,7 @@ void ASTGenericRoleSet::formatImpl(const FormatSettings & settings, FormatState
{
if (std::exchange(need_comma, true))
settings.ostr << ", ";
settings.ostr << backQuoteIfNeed(role);
formatRoleNameOrID(role, id_mode, settings);
}
if (current_user)
......@@ -45,7 +61,7 @@ void ASTGenericRoleSet::formatImpl(const FormatSettings & settings, FormatState
{
if (std::exchange(need_comma, true))
settings.ostr << ", ";
settings.ostr << backQuoteIfNeed(except_role);
formatRoleNameOrID(except_role, id_mode, settings);
}
if (except_current_user)
......
......@@ -16,6 +16,7 @@ public:
bool all = false;
Strings except_names;
bool except_current_user = false;
bool id_mode = false; /// If true then `names` and `except_names` keeps UUIDs, not names.
bool empty() const { return names.empty() && !current_user && !all; }
......
......@@ -121,7 +121,7 @@ ASTPtr ASTGrantQuery::clone() const
void ASTGrantQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << ((kind == Kind::GRANT) ? "GRANT" : "REVOKE")
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << (attach ? "ATTACH " : "") << ((kind == Kind::GRANT) ? "GRANT" : "REVOKE")
<< (settings.hilite ? IAST::hilite_none : "") << " ";
if (kind == Kind::REVOKE)
......
......@@ -24,6 +24,7 @@ public:
REVOKE,
};
Kind kind = Kind::GRANT;
bool attach = false;
AccessRightsElements access_rights_elements;
std::shared_ptr<ASTGenericRoleSet> roles;
std::shared_ptr<ASTGenericRoleSet> to_roles;
......
......@@ -187,12 +187,12 @@ namespace
});
}
bool parseToRoles(IParserBase::Pos & pos, Expected & expected, std::shared_ptr<ASTGenericRoleSet> & roles)
bool parseToRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr<ASTGenericRoleSet> & roles)
{
return IParserBase::wrapParseImpl(pos, [&]
{
ASTPtr node;
if (roles || !ParserKeyword{"TO"}.ignore(pos, expected) || !ParserGenericRoleSet{}.parse(pos, node, expected))
if (roles || !ParserKeyword{"TO"}.ignore(pos, expected) || !ParserGenericRoleSet{}.enableIDMode(id_mode).parse(pos, node, expected))
return false;
roles = std::static_pointer_cast<ASTGenericRoleSet>(node);
......@@ -204,13 +204,21 @@ namespace
bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
bool alter;
if (ParserKeyword{"CREATE QUOTA"}.ignore(pos, expected))
alter = false;
else if (ParserKeyword{"ALTER QUOTA"}.ignore(pos, expected))
alter = true;
bool alter = false;
bool attach = false;
if (attach_mode)
{
if (!ParserKeyword{"ATTACH QUOTA"}.ignore(pos, expected))
return false;
attach = true;
}
else
return false;
{
if (ParserKeyword{"ALTER QUOTA"}.ignore(pos, expected))
alter = true;
else if (!ParserKeyword{"CREATE QUOTA"}.ignore(pos, expected))
return false;
}
bool if_exists = false;
bool if_not_exists = false;
......@@ -248,7 +256,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
if (parseAllLimits(pos, expected, alter, all_limits))
continue;
if (!roles && parseToRoles(pos, expected, roles))
if (!roles && parseToRoles(pos, expected, attach, roles))
continue;
break;
......
......@@ -24,8 +24,14 @@ namespace DB
*/
class ParserCreateQuotaQuery : public IParserBase
{
public:
ParserCreateQuotaQuery & enableAttachMode(bool enable_) { attach_mode = enable_; return *this; }
protected:
const char * getName() const override { return "CREATE QUOTA or ALTER QUOTA query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
private:
bool attach_mode = false;
};
}
#include <Parsers/ParserCreateRoleQuery.h>
#include <Parsers/ASTCreateRoleQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/parseUserName.h>
......@@ -23,13 +25,21 @@ namespace
bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
bool alter;
if (ParserKeyword{"CREATE ROLE"}.ignore(pos, expected))
alter = false;
else if (ParserKeyword{"ALTER ROLE"}.ignore(pos, expected))
alter = true;
bool attach = false;
bool alter = false;
if (attach_mode)
{
if (!ParserKeyword{"ATTACH ROLE"}.ignore(pos, expected))
return false;
attach = true;
}
else
return false;
{
if (ParserKeyword{"ALTER ROLE"}.ignore(pos, expected))
alter = true;
else if (!ParserKeyword{"CREATE ROLE"}.ignore(pos, expected))
return false;
}
bool if_exists = false;
bool if_not_exists = false;
......@@ -59,6 +69,7 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
node = query;
query->alter = alter;
query->attach = attach;
query->if_exists = if_exists;
query->if_not_exists = if_not_exists;
query->or_replace = or_replace;
......
......@@ -13,8 +13,14 @@ namespace DB
*/
class ParserCreateRoleQuery : public IParserBase
{
public:
ParserCreateRoleQuery & enableAttachMode(bool enable) { attach_mode = enable; return *this; }
protected:
const char * getName() const override { return "CREATE ROLE or ALTER ROLE query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
private:
bool attach_mode = false;
};
}
......@@ -6,6 +6,7 @@
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/parseDatabaseAndTableName.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ASTLiteral.h>
......@@ -186,12 +187,13 @@ namespace
});
}
bool parseToRoles(IParserBase::Pos & pos, Expected & expected, std::shared_ptr<ASTGenericRoleSet> & roles)
bool parseToRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr<ASTGenericRoleSet> & roles)
{
return IParserBase::wrapParseImpl(pos, [&]
{
ASTPtr ast;
if (roles || !ParserKeyword{"TO"}.ignore(pos, expected) || !ParserGenericRoleSet{}.parse(pos, ast, expected))
if (roles || !ParserKeyword{"TO"}.ignore(pos, expected)
|| !ParserGenericRoleSet{}.enableIDMode(id_mode).parse(pos, ast, expected))
return false;
roles = std::static_pointer_cast<ASTGenericRoleSet>(ast);
......@@ -203,13 +205,21 @@ namespace
bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
bool alter;
if (ParserKeyword{"CREATE POLICY"}.ignore(pos, expected) || ParserKeyword{"CREATE ROW POLICY"}.ignore(pos, expected))
alter = false;
else if (ParserKeyword{"ALTER POLICY"}.ignore(pos, expected) || ParserKeyword{"ALTER ROW POLICY"}.ignore(pos, expected))
alter = true;
bool alter = false;
bool attach = false;
if (attach_mode)
{
if (!ParserKeyword{"ATTACH POLICY"}.ignore(pos, expected) && !ParserKeyword{"ATTACH ROW POLICY"}.ignore(pos, expected))
return false;
attach = true;
}
else
return false;
{
if (ParserKeyword{"ALTER POLICY"}.ignore(pos, expected) || ParserKeyword{"ALTER ROW POLICY"}.ignore(pos, expected))
alter = true;
else if (!ParserKeyword{"CREATE POLICY"}.ignore(pos, expected) && !ParserKeyword{"CREATE ROW POLICY"}.ignore(pos, expected))
return false;
}
bool if_exists = false;
bool if_not_exists = false;
......@@ -251,7 +261,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
if (parseMultipleConditions(pos, expected, alter, conditions))
continue;
if (!roles && parseToRoles(pos, expected, roles))
if (!roles && parseToRoles(pos, expected, attach, roles))
continue;
break;
......@@ -261,6 +271,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
node = query;
query->alter = alter;
query->attach = attach;
query->if_exists = if_exists;
query->if_not_exists = if_not_exists;
query->or_replace = or_replace;
......
......@@ -23,8 +23,14 @@ namespace DB
*/
class ParserCreateRowPolicyQuery : public IParserBase
{
public:
ParserCreateRowPolicyQuery & enableAttachMode(bool enable_) { attach_mode = enable_; return *this; }
protected:
const char * getName() const override { return "CREATE ROW POLICY or ALTER ROW POLICY query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
private:
bool attach_mode = false;
};
}
......@@ -208,7 +208,7 @@ namespace
}
bool parseDefaultRoles(IParserBase::Pos & pos, Expected & expected, std::shared_ptr<ASTGenericRoleSet> & default_roles)
bool parseDefaultRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr<ASTGenericRoleSet> & default_roles)
{
return IParserBase::wrapParseImpl(pos, [&]
{
......@@ -216,7 +216,7 @@ namespace
return false;
ASTPtr ast;
if (!ParserGenericRoleSet{}.allowCurrentUser(false).parse(pos, ast, expected))
if (!ParserGenericRoleSet{}.enableCurrentUserKeyword(false).enableIDMode(id_mode).parse(pos, ast, expected))
return false;
default_roles = typeid_cast<std::shared_ptr<ASTGenericRoleSet>>(ast);
......@@ -245,13 +245,22 @@ namespace
bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
bool alter;
if (ParserKeyword{"CREATE USER"}.ignore(pos, expected))
alter = false;
else if (ParserKeyword{"ALTER USER"}.ignore(pos, expected))
alter = true;
bool alter = false;
bool attach = false;
if (attach_mode)
{
if (ParserKeyword{"ATTACH USER"}.ignore(pos, expected))
attach = true;
else
return false;
}
else
return false;
{
if (ParserKeyword{"ALTER USER"}.ignore(pos, expected))
alter = true;
else if (!ParserKeyword{"CREATE USER"}.ignore(pos, expected))
return false;
}
bool if_exists = false;
bool if_not_exists = false;
......@@ -294,7 +303,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (!profile && parseProfileName(pos, expected, profile))
continue;
if (!default_roles && parseDefaultRoles(pos, expected, default_roles))
if (!default_roles && parseDefaultRoles(pos, expected, attach, default_roles))
continue;
if (alter)
......@@ -321,6 +330,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
node = query;
query->alter = alter;
query->attach = attach;
query->if_exists = if_exists;
query->if_not_exists = if_not_exists;
query->or_replace = or_replace;
......@@ -330,6 +340,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
query->hosts = std::move(hosts);
query->add_hosts = std::move(add_hosts);
query->remove_hosts = std::move(remove_hosts);
query->default_roles = std::move(default_roles);
query->profile = std::move(profile);
return true;
......
......@@ -19,8 +19,14 @@ namespace DB
*/
class ParserCreateUserQuery : public IParserBase
{
public:
ParserCreateUserQuery & enableAttachMode(bool enable) { attach_mode = enable; return *this; }
protected:
const char * getName() const override { return "CREATE USER or ALTER USER query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
private:
bool attach_mode = false;
};
}
#include <Parsers/ParserGenericRoleSet.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTGenericRoleSet.h>
#include <Parsers/parseUserName.h>
#include <boost/range/algorithm/find.hpp>
......@@ -9,7 +11,39 @@ namespace DB
{
namespace
{
bool parseBeforeExcept(IParserBase::Pos & pos, Expected & expected, bool * all, bool * current_user, Strings & names)
bool parseRoleNameOrID(IParserBase::Pos & pos, Expected & expected, bool parse_id, String & res)
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!parse_id)
return parseRoleName(pos, expected, res);
if (!ParserKeyword{"ID"}.ignore(pos, expected))
return false;
if (!ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected))
return false;
ASTPtr ast;
if (!ParserStringLiteral{}.parse(pos, ast, expected))
return false;
String id = ast->as<ASTLiteral &>().value.safeGet<String>();
if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected))
return false;
res = std::move(id);
return true;
});
}
bool parseBeforeExcept(
IParserBase::Pos & pos,
Expected & expected,
bool id_mode,
bool all_keyword_enabled,
bool current_user_keyword_enabled,
Strings & names,
bool & all,
bool & current_user)
{
return IParserBase::wrapParseImpl(pos, [&]
{
......@@ -22,7 +56,8 @@ namespace
{
}
else if (
current_user && (ParserKeyword{"CURRENT_USER"}.ignore(pos, expected) || ParserKeyword{"currentUser"}.ignore(pos, expected)))
current_user_keyword_enabled
&& (ParserKeyword{"CURRENT_USER"}.ignore(pos, expected) || ParserKeyword{"currentUser"}.ignore(pos, expected)))
{
if (ParserToken{TokenType::OpeningRoundBracket}.ignore(pos, expected))
{
......@@ -31,14 +66,14 @@ namespace
}
res_current_user = true;
}
else if (all && ParserKeyword{"ALL"}.ignore(pos, expected))
else if (all_keyword_enabled && ParserKeyword{"ALL"}.ignore(pos, expected))
{
res_all = true;
}
else
{
String name;
if (!parseUserName(pos, expected, name))
if (!parseRoleNameOrID(pos, expected, id_mode, name))
return false;
res_names.push_back(name);
}
......@@ -47,23 +82,28 @@ namespace
break;
}
if (all)
*all = res_all;
if (current_user)
*current_user = res_current_user;
all = res_all;
current_user = res_current_user;
names = std::move(res_names);
return true;
});
}
bool parseExcept(IParserBase::Pos & pos, Expected & expected, bool * except_current_user, Strings & except_names)
bool parseExceptAndAfterExcept(
IParserBase::Pos & pos,
Expected & expected,
bool id_mode,
bool current_user_keyword_enabled,
Strings & except_names,
bool & except_current_user)
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"EXCEPT"}.ignore(pos, expected))
return false;
return parseBeforeExcept(pos, expected, nullptr, except_current_user, except_names);
bool dummy;
return parseBeforeExcept(pos, expected, id_mode, false, current_user_keyword_enabled, except_names, dummy, except_current_user);
});
}
}
......@@ -77,10 +117,10 @@ bool ParserGenericRoleSet::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
Strings except_names;
bool except_current_user = false;
if (!parseBeforeExcept(pos, expected, (allow_all ? &all : nullptr), (allow_current_user ? &current_user : nullptr), names))
if (!parseBeforeExcept(pos, expected, id_mode, all_keyword, current_user_keyword, names, all, current_user))
return false;
parseExcept(pos, expected, (allow_current_user ? &except_current_user : nullptr), except_names);
parseExceptAndAfterExcept(pos, expected, id_mode, current_user_keyword, except_names, except_current_user);
if (all)
names.clear();
......@@ -91,6 +131,7 @@ bool ParserGenericRoleSet::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
result->all = all;
result->except_names = std::move(except_names);
result->except_current_user = except_current_user;
result->id_mode = id_mode;
node = result;
return true;
}
......
......@@ -11,16 +11,18 @@ namespace DB
class ParserGenericRoleSet : public IParserBase
{
public:
ParserGenericRoleSet & allowAll(bool allow_) { allow_all = allow_; return *this; }
ParserGenericRoleSet & allowCurrentUser(bool allow_) { allow_current_user = allow_; return *this; }
ParserGenericRoleSet & enableAllKeyword(bool enable_) { all_keyword = enable_; return *this; }
ParserGenericRoleSet & enableCurrentUserKeyword(bool enable_) { current_user_keyword = enable_; return *this; }
ParserGenericRoleSet & enableIDMode(bool enable_) { id_mode = enable_; return *this; }
protected:
const char * getName() const override { return "GenericRoleSet"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
private:
bool allow_all = true;
bool allow_current_user = true;
bool all_keyword = true;
bool current_user_keyword = true;
bool id_mode = false;
};
}
......@@ -211,12 +211,12 @@ namespace
}
bool parseRoles(IParser::Pos & pos, Expected & expected, std::shared_ptr<ASTGenericRoleSet> & roles)
bool parseRoles(IParser::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr<ASTGenericRoleSet> & roles)
{
return IParserBase::wrapParseImpl(pos, [&]
{
ASTPtr ast;
if (!ParserGenericRoleSet{}.allowAll(false).allowCurrentUser(false).parse(pos, ast, expected))
if (!ParserGenericRoleSet{}.enableAllKeyword(false).enableCurrentUserKeyword(false).enableIDMode(id_mode).parse(pos, ast, expected))
return false;
roles = typeid_cast<std::shared_ptr<ASTGenericRoleSet>>(ast);
......@@ -242,7 +242,7 @@ namespace
}
ASTPtr ast;
if (!ParserGenericRoleSet{}.allowAll(kind == Kind::REVOKE).parse(pos, ast, expected))
if (!ParserGenericRoleSet{}.enableAllKeyword(kind == Kind::REVOKE).parse(pos, ast, expected))
return false;
to_roles = typeid_cast<std::shared_ptr<ASTGenericRoleSet>>(ast);
......@@ -254,6 +254,14 @@ namespace
bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
bool attach = false;
if (attach_mode)
{
if (!ParserKeyword{"ATTACH"}.ignore(pos, expected))
return false;
attach = true;
}
using Kind = ASTGrantQuery::Kind;
Kind kind;
if (ParserKeyword{"GRANT"}.ignore(pos, expected))
......@@ -275,7 +283,7 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
AccessRightsElements elements;
std::shared_ptr<ASTGenericRoleSet> roles;
if (!parseAccessRightsElements(pos, expected, elements) && !parseRoles(pos, expected, roles))
if (!parseAccessRightsElements(pos, expected, elements) && !parseRoles(pos, expected, attach, roles))
return false;
std::shared_ptr<ASTGenericRoleSet> to_roles;
......@@ -299,6 +307,7 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
node = query;
query->kind = kind;
query->attach = attach;
query->access_rights_elements = std::move(elements);
query->roles = std::move(roles);
query->to_roles = std::move(to_roles);
......
......@@ -11,8 +11,14 @@ namespace DB
*/
class ParserGrantQuery : public IParserBase
{
public:
ParserGrantQuery & enableAttachMode(bool enable) { attach_mode = enable; return *this; }
protected:
const char * getName() const override { return "GRANT or REVOKE query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
private:
bool attach_mode = false;
};
}
......@@ -14,7 +14,7 @@ namespace
return IParserBase::wrapParseImpl(pos, [&]
{
ASTPtr ast;
if (!ParserGenericRoleSet{}.allowCurrentUser(false).parse(pos, ast, expected))
if (!ParserGenericRoleSet{}.enableCurrentUserKeyword(false).parse(pos, ast, expected))
return false;
roles = typeid_cast<std::shared_ptr<ASTGenericRoleSet>>(ast);
......@@ -30,7 +30,7 @@ namespace
return false;
ASTPtr ast;
if (!ParserGenericRoleSet{}.allowAll(false).parse(pos, ast, expected))
if (!ParserGenericRoleSet{}.enableAllKeyword(false).parse(pos, ast, expected))
return false;
to_users = typeid_cast<std::shared_ptr<ASTGenericRoleSet>>(ast);
......
......@@ -87,7 +87,7 @@ void StorageSystemQuotas::fillData(MutableColumns & res_columns, const Context &
storage_name_column.insert(storage_name);
key_type_column.insert(static_cast<UInt8>(quota->key_type));
for (const String & role : quota->roles.toStrings(access_control))
for (const String & role : quota->roles.toStringsWithNames(access_control))
roles_data.insert(role);
roles_offsets.push_back(roles_data.size());
......
......@@ -401,77 +401,82 @@ class ClickHouseCluster:
if not subprocess_call(['docker-compose', 'kill']):
subprocess_call(['docker-compose', 'down', '--volumes'])
logging.info("Unstopped containers killed")
except:
pass
logging.info("Unstopped containers killed")
if destroy_dirs and p.exists(self.instances_dir):
logging.info("Removing instances dir %s", self.instances_dir)
shutil.rmtree(self.instances_dir)
for instance in self.instances.values():
instance.create_dir(destroy_dir=destroy_dirs)
self.docker_client = docker.from_env(version=self.docker_api_version)
common_opts = ['up', '-d', '--force-recreate']
if self.with_zookeeper and self.base_zookeeper_cmd:
subprocess_check_call(self.base_zookeeper_cmd + common_opts)
for command in self.pre_zookeeper_commands:
self.run_kazoo_commands_with_retries(command, repeats=5)
self.wait_zookeeper_to_start(120)
if self.with_mysql and self.base_mysql_cmd:
subprocess_check_call(self.base_mysql_cmd + common_opts)
self.wait_mysql_to_start(120)
if self.with_postgres and self.base_postgres_cmd:
subprocess_check_call(self.base_postgres_cmd + common_opts)
self.wait_postgres_to_start(120)
if self.with_kafka and self.base_kafka_cmd:
subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes'])
self.kafka_docker_id = self.get_instance_docker_id('kafka1')
self.wait_schema_registry_to_start(120)
if self.with_hdfs and self.base_hdfs_cmd:
subprocess_check_call(self.base_hdfs_cmd + common_opts)
self.wait_hdfs_to_start(120)
if self.with_mongo and self.base_mongo_cmd:
subprocess_check_call(self.base_mongo_cmd + common_opts)
self.wait_mongo_to_start(30)
if self.with_redis and self.base_redis_cmd:
subprocess_check_call(self.base_redis_cmd + ['up', '-d', '--force-recreate'])
time.sleep(10)
if self.with_minio and self.base_minio_cmd:
minio_start_cmd = self.base_minio_cmd + common_opts
logging.info("Trying to create Minio instance by command %s", ' '.join(map(str, minio_start_cmd)))
subprocess_check_call(minio_start_cmd)
logging.info("Trying to connect to Minio...")
self.wait_minio_to_start()
clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate']
logging.info("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd)))
subprocess_check_call(clickhouse_start_cmd)
logging.info("ClickHouse instance created")
start_deadline = time.time() + 20.0 # seconds
for instance in self.instances.itervalues():
instance.docker_client = self.docker_client
instance.ip_address = self.get_instance_ip(instance.name)
logging.info("Waiting for ClickHouse start...")
instance.wait_for_start(start_deadline)
logging.info("ClickHouse started")
instance.client = Client(instance.ip_address, command=self.client_bin_path)
self.is_up = True
try:
if destroy_dirs and p.exists(self.instances_dir):
logging.info("Removing instances dir %s", self.instances_dir)
shutil.rmtree(self.instances_dir)
for instance in self.instances.values():
instance.create_dir(destroy_dir=destroy_dirs)
self.docker_client = docker.from_env(version=self.docker_api_version)
common_opts = ['up', '-d', '--force-recreate']
if self.with_zookeeper and self.base_zookeeper_cmd:
subprocess_check_call(self.base_zookeeper_cmd + common_opts)
for command in self.pre_zookeeper_commands:
self.run_kazoo_commands_with_retries(command, repeats=5)
self.wait_zookeeper_to_start(120)
if self.with_mysql and self.base_mysql_cmd:
subprocess_check_call(self.base_mysql_cmd + common_opts)
self.wait_mysql_to_start(120)
if self.with_postgres and self.base_postgres_cmd:
subprocess_check_call(self.base_postgres_cmd + common_opts)
self.wait_postgres_to_start(120)
if self.with_kafka and self.base_kafka_cmd:
subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes'])
self.kafka_docker_id = self.get_instance_docker_id('kafka1')
self.wait_schema_registry_to_start(120)
if self.with_hdfs and self.base_hdfs_cmd:
subprocess_check_call(self.base_hdfs_cmd + common_opts)
self.wait_hdfs_to_start(120)
if self.with_mongo and self.base_mongo_cmd:
subprocess_check_call(self.base_mongo_cmd + common_opts)
self.wait_mongo_to_start(30)
if self.with_redis and self.base_redis_cmd:
subprocess_check_call(self.base_redis_cmd + ['up', '-d', '--force-recreate'])
time.sleep(10)
if self.with_minio and self.base_minio_cmd:
minio_start_cmd = self.base_minio_cmd + common_opts
logging.info("Trying to create Minio instance by command %s", ' '.join(map(str, minio_start_cmd)))
subprocess_check_call(minio_start_cmd)
logging.info("Trying to connect to Minio...")
self.wait_minio_to_start()
clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate']
logging.info("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd)))
subprocess_check_call(clickhouse_start_cmd)
logging.info("ClickHouse instance created")
start_deadline = time.time() + 20.0 # seconds
for instance in self.instances.itervalues():
instance.docker_client = self.docker_client
instance.ip_address = self.get_instance_ip(instance.name)
logging.info("Waiting for ClickHouse start...")
instance.wait_for_start(start_deadline)
logging.info("ClickHouse started")
instance.client = Client(instance.ip_address, command=self.client_bin_path)
self.is_up = True
except BaseException, e:
print "Failed to start cluster: "
print str(e)
raise
def shutdown(self, kill=True):
sanitizer_assert_instance = None
......@@ -686,6 +691,9 @@ class ClickHouseInstance:
self.exec_in_container(["bash", "-c", "pkill {} clickhouse".format("-9" if kill else "")], user='root')
time.sleep(stop_start_wait_sec)
self.exec_in_container(["bash", "-c", "{} --daemon".format(CLICKHOUSE_START_COMMAND)], user=str(os.getuid()))
# wait start
from helpers.test_tools import assert_eq_with_retry
assert_eq_with_retry(self, "select 1", "1", retry_count=int(stop_start_wait_sec / 0.5), sleep_time=0.5)
def exec_in_container(self, cmd, detach=False, **kwargs):
container = self.get_docker_handle()
......
<?xml version="1.0"?>
<yandex>
<access_control_path>/var/lib/clickhouse/access</access_control_path>
</yandex>
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance', config_dir='configs', main_configs=['configs/access_control_path.xml'], stay_alive=True)
@pytest.fixture(scope="module", autouse=True)
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def create_entities():
instance.query("CREATE USER u1")
instance.query("CREATE ROLE rx")
instance.query("CREATE USER u2 IDENTIFIED BY 'qwerty' HOST LOCAL DEFAULT ROLE rx")
instance.query("CREATE ROW POLICY p ON mydb.mytable FOR SELECT USING a<1000 TO u1, u2")
instance.query("CREATE QUOTA q FOR INTERVAL 1 HOUR SET MAX QUERIES = 100 TO ALL EXCEPT rx")
@pytest.fixture(autouse=True)
def drop_entities():
instance.query("DROP USER IF EXISTS u1, u2")
instance.query("DROP ROLE IF EXISTS rx, ry")
instance.query("DROP ROW POLICY IF EXISTS p ON mydb.mytable")
instance.query("DROP QUOTA IF EXISTS q")
def test_create():
create_entities()
def check():
assert instance.query("SHOW CREATE USER u1") == "CREATE USER u1\n"
assert instance.query("SHOW CREATE USER u2") == "CREATE USER u2 HOST LOCAL DEFAULT ROLE rx\n"
assert instance.query("SHOW CREATE ROW POLICY p ON mydb.mytable") == "CREATE POLICY p ON mydb.mytable FOR SELECT USING a < 1000 TO u1, u2\n"
assert instance.query("SHOW CREATE QUOTA q") == "CREATE QUOTA q KEYED BY \\'none\\' FOR INTERVAL 1 HOUR MAX QUERIES = 100 TO ALL EXCEPT rx\n"
assert instance.query("SHOW GRANTS FOR u1") == ""
assert instance.query("SHOW GRANTS FOR u2") == "GRANT rx TO u2\n"
assert instance.query("SHOW GRANTS FOR rx") == ""
check()
instance.restart_clickhouse() # Check persistency
check()
def test_alter():
create_entities()
instance.restart_clickhouse()
instance.query("CREATE ROLE ry")
instance.query("GRANT ry TO u2")
instance.query("ALTER USER u2 DEFAULT ROLE ry")
instance.query("GRANT rx TO ry WITH ADMIN OPTION")
instance.query("GRANT SELECT ON mydb.mytable TO u1")
instance.query("GRANT SELECT ON mydb.* TO rx WITH GRANT OPTION")
def check():
assert instance.query("SHOW CREATE USER u1") == "CREATE USER u1\n"
assert instance.query("SHOW CREATE USER u2") == "CREATE USER u2 HOST LOCAL DEFAULT ROLE ry\n"
assert instance.query("SHOW GRANTS FOR u1") == "GRANT SELECT ON mydb.mytable TO u1\n"
assert instance.query("SHOW GRANTS FOR u2") == "GRANT rx, ry TO u2\n"
assert instance.query("SHOW GRANTS FOR rx") == "GRANT SELECT ON mydb.* TO rx WITH GRANT OPTION\n"
assert instance.query("SHOW GRANTS FOR ry") == "GRANT rx TO ry WITH ADMIN OPTION\n"
check()
instance.restart_clickhouse() # Check persistency
check()
def test_drop():
create_entities()
instance.restart_clickhouse()
instance.query("DROP USER u2")
instance.query("DROP ROLE rx")
instance.query("DROP ROW POLICY p ON mydb.mytable")
instance.query("DROP QUOTA q")
def check():
assert instance.query("SHOW CREATE USER u1") == "CREATE USER u1\n"
assert "User `u2` not found" in instance.query_and_get_error("SHOW CREATE USER u2")
assert "Row policy `p ON mydb.mytable` not found" in instance.query_and_get_error("SHOW CREATE ROW POLICY p ON mydb.mytable")
assert "Quota `q` not found" in instance.query_and_get_error("SHOW CREATE QUOTA q")
check()
instance.restart_clickhouse() # Check persistency
check()
......@@ -1164,7 +1164,6 @@ def test_kill_while_insert(start_cluster):
except:
""""""
time.sleep(0.5)
assert node1.query("SELECT count() FROM {name}".format(name=name)).splitlines() == ["10"]
finally:
......
......@@ -47,7 +47,6 @@ def test_startup_without_zookeeper(start_cluster):
node1.query("INSERT INTO test_table VALUES ('2018-10-01', 1), ('2018-10-02', 2), ('2018-10-03', 3)")
node1.restart_clickhouse()
time.sleep(5)
assert node1.query("SELECT COUNT(*) from test_table") == "3\n"
assert node1.query("SELECT is_readonly from system.replicas where table='test_table'") == "1\n"
......
......@@ -44,5 +44,3 @@ def test_live_view_dependency(started_cluster):
instance.query("CREATE TABLE b_load_second.mt (a Int32) Engine=MergeTree order by tuple()")
instance.query("CREATE LIVE VIEW a_load_first.lv AS SELECT sum(a) FROM b_load_second.mt", settings={'allow_experimental_live_view': 1})
instance.restart_clickhouse()
time.sleep(5)
instance.query("SELECT 1")
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册