mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Add system table for access types.
This commit is contained in:
parent
a14f322723
commit
b1f362b2dc
@ -19,6 +19,7 @@
|
||||
#include <Parsers/ASTShowAccessEntitiesQuery.h>
|
||||
#include <Parsers/ASTShowCreateAccessEntityQuery.h>
|
||||
#include <Parsers/ASTShowGrantsQuery.h>
|
||||
#include <Parsers/ASTShowPrivilegesQuery.h>
|
||||
#include <Parsers/ASTShowProcesslistQuery.h>
|
||||
#include <Parsers/ASTShowTablesQuery.h>
|
||||
#include <Parsers/ASTUseQuery.h>
|
||||
@ -52,6 +53,7 @@
|
||||
#include <Interpreters/InterpreterShowAccessEntitiesQuery.h>
|
||||
#include <Interpreters/InterpreterShowCreateAccessEntityQuery.h>
|
||||
#include <Interpreters/InterpreterShowGrantsQuery.h>
|
||||
#include <Interpreters/InterpreterShowPrivilegesQuery.h>
|
||||
#include <Interpreters/InterpreterShowCreateQuery.h>
|
||||
#include <Interpreters/InterpreterShowProcesslistQuery.h>
|
||||
#include <Interpreters/InterpreterShowTablesQuery.h>
|
||||
@ -228,6 +230,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context &
|
||||
{
|
||||
return std::make_unique<InterpreterShowAccessEntitiesQuery>(query, context);
|
||||
}
|
||||
else if (query->as<ASTShowPrivilegesQuery>())
|
||||
{
|
||||
return std::make_unique<InterpreterShowPrivilegesQuery>(query, context);
|
||||
}
|
||||
else
|
||||
throw Exception("Unknown type of query: " + query->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY);
|
||||
}
|
||||
|
18
src/Interpreters/InterpreterShowPrivilegesQuery.cpp
Normal file
18
src/Interpreters/InterpreterShowPrivilegesQuery.cpp
Normal file
@ -0,0 +1,18 @@
|
||||
#include <Interpreters/InterpreterShowPrivilegesQuery.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
InterpreterShowPrivilegesQuery::InterpreterShowPrivilegesQuery(const ASTPtr & query_ptr_, Context & context_)
|
||||
: query_ptr(query_ptr_), context(context_)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
BlockIO InterpreterShowPrivilegesQuery::execute()
|
||||
{
|
||||
return executeQuery("SELECT * FROM system.privileges", context, true);
|
||||
}
|
||||
|
||||
}
|
26
src/Interpreters/InterpreterShowPrivilegesQuery.h
Normal file
26
src/Interpreters/InterpreterShowPrivilegesQuery.h
Normal file
@ -0,0 +1,26 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class Context;
|
||||
|
||||
class InterpreterShowPrivilegesQuery : public IInterpreter
|
||||
{
|
||||
public:
|
||||
InterpreterShowPrivilegesQuery(const ASTPtr & query_ptr_, Context & context_);
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
bool ignoreQuota() const override { return true; }
|
||||
bool ignoreLimits() const override { return true; }
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context & context;
|
||||
};
|
||||
|
||||
}
|
@ -87,6 +87,7 @@ SRCS(
|
||||
InterpreterShowCreateAccessEntityQuery.cpp
|
||||
InterpreterShowCreateQuery.cpp
|
||||
InterpreterShowGrantsQuery.cpp
|
||||
InterpreterShowPrivilegesQuery.cpp
|
||||
InterpreterShowProcesslistQuery.cpp
|
||||
InterpreterShowTablesQuery.cpp
|
||||
InterpreterSystemQuery.cpp
|
||||
|
0
src/Parsers/ASTShowPrivilegesQuery.cpp
Normal file
0
src/Parsers/ASTShowPrivilegesQuery.cpp
Normal file
17
src/Parsers/ASTShowPrivilegesQuery.h
Normal file
17
src/Parsers/ASTShowPrivilegesQuery.h
Normal file
@ -0,0 +1,17 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/ASTQueryWithOutput.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct ASTShowPrivilegesIDAndQueryName
|
||||
{
|
||||
static constexpr auto ID = "ShowPrivilegesQuery";
|
||||
static constexpr auto Query = "SHOW PRIVILEGES";
|
||||
};
|
||||
|
||||
using ASTShowPrivilegesQuery = ASTQueryWithOutputImpl<ASTShowPrivilegesIDAndQueryName>;
|
||||
|
||||
}
|
@ -17,6 +17,7 @@
|
||||
#include <Parsers/ParserShowAccessEntitiesQuery.h>
|
||||
#include <Parsers/ParserShowCreateAccessEntityQuery.h>
|
||||
#include <Parsers/ParserShowGrantsQuery.h>
|
||||
#include <Parsers/ParserShowPrivilegesQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -40,6 +41,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
ParserShowAccessEntitiesQuery show_access_entities_p;
|
||||
ParserShowCreateAccessEntityQuery show_create_access_entity_p;
|
||||
ParserShowGrantsQuery show_grants_p;
|
||||
ParserShowPrivilegesQuery show_privileges_p;
|
||||
|
||||
ASTPtr query;
|
||||
|
||||
@ -69,7 +71,8 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
|| optimize_p.parse(pos, query, expected)
|
||||
|| watch_p.parse(pos, query, expected)
|
||||
|| show_access_entities_p.parse(pos, query, expected)
|
||||
|| show_grants_p.parse(pos, query, expected);
|
||||
|| show_grants_p.parse(pos, query, expected)
|
||||
|| show_privileges_p.parse(pos, query, expected);
|
||||
|
||||
if (!parsed)
|
||||
return false;
|
||||
|
21
src/Parsers/ParserShowPrivilegesQuery.cpp
Normal file
21
src/Parsers/ParserShowPrivilegesQuery.cpp
Normal file
@ -0,0 +1,21 @@
|
||||
#include <Parsers/ParserShowPrivilegesQuery.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ASTShowPrivilegesQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserShowPrivilegesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
auto query = std::make_shared<ASTShowPrivilegesQuery>();
|
||||
|
||||
if (!ParserKeyword("SHOW PRIVILEGES").ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
node = query;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
18
src/Parsers/ParserShowPrivilegesQuery.h
Normal file
18
src/Parsers/ParserShowPrivilegesQuery.h
Normal file
@ -0,0 +1,18 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Query SHOW PRIVILEGES
|
||||
*/
|
||||
class ParserShowPrivilegesQuery : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override { return "SHOW PRIVILEGES query"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
}
|
@ -45,6 +45,7 @@ SRCS(
|
||||
ASTShowAccessEntitiesQuery.cpp
|
||||
ASTShowCreateAccessEntityQuery.cpp
|
||||
ASTShowGrantsQuery.cpp
|
||||
ASTShowPrivilegesQuery.cpp
|
||||
ASTShowTablesQuery.cpp
|
||||
ASTSubquery.cpp
|
||||
ASTSystemQuery.cpp
|
||||
@ -96,6 +97,7 @@ SRCS(
|
||||
ParserShowAccessEntitiesQuery.cpp
|
||||
ParserShowCreateAccessEntityQuery.cpp
|
||||
ParserShowGrantsQuery.cpp
|
||||
ParserShowPrivilegesQuery.cpp
|
||||
ParserShowTablesQuery.cpp
|
||||
ParserSystemQuery.cpp
|
||||
ParserTablePropertiesQuery.cpp
|
||||
|
137
src/Storages/System/StorageSystemPrivileges.cpp
Normal file
137
src/Storages/System/StorageSystemPrivileges.cpp
Normal file
@ -0,0 +1,137 @@
|
||||
#include <Storages/System/StorageSystemPrivileges.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTExtendedRoleSet.h>
|
||||
#include <Access/AccessControlManager.h>
|
||||
#include <Access/SettingsProfile.h>
|
||||
#include <Access/AccessFlags.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace
|
||||
{
|
||||
enum Level
|
||||
{
|
||||
GROUP = -1,
|
||||
GLOBAL,
|
||||
DATABASE,
|
||||
TABLE,
|
||||
DICTIONARY,
|
||||
VIEW,
|
||||
COLUMN,
|
||||
};
|
||||
|
||||
DataTypeEnum8::Values getLevelEnumValues()
|
||||
{
|
||||
DataTypeEnum8::Values enum_values;
|
||||
enum_values.emplace_back("GLOBAL", static_cast<Int8>(GLOBAL));
|
||||
enum_values.emplace_back("DATABASE", static_cast<Int8>(DATABASE));
|
||||
enum_values.emplace_back("TABLE", static_cast<Int8>(TABLE));
|
||||
enum_values.emplace_back("DICTIONARY", static_cast<Int8>(DICTIONARY));
|
||||
enum_values.emplace_back("VIEW", static_cast<Int8>(VIEW));
|
||||
enum_values.emplace_back("COLUMN", static_cast<Int8>(COLUMN));
|
||||
return enum_values;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
const std::vector<std::pair<String, Int8>> & StorageSystemPrivileges::getAccessTypeEnumValues()
|
||||
{
|
||||
static const std::vector<std::pair<String, Int8>> values = []
|
||||
{
|
||||
std::vector<std::pair<String, Int8>> res;
|
||||
|
||||
#define ADD_ACCESS_TYPE_ENUM_VALUE(name, aliases, node_type, parent_group_name) \
|
||||
res.emplace_back(toString(AccessType::name), static_cast<size_t>(AccessType::name));
|
||||
|
||||
APPLY_FOR_ACCESS_TYPES(ADD_ACCESS_TYPE_ENUM_VALUE)
|
||||
#undef ADD_ACCESS_TYPE_ENUM_VALUE
|
||||
|
||||
return res;
|
||||
}();
|
||||
return values;
|
||||
}
|
||||
|
||||
|
||||
NamesAndTypesList StorageSystemPrivileges::getNamesAndTypes()
|
||||
{
|
||||
NamesAndTypesList names_and_types{
|
||||
{"privilege", std::make_shared<DataTypeEnum8>(getAccessTypeEnumValues())},
|
||||
{"aliases", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
{"level", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeEnum8>(getLevelEnumValues()))},
|
||||
{"parent_group", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeEnum8>(getAccessTypeEnumValues()))},
|
||||
};
|
||||
return names_and_types;
|
||||
}
|
||||
|
||||
|
||||
void StorageSystemPrivileges::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const
|
||||
{
|
||||
size_t column_index = 0;
|
||||
auto & column_access_type = assert_cast<ColumnInt8 &>(*res_columns[column_index++]).getData();
|
||||
auto & column_aliases = assert_cast<ColumnString &>(assert_cast<ColumnArray &>(*res_columns[column_index]).getData());
|
||||
auto & column_aliases_offsets = assert_cast<ColumnArray &>(*res_columns[column_index++]).getOffsets();
|
||||
auto & column_level = assert_cast<ColumnInt8 &>(assert_cast<ColumnNullable &>(*res_columns[column_index]).getNestedColumn()).getData();
|
||||
auto & column_level_null_map = assert_cast<ColumnNullable &>(*res_columns[column_index++]).getNullMapData();
|
||||
auto & column_parent_group = assert_cast<ColumnInt8 &>(assert_cast<ColumnNullable &>(*res_columns[column_index]).getNestedColumn()).getData();
|
||||
auto & column_parent_group_null_map = assert_cast<ColumnNullable &>(*res_columns[column_index++]).getNullMapData();
|
||||
|
||||
auto add_row = [&](AccessType access_type, const std::string_view & aliases, Level max_level, AccessType parent_group)
|
||||
{
|
||||
column_access_type.push_back(static_cast<Int8>(access_type));
|
||||
|
||||
for (size_t pos = 0; pos < aliases.length();)
|
||||
{
|
||||
size_t next_pos = aliases.find_first_of(',', pos);
|
||||
std::string_view alias = aliases.substr(pos, next_pos - pos);
|
||||
pos = ((next_pos == std::string_view::npos) ? next_pos : next_pos + 1);
|
||||
|
||||
while (alias.starts_with(' '))
|
||||
alias.remove_prefix(1);
|
||||
while (alias.ends_with(' '))
|
||||
alias.remove_suffix(1);
|
||||
column_aliases.insertData(alias.data(), alias.length());
|
||||
}
|
||||
column_aliases_offsets.push_back(column_aliases.size());
|
||||
|
||||
if (max_level == GROUP)
|
||||
{
|
||||
column_level.push_back(0);
|
||||
column_level_null_map.push_back(true);
|
||||
}
|
||||
else
|
||||
{
|
||||
column_level.push_back(static_cast<Int8>(max_level));
|
||||
column_level_null_map.push_back(false);
|
||||
}
|
||||
|
||||
if (parent_group == AccessType::NONE)
|
||||
{
|
||||
column_parent_group.push_back(0);
|
||||
column_parent_group_null_map.push_back(true);
|
||||
}
|
||||
else
|
||||
{
|
||||
column_parent_group.push_back(static_cast<Int8>(parent_group));
|
||||
column_parent_group_null_map.push_back(false);
|
||||
}
|
||||
};
|
||||
|
||||
#define STORAGE_SYSTEM_PRIVILEGES_ADD_ROW(name, aliases, node_type, parent_group_name) \
|
||||
add_row(AccessType::name, aliases, node_type, AccessType::parent_group_name);
|
||||
|
||||
APPLY_FOR_ACCESS_TYPES(STORAGE_SYSTEM_PRIVILEGES_ADD_ROW)
|
||||
|
||||
#undef STORAGE_SYSTEM_PRIVILEGES_ADD_ROW
|
||||
}
|
||||
|
||||
}
|
25
src/Storages/System/StorageSystemPrivileges.h
Normal file
25
src/Storages/System/StorageSystemPrivileges.h
Normal file
@ -0,0 +1,25 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
#include <Storages/System/IStorageSystemOneBlock.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class Context;
|
||||
|
||||
/// Implements `privileges` system table, which allows you to get information about access types.
|
||||
class StorageSystemPrivileges final : public ext::shared_ptr_helper<StorageSystemPrivileges>, public IStorageSystemOneBlock<StorageSystemPrivileges>
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "SystemPrivileges"; }
|
||||
static NamesAndTypesList getNamesAndTypes();
|
||||
static const std::vector<std::pair<String, Int8>> & getAccessTypeEnumValues();
|
||||
|
||||
protected:
|
||||
friend struct ext::shared_ptr_helper<StorageSystemPrivileges>;
|
||||
using IStorageSystemOneBlock::IStorageSystemOneBlock;
|
||||
void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const override;
|
||||
};
|
||||
|
||||
}
|
@ -49,6 +49,7 @@
|
||||
#include <Storages/System/StorageSystemQuotaLimits.h>
|
||||
#include <Storages/System/StorageSystemQuotaUsage.h>
|
||||
#include <Storages/System/StorageSystemQuotasUsage.h>
|
||||
#include <Storages/System/StorageSystemPrivileges.h>
|
||||
|
||||
#ifdef OS_LINUX
|
||||
#include <Storages/System/StorageSystemStackTrace.h>
|
||||
@ -88,6 +89,7 @@ void attachSystemTablesLocal(IDatabase & system_database)
|
||||
system_database.attachTable("quota_limits", StorageSystemQuotaLimits::create("quota_limits"));
|
||||
system_database.attachTable("quota_usage", StorageSystemQuotaUsage::create("quota_usage"));
|
||||
system_database.attachTable("quotas_usage", StorageSystemQuotasUsage::create("all_quotas_usage"));
|
||||
system_database.attachTable("privileges", StorageSystemPrivileges::create("privileges"));
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
system_database.attachTable("licenses", StorageSystemLicenses::create("licenses"));
|
||||
#endif
|
||||
|
@ -110,6 +110,7 @@ SRCS(
|
||||
System/StorageSystemParts.cpp
|
||||
System/StorageSystemPartsBase.cpp
|
||||
System/StorageSystemPartsColumns.cpp
|
||||
System/StorageSystemPrivileges.cpp
|
||||
System/StorageSystemProcesses.cpp
|
||||
System/StorageSystemQuotaLimits.cpp
|
||||
System/StorageSystemQuotaUsage.cpp
|
||||
|
113
tests/queries/0_stateless/01271_show_privileges.reference
Normal file
113
tests/queries/0_stateless/01271_show_privileges.reference
Normal file
@ -0,0 +1,113 @@
|
||||
SHOW DATABASES [] DATABASE SHOW
|
||||
SHOW TABLES [] TABLE SHOW
|
||||
SHOW COLUMNS [] COLUMN SHOW
|
||||
SHOW DICTIONARIES [] DICTIONARY SHOW
|
||||
SHOW [] \N ALL
|
||||
SELECT [] COLUMN ALL
|
||||
INSERT [] COLUMN ALL
|
||||
ALTER UPDATE ['UPDATE'] COLUMN ALTER TABLE
|
||||
ALTER DELETE ['DELETE'] COLUMN ALTER TABLE
|
||||
ALTER ADD COLUMN ['ADD COLUMN'] COLUMN ALTER COLUMN
|
||||
ALTER MODIFY COLUMN ['MODIFY COLUMN'] COLUMN ALTER COLUMN
|
||||
ALTER DROP COLUMN ['DROP COLUMN'] COLUMN ALTER COLUMN
|
||||
ALTER COMMENT COLUMN ['COMMENT COLUMN'] COLUMN ALTER COLUMN
|
||||
ALTER CLEAR COLUMN ['CLEAR COLUMN'] COLUMN ALTER COLUMN
|
||||
ALTER RENAME COLUMN ['RENAME COLUMN'] COLUMN ALTER COLUMN
|
||||
ALTER COLUMN [] \N ALTER TABLE
|
||||
ALTER ORDER BY ['ALTER MODIFY ORDER BY','MODIFY ORDER BY'] TABLE ALTER INDEX
|
||||
ALTER ADD INDEX ['ADD INDEX'] TABLE ALTER INDEX
|
||||
ALTER DROP INDEX ['DROP INDEX'] TABLE ALTER INDEX
|
||||
ALTER MATERIALIZE INDEX ['MATERIALIZE INDEX'] TABLE ALTER INDEX
|
||||
ALTER CLEAR INDEX ['CLEAR INDEX'] TABLE ALTER INDEX
|
||||
ALTER INDEX ['INDEX'] \N ALTER TABLE
|
||||
ALTER ADD CONSTRAINT ['ADD CONSTRAINT'] TABLE ALTER CONSTRAINT
|
||||
ALTER DROP CONSTRAINT ['DROP CONSTRAINT'] TABLE ALTER CONSTRAINT
|
||||
ALTER CONSTRAINT ['CONSTRAINT'] \N ALTER TABLE
|
||||
ALTER TTL ['ALTER MODIFY TTL','MODIFY TTL'] TABLE ALTER TABLE
|
||||
ALTER MATERIALIZE TTL ['MATERIALIZE TTL'] TABLE ALTER TABLE
|
||||
ALTER SETTINGS ['ALTER SETTING','ALTER MODIFY SETTING','MODIFY SETTING'] TABLE ALTER TABLE
|
||||
ALTER MOVE PARTITION ['ALTER MOVE PART','MOVE PARTITION','MOVE PART'] TABLE ALTER TABLE
|
||||
ALTER FETCH PARTITION ['FETCH PARTITION'] TABLE ALTER TABLE
|
||||
ALTER FREEZE PARTITION ['FREEZE PARTITION'] TABLE ALTER TABLE
|
||||
ALTER TABLE [] \N ALTER
|
||||
ALTER VIEW REFRESH ['ALTER LIVE VIEW REFRESH','REFRESH VIEW'] VIEW ALTER VIEW
|
||||
ALTER VIEW MODIFY QUERY ['ALTER TABLE MODIFY QUERY'] VIEW ALTER VIEW
|
||||
ALTER VIEW [] \N ALTER
|
||||
ALTER [] \N ALL
|
||||
CREATE DATABASE [] DATABASE CREATE
|
||||
CREATE TABLE [] TABLE CREATE
|
||||
CREATE VIEW [] VIEW CREATE
|
||||
CREATE DICTIONARY [] DICTIONARY CREATE
|
||||
CREATE TEMPORARY TABLE [] GLOBAL CREATE
|
||||
CREATE [] \N ALL
|
||||
DROP DATABASE [] DATABASE DROP
|
||||
DROP TABLE [] TABLE DROP
|
||||
DROP VIEW [] VIEW DROP
|
||||
DROP DICTIONARY [] DICTIONARY DROP
|
||||
DROP [] \N ALL
|
||||
TRUNCATE ['TRUNCATE TABLE'] TABLE ALL
|
||||
OPTIMIZE ['OPTIMIZE TABLE'] TABLE ALL
|
||||
KILL QUERY [] GLOBAL ALL
|
||||
CREATE USER [] GLOBAL ACCESS MANAGEMENT
|
||||
ALTER USER [] GLOBAL ACCESS MANAGEMENT
|
||||
DROP USER [] GLOBAL ACCESS MANAGEMENT
|
||||
CREATE ROLE [] GLOBAL ACCESS MANAGEMENT
|
||||
ALTER ROLE [] GLOBAL ACCESS MANAGEMENT
|
||||
DROP ROLE [] GLOBAL ACCESS MANAGEMENT
|
||||
ROLE ADMIN [] GLOBAL ACCESS MANAGEMENT
|
||||
CREATE ROW POLICY ['CREATE POLICY'] GLOBAL ACCESS MANAGEMENT
|
||||
ALTER ROW POLICY ['ALTER POLICY'] GLOBAL ACCESS MANAGEMENT
|
||||
DROP ROW POLICY ['DROP POLICY'] GLOBAL ACCESS MANAGEMENT
|
||||
CREATE QUOTA [] GLOBAL ACCESS MANAGEMENT
|
||||
ALTER QUOTA [] GLOBAL ACCESS MANAGEMENT
|
||||
DROP QUOTA [] GLOBAL ACCESS MANAGEMENT
|
||||
CREATE SETTINGS PROFILE ['CREATE PROFILE'] GLOBAL ACCESS MANAGEMENT
|
||||
ALTER SETTINGS PROFILE ['ALTER PROFILE'] GLOBAL ACCESS MANAGEMENT
|
||||
DROP SETTINGS PROFILE ['DROP PROFILE'] GLOBAL ACCESS MANAGEMENT
|
||||
SHOW USERS ['SHOW CREATE USER'] GLOBAL SHOW ACCESS
|
||||
SHOW ROLES ['SHOW CREATE ROLE'] GLOBAL SHOW ACCESS
|
||||
SHOW ROW POLICIES ['SHOW POLICIES','SHOW CREATE ROW POLICY','SHOW CREATE POLICY'] GLOBAL SHOW ACCESS
|
||||
SHOW QUOTAS ['SHOW CREATE QUOTA'] GLOBAL SHOW ACCESS
|
||||
SHOW SETTINGS PROFILES ['SHOW PROFILES','SHOW CREATE SETTINGS PROFILE','SHOW CREATE PROFILE'] GLOBAL SHOW ACCESS
|
||||
SHOW ACCESS [] \N ACCESS MANAGEMENT
|
||||
ACCESS MANAGEMENT [] \N ALL
|
||||
SYSTEM SHUTDOWN ['SYSTEM KILL','SHUTDOWN'] GLOBAL SYSTEM
|
||||
SYSTEM DROP DNS CACHE ['SYSTEM DROP DNS','DROP DNS CACHE','DROP DNS'] GLOBAL SYSTEM DROP CACHE
|
||||
SYSTEM DROP MARK CACHE ['SYSTEM DROP MARK','DROP MARK CACHE','DROP MARKS'] GLOBAL SYSTEM DROP CACHE
|
||||
SYSTEM DROP UNCOMPRESSED CACHE ['SYSTEM DROP UNCOMPRESSED','DROP UNCOMPRESSED CACHE','DROP UNCOMPRESSED'] GLOBAL SYSTEM DROP CACHE
|
||||
SYSTEM DROP COMPILED EXPRESSION CACHE ['SYSTEM DROP COMPILED EXPRESSION','DROP COMPILED EXPRESSION CACHE','DROP COMPILED EXPRESSIONS'] GLOBAL SYSTEM DROP CACHE
|
||||
SYSTEM DROP CACHE ['DROP CACHE'] \N SYSTEM
|
||||
SYSTEM RELOAD CONFIG ['RELOAD CONFIG'] GLOBAL SYSTEM RELOAD
|
||||
SYSTEM RELOAD DICTIONARY ['SYSTEM RELOAD DICTIONARIES','RELOAD DICTIONARY','RELOAD DICTIONARIES'] GLOBAL SYSTEM RELOAD
|
||||
SYSTEM RELOAD EMBEDDED DICTIONARIES ['RELOAD EMBEDDED DICTIONARIES'] GLOBAL SYSTEM RELOAD
|
||||
SYSTEM RELOAD [] \N SYSTEM
|
||||
SYSTEM MERGES ['SYSTEM STOP MERGES','SYSTEM START MERGES','STOP_MERGES','START MERGES'] TABLE SYSTEM
|
||||
SYSTEM TTL MERGES ['SYSTEM STOP TTL MERGES','SYSTEM START TTL MERGES','STOP TTL MERGES','START TTL MERGES'] TABLE SYSTEM
|
||||
SYSTEM FETCHES ['SYSTEM STOP FETCHES','SYSTEM START FETCHES','STOP FETCHES','START FETCHES'] TABLE SYSTEM
|
||||
SYSTEM MOVES ['SYSTEM STOP MOVES','SYSTEM START MOVES','STOP MOVES','START MOVES'] TABLE SYSTEM
|
||||
SYSTEM DISTRIBUTED SENDS ['SYSTEM STOP DISTRIBUTED SENDS','SYSTEM START DISTRIBUTED SENDS','STOP DISTRIBUTED SENDS','START DISTRIBUTED SENDS'] TABLE SYSTEM SENDS
|
||||
SYSTEM REPLICATED SENDS ['SYSTEM STOP REPLICATED SENDS','SYSTEM START REPLICATED SENDS','STOP_REPLICATED_SENDS','START REPLICATED SENDS'] TABLE SYSTEM SENDS
|
||||
SYSTEM SENDS ['SYSTEM STOP SENDS','SYSTEM START SENDS','STOP SENDS','START SENDS'] \N SYSTEM
|
||||
SYSTEM REPLICATION QUEUES ['SYSTEM STOP REPLICATION QUEUES','SYSTEM START REPLICATION QUEUES','STOP_REPLICATION_QUEUES','START REPLICATION QUEUES'] TABLE SYSTEM
|
||||
SYSTEM SYNC REPLICA ['SYNC REPLICA'] TABLE SYSTEM
|
||||
SYSTEM RESTART REPLICA ['RESTART REPLICA'] TABLE SYSTEM
|
||||
SYSTEM FLUSH DISTRIBUTED ['FLUSH DISTRIBUTED'] TABLE SYSTEM FLUSH
|
||||
SYSTEM FLUSH LOGS ['FLUSH LOGS'] GLOBAL SYSTEM FLUSH
|
||||
SYSTEM FLUSH [] \N SYSTEM
|
||||
SYSTEM [] \N ALL
|
||||
dictGet ['dictHas','dictGetHierarchy','dictIsIn'] DICTIONARY ALL
|
||||
addressToLine [] GLOBAL INTROSPECTION
|
||||
addressToSymbol [] GLOBAL INTROSPECTION
|
||||
demangle [] GLOBAL INTROSPECTION
|
||||
INTROSPECTION ['INTROSPECTION FUNCTIONS'] \N ALL
|
||||
FILE [] GLOBAL SOURCES
|
||||
URL [] GLOBAL SOURCES
|
||||
REMOTE [] GLOBAL SOURCES
|
||||
MYSQL [] GLOBAL SOURCES
|
||||
ODBC [] GLOBAL SOURCES
|
||||
JDBC [] GLOBAL SOURCES
|
||||
HDFS [] GLOBAL SOURCES
|
||||
S3 [] GLOBAL SOURCES
|
||||
SOURCES [] \N ALL
|
||||
ALL ['ALL PRIVILEGES'] \N \N
|
||||
NONE ['USAGE','NO PRIVILEGES'] \N \N
|
1
tests/queries/0_stateless/01271_show_privileges.sql
Normal file
1
tests/queries/0_stateless/01271_show_privileges.sql
Normal file
@ -0,0 +1 @@
|
||||
SHOW PRIVILEGES;
|
Loading…
Reference in New Issue
Block a user