mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
add id to insertImpl
This commit is contained in:
parent
21e12ea084
commit
13d9952227
@ -525,12 +525,14 @@ scope_guard AccessControl::subscribeForChanges(const std::vector<UUID> & ids, co
|
||||
return changes_notifier->subscribeForChanges(ids, handler);
|
||||
}
|
||||
|
||||
std::optional<UUID> AccessControl::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional<UUID> set_id)
|
||||
bool AccessControl::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists)
|
||||
{
|
||||
auto id = MultipleAccessStorage::insertImpl(entity, replace_if_exists, throw_if_exists, set_id);
|
||||
if (id)
|
||||
if (MultipleAccessStorage::insertImpl(id, entity, replace_if_exists, throw_if_exists))
|
||||
{
|
||||
changes_notifier->sendNotifications();
|
||||
return id;
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
bool AccessControl::removeImpl(const UUID & id, bool throw_if_not_exists)
|
||||
|
@ -229,7 +229,7 @@ private:
|
||||
class CustomSettingsPrefixes;
|
||||
class PasswordComplexityRules;
|
||||
|
||||
std::optional<UUID> insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional<UUID> set_id) override;
|
||||
bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
|
||||
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
|
||||
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
|
||||
|
||||
|
@ -498,20 +498,10 @@ std::optional<std::pair<String, AccessEntityType>> DiskAccessStorage::readNameWi
|
||||
}
|
||||
|
||||
|
||||
std::optional<UUID> DiskAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional<UUID> set_id)
|
||||
{
|
||||
UUID id = set_id ? *set_id : generateRandomID();
|
||||
if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists, /* write_on_disk= */ true))
|
||||
return id;
|
||||
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
|
||||
bool DiskAccessStorage::insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk)
|
||||
bool DiskAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, write_on_disk);
|
||||
return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, /* write_on_disk = */ true);
|
||||
}
|
||||
|
||||
|
||||
@ -745,7 +735,7 @@ void DiskAccessStorage::restoreFromBackup(RestorerFromBackup & restorer)
|
||||
restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists]
|
||||
{
|
||||
for (const auto & [id, entity] : my_entities)
|
||||
insertWithID(id, entity, replace_if_exists, throw_if_exists, /* write_on_disk= */ true);
|
||||
insert(id, entity, replace_if_exists, throw_if_exists);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -39,7 +39,7 @@ private:
|
||||
std::vector<UUID> findAllImpl(AccessEntityType type) const override;
|
||||
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override;
|
||||
std::optional<std::pair<String, AccessEntityType>> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override;
|
||||
std::optional<UUID> insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional<UUID> set_id) override;
|
||||
bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
|
||||
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
|
||||
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
|
||||
|
||||
@ -53,7 +53,6 @@ private:
|
||||
void listsWritingThreadFunc() TSA_NO_THREAD_SAFETY_ANALYSIS;
|
||||
void stopListsWritingThread();
|
||||
|
||||
bool insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk);
|
||||
bool insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk) TSA_REQUIRES(mutex);
|
||||
bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex);
|
||||
bool removeNoLock(const UUID & id, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex);
|
||||
|
@ -180,14 +180,21 @@ UUID IAccessStorage::insert(const AccessEntityPtr & entity)
|
||||
|
||||
std::optional<UUID> IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists)
|
||||
{
|
||||
return insert(entity, replace_if_exists, throw_if_exists, /* set_id = */ std::nullopt);
|
||||
auto id = generateRandomID();
|
||||
|
||||
if (insert(id, entity, replace_if_exists, throw_if_exists))
|
||||
return id;
|
||||
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
std::optional<UUID> IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional<UUID> set_id)
|
||||
|
||||
bool IAccessStorage::insert(const DB::UUID & id, const DB::AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists)
|
||||
{
|
||||
return insertImpl(entity, replace_if_exists, throw_if_exists, set_id);
|
||||
return insertImpl(id, entity, replace_if_exists, throw_if_exists);
|
||||
}
|
||||
|
||||
|
||||
std::vector<UUID> IAccessStorage::insert(const std::vector<AccessEntityPtr> & multiple_entities, bool replace_if_exists, bool throw_if_exists)
|
||||
{
|
||||
return insert(multiple_entities, /* ids = */ {}, replace_if_exists, throw_if_exists);
|
||||
@ -216,16 +223,16 @@ std::vector<UUID> IAccessStorage::insert(const std::vector<AccessEntityPtr> & mu
|
||||
{
|
||||
const auto & entity = multiple_entities[i];
|
||||
|
||||
std::optional<UUID> id;
|
||||
UUID id;
|
||||
if (!ids.empty())
|
||||
id = ids[i];
|
||||
else
|
||||
id = generateRandomID();
|
||||
|
||||
auto new_id = insertImpl(entity, replace_if_exists, throw_if_exists, id);
|
||||
|
||||
if (new_id)
|
||||
if (insertImpl(id, entity, replace_if_exists, throw_if_exists))
|
||||
{
|
||||
successfully_inserted.push_back(entity);
|
||||
new_ids.push_back(*new_id);
|
||||
new_ids.push_back(id);
|
||||
}
|
||||
}
|
||||
return new_ids;
|
||||
@ -274,7 +281,7 @@ std::vector<UUID> IAccessStorage::insertOrReplace(const std::vector<AccessEntity
|
||||
}
|
||||
|
||||
|
||||
std::optional<UUID> IAccessStorage::insertImpl(const AccessEntityPtr & entity, bool, bool, std::optional<UUID>)
|
||||
bool IAccessStorage::insertImpl(const UUID &, const AccessEntityPtr & entity, bool, bool)
|
||||
{
|
||||
if (isReadOnly())
|
||||
throwReadonlyCannotInsert(entity->getType(), entity->getName());
|
||||
|
@ -3,6 +3,8 @@
|
||||
#include <Access/IAccessEntity.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Core/UUID.h>
|
||||
#include <Parsers/IParser.h>
|
||||
#include <Parsers/parseIdentifierOrStringLiteral.h>
|
||||
#include <functional>
|
||||
#include <optional>
|
||||
#include <vector>
|
||||
@ -132,7 +134,7 @@ public:
|
||||
/// Throws an exception if the specified name already exists.
|
||||
UUID insert(const AccessEntityPtr & entity);
|
||||
std::optional<UUID> insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists);
|
||||
std::optional<UUID> insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional<UUID> set_id);
|
||||
bool insert(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists);
|
||||
std::vector<UUID> insert(const std::vector<AccessEntityPtr> & multiple_entities, bool replace_if_exists = false, bool throw_if_exists = true);
|
||||
std::vector<UUID> insert(const std::vector<AccessEntityPtr> & multiple_entities, const std::vector<UUID> & ids, bool replace_if_exists = false, bool throw_if_exists = true);
|
||||
|
||||
@ -185,7 +187,7 @@ protected:
|
||||
virtual std::vector<UUID> findAllImpl(AccessEntityType type) const = 0;
|
||||
virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const = 0;
|
||||
virtual std::optional<std::pair<String, AccessEntityType>> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const;
|
||||
virtual std::optional<UUID> insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional<UUID> set_id);
|
||||
virtual bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists);
|
||||
virtual bool removeImpl(const UUID & id, bool throw_if_not_exists);
|
||||
virtual bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists);
|
||||
virtual std::optional<UUID> authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const;
|
||||
@ -284,4 +286,9 @@ std::vector<std::pair<UUID, std::shared_ptr<const EntityClassT>>> IAccessStorage
|
||||
return entities;
|
||||
}
|
||||
|
||||
inline bool parseAccessStorageName(IParser::Pos & pos, Expected & expected, String & storage_name)
|
||||
{
|
||||
return parseIdentifierOrStringLiteral(pos, expected, storage_name);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -63,17 +63,7 @@ AccessEntityPtr MemoryAccessStorage::readImpl(const UUID & id, bool throw_if_not
|
||||
}
|
||||
|
||||
|
||||
std::optional<UUID> MemoryAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional<UUID> set_id)
|
||||
{
|
||||
UUID id = set_id ? *set_id : generateRandomID();
|
||||
if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists))
|
||||
return id;
|
||||
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
|
||||
bool MemoryAccessStorage::insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists)
|
||||
bool MemoryAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists);
|
||||
@ -300,7 +290,7 @@ void MemoryAccessStorage::restoreFromBackup(RestorerFromBackup & restorer)
|
||||
restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists]
|
||||
{
|
||||
for (const auto & [id, entity] : my_entities)
|
||||
insertWithID(id, entity, replace_if_exists, throw_if_exists);
|
||||
insert(id, entity, replace_if_exists, throw_if_exists);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -22,11 +22,6 @@ public:
|
||||
|
||||
const char * getStorageType() const override { return STORAGE_TYPE; }
|
||||
|
||||
/// Inserts an entity with a specified ID.
|
||||
/// If `replace_if_exists == true` it can replace an existing entry with such ID and also remove an existing entry
|
||||
/// with such name & type.
|
||||
bool insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists);
|
||||
|
||||
/// Removes all entities except the specified list `ids_to_keep`.
|
||||
/// The function skips IDs not contained in the storage.
|
||||
void removeAllExcept(const std::vector<UUID> & ids_to_keep);
|
||||
@ -44,7 +39,7 @@ private:
|
||||
std::optional<UUID> findImpl(AccessEntityType type, const String & name) const override;
|
||||
std::vector<UUID> findAllImpl(AccessEntityType type) const override;
|
||||
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override;
|
||||
std::optional<UUID> insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional<UUID> set_id) override;
|
||||
bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
|
||||
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
|
||||
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
|
||||
|
||||
|
@ -316,7 +316,7 @@ void MultipleAccessStorage::reload(ReloadMode reload_mode)
|
||||
}
|
||||
|
||||
|
||||
std::optional<UUID> MultipleAccessStorage::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional<UUID> set_id)
|
||||
bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists)
|
||||
{
|
||||
std::shared_ptr<IAccessStorage> storage_for_insertion;
|
||||
|
||||
@ -339,13 +339,14 @@ std::optional<UUID> MultipleAccessStorage::insertImpl(const AccessEntityPtr & en
|
||||
getStorageName());
|
||||
}
|
||||
|
||||
auto id = storage_for_insertion->insert(entity, replace_if_exists, throw_if_exists, set_id);
|
||||
if (id)
|
||||
if (storage_for_insertion->insert(id, entity, replace_if_exists, throw_if_exists))
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
ids_cache.set(*id, storage_for_insertion);
|
||||
ids_cache.set(id, storage_for_insertion);
|
||||
return true;
|
||||
}
|
||||
return id;
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
|
@ -63,7 +63,7 @@ protected:
|
||||
std::vector<UUID> findAllImpl(AccessEntityType type) const override;
|
||||
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override;
|
||||
std::optional<std::pair<String, AccessEntityType>> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override;
|
||||
std::optional<UUID> insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional<UUID> set_id) override;
|
||||
bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
|
||||
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
|
||||
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
|
||||
std::optional<UUID> authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override;
|
||||
|
@ -108,17 +108,7 @@ static void retryOnZooKeeperUserError(size_t attempts, Func && function)
|
||||
}
|
||||
}
|
||||
|
||||
std::optional<UUID> ReplicatedAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional<UUID> set_id)
|
||||
{
|
||||
const UUID id = set_id ? *set_id : generateRandomID();
|
||||
if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists))
|
||||
return id;
|
||||
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
|
||||
bool ReplicatedAccessStorage::insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists)
|
||||
bool ReplicatedAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists)
|
||||
{
|
||||
const AccessEntityTypeInfo type_info = AccessEntityTypeInfo::get(new_entity->getType());
|
||||
const String & name = new_entity->getName();
|
||||
@ -619,7 +609,7 @@ AccessEntityPtr ReplicatedAccessStorage::tryReadEntityFromZooKeeper(const zkutil
|
||||
void ReplicatedAccessStorage::setEntityNoLock(const UUID & id, const AccessEntityPtr & entity)
|
||||
{
|
||||
LOG_DEBUG(getLogger(), "Setting id {} to entity named {}", toString(id), entity->getName());
|
||||
memory_storage.insertWithID(id, entity, /* replace_if_exists= */ true, /* throw_if_exists= */ false);
|
||||
memory_storage.insert(id, entity, /* replace_if_exists= */ true, /* throw_if_exists= */ false);
|
||||
}
|
||||
|
||||
|
||||
@ -711,7 +701,7 @@ void ReplicatedAccessStorage::restoreFromBackup(RestorerFromBackup & restorer)
|
||||
restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists]
|
||||
{
|
||||
for (const auto & [id, entity] : my_entities)
|
||||
insertWithID(id, entity, replace_if_exists, throw_if_exists);
|
||||
insert(id, entity, replace_if_exists, throw_if_exists);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -46,11 +46,10 @@ private:
|
||||
std::unique_ptr<ThreadFromGlobalPool> watching_thread;
|
||||
std::shared_ptr<ConcurrentBoundedQueue<UUID>> watched_queue;
|
||||
|
||||
std::optional<UUID> insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional<UUID> set_id) override;
|
||||
bool insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) override;
|
||||
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
|
||||
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
|
||||
|
||||
bool insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists);
|
||||
bool insertZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists);
|
||||
bool removeZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, bool throw_if_not_exists);
|
||||
bool updateZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists);
|
||||
|
@ -1,11 +1,11 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Access/IAccessStorage.h>
|
||||
#include <Parsers/ASTIdentifier_fwd.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/Access/ASTCreateQuotaQuery.h>
|
||||
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
||||
#include <Parsers/Access/ParserCreateQuotaQuery.h>
|
||||
#include <Parsers/Access/ParserRolesOrUsersSet.h>
|
||||
#include <Parsers/Access/parseUserName.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
@ -312,7 +312,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
if (cluster.empty() && parseOnCluster(pos, expected, cluster))
|
||||
continue;
|
||||
|
||||
if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name))
|
||||
if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name))
|
||||
continue;
|
||||
|
||||
break;
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Access/IAccessStorage.h>
|
||||
#include <Parsers/Access/ParserCreateRoleQuery.h>
|
||||
#include <Parsers/Access/ASTCreateRoleQuery.h>
|
||||
#include <Parsers/Access/ASTSettingsProfileElement.h>
|
||||
@ -111,7 +112,7 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
if (cluster.empty() && parseOnCluster(pos, expected, cluster))
|
||||
continue;
|
||||
|
||||
if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name))
|
||||
if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name))
|
||||
continue;
|
||||
|
||||
break;
|
||||
|
@ -1,10 +1,10 @@
|
||||
#include <Access/IAccessStorage.h>
|
||||
#include <Parsers/Access/ParserCreateRowPolicyQuery.h>
|
||||
#include <Parsers/Access/ASTCreateRowPolicyQuery.h>
|
||||
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
||||
#include <Parsers/Access/ASTRowPolicyName.h>
|
||||
#include <Parsers/Access/ParserRolesOrUsersSet.h>
|
||||
#include <Parsers/Access/ParserRowPolicyName.h>
|
||||
#include <Parsers/Access/parseUserName.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/parseDatabaseAndTableName.h>
|
||||
@ -273,7 +273,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
if (cluster.empty() && parseOnCluster(pos, expected, cluster))
|
||||
continue;
|
||||
|
||||
if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name))
|
||||
if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name))
|
||||
continue;
|
||||
|
||||
break;
|
||||
|
@ -1,10 +1,10 @@
|
||||
#include <Access/IAccessStorage.h>
|
||||
#include <Parsers/Access/ParserCreateSettingsProfileQuery.h>
|
||||
#include <Parsers/Access/ASTCreateSettingsProfileQuery.h>
|
||||
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
||||
#include <Parsers/Access/ASTSettingsProfileElement.h>
|
||||
#include <Parsers/Access/ParserSettingsProfileElement.h>
|
||||
#include <Parsers/Access/ParserRolesOrUsersSet.h>
|
||||
#include <Parsers/Access/parseUserName.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
@ -132,7 +132,7 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec
|
||||
if (cluster.empty() && parseOnCluster(pos, expected, cluster))
|
||||
continue;
|
||||
|
||||
if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name))
|
||||
if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name))
|
||||
continue;
|
||||
|
||||
break;
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Access/IAccessStorage.h>
|
||||
#include <Parsers/Access/ParserCreateUserQuery.h>
|
||||
#include <Parsers/Access/ASTCreateUserQuery.h>
|
||||
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
||||
@ -481,7 +482,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
}
|
||||
}
|
||||
|
||||
if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name))
|
||||
if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name))
|
||||
continue;
|
||||
|
||||
break;
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Access/IAccessStorage.h>
|
||||
#include <Parsers/Access/ParserDropAccessEntityQuery.h>
|
||||
#include <Parsers/Access/ASTDropAccessEntityQuery.h>
|
||||
#include <Parsers/Access/ParserRowPolicyName.h>
|
||||
@ -78,7 +79,7 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
}
|
||||
|
||||
if (ParserKeyword{"FROM"}.ignore(pos, expected))
|
||||
parseStorageName(pos, expected, storage_name);
|
||||
parseAccessStorageName(pos, expected, storage_name);
|
||||
|
||||
if (cluster.empty())
|
||||
parseOnCluster(pos, expected, cluster);
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Access/IAccessStorage.h>
|
||||
#include <Parsers/Access/ParserMoveAccessEntityQuery.h>
|
||||
#include <Parsers/Access/ASTMoveAccessEntityQuery.h>
|
||||
#include <Parsers/Access/ParserRowPolicyName.h>
|
||||
@ -73,7 +74,7 @@ bool ParserMoveAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!ParserKeyword{"TO"}.ignore(pos, expected) || !parseStorageName(pos, expected, storage_name))
|
||||
if (!ParserKeyword{"TO"}.ignore(pos, expected) || !parseAccessStorageName(pos, expected, storage_name))
|
||||
return false;
|
||||
|
||||
if (cluster.empty())
|
||||
|
@ -35,9 +35,4 @@ inline bool parseRoleNames(IParser::Pos & pos, Expected & expected, Strings & ro
|
||||
return parseUserNames(pos, expected, role_names);
|
||||
}
|
||||
|
||||
inline bool parseStorageName(IParser::Pos & pos, Expected & expected, String & storage_name)
|
||||
{
|
||||
return parseIdentifierOrStringLiteral(pos, expected, storage_name);
|
||||
}
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user