mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Merge pull request #72103 from vitlibar/get-rid-of-code-duplication-after-check-grant
Get rid of code duplication after adding CHECK GRANT
This commit is contained in:
commit
4e56c026cd
@ -9,6 +9,12 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INVALID_GRANT;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
void formatOptions(bool grant_option, bool is_partial_revoke, String & result)
|
||||
@ -211,18 +217,43 @@ AccessRightsElement::AccessRightsElement(
|
||||
{
|
||||
}
|
||||
|
||||
void AccessRightsElement::eraseNonGrantable()
|
||||
AccessFlags AccessRightsElement::getGrantableFlags() const
|
||||
{
|
||||
if (isGlobalWithParameter() && !anyParameter())
|
||||
access_flags &= AccessFlags::allFlagsGrantableOnGlobalWithParameterLevel();
|
||||
return access_flags & AccessFlags::allFlagsGrantableOnGlobalWithParameterLevel();
|
||||
else if (!anyColumn())
|
||||
access_flags &= AccessFlags::allFlagsGrantableOnColumnLevel();
|
||||
return access_flags & AccessFlags::allFlagsGrantableOnColumnLevel();
|
||||
else if (!anyTable())
|
||||
access_flags &= AccessFlags::allFlagsGrantableOnTableLevel();
|
||||
return access_flags & AccessFlags::allFlagsGrantableOnTableLevel();
|
||||
else if (!anyDatabase())
|
||||
access_flags &= AccessFlags::allFlagsGrantableOnDatabaseLevel();
|
||||
return access_flags & AccessFlags::allFlagsGrantableOnDatabaseLevel();
|
||||
else
|
||||
access_flags &= AccessFlags::allFlagsGrantableOnGlobalLevel();
|
||||
return access_flags & AccessFlags::allFlagsGrantableOnGlobalLevel();
|
||||
}
|
||||
|
||||
void AccessRightsElement::throwIfNotGrantable() const
|
||||
{
|
||||
if (empty())
|
||||
return;
|
||||
auto grantable_flags = getGrantableFlags();
|
||||
if (grantable_flags)
|
||||
return;
|
||||
|
||||
if (!anyColumn())
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the column level", access_flags.toString());
|
||||
if (!anyTable())
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the table level", access_flags.toString());
|
||||
if (!anyDatabase())
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the database level", access_flags.toString());
|
||||
if (!anyParameter())
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the global with parameter level", access_flags.toString());
|
||||
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted", access_flags.toString());
|
||||
}
|
||||
|
||||
void AccessRightsElement::eraseNotGrantable()
|
||||
{
|
||||
access_flags = getGrantableFlags();
|
||||
}
|
||||
|
||||
void AccessRightsElement::replaceEmptyDatabase(const String & current_database)
|
||||
@ -251,11 +282,17 @@ bool AccessRightsElements::sameOptions() const
|
||||
return (size() < 2) || std::all_of(std::next(begin()), end(), [this](const AccessRightsElement & e) { return e.sameOptions(front()); });
|
||||
}
|
||||
|
||||
void AccessRightsElements::eraseNonGrantable()
|
||||
void AccessRightsElements::throwIfNotGrantable() const
|
||||
{
|
||||
for (const auto & element : *this)
|
||||
element.throwIfNotGrantable();
|
||||
}
|
||||
|
||||
void AccessRightsElements::eraseNotGrantable()
|
||||
{
|
||||
std::erase_if(*this, [](AccessRightsElement & element)
|
||||
{
|
||||
element.eraseNonGrantable();
|
||||
element.eraseNotGrantable();
|
||||
return element.empty();
|
||||
});
|
||||
}
|
||||
@ -269,4 +306,45 @@ void AccessRightsElements::replaceEmptyDatabase(const String & current_database)
|
||||
String AccessRightsElements::toString() const { return toStringImpl(*this, true); }
|
||||
String AccessRightsElements::toStringWithoutOptions() const { return toStringImpl(*this, false); }
|
||||
|
||||
void AccessRightsElements::formatElementsWithoutOptions(WriteBuffer & buffer, bool hilite) const
|
||||
{
|
||||
bool no_output = true;
|
||||
for (size_t i = 0; i != size(); ++i)
|
||||
{
|
||||
const auto & element = (*this)[i];
|
||||
auto keywords = element.access_flags.toKeywords();
|
||||
if (keywords.empty() || (!element.anyColumn() && element.columns.empty()))
|
||||
continue;
|
||||
|
||||
for (const auto & keyword : keywords)
|
||||
{
|
||||
if (!std::exchange(no_output, false))
|
||||
buffer << ", ";
|
||||
|
||||
buffer << (hilite ? IAST::hilite_keyword : "") << keyword << (hilite ? IAST::hilite_none : "");
|
||||
if (!element.anyColumn())
|
||||
element.formatColumnNames(buffer);
|
||||
}
|
||||
|
||||
bool next_element_on_same_db_and_table = false;
|
||||
if (i != size() - 1)
|
||||
{
|
||||
const auto & next_element = (*this)[i + 1];
|
||||
if (element.sameDatabaseAndTableAndParameter(next_element))
|
||||
{
|
||||
next_element_on_same_db_and_table = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (!next_element_on_same_db_and_table)
|
||||
{
|
||||
buffer << " ";
|
||||
element.formatONClause(buffer, hilite);
|
||||
}
|
||||
}
|
||||
|
||||
if (no_output)
|
||||
buffer << (hilite ? IAST::hilite_keyword : "") << "USAGE ON " << (hilite ? IAST::hilite_none : "") << "*.*";
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -79,8 +79,14 @@ struct AccessRightsElement
|
||||
return (grant_option == other.grant_option) && (is_partial_revoke == other.is_partial_revoke);
|
||||
}
|
||||
|
||||
/// Returns only those flags which can be granted.
|
||||
AccessFlags getGrantableFlags() const;
|
||||
|
||||
/// Throws an exception if some flags can't be granted.
|
||||
void throwIfNotGrantable() const;
|
||||
|
||||
/// Resets flags which cannot be granted.
|
||||
void eraseNonGrantable();
|
||||
void eraseNotGrantable();
|
||||
|
||||
bool isEmptyDatabase() const { return database.empty() and !anyDatabase(); }
|
||||
|
||||
@ -110,8 +116,11 @@ public:
|
||||
bool sameDatabaseAndTable() const;
|
||||
bool sameOptions() const;
|
||||
|
||||
/// Throws an exception if some flags can't be granted.
|
||||
void throwIfNotGrantable() const;
|
||||
|
||||
/// Resets flags which cannot be granted.
|
||||
void eraseNonGrantable();
|
||||
void eraseNotGrantable();
|
||||
|
||||
/// If the database is empty, replaces it with `current_database`. Otherwise does nothing.
|
||||
void replaceEmptyDatabase(const String & current_database);
|
||||
@ -119,6 +128,7 @@ public:
|
||||
/// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table".
|
||||
String toString() const;
|
||||
String toStringWithoutOptions() const;
|
||||
void formatElementsWithoutOptions(WriteBuffer & buffer, bool hilite) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,25 +1,13 @@
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Interpreters/Access/InterpreterCheckGrantQuery.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Parsers/Access/ASTCheckGrantQuery.h>
|
||||
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
||||
#include <Access/AccessControl.h>
|
||||
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Access/Role.h>
|
||||
#include <Access/RolesOrUsersSet.h>
|
||||
#include <Access/User.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/removeOnClusterClauseIfNeeded.h>
|
||||
#include <Interpreters/QueryLog.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
#include <boost/range/algorithm/copy.hpp>
|
||||
#include <boost/range/algorithm/set_algorithm.hpp>
|
||||
#include <boost/range/algorithm_ext/erase.hpp>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterFactory.h>
|
||||
#include <Parsers/Access/ASTCheckGrantQuery.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
#include "Storages/IStorage.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -27,21 +15,19 @@ namespace DB
|
||||
BlockIO InterpreterCheckGrantQuery::execute()
|
||||
{
|
||||
auto & query = query_ptr->as<ASTCheckGrantQuery &>();
|
||||
query.access_rights_elements.eraseNonGrantable();
|
||||
|
||||
auto current_user_access = getContext()->getAccess();
|
||||
|
||||
/// Collect access rights elements which will be checked.
|
||||
AccessRightsElements & elements_to_check_grant = query.access_rights_elements;
|
||||
|
||||
/// Replacing empty database with the default. This step must be done before replication to avoid privilege escalation.
|
||||
String current_database = getContext()->getCurrentDatabase();
|
||||
elements_to_check_grant.replaceEmptyDatabase(current_database);
|
||||
query.access_rights_elements.replaceEmptyDatabase(current_database);
|
||||
bool user_is_granted = current_user_access->isGranted(elements_to_check_grant);
|
||||
|
||||
auto current_user_access = getContext()->getAccess();
|
||||
bool is_granted = current_user_access->isGranted(elements_to_check_grant);
|
||||
|
||||
BlockIO res;
|
||||
res.pipeline = QueryPipeline(
|
||||
std::make_shared<SourceFromSingleChunk>(Block{{ColumnUInt8::create(1, user_is_granted), std::make_shared<DataTypeUInt8>(), "result"}}));
|
||||
std::make_shared<SourceFromSingleChunk>(Block{{ColumnUInt8::create(1, is_granted), std::make_shared<DataTypeUInt8>(), "result"}}));
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/UUID.h>
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
@ -8,10 +7,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ASTCheckGrantQuery;
|
||||
struct User;
|
||||
struct Role;
|
||||
|
||||
class InterpreterCheckGrantQuery : public IInterpreter, WithMutableContext
|
||||
{
|
||||
public:
|
||||
|
@ -418,7 +418,7 @@ BlockIO InterpreterGrantQuery::execute()
|
||||
auto & query = updated_query->as<ASTGrantQuery &>();
|
||||
|
||||
query.replaceCurrentUserTag(getContext()->getUserName());
|
||||
query.access_rights_elements.eraseNonGrantable();
|
||||
query.access_rights_elements.eraseNotGrantable();
|
||||
|
||||
if (!query.access_rights_elements.sameOptions())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Elements of an ASTGrantQuery are expected to have the same options");
|
||||
|
@ -1,98 +1,11 @@
|
||||
#include <Parsers/Access/ASTCheckGrantQuery.h>
|
||||
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
||||
#include <Common/quoteString.h>
|
||||
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
void formatColumnNames(const Strings & columns, const IAST::FormatSettings & settings)
|
||||
{
|
||||
settings.ostr << "(";
|
||||
bool need_comma = false;
|
||||
for (const auto & column : columns)
|
||||
{
|
||||
if (std::exchange(need_comma, true))
|
||||
settings.ostr << ", ";
|
||||
settings.ostr << backQuoteIfNeed(column);
|
||||
}
|
||||
settings.ostr << ")";
|
||||
}
|
||||
|
||||
|
||||
void formatONClause(const AccessRightsElement & element, const IAST::FormatSettings & settings)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "ON " << (settings.hilite ? IAST::hilite_none : "");
|
||||
if (element.isGlobalWithParameter())
|
||||
{
|
||||
if (element.anyParameter())
|
||||
settings.ostr << "*";
|
||||
else
|
||||
settings.ostr << backQuoteIfNeed(element.parameter);
|
||||
}
|
||||
else if (element.anyDatabase())
|
||||
{
|
||||
settings.ostr << "*.*";
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!element.database.empty())
|
||||
settings.ostr << backQuoteIfNeed(element.database) << ".";
|
||||
if (element.anyDatabase())
|
||||
settings.ostr << "*";
|
||||
else
|
||||
settings.ostr << backQuoteIfNeed(element.table);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void formatElementsWithoutOptions(const AccessRightsElements & elements, const IAST::FormatSettings & settings)
|
||||
{
|
||||
bool no_output = true;
|
||||
for (size_t i = 0; i != elements.size(); ++i)
|
||||
{
|
||||
const auto & element = elements[i];
|
||||
auto keywords = element.access_flags.toKeywords();
|
||||
if (keywords.empty() || (!element.anyColumn() && element.columns.empty()))
|
||||
continue;
|
||||
|
||||
for (const auto & keyword : keywords)
|
||||
{
|
||||
if (!std::exchange(no_output, false))
|
||||
settings.ostr << ", ";
|
||||
|
||||
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << keyword << (settings.hilite ? IAST::hilite_none : "");
|
||||
if (!element.anyColumn())
|
||||
formatColumnNames(element.columns, settings);
|
||||
}
|
||||
|
||||
bool next_element_on_same_db_and_table = false;
|
||||
if (i != elements.size() - 1)
|
||||
{
|
||||
const auto & next_element = elements[i + 1];
|
||||
if (element.sameDatabaseAndTableAndParameter(next_element))
|
||||
{
|
||||
next_element_on_same_db_and_table = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (!next_element_on_same_db_and_table)
|
||||
{
|
||||
settings.ostr << " ";
|
||||
formatONClause(element, settings);
|
||||
}
|
||||
}
|
||||
|
||||
if (no_output)
|
||||
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "USAGE ON " << (settings.hilite ? IAST::hilite_none : "") << "*.*";
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
String ASTCheckGrantQuery::getID(char) const
|
||||
{
|
||||
return "CheckGrantQuery";
|
||||
@ -113,8 +26,7 @@ void ASTCheckGrantQuery::formatImpl(const FormatSettings & settings, FormatState
|
||||
<< (settings.hilite ? IAST::hilite_none : "");
|
||||
|
||||
settings.ostr << " ";
|
||||
|
||||
formatElementsWithoutOptions(access_rights_elements, settings);
|
||||
access_rights_elements.formatElementsWithoutOptions(settings.ostr, settings.hilite);
|
||||
}
|
||||
|
||||
|
||||
|
@ -2,13 +2,10 @@
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Access/Common/AccessRightsElement.h>
|
||||
#include <Parsers/ASTQueryWithOnCluster.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTRolesOrUsersSet;
|
||||
|
||||
|
||||
/** Parses queries like
|
||||
* CHECK GRANT access_type[(column_name [,...])] [,...] ON {db.table|db.*|*.*|table|*}
|
||||
@ -24,4 +21,5 @@ public:
|
||||
void replaceEmptyDatabase(const String & current_database);
|
||||
QueryKind getQueryKind() const override { return QueryKind::Check; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <Parsers/Access/ASTGrantQuery.h>
|
||||
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
@ -13,52 +12,10 @@ namespace ErrorCodes
|
||||
|
||||
namespace
|
||||
{
|
||||
void formatElementsWithoutOptions(const AccessRightsElements & elements, const IAST::FormatSettings & settings)
|
||||
{
|
||||
bool no_output = true;
|
||||
for (size_t i = 0; i != elements.size(); ++i)
|
||||
{
|
||||
const auto & element = elements[i];
|
||||
auto keywords = element.access_flags.toKeywords();
|
||||
if (keywords.empty() || (!element.anyColumn() && element.columns.empty()))
|
||||
continue;
|
||||
|
||||
for (const auto & keyword : keywords)
|
||||
{
|
||||
if (!std::exchange(no_output, false))
|
||||
settings.ostr << ", ";
|
||||
|
||||
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << keyword << (settings.hilite ? IAST::hilite_none : "");
|
||||
if (!element.anyColumn())
|
||||
element.formatColumnNames(settings.ostr);
|
||||
}
|
||||
|
||||
bool next_element_on_same_db_and_table = false;
|
||||
if (i != elements.size() - 1)
|
||||
{
|
||||
const auto & next_element = elements[i + 1];
|
||||
if (element.sameDatabaseAndTableAndParameter(next_element))
|
||||
{
|
||||
next_element_on_same_db_and_table = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (!next_element_on_same_db_and_table)
|
||||
{
|
||||
settings.ostr << " ";
|
||||
element.formatONClause(settings.ostr, settings.hilite);
|
||||
}
|
||||
}
|
||||
|
||||
if (no_output)
|
||||
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "USAGE ON " << (settings.hilite ? IAST::hilite_none : "") << "*.*";
|
||||
}
|
||||
|
||||
|
||||
void formatCurrentGrantsElements(const AccessRightsElements & elements, const IAST::FormatSettings & settings)
|
||||
{
|
||||
settings.ostr << "(";
|
||||
formatElementsWithoutOptions(elements, settings);
|
||||
elements.formatElementsWithoutOptions(settings.ostr, settings.hilite);
|
||||
settings.ostr << ")";
|
||||
}
|
||||
}
|
||||
@ -122,7 +79,7 @@ void ASTGrantQuery::formatImpl(const FormatSettings & settings, FormatState &, F
|
||||
}
|
||||
else
|
||||
{
|
||||
formatElementsWithoutOptions(access_rights_elements, settings);
|
||||
access_rights_elements.formatElementsWithoutOptions(settings.ostr, settings.hilite);
|
||||
}
|
||||
|
||||
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << (is_revoke ? " FROM " : " TO ")
|
||||
|
@ -1,219 +1,24 @@
|
||||
#include <Parsers/ASTIdentifier_fwd.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/Access/ASTCheckGrantQuery.h>
|
||||
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
||||
#include <Parsers/Access/ParserCheckGrantQuery.h>
|
||||
#include <Parsers/Access/ParserRolesOrUsersSet.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/parseDatabaseAndTableName.h>
|
||||
#include <boost/algorithm/string/predicate.hpp>
|
||||
#include <boost/range/algorithm_ext/erase.hpp>
|
||||
|
||||
#include <Access/Common/AccessRightsElement.h>
|
||||
#include <Parsers/Access/ASTCheckGrantQuery.h>
|
||||
#include <Parsers/Access/parseAccessRightsElements.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INVALID_GRANT;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
bool parseAccessFlags(IParser::Pos & pos, Expected & expected, AccessFlags & access_flags)
|
||||
{
|
||||
static constexpr auto is_one_of_access_type_words = [](IParser::Pos & pos_)
|
||||
{
|
||||
if (pos_->type != TokenType::BareWord)
|
||||
return false;
|
||||
std::string_view word{pos_->begin, pos_->size()};
|
||||
return !(boost::iequals(word, toStringView(Keyword::ON)) || boost::iequals(word, toStringView(Keyword::TO)) || boost::iequals(word, toStringView(Keyword::FROM)));
|
||||
};
|
||||
|
||||
expected.add(pos, "access type");
|
||||
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
if (!is_one_of_access_type_words(pos))
|
||||
return false;
|
||||
|
||||
String str;
|
||||
do
|
||||
{
|
||||
if (!str.empty())
|
||||
str += " ";
|
||||
str += std::string_view(pos->begin, pos->size());
|
||||
++pos;
|
||||
}
|
||||
while (is_one_of_access_type_words(pos));
|
||||
|
||||
try
|
||||
{
|
||||
access_flags = AccessFlags{str};
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
bool parseColumnNames(IParser::Pos & pos, Expected & expected, Strings & columns)
|
||||
{
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
if (!ParserToken{TokenType::OpeningRoundBracket}.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
ASTPtr ast;
|
||||
if (!ParserList{std::make_unique<ParserIdentifier>(), std::make_unique<ParserToken>(TokenType::Comma), false}.parse(pos, ast, expected))
|
||||
return false;
|
||||
|
||||
Strings res_columns;
|
||||
for (const auto & child : ast->children)
|
||||
res_columns.emplace_back(getIdentifierName(child));
|
||||
|
||||
if (!ParserToken{TokenType::ClosingRoundBracket}.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
columns = std::move(res_columns);
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
bool parseAccessFlagsWithColumns(IParser::Pos & pos, Expected & expected,
|
||||
std::vector<std::pair<AccessFlags, Strings>> & access_and_columns)
|
||||
{
|
||||
std::vector<std::pair<AccessFlags, Strings>> res;
|
||||
|
||||
auto parse_access_and_columns = [&]
|
||||
{
|
||||
AccessFlags access_flags;
|
||||
if (!parseAccessFlags(pos, expected, access_flags))
|
||||
return false;
|
||||
|
||||
Strings columns;
|
||||
parseColumnNames(pos, expected, columns);
|
||||
res.emplace_back(access_flags, std::move(columns));
|
||||
return true;
|
||||
};
|
||||
|
||||
if (!ParserList::parseUtil(pos, expected, parse_access_and_columns, false))
|
||||
return false;
|
||||
|
||||
access_and_columns = std::move(res);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool parseElements(IParser::Pos & pos, Expected & expected, AccessRightsElements & elements)
|
||||
{
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
AccessRightsElements res_elements;
|
||||
|
||||
auto parse_around_on = [&]
|
||||
{
|
||||
std::vector<std::pair<AccessFlags, Strings>> access_and_columns;
|
||||
if (!parseAccessFlagsWithColumns(pos, expected, access_and_columns))
|
||||
return false;
|
||||
|
||||
String database_name, table_name, parameter;
|
||||
|
||||
size_t is_global_with_parameter = 0;
|
||||
for (const auto & elem : access_and_columns)
|
||||
{
|
||||
if (elem.first.isGlobalWithParameter())
|
||||
++is_global_with_parameter;
|
||||
}
|
||||
|
||||
if (!ParserKeyword{Keyword::ON}.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
bool wildcard = false;
|
||||
bool default_database = false;
|
||||
if (is_global_with_parameter && is_global_with_parameter == access_and_columns.size())
|
||||
{
|
||||
ASTPtr parameter_ast;
|
||||
if (!ParserToken{TokenType::Asterisk}.ignore(pos, expected))
|
||||
{
|
||||
if (ParserIdentifier{}.parse(pos, parameter_ast, expected))
|
||||
parameter = getIdentifierName(parameter_ast);
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
if (ParserToken{TokenType::Asterisk}.ignore(pos, expected))
|
||||
wildcard = true;
|
||||
}
|
||||
else if (!parseDatabaseAndTableNameOrAsterisks(pos, expected, database_name, table_name, wildcard, default_database))
|
||||
return false;
|
||||
|
||||
for (auto & [access_flags, columns] : access_and_columns)
|
||||
{
|
||||
AccessRightsElement element;
|
||||
element.access_flags = access_flags;
|
||||
element.columns = std::move(columns);
|
||||
element.database = database_name;
|
||||
element.table = table_name;
|
||||
element.parameter = parameter;
|
||||
element.wildcard = wildcard;
|
||||
element.default_database = default_database;
|
||||
res_elements.emplace_back(std::move(element));
|
||||
}
|
||||
|
||||
return true;
|
||||
};
|
||||
|
||||
if (!ParserList::parseUtil(pos, expected, parse_around_on, false))
|
||||
return false;
|
||||
|
||||
elements = std::move(res_elements);
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
void throwIfNotGrantable(AccessRightsElements & elements)
|
||||
{
|
||||
std::erase_if(elements, [](AccessRightsElement & element)
|
||||
{
|
||||
if (element.empty())
|
||||
return true;
|
||||
auto old_flags = element.access_flags;
|
||||
element.eraseNonGrantable();
|
||||
if (!element.empty())
|
||||
return false;
|
||||
|
||||
if (!element.anyColumn())
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot check grant on the column level", old_flags.toString());
|
||||
else if (!element.anyTable())
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot check grant on the table level", old_flags.toString());
|
||||
else if (!element.anyDatabase())
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot check grant on the database level", old_flags.toString());
|
||||
else if (!element.anyParameter())
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot check grant on the global with parameter level", old_flags.toString());
|
||||
else
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot check grant", old_flags.toString());
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
bool ParserCheckGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
if (!ParserKeyword{Keyword::CHECK_GRANT}.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
|
||||
AccessRightsElements elements;
|
||||
|
||||
if (!parseElements(pos, expected, elements))
|
||||
if (!parseAccessRightsElementsWithoutOptions(pos, expected, elements))
|
||||
return false;
|
||||
|
||||
throwIfNotGrantable(elements);
|
||||
elements.throwIfNotGrantable();
|
||||
|
||||
auto query = std::make_shared<ASTCheckGrantQuery>();
|
||||
node = query;
|
||||
|
@ -1,190 +1,29 @@
|
||||
#include <Parsers/ASTIdentifier_fwd.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/Access/ParserGrantQuery.h>
|
||||
|
||||
#include <Access/Common/AccessRightsElement.h>
|
||||
#include <Parsers/ASTQueryWithOnCluster.h>
|
||||
#include <Parsers/Access/ASTGrantQuery.h>
|
||||
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
||||
#include <Parsers/Access/ParserGrantQuery.h>
|
||||
#include <Parsers/Access/ParserRolesOrUsersSet.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/Access/parseAccessRightsElements.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/parseDatabaseAndTableName.h>
|
||||
#include <boost/algorithm/string/predicate.hpp>
|
||||
#include <boost/range/algorithm_ext/erase.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INVALID_GRANT;
|
||||
extern const int SYNTAX_ERROR;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
bool parseAccessFlags(IParser::Pos & pos, Expected & expected, AccessFlags & access_flags)
|
||||
{
|
||||
static constexpr auto is_one_of_access_type_words = [](IParser::Pos & pos_)
|
||||
{
|
||||
if (pos_->type != TokenType::BareWord)
|
||||
return false;
|
||||
std::string_view word{pos_->begin, pos_->size()};
|
||||
return !(boost::iequals(word, toStringView(Keyword::ON)) || boost::iequals(word, toStringView(Keyword::TO)) || boost::iequals(word, toStringView(Keyword::FROM)));
|
||||
};
|
||||
|
||||
expected.add(pos, "access type");
|
||||
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
if (!is_one_of_access_type_words(pos))
|
||||
return false;
|
||||
|
||||
String str;
|
||||
do
|
||||
{
|
||||
if (!str.empty())
|
||||
str += " ";
|
||||
str += std::string_view(pos->begin, pos->size());
|
||||
++pos;
|
||||
}
|
||||
while (is_one_of_access_type_words(pos));
|
||||
|
||||
try
|
||||
{
|
||||
access_flags = AccessFlags{str};
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
bool parseColumnNames(IParser::Pos & pos, Expected & expected, Strings & columns)
|
||||
{
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
if (!ParserToken{TokenType::OpeningRoundBracket}.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
ASTPtr ast;
|
||||
if (!ParserList{std::make_unique<ParserIdentifier>(), std::make_unique<ParserToken>(TokenType::Comma), false}.parse(pos, ast, expected))
|
||||
return false;
|
||||
|
||||
Strings res_columns;
|
||||
for (const auto & child : ast->children)
|
||||
res_columns.emplace_back(getIdentifierName(child));
|
||||
|
||||
if (!ParserToken{TokenType::ClosingRoundBracket}.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
columns = std::move(res_columns);
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
bool parseAccessFlagsWithColumns(IParser::Pos & pos, Expected & expected,
|
||||
std::vector<std::pair<AccessFlags, Strings>> & access_and_columns)
|
||||
{
|
||||
std::vector<std::pair<AccessFlags, Strings>> res;
|
||||
|
||||
auto parse_access_and_columns = [&]
|
||||
{
|
||||
AccessFlags access_flags;
|
||||
if (!parseAccessFlags(pos, expected, access_flags))
|
||||
return false;
|
||||
|
||||
Strings columns;
|
||||
parseColumnNames(pos, expected, columns);
|
||||
res.emplace_back(access_flags, std::move(columns));
|
||||
return true;
|
||||
};
|
||||
|
||||
if (!ParserList::parseUtil(pos, expected, parse_access_and_columns, false))
|
||||
return false;
|
||||
|
||||
access_and_columns = std::move(res);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool parseElementsWithoutOptions(IParser::Pos & pos, Expected & expected, AccessRightsElements & elements)
|
||||
{
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
AccessRightsElements res_elements;
|
||||
|
||||
auto parse_around_on = [&]
|
||||
{
|
||||
std::vector<std::pair<AccessFlags, Strings>> access_and_columns;
|
||||
if (!parseAccessFlagsWithColumns(pos, expected, access_and_columns))
|
||||
return false;
|
||||
|
||||
String database_name, table_name, parameter;
|
||||
|
||||
size_t is_global_with_parameter = 0;
|
||||
for (const auto & elem : access_and_columns)
|
||||
{
|
||||
if (elem.first.isGlobalWithParameter())
|
||||
++is_global_with_parameter;
|
||||
}
|
||||
|
||||
if (!ParserKeyword{Keyword::ON}.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
bool wildcard = false;
|
||||
bool default_database = false;
|
||||
if (is_global_with_parameter && is_global_with_parameter == access_and_columns.size())
|
||||
{
|
||||
ASTPtr parameter_ast;
|
||||
if (!ParserToken{TokenType::Asterisk}.ignore(pos, expected))
|
||||
{
|
||||
if (ParserIdentifier{}.parse(pos, parameter_ast, expected))
|
||||
parameter = getIdentifierName(parameter_ast);
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
if (ParserToken{TokenType::Asterisk}.ignore(pos, expected))
|
||||
wildcard = true;
|
||||
}
|
||||
else if (!parseDatabaseAndTableNameOrAsterisks(pos, expected, database_name, table_name, wildcard, default_database))
|
||||
return false;
|
||||
|
||||
for (auto & [access_flags, columns] : access_and_columns)
|
||||
{
|
||||
if (wildcard && !columns.empty())
|
||||
return false;
|
||||
|
||||
AccessRightsElement element;
|
||||
element.access_flags = access_flags;
|
||||
element.columns = std::move(columns);
|
||||
element.database = database_name;
|
||||
element.table = table_name;
|
||||
element.parameter = parameter;
|
||||
element.wildcard = wildcard;
|
||||
element.default_database = default_database;
|
||||
res_elements.emplace_back(std::move(element));
|
||||
}
|
||||
|
||||
return true;
|
||||
};
|
||||
|
||||
if (!ParserList::parseUtil(pos, expected, parse_around_on, false))
|
||||
return false;
|
||||
|
||||
elements = std::move(res_elements);
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
bool parseCurrentGrants(IParser::Pos & pos, Expected & expected, AccessRightsElements & elements)
|
||||
{
|
||||
if (ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected))
|
||||
{
|
||||
if (!parseElementsWithoutOptions(pos, expected, elements))
|
||||
if (!parseAccessRightsElementsWithoutOptions(pos, expected, elements))
|
||||
return false;
|
||||
|
||||
if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected))
|
||||
@ -214,30 +53,6 @@ namespace
|
||||
return true;
|
||||
}
|
||||
|
||||
void throwIfNotGrantable(AccessRightsElements & elements)
|
||||
{
|
||||
std::erase_if(elements, [](AccessRightsElement & element)
|
||||
{
|
||||
if (element.empty())
|
||||
return true;
|
||||
auto old_flags = element.access_flags;
|
||||
element.eraseNonGrantable();
|
||||
if (!element.empty())
|
||||
return false;
|
||||
|
||||
if (!element.anyColumn())
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the column level", old_flags.toString());
|
||||
if (!element.anyTable())
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the table level", old_flags.toString());
|
||||
if (!element.anyDatabase())
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the database level", old_flags.toString());
|
||||
if (!element.anyParameter())
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted on the global with parameter level", old_flags.toString());
|
||||
|
||||
throw Exception(ErrorCodes::INVALID_GRANT, "{} cannot be granted", old_flags.toString());
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
bool parseRoles(IParser::Pos & pos, Expected & expected, bool is_revoke, bool id_mode, std::shared_ptr<ASTRolesOrUsersSet> & roles)
|
||||
{
|
||||
@ -323,7 +138,7 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!parseElementsWithoutOptions(pos, expected, elements) && !parseRoles(pos, expected, is_revoke, attach_mode, roles))
|
||||
if (!parseAccessRightsElementsWithoutOptions(pos, expected, elements) && !parseRoles(pos, expected, is_revoke, attach_mode, roles))
|
||||
return false;
|
||||
}
|
||||
|
||||
@ -373,13 +188,8 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
replace_access = true;
|
||||
}
|
||||
|
||||
if (!is_revoke)
|
||||
{
|
||||
if (attach_mode)
|
||||
elements.eraseNonGrantable();
|
||||
else
|
||||
throwIfNotGrantable(elements);
|
||||
}
|
||||
if (!is_revoke && !attach_mode)
|
||||
elements.throwIfNotGrantable();
|
||||
|
||||
auto query = std::make_shared<ASTGrantQuery>();
|
||||
node = query;
|
||||
|
178
src/Parsers/Access/parseAccessRightsElements.cpp
Normal file
178
src/Parsers/Access/parseAccessRightsElements.cpp
Normal file
@ -0,0 +1,178 @@
|
||||
#include <Parsers/Access/parseAccessRightsElements.h>
|
||||
|
||||
#include <Access/Common/AccessRightsElement.h>
|
||||
#include <Parsers/ASTIdentifier_fwd.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
#include <Parsers/parseDatabaseAndTableName.h>
|
||||
|
||||
#include <boost/algorithm/string/predicate.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
bool parseColumnNames(IParser::Pos & pos, Expected & expected, Strings & columns)
|
||||
{
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
if (!ParserToken{TokenType::OpeningRoundBracket}.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
ASTPtr ast;
|
||||
if (!ParserList{std::make_unique<ParserIdentifier>(), std::make_unique<ParserToken>(TokenType::Comma), false}.parse(pos, ast, expected))
|
||||
return false;
|
||||
|
||||
Strings res_columns;
|
||||
for (const auto & child : ast->children)
|
||||
res_columns.emplace_back(getIdentifierName(child));
|
||||
|
||||
if (!ParserToken{TokenType::ClosingRoundBracket}.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
columns = std::move(res_columns);
|
||||
return true;
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
bool parseAccessFlags(IParser::Pos & pos, Expected & expected, AccessFlags & access_flags)
|
||||
{
|
||||
static constexpr auto is_one_of_access_type_words = [](IParser::Pos & pos_)
|
||||
{
|
||||
if (pos_->type != TokenType::BareWord)
|
||||
return false;
|
||||
std::string_view word{pos_->begin, pos_->size()};
|
||||
return !(boost::iequals(word, toStringView(Keyword::ON)) || boost::iequals(word, toStringView(Keyword::TO)) || boost::iequals(word, toStringView(Keyword::FROM)));
|
||||
};
|
||||
|
||||
expected.add(pos, "access type");
|
||||
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
if (!is_one_of_access_type_words(pos))
|
||||
return false;
|
||||
|
||||
String str;
|
||||
do
|
||||
{
|
||||
if (!str.empty())
|
||||
str += " ";
|
||||
str += std::string_view(pos->begin, pos->size());
|
||||
++pos;
|
||||
}
|
||||
while (is_one_of_access_type_words(pos));
|
||||
|
||||
try
|
||||
{
|
||||
access_flags = AccessFlags{str};
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
bool parseAccessFlagsWithColumns(IParser::Pos & pos, Expected & expected,
|
||||
std::vector<std::pair<AccessFlags, Strings>> & access_and_columns)
|
||||
{
|
||||
std::vector<std::pair<AccessFlags, Strings>> res;
|
||||
|
||||
auto parse_access_and_columns = [&]
|
||||
{
|
||||
AccessFlags access_flags;
|
||||
if (!parseAccessFlags(pos, expected, access_flags))
|
||||
return false;
|
||||
|
||||
Strings columns;
|
||||
parseColumnNames(pos, expected, columns);
|
||||
res.emplace_back(access_flags, std::move(columns));
|
||||
return true;
|
||||
};
|
||||
|
||||
if (!ParserList::parseUtil(pos, expected, parse_access_and_columns, false))
|
||||
return false;
|
||||
|
||||
access_and_columns = std::move(res);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool parseAccessRightsElementsWithoutOptions(IParser::Pos & pos, Expected & expected, AccessRightsElements & elements)
|
||||
{
|
||||
return IParserBase::wrapParseImpl(pos, [&]
|
||||
{
|
||||
AccessRightsElements res_elements;
|
||||
|
||||
auto parse_around_on = [&]
|
||||
{
|
||||
std::vector<std::pair<AccessFlags, Strings>> access_and_columns;
|
||||
if (!parseAccessFlagsWithColumns(pos, expected, access_and_columns))
|
||||
return false;
|
||||
|
||||
String database_name, table_name, parameter;
|
||||
|
||||
size_t is_global_with_parameter = 0;
|
||||
for (const auto & elem : access_and_columns)
|
||||
{
|
||||
if (elem.first.isGlobalWithParameter())
|
||||
++is_global_with_parameter;
|
||||
}
|
||||
|
||||
if (!ParserKeyword{Keyword::ON}.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
bool wildcard = false;
|
||||
bool default_database = false;
|
||||
if (is_global_with_parameter && is_global_with_parameter == access_and_columns.size())
|
||||
{
|
||||
ASTPtr parameter_ast;
|
||||
if (!ParserToken{TokenType::Asterisk}.ignore(pos, expected))
|
||||
{
|
||||
if (ParserIdentifier{}.parse(pos, parameter_ast, expected))
|
||||
parameter = getIdentifierName(parameter_ast);
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
if (ParserToken{TokenType::Asterisk}.ignore(pos, expected))
|
||||
wildcard = true;
|
||||
}
|
||||
else if (!parseDatabaseAndTableNameOrAsterisks(pos, expected, database_name, table_name, wildcard, default_database))
|
||||
return false;
|
||||
|
||||
for (auto & [access_flags, columns] : access_and_columns)
|
||||
{
|
||||
if (wildcard && !columns.empty())
|
||||
return false;
|
||||
|
||||
AccessRightsElement element;
|
||||
element.access_flags = access_flags;
|
||||
element.columns = std::move(columns);
|
||||
element.database = database_name;
|
||||
element.table = table_name;
|
||||
element.parameter = parameter;
|
||||
element.wildcard = wildcard;
|
||||
element.default_database = default_database;
|
||||
res_elements.emplace_back(std::move(element));
|
||||
}
|
||||
|
||||
return true;
|
||||
};
|
||||
|
||||
if (!ParserList::parseUtil(pos, expected, parse_around_on, false))
|
||||
return false;
|
||||
|
||||
elements = std::move(res_elements);
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
}
|
24
src/Parsers/Access/parseAccessRightsElements.h
Normal file
24
src/Parsers/Access/parseAccessRightsElements.h
Normal file
@ -0,0 +1,24 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <Parsers/IParser.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class AccessFlags;
|
||||
class AccessRightsElements;
|
||||
|
||||
/// Parses a list of privileges, for example "SELECT, INSERT".
|
||||
bool parseAccessFlags(IParser::Pos & pos, Expected & expected, AccessFlags & access_flags);
|
||||
|
||||
/// Parses a list of privileges which can be written with lists of columns.
|
||||
/// For example "SELECT(a), INSERT(b, c), DROP".
|
||||
bool parseAccessFlagsWithColumns(IParser::Pos & pos, Expected & expected,
|
||||
std::vector<std::pair<AccessFlags, Strings>> & access_and_columns);
|
||||
|
||||
/// Parses a list of privileges with columns and tables or databases or wildcards,
|
||||
/// For examples, "SELECT(a), INSERT(b,c) ON mydb.mytable, DROP ON mydb.*"
|
||||
bool parseAccessRightsElementsWithoutOptions(IParser::Pos & pos, Expected & expected, AccessRightsElements & elements);
|
||||
|
||||
}
|
@ -1,35 +1,42 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-parallel
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
db=${CLICKHOUSE_DATABASE}
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "DROP USER IF EXISTS user_03234; DROP TABLE IF EXISTS ${db}.tb;CREATE USER user_03234; GRANT SELECT ON ${db}.tb TO user_03234;"
|
||||
db=${CLICKHOUSE_DATABASE}
|
||||
user=user_${CLICKHOUSE_TEST_UNIQUE_NAME}
|
||||
role=role_${CLICKHOUSE_TEST_UNIQUE_NAME}
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "DROP ROLE IF EXISTS $role; DROP USER IF EXISTS $user; DROP TABLE IF EXISTS ${db}.tb;"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "CREATE USER $user; GRANT SELECT ON ${db}.tb TO $user;"
|
||||
|
||||
# Has been granted but not table not exists
|
||||
# expected to 1
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=user_03234" --data-binary "CHECK GRANT SELECT ON ${db}.tb"
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=$user" --data-binary "CHECK GRANT SELECT ON ${db}.tb"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "CREATE TABLE ${db}.tb (\`content\` UInt64) ENGINE = MergeTree ORDER BY content; INSERT INTO ${db}.tb VALUES (1);"
|
||||
# Has been granted and table exists
|
||||
# expected to 1
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=user_03234" --data-binary "CHECK GRANT SELECT ON ${db}.tb"
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=$user" --data-binary "CHECK GRANT SELECT ON ${db}.tb"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "REVOKE SELECT ON ${db}.tb FROM user_03234;"
|
||||
${CLICKHOUSE_CLIENT} --query "REVOKE SELECT ON ${db}.tb FROM $user;"
|
||||
# Has not been granted but table exists
|
||||
# expected to 0
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=user_03234" --data-binary "CHECK GRANT SELECT ON ${db}.tb"
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=$user" --data-binary "CHECK GRANT SELECT ON ${db}.tb"
|
||||
|
||||
# Role
|
||||
# expected to 1
|
||||
${CLICKHOUSE_CLIENT} --query "DROP ROLE IF EXISTS role_03234;CREATE ROLE role_03234;GRANT SELECT ON ${db}.tb TO role_03234;GRANT role_03234 TO user_03234"
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=user_03234" --data-binary "SET ROLE role_03234"
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=user_03234" --data-binary "CHECK GRANT SELECT ON ${db}.tb"
|
||||
${CLICKHOUSE_CLIENT} --query "CREATE ROLE $role; GRANT SELECT ON ${db}.tb TO $role; GRANT $role TO $user"
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=$user" --data-binary "SET ROLE $role"
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=$user" --data-binary "CHECK GRANT SELECT ON ${db}.tb"
|
||||
|
||||
# wildcard
|
||||
${CLICKHOUSE_CLIENT} --query "GRANT SELECT ON ${db}.tbb* TO user_03234;"
|
||||
${CLICKHOUSE_CLIENT} --query "GRANT SELECT ON ${db}.tbb* TO $user;"
|
||||
# expected to 1
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=user_03234" --data-binary "CHECK GRANT SELECT ON ${db}.tbb1"
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=$user" --data-binary "CHECK GRANT SELECT ON ${db}.tbb1"
|
||||
# expected to 1
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=user_03234" --data-binary "CHECK GRANT SELECT ON ${db}.tbb2*"
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=$user" --data-binary "CHECK GRANT SELECT ON ${db}.tbb2*"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "DROP ROLE $role; DROP USER $user; DROP TABLE $db.tb;"
|
||||
|
Loading…
Reference in New Issue
Block a user