testflows RBAC tests for views, distributed tables, public tables, and privileges: alter, updated select, updated insert, and show tables

This commit is contained in:
MyroTk 2020-10-15 22:23:49 +02:00
parent 41f1fd8fa0
commit 708fedbcf8
59 changed files with 12433 additions and 867 deletions

View File

@ -2,7 +2,7 @@
import sys import sys
from testflows.core import * from testflows.core import *
append_path(sys.path, "..") append_path(sys.path, "..")
from helpers.cluster import Cluster from helpers.cluster import Cluster
from helpers.argparser import argparser from helpers.argparser import argparser
@ -10,13 +10,13 @@ from helpers.argparser import argparser
@TestFeature @TestFeature
@Name("example") @Name("example")
@ArgumentParser(argparser) @ArgumentParser(argparser)
def regression(self, local, clickhouse_binary_path): def regression(self, local, clickhouse_binary_path, stress=None, parallel=None):
"""Simple example of how you can use TestFlows to test ClickHouse. """Simple example of how you can use TestFlows to test ClickHouse.
""" """
nodes = { nodes = {
"clickhouse": ("clickhouse1",), "clickhouse": ("clickhouse1",),
} }
with Cluster(local, clickhouse_binary_path, nodes=nodes) as cluster: with Cluster(local, clickhouse_binary_path, nodes=nodes) as cluster:
self.context.cluster = cluster self.context.cluster = cluster

View File

@ -1,5 +1,12 @@
import os import os
def onoff(v):
if v in ["yes", "1", "on"]:
return True
elif v in ["no", "0", "off"]:
return False
raise ValueError(f"invalid {v}")
def argparser(parser): def argparser(parser):
"""Default argument parser for regressions. """Default argument parser for regressions.
""" """
@ -10,4 +17,10 @@ def argparser(parser):
parser.add_argument("--clickhouse-binary-path", parser.add_argument("--clickhouse-binary-path",
type=str, dest="clickhouse_binary_path", type=str, dest="clickhouse_binary_path",
help="path to ClickHouse binary, default: /usr/bin/clickhouse", metavar="path", help="path to ClickHouse binary, default: /usr/bin/clickhouse", metavar="path",
default=os.getenv("CLICKHOUSE_TESTS_SERVER_BIN_PATH", "/usr/bin/clickhouse")) default=os.getenv("CLICKHOUSE_TESTS_SERVER_BIN_PATH", "/usr/bin/clickhouse"))
parser.add_argument("--stress", action="store_true", default=False,
help="enable stress testing (might take a long time)")
parser.add_argument("--parallel", type=onoff, default=True, choices=["yes", "no", "on", "off", 0, 1],
help="enable parallelism for tests that support it")

89
tests/testflows/helpers/cluster.py Normal file → Executable file
View File

@ -7,6 +7,7 @@ import tempfile
from testflows.core import * from testflows.core import *
from testflows.asserts import error from testflows.asserts import error
from testflows.connect import Shell from testflows.connect import Shell
from testflows.uexpect import ExpectTimeoutError
class QueryRuntimeException(Exception): class QueryRuntimeException(Exception):
"""Exception during query execution on the server. """Exception during query execution on the server.
@ -78,32 +79,43 @@ class ClickHouseNode(Node):
def query(self, sql, message=None, exitcode=None, steps=True, no_checks=False, def query(self, sql, message=None, exitcode=None, steps=True, no_checks=False,
raise_on_exception=False, step=By, settings=None, *args, **kwargs): raise_on_exception=False, step=By, settings=None, *args, **kwargs):
"""Execute and check query. """Execute and check query.
:param sql: sql query :param sql: sql query
:param message: expected message that should be in the output, default: None :param message: expected message that should be in the output, default: None
:param exitcode: expected exitcode, default: None :param exitcode: expected exitcode, default: None
""" """
settings = list(settings or [])
if hasattr(current().context, "default_query_settings"):
settings += current().context.default_query_settings
if len(sql) > 1024: if len(sql) > 1024:
with tempfile.NamedTemporaryFile("w", encoding="utf-8") as query: with tempfile.NamedTemporaryFile("w", encoding="utf-8") as query:
query.write(sql) query.write(sql)
query.flush() query.flush()
command = f"cat \"{query.name}\" | {self.cluster.docker_compose} exec -T {self.name} clickhouse client -n" command = f"cat \"{query.name}\" | {self.cluster.docker_compose} exec -T {self.name} clickhouse client -n"
for setting in settings or []: for setting in settings:
name, value = setting name, value = setting
command += f" --{name} \"{value}\"" command += f" --{name} \"{value}\""
description = f""" description = f"""
echo -e \"{sql[:100]}...\" > {query.name} echo -e \"{sql[:100]}...\" > {query.name}
{command} {command}
""" """
with step("executing command", description=description) if steps else NullStep(): with step("executing command", description=description, format_description=False) if steps else NullStep():
r = self.cluster.bash(None)(command, *args, **kwargs) try:
r = self.cluster.bash(None)(command, *args, **kwargs)
except ExpectTimeoutError:
self.cluster.close_bash(None)
else: else:
command = f"echo -e \"{sql}\" | clickhouse client -n" command = f"echo -e \"{sql}\" | clickhouse client -n"
for setting in settings or []: for setting in settings:
name, value = setting name, value = setting
command += f" --{name} \"{value}\"" command += f" --{name} \"{value}\""
with step("executing command", description=command) if steps else NullStep(): with step("executing command", description=command, format_description=False) if steps else NullStep():
r = self.cluster.bash(self.name)(command, *args, **kwargs) try:
r = self.cluster.bash(self.name)(command, *args, **kwargs)
except ExpectTimeoutError:
self.cluster.close_bash(self.name)
raise
if no_checks: if no_checks:
return r return r
@ -134,6 +146,7 @@ class Cluster(object):
docker_compose="docker-compose", docker_compose_project_dir=None, docker_compose="docker-compose", docker_compose_project_dir=None,
docker_compose_file="docker-compose.yml"): docker_compose_file="docker-compose.yml"):
self.terminating = False
self._bash = {} self._bash = {}
self.clickhouse_binary_path = clickhouse_binary_path self.clickhouse_binary_path = clickhouse_binary_path
self.configs_dir = configs_dir self.configs_dir = configs_dir
@ -183,11 +196,19 @@ class Cluster(object):
def bash(self, node, timeout=120): def bash(self, node, timeout=120):
"""Returns thread-local bash terminal """Returns thread-local bash terminal
to a specific node. to a specific node.
:param node: name of the service :param node: name of the service
""" """
test = current()
if self.terminating:
if test and (test.cflags & MANDATORY):
pass
else:
raise InterruptedError("terminating")
current_thread = threading.current_thread() current_thread = threading.current_thread()
id = f"{current_thread.ident}-{node}" id = f"{current_thread.name}-{node}"
with self.lock: with self.lock:
if self._bash.get(id) is None: if self._bash.get(id) is None:
if node is None: if node is None:
@ -196,9 +217,30 @@ class Cluster(object):
self._bash[id] = Shell(command=[ self._bash[id] = Shell(command=[
"/bin/bash", "--noediting", "-c", f"{self.docker_compose} exec {node} bash --noediting" "/bin/bash", "--noediting", "-c", f"{self.docker_compose} exec {node} bash --noediting"
], name=node).__enter__() ], name=node).__enter__()
self._bash[id].timeout = timeout self._bash[id].timeout = timeout
# clean up any stale open shells for threads that have exited
active_thread_names = {thread.name for thread in threading.enumerate()}
for bash_id in list(self._bash.keys()):
thread_name, node_name = bash_id.rsplit("-", 1)
if thread_name not in active_thread_names:
self._bash[bash_id].__exit__(None, None, None)
del self._bash[bash_id]
return self._bash[id] return self._bash[id]
def close_bash(self, node):
current_thread = threading.current_thread()
id = f"{current_thread.name}-{node}"
with self.lock:
if self._bash.get(id) is None:
return
self._bash[id].__exit__(None, None, None)
del self._bash[id]
def __enter__(self): def __enter__(self):
with Given("docker-compose cluster"): with Given("docker-compose cluster"):
self.up() self.up()
@ -210,20 +252,21 @@ class Cluster(object):
self.down() self.down()
finally: finally:
with self.lock: with self.lock:
for shell in list(self._bash.values()): for shell in self._bash.values():
shell.__exit__(type, value, traceback) shell.__exit__(type, value, traceback)
def node(self, name): def node(self, name):
"""Get object with node bound methods. """Get object with node bound methods.
:param name: name of service name :param name: name of service name
""" """
if name.startswith("clickhouse"): if name.startswith("clickhouse"):
return ClickHouseNode(self, name) return ClickHouseNode(self, name)
return Node(self, name) return Node(self, name)
def down(self, timeout=120): def down(self, timeout=300):
"""Bring cluster down by executing docker-compose down.""" """Bring cluster down by executing docker-compose down."""
self.terminating = True
try: try:
bash = self.bash(None) bash = self.bash(None)
with self.lock: with self.lock:
@ -235,7 +278,7 @@ class Cluster(object):
else: else:
self._bash[id] = shell self._bash[id] = shell
finally: finally:
return self.command(None, f"{self.docker_compose} down", timeout=timeout) return self.command(None, f"{self.docker_compose} down", bash=bash, timeout=timeout)
def up(self, timeout=30*60): def up(self, timeout=30*60):
if self.local: if self.local:
@ -264,7 +307,7 @@ class Cluster(object):
if cmd.exitcode != 0: if cmd.exitcode != 0:
continue continue
with And("executing docker-compose down just in case it is up"): with And("executing docker-compose down just in case it is up"):
cmd = self.command(None, f"{self.docker_compose} down 2>&1 | tee", exitcode=None, timeout=timeout) cmd = self.command(None, f"{self.docker_compose} down --remove-orphans 2>&1 | tee", exitcode=None, timeout=timeout)
if cmd.exitcode != 0: if cmd.exitcode != 0:
continue continue
with And("executing docker-compose up"): with And("executing docker-compose up"):
@ -285,22 +328,26 @@ class Cluster(object):
for name in self.nodes["clickhouse"]: for name in self.nodes["clickhouse"]:
self.node(name).wait_healthy() self.node(name).wait_healthy()
def command(self, node, command, message=None, exitcode=None, steps=True, *args, **kwargs): def command(self, node, command, message=None, exitcode=None, steps=True, bash=None, *args, **kwargs):
"""Execute and check command. """Execute and check command.
:param node: name of the service :param node: name of the service
:param command: command :param command: command
:param message: expected message that should be in the output, default: None :param message: expected message that should be in the output, default: None
:param exitcode: expected exitcode, default: None :param exitcode: expected exitcode, default: None
:param steps: don't break command into steps, default: True :param steps: don't break command into steps, default: True
""" """
debug(f"command() {node}, {command}") with By("executing command", description=command, format_description=False) if steps else NullStep():
with By("executing command", description=command) if steps else NullStep(): if bash is None:
r = self.bash(node)(command, *args, **kwargs) bash = self.bash(node)
try:
r = bash(command, *args, **kwargs)
except ExpectTimeoutError:
self.close_bash(node)
raise
if exitcode is not None: if exitcode is not None:
with Then(f"exitcode should be {exitcode}") if steps else NullStep(): with Then(f"exitcode should be {exitcode}", format_name=False) if steps else NullStep():
assert r.exitcode == exitcode, error(r.output) assert r.exitcode == exitcode, error(r.output)
if message is not None: if message is not None:
with Then(f"output should contain message", description=message) if steps else NullStep(): with Then(f"output should contain message", description=message, format_description=False) if steps else NullStep():
assert message in r.output, error(r.output) assert message in r.output, error(r.output)
return r return r

View File

@ -2,7 +2,7 @@
import sys import sys
from testflows.core import * from testflows.core import *
append_path(sys.path, "..") append_path(sys.path, "..")
from helpers.cluster import Cluster from helpers.cluster import Cluster
from helpers.argparser import argparser from helpers.argparser import argparser
@ -33,13 +33,13 @@ xfails = {
RQ_SRS_007_LDAP_Authentication("1.0") RQ_SRS_007_LDAP_Authentication("1.0")
) )
@XFails(xfails) @XFails(xfails)
def regression(self, local, clickhouse_binary_path): def regression(self, local, clickhouse_binary_path, stress=None, parallel=None):
"""ClickHouse integration with LDAP regression module. """ClickHouse integration with LDAP regression module.
""" """
nodes = { nodes = {
"clickhouse": ("clickhouse1", "clickhouse2", "clickhouse3"), "clickhouse": ("clickhouse1", "clickhouse2", "clickhouse3"),
} }
with Cluster(local, clickhouse_binary_path, nodes=nodes) as cluster: with Cluster(local, clickhouse_binary_path, nodes=nodes) as cluster:
self.context.cluster = cluster self.context.cluster = cluster

View File

@ -58,9 +58,44 @@
<port>9440</port> <port>9440</port>
<secure>1</secure> <secure>1</secure>
</replica> </replica>
</shard> </shard>
</replicated_cluster_secure> </replicated_cluster_secure>
<cluster23> <cluster1>
<shard>
<replica>
<host>clickhouse1</host>
<port>9440</port>
<secure>1</secure>
</replica>
</shard>
</cluster1>
<sharded_cluster12>
<shard>
<replica>
<host>clickhouse1</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>clickhouse2</host>
<port>9000</port>
</replica>
</shard>
</sharded_cluster12>
<one_shard_cluster12>
<shard>
<replica>
<host>clickhouse1</host>
<port>9000</port>
</replica>
<replica>
<host>clickhouse2</host>
<port>9000</port>
</replica>
</shard>
</one_shard_cluster12>
<sharded_cluster23>
<shard> <shard>
<replica> <replica>
<host>clickhouse2</host> <host>clickhouse2</host>
@ -73,8 +108,20 @@
<port>9000</port> <port>9000</port>
</replica> </replica>
</shard> </shard>
</cluster23> </sharded_cluster23>
<sharded_cluster> <one_shard_cluster23>
<shard>
<replica>
<host>clickhouse2</host>
<port>9000</port>
</replica>
<replica>
<host>clickhouse3</host>
<port>9000</port>
</replica>
</shard>
</one_shard_cluster23>
<sharded_cluster>
<shard> <shard>
<replica> <replica>
<host>clickhouse1</host> <host>clickhouse1</host>
@ -94,6 +141,22 @@
</replica> </replica>
</shard> </shard>
</sharded_cluster> </sharded_cluster>
<one_shard_cluster>
<shard>
<replica>
<host>clickhouse1</host>
<port>9000</port>
</replica>
<replica>
<host>clickhouse2</host>
<port>9000</port>
</replica>
<replica>
<host>clickhouse3</host>
<port>9000</port>
</replica>
</shard>
</one_shard_cluster>
<sharded_cluster_secure> <sharded_cluster_secure>
<shard> <shard>
<replica> <replica>

View File

@ -3,6 +3,7 @@
<server> <server>
<certificateFile>/etc/clickhouse-server/ssl/server.crt</certificateFile> <certificateFile>/etc/clickhouse-server/ssl/server.crt</certificateFile>
<privateKeyFile>/etc/clickhouse-server/ssl/server.key</privateKeyFile> <privateKeyFile>/etc/clickhouse-server/ssl/server.key</privateKeyFile>
<dhParamsFile>/etc/clickhouse-server/ssl/dhparam.pem</dhParamsFile>
<verificationMode>none</verificationMode> <verificationMode>none</verificationMode>
<cacheSessions>true</cacheSessions> <cacheSessions>true</cacheSessions>
</server> </server>

View File

@ -69,7 +69,7 @@
<!-- Listen specified host. use :: (wildcard IPv6 address), if you want to accept connections both with IPv4 and IPv6 from everywhere. --> <!-- Listen specified host. use :: (wildcard IPv6 address), if you want to accept connections both with IPv4 and IPv6 from everywhere. -->
<!-- <listen_host>::</listen_host> --> <!-- <listen_host>::</listen_host> -->
<!-- Same for hosts with disabled ipv6: --> <!-- Same for hosts with disabled ipv6: -->
<!-- <listen_host>0.0.0.0</listen_host> --> <listen_host>0.0.0.0</listen_host>
<!-- Default values - try listen localhost on ipv4 and ipv6: --> <!-- Default values - try listen localhost on ipv4 and ipv6: -->
<!-- <!--
@ -120,6 +120,18 @@
<!-- Path to folder where users and roles created by SQL commands are stored. --> <!-- Path to folder where users and roles created by SQL commands are stored. -->
<access_control_path>/var/lib/clickhouse/access/</access_control_path> <access_control_path>/var/lib/clickhouse/access/</access_control_path>
<!-- Sources to read users, roles, access rights, profiles of settings, quotas. -->
<user_directories>
<users_xml>
<!-- Path to configuration file with predefined users. -->
<path>users.xml</path>
</users_xml>
<local_directory>
<!-- Path to folder where users created by SQL commands are stored. -->
<path>/var/lib/clickhouse/access/</path>
</local_directory>
</user_directories>
<!-- Path to configuration file with users, access rights, profiles of settings, quotas. --> <!-- Path to configuration file with users, access rights, profiles of settings, quotas. -->
<users_config>users.xml</users_config> <users_config>users.xml</users_config>
@ -160,7 +172,7 @@
<!-- Configuration of clusters that could be used in Distributed tables. <!-- Configuration of clusters that could be used in Distributed tables.
https://clickhouse.yandex/docs/en/table_engines/distributed/ https://clickhouse.yandex/docs/en/table_engines/distributed/
--> -->
<remote_servers incl="clickhouse_remote" > <remote_servers incl="clickhouse_remote_servers" >
<!-- Test only shard config for testing distributed storage --> <!-- Test only shard config for testing distributed storage -->
<test_shard_localhost> <test_shard_localhost>
<shard> <shard>
@ -220,7 +232,7 @@
See https://clickhouse.yandex/docs/en/table_engines/replication/ See https://clickhouse.yandex/docs/en/table_engines/replication/
--> -->
<zookeeper incl="zookeeper" optional="true" /> <zookeeper incl="zookeeper-servers" optional="true" />
<!-- Substitutions for parameters of replicated tables. <!-- Substitutions for parameters of replicated tables.
Optional. If you don't use replicated tables, you could omit that. Optional. If you don't use replicated tables, you could omit that.
@ -353,7 +365,7 @@
<!-- Uncomment if you want data to be compressed 30-100% better. <!-- Uncomment if you want data to be compressed 30-100% better.
Don't do that if you just started using ClickHouse. Don't do that if you just started using ClickHouse.
--> -->
<compression incl="compression"> <compression incl="clickhouse_compression">
<!-- <!--
<!- - Set of variants. Checked in order. Last matching case wins. If nothing matches, lz4 will be used. - -> <!- - Set of variants. Checked in order. Last matching case wins. If nothing matches, lz4 will be used. - ->
<case> <case>

View File

4
tests/testflows/rbac/docker-compose/docker-compose.yml Normal file → Executable file
View File

@ -45,7 +45,7 @@ services:
zookeeper: zookeeper:
condition: service_healthy condition: service_healthy
# dummy service which does nothing, but allows to postpone # dummy service which does nothing, but allows to postpone
# 'docker-compose up -d' till all dependecies will go healthy # 'docker-compose up -d' till all dependecies will go healthy
all_services_ready: all_services_ready:
image: hello-world image: hello-world
@ -57,4 +57,4 @@ services:
clickhouse3: clickhouse3:
condition: service_healthy condition: service_healthy
zookeeper: zookeeper:
condition: service_healthy condition: service_healthy

View File

@ -10,9 +10,9 @@ services:
ZOO_MY_ID: 1 ZOO_MY_ID: 1
healthcheck: healthcheck:
test: echo stat | nc localhost 2181 test: echo stat | nc localhost 2181
interval: 10s interval: 3s
timeout: 10s timeout: 2s
retries: 3 retries: 5
start_period: 300s start_period: 2s
security_opt: security_opt:
- label:disable - label:disable

View File

View File

@ -0,0 +1,124 @@
import uuid
from multiprocessing.dummy import Pool
from contextlib import contextmanager
from testflows.core import *
from rbac.helper.tables import table_types
def join(tasks):
"""Join all parallel tests.
"""
exc = None
while tasks:
try:
tasks[0].get()
tasks.pop(0)
except KeyboardInterrupt as e:
current().context.cluster.terminating = True
continue
except Exception as e:
tasks.pop(0)
if exc is None:
exc = e
current().context.cluster.terminating = True
if exc is not None:
raise exc
def start(pool, tasks, scenario, kwargs=None):
"""Start parallel test.
"""
if kwargs is None:
kwargs = {}
task = pool.apply_async(scenario, [], kwargs)
tasks.append(task)
return task
def run_scenario(pool, tasks, scenario, kwargs=None):
if kwargs is None:
kwargs = {}
if current().context.parallel:
start(pool, tasks, scenario, kwargs)
else:
scenario(**kwargs)
def permutations(table_count=1):
return [*range((1 << table_count)-1)]
def getuid():
return str(uuid.uuid1()).replace('-', '_')
@contextmanager
def table(node, name, table_type_name="MergeTree"):
table_type = table_types[table_type_name]
try:
names = name.split(",")
for name in names:
with Given(f"I have {name} with engine {table_type_name}"):
node.query(f"DROP TABLE IF EXISTS {name}")
node.query(table_type.create_statement.format(name=name))
yield
finally:
for name in names:
with Finally(f"I drop the table {name}"):
if table_type.cluster:
node.query(f"DROP TABLE IF EXISTS {name} ON CLUSTER {table_type.cluster}")
else:
node.query(f"DROP TABLE IF EXISTS {name}")
@contextmanager
def user(node, name):
try:
names = name.split(",")
for name in names:
with Given("I have a user"):
node.query(f"CREATE USER OR REPLACE {name}")
yield
finally:
for name in names:
with Finally("I drop the user"):
node.query(f"DROP USER IF EXISTS {name}")
@contextmanager
def role(node, role):
try:
roles = role.split(",")
for role in roles:
with Given("I have a role"):
node.query(f"CREATE ROLE OR REPLACE {role}")
yield
finally:
for role in roles:
with Finally("I drop the role"):
node.query(f"DROP ROLE IF EXISTS {role}")
tables = {
"table0" : 1 << 0,
"table1" : 1 << 1,
"table2" : 1 << 2,
"table3" : 1 << 3,
"table4" : 1 << 4,
"table5" : 1 << 5,
"table6" : 1 << 6,
"table7" : 1 << 7,
}
@contextmanager
def grant_select_on_table(node, grants, target_name, *table_names):
try:
tables_granted = []
for table_number in range(len(table_names)):
if(grants & tables[f"table{table_number}"]):
with When(f"I grant select privilege on {table_names[table_number]}"):
node.query(f"GRANT SELECT ON {table_names[table_number]} TO {target_name}")
tables_granted.append(f'{table_names[table_number]}')
yield (', ').join(tables_granted)
finally:
for table_number in range(len(table_names)):
with Finally(f"I revoke the select privilege on {table_names[table_number]}"):
node.query(f"REVOKE SELECT ON {table_names[table_number]} FROM {target_name}")

View File

@ -0,0 +1,123 @@
## Syntax
# Errors: not found
not_found = "Exception: There is no {type} `{name}`"
def user_not_found_in_disk(name):
return (192,not_found.format(type="user",name=name))
def role_not_found_in_disk(name):
return (255,not_found.format(type="role",name=name))
def settings_profile_not_found_in_disk(name):
return (180,not_found.format(type="settings profile",name=name))
def quota_not_found_in_disk(name):
return (199,not_found.format(type="quota",name=name))
def row_policy_not_found_in_disk(name):
return (11,not_found.format(type="row policy",name=name))
def table_does_not_exist(name):
return(60,"Exception: Table {name} doesn't exist".format(name=name))
# Errors: cannot_rename
cannot_rename = "Exception: {type} `{name}`: cannot rename to `{name_new}` because {type} `{name_new}` already exists"
cannot_rename_exitcode = 237
def cannot_rename_user(name,name_new):
return (cannot_rename_exitcode, cannot_rename.format(type="user", name=name, name_new=name_new))
def cannot_rename_role(name,name_new):
return (cannot_rename_exitcode, cannot_rename.format(type="role", name=name, name_new=name_new))
def cannot_rename_settings_profile(name,name_new):
return (cannot_rename_exitcode, cannot_rename.format(type="settings profile", name=name, name_new=name_new))
def cannot_rename_quota(name,name_new):
return (cannot_rename_exitcode, cannot_rename.format(type="quota", name=name, name_new=name_new))
def cannot_rename_row_policy(name,name_new):
return (cannot_rename_exitcode, cannot_rename.format(type="row policy", name=name, name_new=name_new))
# Errors: cannot insert
cannot_insert = "Exception: {type} `{name}`: cannot insert because {type} `{name}` already exists"
cannot_insert_exitcode = 237
def cannot_insert_user(name):
return (cannot_insert_exitcode, cannot_insert.format(type="user",name=name))
def cannot_insert_role(name):
return (cannot_insert_exitcode, cannot_insert.format(type="role",name=name))
def cannot_insert_settings_profile(name):
return (cannot_insert_exitcode, cannot_insert.format(type="settings profile",name=name))
def cannot_insert_quota(name):
return (cannot_insert_exitcode, cannot_insert.format(type="quota",name=name))
def cannot_insert_row_policy(name):
return (cannot_insert_exitcode, cannot_insert.format(type="row policy",name=name))
# Error: default is readonly
cannot_remove_default = "Exception: Cannot remove {type} `default` from users.xml because this storage is readonly"
cannot_remove_default_exitcode = 239
def cannot_update_default():
return (cannot_remove_default_exitcode, "Exception: Cannot update user `default` in users.xml because this storage is readonly")
def cannot_remove_user_default():
return (cannot_remove_default_exitcode, cannot_remove_default.format(type="user"))
def cannot_remove_settings_profile_default():
return (cannot_remove_default_exitcode, cannot_remove_default.format(type="settings profile"))
def cannot_remove_quota_default():
return (cannot_remove_default_exitcode, cannot_remove_default.format(type="quota"))
# Other syntax errors
def unknown_setting(setting):
return (115, f"Exception: Unknown setting {setting}.")
def cluster_not_found(cluster):
return (170, f"Exception: Requested cluster '{cluster}' not found.")
## Privileges
def not_enough_privileges(name):
return (241, f"Exception: {name}: Not enough privileges.")
def cannot_parse_string_as_float(string):
return (6, f"Exception: Cannot parse string '{string}' as Float64")
def missing_columns(name):
return (47, f"Exception: Missing columns: '{name}' while processing query")
# Errors: wrong name
wrong_name = "Exception: Wrong {type} name. Cannot find {type} `{name}` to drop"
def wrong_column_name(name):
return (10, wrong_name.format(type="column",name=name))
def wrong_index_name(name):
return (36, wrong_name.format(type="index",name=name))
def wrong_constraint_name(name):
return (36, wrong_name.format(type="constraint",name=name))
# Errors: cannot add
cannot_add = "Exception: Cannot add index {name}: index with this name already exists"
cannot_add_exitcode = 44
def cannot_add_index(name):
return (cannot_add_exitcode, cannot_add.format(name=name))
def cannot_add_constraint(name):
return (cannot_add_exitcode, cannot_add.format(name=name))

View File

@ -0,0 +1,41 @@
from collections import namedtuple
table_tuple = namedtuple("table_tuple", "create_statement cluster")
table_types = {
"MergeTree": table_tuple("CREATE TABLE {name} (d DATE, a String, b UInt8, x String, y Int8) ENGINE = MergeTree() PARTITION BY y ORDER BY d", None),
"ReplacingMergeTree": table_tuple("CREATE TABLE {name} (d DATE, a String, b UInt8, x String, y Int8) ENGINE = ReplacingMergeTree() PARTITION BY y ORDER BY d", None),
"SummingMergeTree": table_tuple("CREATE TABLE {name} (d DATE, a String, b UInt8 DEFAULT 1, x String, y Int8) ENGINE = SummingMergeTree() PARTITION BY y ORDER BY d", None),
"AggregatingMergeTree": table_tuple("CREATE TABLE {name} (d DATE, a String, b UInt8, x String, y Int8) ENGINE = AggregatingMergeTree() PARTITION BY y ORDER BY d", None),
"CollapsingMergeTree": table_tuple("CREATE TABLE {name} (d Date, a String, b UInt8, x String, y Int8, sign Int8 DEFAULT 1) ENGINE = CollapsingMergeTree(sign) PARTITION BY y ORDER BY d", None),
"VersionedCollapsingMergeTree": table_tuple("CREATE TABLE {name} (d Date, a String, b UInt8, x String, y Int8, version UInt64, sign Int8 DEFAULT 1) ENGINE = VersionedCollapsingMergeTree(sign, version) PARTITION BY y ORDER BY d", None),
"GraphiteMergeTree": table_tuple("CREATE TABLE {name} (d Date, a String, b UInt8, x String, y Int8, Path String, Time DateTime, Value Float64, col UInt64, Timestamp Int64) ENGINE = GraphiteMergeTree('graphite_rollup_example') PARTITION BY y ORDER by d", None),
"ReplicatedMergeTree-sharded_cluster": table_tuple("CREATE TABLE {name} ON CLUSTER sharded_cluster (d DATE, a String, b UInt8, x String, y Int8) \
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{{shard}}/{name}', '{{replica}}') PARTITION BY y ORDER BY d", "sharded_cluster"),
"ReplicatedMergeTree-one_shard_cluster": table_tuple("CREATE TABLE {name} ON CLUSTER one_shard_cluster (d DATE, a String, b UInt8, x String, y Int8) \
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{{shard}}/{name}', '{{replica}}') PARTITION BY y ORDER BY d", "one_shard_cluster"),
"ReplicatedReplacingMergeTree-sharded_cluster": table_tuple("CREATE TABLE {name} ON CLUSTER sharded_cluster (d DATE, a String, b UInt8, x String, y Int8) \
ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/{{shard}}/{name}', '{{replica}}') PARTITION BY y ORDER BY d", "sharded_cluster"),
"ReplicatedReplacingMergeTree-one_shard_cluster": table_tuple("CREATE TABLE {name} ON CLUSTER one_shard_cluster (d DATE, a String, b UInt8, x String, y Int8) \
ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/{{shard}}/{name}', '{{replica}}') PARTITION BY y ORDER BY d", "one_shard_cluster"),
"ReplicatedSummingMergeTree-sharded_cluster": table_tuple("CREATE TABLE {name} ON CLUSTER sharded_cluster (d DATE, a String, b UInt8 DEFAULT 1, x String, y Int8) \
ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{{shard}}/{name}', '{{replica}}') PARTITION BY y ORDER BY d", "sharded_cluster"),
"ReplicatedSummingMergeTree-one_shard_cluster": table_tuple("CREATE TABLE {name} ON CLUSTER one_shard_cluster (d DATE, a String, b UInt8 DEFAULT 1, x String, y Int8) \
ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{{shard}}/{name}', '{{replica}}') PARTITION BY y ORDER BY d", "one_shard_cluster"),
"ReplicatedAggregatingMergeTree-sharded_cluster": table_tuple("CREATE TABLE {name} ON CLUSTER sharded_cluster (d DATE, a String, b UInt8, x String, y Int8) \
ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/{{shard}}/{name}', '{{replica}}') PARTITION BY y ORDER BY d", "sharded_cluster"),
"ReplicatedAggregatingMergeTree-one_shard_cluster": table_tuple("CREATE TABLE {name} ON CLUSTER one_shard_cluster (d DATE, a String, b UInt8, x String, y Int8) \
ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/{{shard}}/{name}', '{{replica}}') PARTITION BY y ORDER BY d", "one_shard_cluster"),
"ReplicatedCollapsingMergeTree-sharded_cluster": table_tuple("CREATE TABLE {name} ON CLUSTER sharded_cluster (d Date, a String, b UInt8, x String, y Int8, sign Int8 DEFAULT 1) \
ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/{{shard}}/{name}', '{{replica}}', sign) PARTITION BY y ORDER BY d", "sharded_cluster"),
"ReplicatedCollapsingMergeTree-one_shard_cluster": table_tuple("CREATE TABLE {name} ON CLUSTER one_shard_cluster (d Date, a String, b UInt8, x String, y Int8, sign Int8 DEFAULT 1) \
ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/{{shard}}/{name}', '{{replica}}', sign) PARTITION BY y ORDER BY d", "one_shard_cluster"),
"ReplicatedVersionedCollapsingMergeTree-sharded_cluster": table_tuple("CREATE TABLE {name} ON CLUSTER sharded_cluster (d Date, a String, b UInt8, x String, y Int8, version UInt64, sign Int8 DEFAULT 1) \
ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/tables/{{shard}}/{name}', '{{replica}}', sign, version) PARTITION BY y ORDER BY d", "sharded_cluster"),
"ReplicatedVersionedCollapsingMergeTree-one_shard_cluster": table_tuple("CREATE TABLE {name} ON CLUSTER one_shard_cluster (d Date, a String, b UInt8, x String, y Int8, version UInt64, sign Int8 DEFAULT 1) \
ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/tables/{{shard}}/{name}', '{{replica}}', sign, version) PARTITION BY y ORDER BY d", "one_shard_cluster"),
"ReplicatedGraphiteMergeTree-sharded_cluster": table_tuple("CREATE TABLE {name} ON CLUSTER sharded_cluster (d Date, a String, b UInt8, x String, y Int8, Path String, Time DateTime, Value Float64, col UInt64, Timestamp Int64) \
ENGINE = ReplicatedGraphiteMergeTree('/clickhouse/tables/{{shard}}/{name}', '{{replica}}', 'graphite_rollup_example') PARTITION BY y ORDER BY d", "sharded_cluster"),
"ReplicatedGraphiteMergeTree-one_shard_cluster": table_tuple("CREATE TABLE {name} ON CLUSTER one_shard_cluster (d Date, a String, b UInt8, x String, y Int8, Path String, Time DateTime, Value Float64, col UInt64, Timestamp Int64) \
ENGINE = ReplicatedGraphiteMergeTree('/clickhouse/tables/{{shard}}/{name}', '{{replica}}', 'graphite_rollup_example') PARTITION BY y ORDER BY d", "one_shard_cluster"),
}

View File

@ -1,51 +1,106 @@
#!/usr/bin/env python3 #!/usr/bin/env python3
import sys import sys
from testflows.core import * from testflows.core import *
append_path(sys.path, "..") append_path(sys.path, "..")
from helpers.cluster import Cluster from helpers.cluster import Cluster
from helpers.argparser import argparser from helpers.argparser import argparser
from rbac.requirements import *
issue_12507 = "https://github.com/ClickHouse/ClickHouse/issues/12507" issue_14091 = "https://github.com/ClickHouse/ClickHouse/issues/14091"
issue_12510 = "https://github.com/ClickHouse/ClickHouse/issues/12510" issue_14149 = "https://github.com/ClickHouse/ClickHouse/issues/14149"
issue_12600 = "https://github.com/ClickHouse/ClickHouse/issues/12600" issue_14224 = "https://github.com/ClickHouse/ClickHouse/issues/14224"
issue_14418 = "https://github.com/ClickHouse/ClickHouse/issues/14418"
issue_14451 = "https://github.com/ClickHouse/ClickHouse/issues/14451"
issue_14566 = "https://github.com/ClickHouse/ClickHouse/issues/14566"
issue_14674 = "https://github.com/ClickHouse/ClickHouse/issues/14674"
issue_14810 = "https://github.com/ClickHouse/ClickHouse/issues/14810"
issue_15165 = "https://github.com/ClickHouse/ClickHouse/issues/15165"
issue_15980 = "https://github.com/ClickHouse/ClickHouse/issues/15980"
xfails = { xfails = {
"syntax/show create quota/I show create quota current": "syntax/show create quota/I show create quota current":
[(Fail, "https://github.com/ClickHouse/ClickHouse/issues/12495")], [(Fail, "https://github.com/ClickHouse/ClickHouse/issues/12495")],
"syntax/create role/I create role that already exists, throws exception": "views/:/create with subquery privilege granted directly or via role/:":
[(Fail, issue_12510)], [(Fail, issue_14091)],
"syntax/create user/I create user with if not exists, user does exist": "views/:/create with join query privilege granted directly or via role/:":
[(Fail, issue_12507)], [(Fail, issue_14091)],
"syntax/create row policy/I create row policy if not exists, policy does exist": "views/:/create with union query privilege granted directly or via role/:":
[(Fail, issue_12507)], [(Fail, issue_14091)],
"syntax/create quota/I create quota if not exists, quota does exist": "views/:/create with join union subquery privilege granted directly or via role/:":
[(Fail, issue_12507)], [(Fail, issue_14091)],
"syntax/create role/I create role if not exists, role does exist": "views/:/create with nested views privilege granted directly or via role/:":
[(Fail, issue_12507)], [(Fail, issue_14091)],
"syntax/create settings profile/I create settings profile if not exists, profile does exist": "views/view/select with join query privilege granted directly or via role/:":
[(Fail, issue_12507)], [(Fail, issue_14149)],
"syntax/grant privilege/grant privileges/privilege='dictGet', on=('db0.table0', 'db0.*', '*.*', 'tb0', '*'), allow_introspection=False": "views/view/select with join union subquery privilege granted directly or via role/:":
[(Fail, issue_12600)], [(Fail, issue_14149)],
"syntax/grant privilege/grant privileges/privilege='CREATE', on=('db0.table0', 'db0.*', '*.*', 'tb0', '*'), allow_introspection=False": "views/view/select with nested views privilege granted directly or via role/:":
[(Fail, issue_12600)], [(Fail, issue_14149)],
"syntax/grant privilege/grant privileges/privilege='DROP', on=('db0.table0', 'db0.*', '*.*', 'tb0', '*'), allow_introspection=False": "views/live view/refresh with privilege granted directly or via role/:":
[(Fail, issue_12600)], [(Fail, issue_14224)],
"syntax/grant privilege/grant privileges/privilege='TRUNCATE', on=('db0.table0', 'db0.*', '*.*', 'tb0', '*'), allow_introspection=False": "views/live view/refresh with privilege revoked directly or from role/:":
[(Fail, issue_12600)], [(Fail, issue_14224)],
"syntax/grant privilege/grant privileges/privilege='OPTIMIZE', on=('db0.table0', 'db0.*', '*.*', 'tb0', '*'), allow_introspection=False": "views/live view/select:":
[(Fail, issue_12600)], [(Fail, issue_14418)],
"syntax/grant privilege/grant privileges/privilege='SYSTEM', on=('db0.table0', 'db0.*', '*.*', 'tb0', '*'), allow_introspection=False": "views/live view/select:/:":
[(Fail, issue_12600)], [(Fail, issue_14418)],
"views/materialized view/select with:":
[(Fail, issue_14451)],
"views/materialized view/select with:/:":
[(Fail, issue_14451)],
"views/materialized view/modify query:":
[(Fail, issue_14674)],
"views/materialized view/modify query:/:":
[(Fail, issue_14674)],
"views/materialized view/insert on source table privilege granted directly or via role/:":
[(Fail, issue_14810)],
"privileges/alter ttl/table_type=:/user with some privileges":
[(Fail, issue_14566)],
"privileges/alter ttl/table_type=:/role with some privileges":
[(Fail, issue_14566)],
"privileges/alter ttl/table_type=:/user with privileges on cluster":
[(Fail, issue_14566)],
"privileges/alter ttl/table_type=:/user with privileges from user with grant option":
[(Fail, issue_14566)],
"privileges/alter ttl/table_type=:/user with privileges from role with grant option":
[(Fail, issue_14566)],
"privileges/alter ttl/table_type=:/role with privileges from user with grant option":
[(Fail, issue_14566)],
"privileges/alter ttl/table_type=:/role with privileges from role with grant option":
[(Fail, issue_14566)],
"privileges/distributed table/:/special cases/insert with table on source table of materialized view:":
[(Fail, issue_14810)],
"privileges/distributed table/cluster tests/cluster='sharded*":
[(Fail, issue_15165)],
"privileges/distributed table/cluster tests/cluster=:/special cases/insert with table on source table of materialized view privilege granted directly or via role/:":
[(Fail, issue_14810)],
"/rbac/views/materialized view/select from implicit target table privilege granted directly or via role/select from implicit target table, privilege granted directly":
[(Fail, ".inner table is not created as expected")],
"/rbac/views/materialized view/insert on target table privilege granted directly or via role/insert on target table, privilege granted through a role":
[(Fail, ".inner table is not created as expected")],
"/rbac/views/materialized view/select from implicit target table privilege granted directly or via role/select from implicit target table, privilege granted through a role":
[(Fail, ".inner table is not created as expected")],
"/rbac/views/materialized view/insert on target table privilege granted directly or via role/insert on target table, privilege granted directly":
[(Fail, ".inner table is not created as expected")],
"/rbac/views/materialized view/select from source table privilege granted directly or via role/select from implicit target table, privilege granted directly":
[(Fail, ".inner table is not created as expected")],
"/rbac/views/materialized view/select from source table privilege granted directly or via role/select from implicit target table, privilege granted through a role":
[(Fail, ".inner table is not created as expected")],
}
xflags = {
"privileges/alter index/table_type='ReplicatedVersionedCollapsingMergeTree-sharded_cluster'/role with privileges from role with grant option/granted=:/I try to ALTER INDEX with given privileges/I check order by when privilege is granted":
(SKIP, 0)
} }
@TestModule @TestModule
@ArgumentParser(argparser) @ArgumentParser(argparser)
@XFails(xfails) @XFails(xfails)
@XFlags(xflags)
@Name("rbac") @Name("rbac")
def regression(self, local, clickhouse_binary_path): def regression(self, local, clickhouse_binary_path, stress=None, parallel=None):
"""RBAC regression. """RBAC regression.
""" """
nodes = { nodes = {
@ -54,9 +109,14 @@ def regression(self, local, clickhouse_binary_path):
} }
with Cluster(local, clickhouse_binary_path, nodes=nodes) as cluster: with Cluster(local, clickhouse_binary_path, nodes=nodes) as cluster:
self.context.cluster = cluster self.context.cluster = cluster
self.context.stress = stress
if parallel is not None:
self.context.parallel = parallel
Feature(run=load("rbac.tests.syntax.feature", "feature"), flags=TE) Feature(run=load("rbac.tests.syntax.feature", "feature"), flags=TE)
Feature(run=load("rbac.tests.privileges.feature", "feature"), flags=TE) Feature(run=load("rbac.tests.privileges.feature", "feature"), flags=TE)
Feature(run=load("rbac.tests.views.feature", "feature"), flags=TE)
if main(): if main():
regression() regression()

View File

@ -1 +1 @@
from .requirements import * from .requirements import *

File diff suppressed because it is too large Load Diff

1332
tests/testflows/rbac/requirements/requirements.py Normal file → Executable file

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,7 @@
# Copyright 2020, Altinity LTD. All Rights Reserved.
#
# All information contained herein is, and remains the property
# of Altinity LTD. Any dissemination of this information or
# reproduction of this material is strictly forbidden unless
# prior written permission is obtained from Altinity LTD.
#

View File

@ -0,0 +1,993 @@
import json
from testflows.core import *
from testflows.asserts import error
from rbac.requirements import *
from rbac.helper.common import *
import rbac.helper.errors as errors
from rbac.helper.tables import table_types
subprivileges = {
"ADD COLUMN" : 1 << 0,
"CLEAR COLUMN" : 1 << 1,
"MODIFY COLUMN" : 1 << 2,
"RENAME COLUMN": 1 << 3,
"COMMENT COLUMN": 1 << 4,
"DROP COLUMN": 1 << 5,
}
aliases = {
"ADD COLUMN" : ["ALTER ADD COLUMN", "ADD COLUMN"],
"CLEAR COLUMN": ["ALTER CLEAR COLUMN", "CLEAR COLUMN"],
"MODIFY COLUMN" : ["ALTER MODIFY COLUMN", "MODIFY COLUMN"],
"RENAME COLUMN" : ["ALTER RENAME COLUMN", "RENAME COLUMN"],
"COMMENT COLUMN": ["ALTER COMMENT COLUMN", "COMMENT COLUMN"],
"DROP COLUMN": ["ALTER DROP COLUMN", "DROP COLUMN"],
"ALTER COLUMN" : ["ALTER COLUMN"], #super-privilege
}
# extra permutation is for 'ALTER COLUMN' super-privilege
permutation_count = (1 << len(subprivileges))
def permutations(table_type):
"""Uses stress flag and table type, returns list of all permutations to run
Stress test (stress=True): all permutations for all tables
"""
if current().context.stress:
return [*range(permutation_count + len(aliases["ALTER COLUMN"]))]
else:
# Selected permutations currently stand as [1,2,4,8,16,32,0,42,63,64] that maps to
# testing
# [
# "ADD COLUMN", "CLEAR COLUMN", "MODIFY COLUMN", "RENAME COLUMN",
# "COMMENT COLUMN", "DROP COLUMN", "NONE", "DROP, RENAME, CLEAR", all, and
# "ALTER COLUMN"
# ]
return [1 << index for index in range(len(subprivileges))] + \
[0, int('101010', 2), permutation_count-1, permutation_count]
def alter_column_privileges(grants: int):
"""Takes in an integer, and returns the corresponding set of tests to grant and
not grant using the binary string. Each integer corresponds to a unique permutation
of grants.
Columns represents columns to grant privileges on, with format "col1,col2,col3"
"""
note(grants)
privileges = []
# extra iteration for ALTER COLUMN
if grants >= permutation_count:
privileges.append(aliases["ALTER COLUMN"][grants-permutation_count])
elif grants==0: # No privileges
privileges.append("NONE")
else:
if (grants & subprivileges["ADD COLUMN"]):
privileges.append(aliases["ADD COLUMN"][grants % len(aliases["ADD COLUMN"])])
if (grants & subprivileges["CLEAR COLUMN"]):
privileges.append(aliases["CLEAR COLUMN"][grants % len(aliases["CLEAR COLUMN"])])
if (grants & subprivileges["MODIFY COLUMN"]):
privileges.append(aliases["MODIFY COLUMN"][grants % len(aliases["MODIFY COLUMN"])])
if (grants & subprivileges["RENAME COLUMN"]):
privileges.append(aliases["RENAME COLUMN"][grants % len(aliases["RENAME COLUMN"])])
if (grants & subprivileges["COMMENT COLUMN"]):
privileges.append(aliases["COMMENT COLUMN"][grants % len(aliases["COMMENT COLUMN"])])
if (grants & subprivileges["DROP COLUMN"]):
privileges.append(aliases["DROP COLUMN"][grants % len(aliases["DROP COLUMN"])])
note(f"Testing privileges: {privileges}")
return ', '.join(privileges)
def on_columns(privileges, columns):
"""For column-based tests. Takes in string output of alter_column_privileges()
and adds columns for those privileges.
"""
privileges = privileges.split(',')
privileges = [privilege + f"({columns})" for privilege in privileges]
return ', '.join(privileges)
def alter_column_privilege_handler(grants, table, user, node, columns=None):
"""For all 6 subprivileges, if the privilege is granted: run test to ensure correct behavior,
and if the privilege is not granted, run test to ensure correct behavior there as well
If `columns` are passed in, they must be columns that do not exist on the table.
This is necessary for full testing (add column, drop column, modify column, etc.).
"""
note(f"GRANTS: {grants}")
# testing ALTER COLUMN is the same as testing all subprivileges
if grants > permutation_count-1:
grants = permutation_count-1
# if 'columns' is not passed then one iteration with column = None
columns = columns.split(",") if columns != None else [None]
for column in columns:
# will always run 6 tests per column depending on granted privileges
if (grants & subprivileges["ADD COLUMN"]):
with When("I check add column when privilege is granted"):
check_add_column_when_privilege_is_granted(table, user, node, column)
else:
with When("I check add column when privilege is not granted"):
check_add_column_when_privilege_is_not_granted(table, user, node, column)
if (grants & subprivileges["CLEAR COLUMN"]):
with When("I check clear column when privilege is granted"):
check_clear_column_when_privilege_is_granted(table, user, node, column)
else:
with When("I check clear column when privilege is not granted"):
check_clear_column_when_privilege_is_not_granted(table, user, node, column)
if (grants & subprivileges["MODIFY COLUMN"]):
with When("I check modify column when privilege is granted"):
check_modify_column_when_privilege_is_granted(table, user, node, column)
else:
with When("I check modify column when privilege is not granted"):
check_modify_column_when_privilege_is_not_granted(table, user, node, column)
if (grants & subprivileges["RENAME COLUMN"]):
with When("I check rename column when privilege is granted"):
check_rename_column_when_privilege_is_granted(table, user, node, column)
else:
with When("I check rename column when privilege is not granted"):
check_rename_column_when_privilege_is_not_granted(table, user, node, column)
if (grants & subprivileges["COMMENT COLUMN"]):
with When("I check comment column when privilege is granted"):
check_comment_column_when_privilege_is_granted(table, user, node, column)
else:
with When("I check comment column when privilege is not granted"):
check_comment_column_when_privilege_is_not_granted(table, user, node, column)
if (grants & subprivileges["DROP COLUMN"]):
with When("I check drop column when privilege is granted"):
check_drop_column_when_privilege_is_granted(table, user, node, column)
else:
with When("I check drop column when privilege is not granted"):
check_drop_column_when_privilege_is_not_granted(table, user, node, column)
def check_add_column_when_privilege_is_granted(table, user, node, column=None):
"""Ensures ADD COLUMN runs as expected when the privilege is granted
to the specified user.
"""
if column is None:
column = 'add'
with Given(f"I add column '{column}'"):
node.query(f"ALTER TABLE {table} ADD COLUMN {column} String",
settings = [("user", user)])
with Then("I insert data to tree"):
node.query(f"INSERT INTO {table} ({column}) VALUES ('3.4')") #String
with Then("I verify that the column was successfully added"):
column_data = node.query(f"SELECT {column} FROM {table} FORMAT JSONEachRow").output
column_data_list = column_data.split('\n')
output_rows = [{f"{column}":"3.4"}, {f"{column}":""}]
for row in column_data_list:
assert json.loads(row) in output_rows, error()
with Finally(f"I drop column '{column}'"):
node.query(f"ALTER TABLE {table} DROP COLUMN {column}")
def check_clear_column_when_privilege_is_granted(table, user, node, column=None):
"""Ensures CLEAR COLUMN runs as expected when the privilege is granted
to the specified user.
"""
if column is None:
column = 'clear'
with Given(f"I add the column {column}"):
node.query(f"ALTER TABLE {table} ADD COLUMN {column} String")
with And("I add some data to column"):
node.query(f"INSERT INTO {table} ({column}) VALUES ('ready to be cleared')")
with When(f"I clear column '{column}'"):
node.query(f"ALTER TABLE {table} CLEAR COLUMN {column}",
settings = [("user", user)])
with Then("I verify that the column was successfully cleared"):
column_data = node.query(f"SELECT {column} FROM {table} FORMAT JSONEachRow").output
column_data_list = column_data.split('\n')
for row in column_data_list:
assert json.loads(row) == {f"{column}":""}, error()
with Finally(f"I drop column '{column}'"):
node.query(f"ALTER TABLE {table} DROP COLUMN {column}")
def check_modify_column_when_privilege_is_granted(table, user, node, column=None):
"""Ensures MODIFY COLUMN runs as expected when the privilege is granted
to the specified user.
"""
if column is None:
column = 'modify'
with Given(f"I add the column {column}"):
node.query(f"ALTER TABLE {table} ADD COLUMN {column} String")
with When(f"I insert some data into column {column}"):
node.query(f"INSERT INTO {table} ({column}) VALUES ('3.4')")
with When(f"I modify column '{column}' to type Float"):
node.query(f"ALTER TABLE {table} MODIFY COLUMN {column} Float64",
settings = [("user", user)])
with And("I run optimize table to ensure above UPDATE command is done"):
node.query(f"OPTIMIZE TABLE {table} FINAL", timeout=900)
with Then("I verify that the column type was modified"):
with When(f"I try to insert a String (old type) to column {column}, throws exception"):
exitcode, message = errors.cannot_parse_string_as_float('hello')
node.query(f"INSERT INTO {table} ({column}) VALUES ('hello')",
exitcode=exitcode, message=message)
with And(f"I try to insert float data (correct type) to column {column}, will accept"):
node.query(f"INSERT INTO {table} ({column}) VALUES (30.01)")
with And("I verify that the date was inserted correctly"):
column_data = node.query(f"SELECT {column} FROM {table} FORMAT JSONEachRow").output
column_data_list = column_data.split('\n')
output_rows = [{f"{column}":30.01}, {f"{column}":3.4}, {f"{column}":0}]
for row in column_data_list:
assert json.loads(row) in output_rows, error()
with Finally(f"I drop column '{column}'"):
node.query(f"ALTER TABLE {table} DROP COLUMN {column}")
def check_rename_column_when_privilege_is_granted(table, user, node, column=None):
"""Ensures RENAME COLUMN runs as expected when the privilege is granted
to the specified user.
"""
if column is None:
column = 'rename'
new_column = f"{column}_new"
with Given(f"I add the column {column}"):
node.query(f"ALTER TABLE {table} ADD COLUMN {column} String")
with And("I get the initial contents of the column"):
# could be either str or float depending on MODIFY COLUMN
initial_column_data = node.query(f"SELECT {column} FROM {table} ORDER BY {column}"
" FORMAT JSONEachRow").output
with When(f"I rename column '{column}' to '{new_column}'"):
node.query(f"ALTER TABLE {table} RENAME COLUMN {column} TO {new_column}",
settings = [("user", user)])
with Then("I verify that the column was successfully renamed"):
with When("I verify that the original column does not exist"):
exitcode, message = errors.missing_columns(column)
node.query(f"SELECT {column} FROM {table} FORMAT JSONEachRow",
exitcode=exitcode, message=message)
with And("I verify that the new column does exist as expected, with same values"):
new_column_data = node.query(f"SELECT {new_column} FROM {table} ORDER BY"
f" {new_column} FORMAT JSONEachRow").output
if initial_column_data == '':
assert initial_column_data == new_column_data, error()
else:
new_column_data_list = new_column_data.split('\n')
initial_column_data_list = initial_column_data.split('\n')
for new, initial in zip(new_column_data_list, initial_column_data_list):
assert json.loads(new)[new_column] == json.loads(initial)[column], error()
with Finally(f"I use default user to undo rename"):
node.query(f"ALTER TABLE {table} RENAME COLUMN {new_column} TO {column}")
with Finally(f"I drop column '{column}'"):
node.query(f"ALTER TABLE {table} DROP COLUMN {column}")
def check_comment_column_when_privilege_is_granted(table, user, node, column='x'):
"""Ensures COMMENT COLUMN runs as expected when the privilege is granted
to the specified user.
"""
if column is None:
column = 'comment'
with Given(f"I add the column {column}"):
node.query(f"ALTER TABLE {table} ADD COLUMN {column} String")
with And(f"I alter {column} with comment"):
node.query(f"ALTER TABLE {table} COMMENT COLUMN {column} 'This is a comment.'",
settings = [("user", user)])
with Then(f"I verify that the specified comment is present for {column}"):
table_data = node.query(f"DESCRIBE TABLE {table} FORMAT JSONEachRow").output
table_data_list = table_data.split('\n')
for row in table_data_list:
row = json.loads(row)
if row['name'] == column:
assert row['comment'] == "This is a comment.", error()
with Finally(f"I drop column '{column}'"):
node.query(f"ALTER TABLE {table} DROP COLUMN {column}")
return
# did not find a match, so cleanup column and throw an error
with Finally(f"I drop column '{column}'"):
node.query(f"ALTER TABLE {table} DROP COLUMN {column}")
error()
def check_drop_column_when_privilege_is_granted(table, user, node, column=None):
"""Ensures DROP COLUMN runs as expected when the privilege is granted
to the specified user.
"""
with When("I try to drop nonexistent column, throws exception"):
# if user has privilege for all columns, error is 'wrong column name'
if column:
exitcode, message = errors.not_enough_privileges(user)
else:
exitcode, message = errors.wrong_column_name("fake_column")
node.query(f"ALTER TABLE {table} DROP COLUMN fake_column",
settings = [("user", user)], exitcode=exitcode, message=message)
if column is None:
column = 'drop'
with Given(f"I add the column {column}"):
node.query(f"ALTER TABLE {table} ADD COLUMN {column} String")
with Then(f"I drop column {column} which exists"):
node.query(f"ALTER TABLE {table} DROP COLUMN {column}",
settings = [("user", user)])
with And(f"I verify that {column} has been dropped"):
exitcode, message = errors.wrong_column_name(column)
node.query(f"ALTER TABLE {table} DROP COLUMN {column}",
settings = [("user", user)], exitcode=exitcode, message=message)
def check_add_column_when_privilege_is_not_granted(table, user, node, column=None):
"""Ensures ADD COLUMN errors as expected without the required privilege
for the specified user.
"""
if column is None:
column = 'add'
with When("I try to use privilege that has not been granted"):
exitcode, message = errors.not_enough_privileges(user)
node.query(f"ALTER TABLE {table} ADD COLUMN {column} String",
settings = [("user", user)], exitcode=exitcode, message=message)
def check_clear_column_when_privilege_is_not_granted(table, user, node, column=None):
"""Ensures CLEAR COLUMN errors as expected without the required privilege
for the specified user.
"""
if column is None:
column = 'clear'
with When("I try to use privilege that has not been granted"):
exitcode, message = errors.not_enough_privileges(user)
node.query(f"ALTER TABLE {table} CLEAR COLUMN {column}",
settings = [("user", user)], exitcode=exitcode, message=message)
def check_modify_column_when_privilege_is_not_granted(table, user, node, column=None):
"""Ensures MODIFY COLUMN errors as expected without the required privilege
for the specified user.
"""
if column is None:
column = 'modify'
with When("I try to use privilege that has not been granted"):
exitcode, message = errors.not_enough_privileges(user)
node.query(f"ALTER TABLE {table} MODIFY COLUMN {column} String",
settings = [("user", user)], exitcode=exitcode, message=message)
def check_rename_column_when_privilege_is_not_granted(table, user, node, column=None):
"""Ensures RENAME COLUMN errors as expected without the required privilege
for the specified user.
"""
if column is None:
column = 'rename'
new_column = f"{column}_new"
with When("I try to use privilege that has not been granted"):
exitcode, message = errors.not_enough_privileges(user)
node.query(f"ALTER TABLE {table} RENAME COLUMN {column} TO {new_column}",
settings = [("user", user)], exitcode=exitcode, message=message)
def check_comment_column_when_privilege_is_not_granted(table, user, node, column=None):
"""Ensures COMMENT COLUMN errors as expected without the required privilege
for the specified user.
"""
if column is None:
column = 'comment'
with When("I try to use privilege that has not been granted"):
exitcode, message = errors.not_enough_privileges(user)
node.query(f"ALTER TABLE {table} COMMENT COLUMN {column} 'This is a comment.'",
settings = [("user", user)], exitcode=exitcode, message=message)
def check_drop_column_when_privilege_is_not_granted(table, user, node, column=None):
"""Ensures DROP COLUMN errors as expected without the required privilege
for the specified user.
"""
if column is None:
column = 'drop'
with When("I try to use privilege that has not been granted"):
exitcode, message = errors.not_enough_privileges(user)
node.query(f"ALTER TABLE {table} DROP COLUMN {column}",
settings = [("user", user)], exitcode=exitcode, message=message)
@TestScenario
def user_with_some_privileges(self, permutation, table_type, node=None):
"""Check that user with some privileges of ALTER COLUMN is able
to alter the table for privileges granted, and not for privileges not granted.
"""
privileges = alter_column_privileges(permutation)
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name):
with Given("I first grant the privileges"):
node.query(f"GRANT {privileges} ON {table_name} TO {user_name}")
with Then(f"I try to ALTER COLUMN"):
alter_column_privilege_handler(permutation, table_name, user_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterColumn_Revoke("1.0"),
)
def user_with_revoked_privileges(self, permutation, table_type, node=None):
"""Check that user is unable to alter columns on table after ALTER COLUMN privilege
on that table has been revoked from the user.
"""
privileges = alter_column_privileges(permutation)
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name):
with Given("I first grant the privileges"):
node.query(f"GRANT {privileges} ON {table_name} TO {user_name}")
with And("I then revoke the privileges"):
node.query(f"REVOKE {privileges} ON {table_name} FROM {user_name}")
with When(f"I try to ALTER COLUMN"):
# No privileges granted
alter_column_privilege_handler(0, table_name, user_name, node)
@TestScenario
@Examples("grant_columns revoke_columns alter_columns_fail", [
("t1", "t1", "t2"),
("t1,t3", "t1", "t2"),
("t1,t3,t4", "t1,t3,t4", "t2"),
])
def user_with_privileges_on_columns(self, table_type, permutation, node=None):
"""Passes in examples to user_column_privileges() below to test granting
of sub-privileges on columns
"""
examples=Examples("grant_columns revoke_columns alter_columns_fail table_type permutation",
[tuple(list(row)+[table_type, permutation]) for row in self.examples])
pool = Pool(3)
try:
tasks = []
try:
for example in examples:
run_scenario(pool, tasks, Scenario(test=user_column_privileges, examples=examples))
finally:
join(tasks)
finally:
pool.close()
@TestOutline
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterColumn_Column("1.0"),
)
def user_column_privileges(self, grant_columns, revoke_columns, alter_columns_fail, table_type,
permutation, node=None):
"""Check that user is able to alter on granted columns
and unable to alter on not granted or revoked columns.
"""
privileges = alter_column_privileges(permutation)
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
privileges_on_columns = on_columns(privileges, grant_columns)
with When(f"granted={privileges_on_columns}"):
with table(node, table_name, table_type), user(node, user_name):
with When(f"I grant subprivileges"):
node.query(f"GRANT {privileges_on_columns} ON {table_name} TO {user_name}")
if alter_columns_fail is not None:
with When(f"I try to alter on not granted columns, fails"):
# Permutation 0: no privileges for any permutation on these columns
alter_column_privilege_handler(0, table_name, user_name, node, columns=alter_columns_fail)
with Then(f"I try to ALTER COLUMN"):
alter_column_privilege_handler(permutation, table_name, user_name, node, columns=grant_columns)
if revoke_columns is not None:
with When(f"I revoke alter column privilege for columns"):
node.query(f"REVOKE {privileges_on_columns} ON {table_name} FROM {user_name}")
with And("I try to alter revoked columns"):
alter_column_privilege_handler(0, table_name, user_name, node, columns=alter_columns_fail)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterColumn_Grant("1.0"),
)
def role_with_some_privileges(self, permutation, table_type, node=None):
"""Check that user can alter column on a table after it is granted a role that
has the alter column privilege for that table.
"""
privileges = alter_column_privileges(permutation)
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name), role(node, role_name):
with Given("I grant the alter column privilege to a role"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name}")
with And("I grant role to the user"):
node.query(f"GRANT {role_name} TO {user_name}")
with Then(f"I try to ALTER COLUMN"):
alter_column_privilege_handler(permutation, table_name, user_name, node)
@TestScenario
def user_with_revoked_role(self, permutation, table_type, node=None):
"""Check that user with a role that has alter column privilege on a table is unable to
alter column from that table after the role with privilege has been revoked from the user.
"""
privileges = alter_column_privileges(permutation)
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name), role(node, role_name):
with When("I grant privileges to a role"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name}")
with And("I grant the role to a user"):
node.query(f"GRANT {role_name} TO {user_name}")
with And("I revoke the role from the user"):
node.query(f"REVOKE {role_name} FROM {user_name}")
with And("I alter column on the table"):
# Permutation 0: no privileges for any permutation on these columns
alter_column_privilege_handler(0, table_name, user_name, node)
@TestScenario
@Examples("grant_columns revoke_columns alter_columns_fail", [
("t1", "t1", "t2"),
("t1,t3", "t1", "t2"),
("t1,t3,t4", "t1,t3,t4", "t2"),
])
def role_with_privileges_on_columns(self, table_type, permutation, node=None):
"""Passes in examples to role_column_privileges() below to test granting
of subprivileges on columns
"""
examples=Examples("grant_columns revoke_columns alter_columns_fail table_type permutation",
[tuple(list(row)+[table_type, permutation]) for row in self.examples])
pool = Pool(3)
try:
tasks = []
try:
for example in examples:
run_scenario(pool, tasks, Scenario(test=role_column_privileges, examples=examples))
finally:
join(tasks)
finally:
pool.close()
@TestOutline
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterColumn_Column("1.0"),
)
def role_column_privileges(self, grant_columns, revoke_columns, alter_columns_fail, table_type,
permutation, node=None):
"""Check that user is able to alter column from granted columns and unable
to alter column from not granted or revoked columns.
"""
privileges = alter_column_privileges(permutation)
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
privileges_on_columns = on_columns(privileges, grant_columns)
with When(f"granted={privileges_on_columns}"):
with table(node, table_name, table_type), user(node, user_name), role(node, role_name):
with When(f"I grant subprivileges"):
node.query(f"GRANT {privileges_on_columns} ON {table_name} TO {role_name}")
with And("I grant the role to a user"):
node.query(f"GRANT {role_name} TO {user_name}")
if alter_columns_fail is not None:
with When(f"I try to alter on not granted columns, fails"):
# Permutation 0: no privileges for any permutation on these columns
alter_column_privilege_handler(0, table_name, user_name, node, columns=alter_columns_fail)
with Then(f"I try to ALTER COLUMN"):
alter_column_privilege_handler(permutation, table_name, user_name, node, columns=grant_columns)
if revoke_columns is not None:
with When(f"I revoke alter column privilege for columns"):
node.query(f"REVOKE {privileges_on_columns} ON {table_name} FROM {role_name}")
with And("I try to alter failed columns"):
alter_column_privilege_handler(0, table_name, user_name, node, columns=revoke_columns)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterColumn_Cluster("1.0"),
)
def user_with_privileges_on_cluster(self, permutation, table_type, node=None):
"""Check that user is able to alter column on a table with
privilege granted on a cluster.
"""
privileges = alter_column_privileges(permutation)
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
with When(f"granted={privileges}"):
with table(node, table_name, table_type):
try:
with Given("I have a user on a cluster"):
node.query(f"CREATE USER OR REPLACE {user_name} ON CLUSTER sharded_cluster")
with When("I grant alter column privileges on a cluster"):
node.query(f"GRANT ON CLUSTER sharded_cluster {privileges} ON {table_name} TO {user_name}")
with Then(f"I try to ALTER COLUMN"):
alter_column_privilege_handler(permutation, table_name, user_name, node)
finally:
with Finally("I drop the user on a cluster"):
node.query(f"DROP USER {user_name} ON CLUSTER sharded_cluster")
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Grant("1.0"),
)
def user_with_privileges_from_user_with_grant_option(self, permutation, table_type, node=None):
"""Check that user is able to alter column on a table when granted privilege
from another user with grant option.
"""
privileges = alter_column_privileges(permutation)
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with When("I grant privileges with grant option to user"):
node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And("I grant privileges to another user via grant option"):
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}",
settings = [("user", user0_name)])
with Then(f"I try to ALTER COLUMN"):
alter_column_privilege_handler(permutation, table_name, user1_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Grant("1.0"),
)
def role_with_privileges_from_user_with_grant_option(self, permutation, table_type, node=None):
"""Check that user is able to alter column on a table when granted a role with
alter column privilege that was granted by another user with grant option.
"""
privileges = alter_column_privileges(permutation)
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with When("I grant subprivileges with grant option to user"):
node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And("I grant privileges to a role via grant option"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name}",
settings = [("user", user0_name)])
with And("I grant the role to another user"):
node.query(f"GRANT {role_name} TO {user1_name}")
with Then(f"I try to ALTER COLUMN"):
alter_column_privilege_handler(permutation, table_name, user1_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Grant("1.0"),
)
def user_with_privileges_from_role_with_grant_option(self, permutation, table_type, node=None):
"""Check that user is able to alter column on a table when granted privilege from
a role with grant option
"""
privileges = alter_column_privileges(permutation)
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with When(f"I grant privileges with grant option to a role"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION")
with When("I grant role to a user"):
node.query(f"GRANT {role_name} TO {user0_name}")
with And("I grant privileges to a user via grant option"):
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}",
settings = [("user", user0_name)])
with Then(f"I try to ALTER COLUMN"):
alter_column_privilege_handler(permutation, table_name, user1_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Grant("1.0"),
)
def role_with_privileges_from_role_with_grant_option(self, permutation, table_type, node=None):
"""Check that a user is able to alter column on a table with a role that was
granted privilege by another role with grant option
"""
privileges = alter_column_privileges(permutation)
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role0_name = f"role0_{getuid()}"
role1_name = f"role1_{getuid()}"
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role0_name), role(node, role1_name):
with When(f"I grant privilege with grant option to role"):
node.query(f"GRANT {privileges} ON {table_name} TO {role0_name} WITH GRANT OPTION")
with And("I grant the role to a user"):
node.query(f"GRANT {role0_name} TO {user0_name}")
with And("I grant privileges to another role via grant option"):
node.query(f"GRANT {privileges} ON {table_name} TO {role1_name}",
settings = [("user", user0_name)])
with And("I grant the second role to another user"):
node.query(f"GRANT {role1_name} TO {user1_name}")
with Then(f"I try to ALTER COLUMN"):
alter_column_privilege_handler(permutation, table_name, user1_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_user_via_user_with_grant_option(self, permutation, table_type, node=None):
"""Check that user is unable to revoke a privilege they don't have access to from a user.
"""
# This test does not apply when no privileges are granted
privileges = alter_column_privileges(permutation)
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with When(f"I grant privileges with grant option to user"):
granted_privileges = on_columns(privileges, "d")
node.query(f"GRANT {granted_privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with Then("I revoke privilege on a column the user with grant option does not have access to"):
exitcode, message = errors.not_enough_privileges(user0_name)
revoked_privileges = on_columns(privileges, "b")
if permutation is 0:
"""Revoking NONE always succeeds, regardless of privileges.
"""
node.query(f"REVOKE {revoked_privileges} ON {table_name} FROM {user1_name}", settings=[("user", user0_name)])
else:
node.query(f"REVOKE {revoked_privileges} ON {table_name} FROM {user1_name}", settings=[("user", user0_name)],
exitcode=exitcode, message=message)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_role_via_user_with_grant_option(self, permutation, table_type, node=None):
"""Check that user is unable to revoke a privilege they don't have access to from a role.
"""
# This test does not apply when no privileges are granted
privileges = alter_column_privileges(permutation)
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name), role(node, role_name):
with When(f"I grant privileges with grant option to user"):
granted_privileges = on_columns(privileges, "d")
node.query(f"GRANT {granted_privileges} ON {table_name} TO {user_name} WITH GRANT OPTION")
with Then("I revoke privilege on a column the user with grant option does not have access to"):
exitcode, message = errors.not_enough_privileges(user_name)
revoked_privileges = on_columns(privileges, "b")
if permutation is 0:
"""Revoking NONE always succeeds, regardless of privileges.
"""
node.query(f"REVOKE {revoked_privileges} ON {table_name} FROM {role_name}", settings=[("user", user_name)])
else:
node.query(f"REVOKE {revoked_privileges} ON {table_name} FROM {role_name}", settings=[("user", user_name)],
exitcode=exitcode, message=message)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_user_via_role_with_grant_option(self, permutation, table_type, node=None):
"""Check that user with a role is unable to revoke a privilege they don't have access to from a user.
"""
# This test does not apply when no privileges are granted
privileges = alter_column_privileges(permutation)
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with When(f"I grant privileges with grant option on column d to a role"):
granted_privileges = on_columns(privileges, "d")
node.query(f"GRANT {granted_privileges} ON {table_name} TO {role_name} WITH GRANT OPTION")
with And("I grant the role to a user"):
node.query(f"GRANT {role_name} TO {user0_name}")
with Then("I revoke privilege on a column the user with grant option does not have access to"):
exitcode, message = errors.not_enough_privileges(user0_name)
revoked_privileges = on_columns(privileges, "b")
if permutation is 0:
"""Revoking NONE always succeeds, regardless of privileges.
"""
node.query(f"REVOKE {revoked_privileges} ON {table_name} FROM {user1_name}", settings=[("user", user0_name)])
else:
node.query(f"REVOKE {revoked_privileges} ON {table_name} FROM {user1_name}", settings=[("user", user0_name)],
exitcode=exitcode, message=message)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_role_via_role_with_grant_option(self, permutation, table_type, node=None):
"""Check that user with a role is unable to revoke a privilege they don't have access to from a role.
"""
# This test does not apply when no privileges are granted
privileges = alter_column_privileges(permutation)
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
role0_name = f"role0_{getuid()}"
role1_name = f"role1_{getuid()}"
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name):
with role(node, role0_name), role(node, role1_name):
with When(f"I grant privileges with grant option to a role"):
granted_privileges = on_columns(privileges, "d")
node.query(f"GRANT {granted_privileges} ON {table_name} TO {user_name} WITH GRANT OPTION")
with And("I grant the role to a user"):
node.query(f"GRANT {role0_name} TO {user_name}")
with Then("I revoke privilege on a column the user with grant option does not have access to"):
exitcode, message = errors.not_enough_privileges(user_name)
revoked_privileges = on_columns(privileges, "b")
if permutation is 0:
node.query(f"REVOKE {revoked_privileges} ON {table_name} FROM {role1_name}", settings=[("user", user_name)])
else:
node.query(f"REVOKE {revoked_privileges} ON {table_name} FROM {role1_name}", settings=[("user", user_name)],
exitcode=exitcode, message=message)
@TestFeature
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterColumn("1.0"),
RQ_SRS_006_RBAC_Privileges_AlterColumn_TableEngines("1.0")
)
@Examples("table_type", [
(key,) for key in table_types.keys()
])
@Name("alter column")
def feature(self, node="clickhouse1", stress=None, parallel=None):
"""Runs test suites above which check correctness over scenarios and permutations.
"""
self.context.node = self.context.cluster.node(node)
if parallel is not None:
self.context.parallel = parallel
if stress is not None:
self.context.stress = stress
for example in self.examples:
table_type, = example
if table_type != "MergeTree" and not self.context.stress:
continue
with Example(str(example)):
pool = Pool(15)
try:
tasks = []
try:
for permutation in permutations(table_type):
privileges = alter_column_privileges(permutation)
for scenario in loads(current_module(), Scenario):
with Suite(name=f"{privileges}"):
run_scenario(pool, tasks, Scenario(test=scenario), {"table_type": table_type, "permutation": permutation})
finally:
join(tasks)
finally:
pool.close()

View File

@ -0,0 +1,629 @@
import json
from testflows.core import *
from testflows.core import threading
from testflows.asserts import error
from rbac.requirements import *
from rbac.helper.common import *
import rbac.helper.errors as errors
from rbac.helper.tables import table_types
subprivileges = {
"ADD CONSTRAINT" : 1 << 0,
"DROP CONSTRAINT" : 1 << 1,
}
aliases = {
"ADD CONSTRAINT" : ["ALTER ADD CONSTRAINT", "ADD CONSTRAINT"],
"DROP CONSTRAINT": ["ALTER DROP CONSTRAINT", "DROP CONSTRAINT"],
"ALTER CONSTRAINT": ["ALTER CONSTRAINT", "CONSTRAINT"] # super-privilege
}
# Extra permutation is for 'ALTER CONSTRAINT' super-privilege
permutation_count = (1 << len(subprivileges))
def permutations():
"""Returns list of all permutations to run.
Currently includes NONE, ADD, DROP, both, and ALTER CONSTRAINT
"""
return [*range(permutation_count + len(aliases["ALTER CONSTRAINT"]))]
def alter_constraint_privileges(grants: int):
"""Takes in an integer, and returns the corresponding set of tests to grant and
not grant using the binary string. Each integer corresponds to a unique permutation
of grants.
"""
note(grants)
privileges = []
# Extra iteration for ALTER CONSTRAINT
if grants >= permutation_count:
privileges.append(aliases["ALTER CONSTRAINT"][grants-permutation_count])
elif grants==0: # No privileges
privileges.append("NONE")
else:
if (grants & subprivileges["ADD CONSTRAINT"]):
privileges.append(aliases["ADD CONSTRAINT"][grants % len(aliases["ADD CONSTRAINT"])])
if (grants & subprivileges["DROP CONSTRAINT"]):
privileges.append(aliases["DROP CONSTRAINT"][grants % len(aliases["DROP CONSTRAINT"])])
note(f"Testing these privileges: {privileges}")
return ', '.join(privileges)
def alter_constraint_privilege_handler(grants, table, user, node):
"""For all 2 subprivileges, if the privilege is granted: run test to ensure correct behavior,
and if the privilege is not granted, run test to ensure correct behavior there as well
"""
# Testing ALTER CONSTRAINT and CONSTRAINT is the same as testing all subprivileges
if grants > permutation_count-1:
grants = permutation_count-1
if (grants & subprivileges["ADD CONSTRAINT"]):
with When("I check add constraint when privilege is granted"):
check_add_constraint_when_privilege_is_granted(table, user, node)
else:
with When("I check add constraint when privilege is not granted"):
check_add_constraint_when_privilege_is_not_granted(table, user, node)
if (grants & subprivileges["DROP CONSTRAINT"]):
with When("I check drop constraint when privilege is granted"):
check_drop_constraint_when_privilege_is_granted(table, user, node)
else:
with When("I check drop constraint when privilege is not granted"):
check_drop_constraint_when_privilege_is_not_granted(table, user, node)
def check_add_constraint_when_privilege_is_granted(table, user, node):
"""Ensures ADD CONSTRAINT runs as expected when the privilege is granted to the specified user
"""
constraint = "add"
with Given(f"I add constraint '{constraint}'"):
node.query(f"ALTER TABLE {table} ADD CONSTRAINT {constraint} CHECK x>5",
settings = [("user", user)])
with Then("I verify that the constraint is in the table"):
output = json.loads(node.query(f"SHOW CREATE TABLE {table} FORMAT JSONEachRow").output)
assert f"CONSTRAINT {constraint} CHECK x > 5" in output['statement'], error()
with Finally(f"I drop constraint {constraint}"):
node.query(f"ALTER TABLE {table} DROP constraint {constraint}")
def check_drop_constraint_when_privilege_is_granted(table, user, node):
"""Ensures DROP CONSTRAINT runs as expected when the privilege is granted to the specified user
"""
with But("I try to drop nonexistent constraint, throws exception"):
exitcode, message = errors.wrong_constraint_name("fake_constraint")
node.query(f"ALTER TABLE {table} DROP CONSTRAINT fake_constraint",
settings = [("user", user)], exitcode=exitcode, message=message)
constraint = "drop"
with Given(f"I add the constraint for this test"):
node.query(f"ALTER TABLE {table} ADD CONSTRAINT {constraint} CHECK x>5")
with Then(f"I drop constraint {constraint} which exists"):
node.query(f"ALTER TABLE {table} DROP CONSTRAINT {constraint}",
settings = [("user", user)])
with Then("I verify that the constraint is not in the table"):
output = json.loads(node.query(f"SHOW CREATE TABLE {table} FORMAT JSONEachRow").output)
assert f"CONSTRAINT {constraint} CHECK x > 5" not in output['statement'], error()
def check_add_constraint_when_privilege_is_not_granted(table, user, node):
"""Ensures ADD CONSTRAINT errors as expected without the required privilege for the specified user
"""
constraint = "add"
with When("I try to use privilege that has not been granted"):
exitcode, message = errors.not_enough_privileges(user)
node.query(f"ALTER TABLE {table} ADD CONSTRAINT {constraint} CHECK x>5",
settings = [("user", user)], exitcode=exitcode, message=message)
def check_drop_constraint_when_privilege_is_not_granted(table, user, node):
"""Ensures DROP CONSTRAINT errors as expected without the required privilege for the specified user
"""
constraint = "drop"
with When("I try to use privilege that has not been granted"):
exitcode, message = errors.not_enough_privileges(user)
node.query(f"ALTER TABLE {table} DROP CONSTRAINT {constraint}",
settings = [("user", user)], exitcode=exitcode, message=message)
@TestScenario
def user_with_some_privileges(self, table_type, node=None):
"""Check that user with any permutation of ALTER CONSTRAINT subprivileges is able
to alter the table for privileges granted, and not for privileges not granted.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
for permutation in permutations():
privileges = alter_constraint_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name):
with Given("I first grant the privileges"):
node.query(f"GRANT {privileges} ON {table_name} TO {user_name}")
with Then(f"I try to ALTER CONSTRAINT"):
alter_constraint_privilege_handler(permutation, table_name, user_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterConstraint_Revoke("1.0"),
)
def user_with_revoked_privileges(self, table_type, node=None):
"""Check that user is unable to ALTER CONSTRAINTs on table after ALTER CONSTRAINT privilege
on that table has been revoked from the user.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
for permutation in permutations():
privileges = alter_constraint_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name):
with Given("I first grant the privileges"):
node.query(f"GRANT {privileges} ON {table_name} TO {user_name}")
with And("I then revoke the privileges"):
node.query(f"REVOKE {privileges} ON {table_name} FROM {user_name}")
with When(f"I try to ALTER CONSTRAINT"):
# Permutation 0: no privileges
alter_constraint_privilege_handler(0, table_name, user_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterConstraint_Grant("1.0"),
)
def role_with_some_privileges(self, table_type, node=None):
"""Check that user can ALTER CONSTRAINT on a table after it is granted a role that
has the ALTER CONSTRAINT privilege for that table.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations():
privileges = alter_constraint_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name), role(node, role_name):
with Given("I grant the ALTER CONSTRAINT privilege to a role"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name}")
with And("I grant role to the user"):
node.query(f"GRANT {role_name} TO {user_name}")
with Then(f"I try to ALTER CONSTRAINT"):
alter_constraint_privilege_handler(permutation, table_name, user_name, node)
@TestScenario
def user_with_revoked_role(self, table_type, node=None):
"""Check that user with a role that has ALTER CONSTRAINT privilege on a table is unable to
ALTER CONSTRAINT from that table after the role with privilege has been revoked from the user.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations():
privileges = alter_constraint_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name), role(node, role_name):
with When("I grant privileges to a role"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name}")
with And("I grant the role to a user"):
node.query(f"GRANT {role_name} TO {user_name}")
with And("I revoke the role from the user"):
node.query(f"REVOKE {role_name} FROM {user_name}")
with And("I ALTER CONSTRAINT on the table"):
# Permutation 0: no privileges for any permutation
alter_constraint_privilege_handler(0, table_name, user_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterConstraint_Cluster("1.0"),
)
def user_with_privileges_on_cluster(self, table_type, node=None):
"""Check that user is able to ALTER CONSTRAINT on a table with
privilege granted on a cluster.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
for permutation in permutations():
privileges = alter_constraint_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type):
try:
with Given("I have a user on a cluster"):
node.query(f"CREATE USER OR REPLACE {user_name} ON CLUSTER sharded_cluster")
with When("I grant ALTER CONSTRAINT privileges on a cluster"):
node.query(f"GRANT ON CLUSTER sharded_cluster {privileges} ON {table_name} TO {user_name}")
with Then(f"I try to ALTER CONSTRAINT"):
alter_constraint_privilege_handler(permutation, table_name, user_name, node)
finally:
with Finally("I drop the user on a cluster"):
node.query(f"DROP USER {user_name} ON CLUSTER sharded_cluster")
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Grant("1.0"),
)
def user_with_privileges_from_user_with_grant_option(self, table_type, node=None):
"""Check that user is able to ALTER CONSTRAINT on a table when granted privilege
from another user with grant option.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
for permutation in permutations():
privileges = alter_constraint_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type),user(node, user0_name), user(node, user1_name):
with When("I grant privileges with grant option to user"):
node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And("I grant privileges to another user via grant option"):
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}",
settings = [("user", user0_name)])
with Then(f"I try to ALTER CONSTRAINT"):
alter_constraint_privilege_handler(permutation, table_name, user1_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Grant("1.0"),
)
def role_with_privileges_from_user_with_grant_option(self, table_type, node=None):
"""Check that user is able to ALTER CONSTRAINT on a table when granted a role with
ALTER CONSTRAINT privilege that was granted by another user with grant option.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations():
privileges = alter_constraint_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with When("I grant subprivileges with grant option to user"):
node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And("I grant privileges to a role via grant option"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name}",
settings = [("user", user0_name)])
with And("I grant the role to another user"):
node.query(f"GRANT {role_name} TO {user1_name}")
with Then(f"I try to ALTER CONSTRAINT"):
alter_constraint_privilege_handler(permutation, table_name, user1_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Grant("1.0"),
)
def user_with_privileges_from_role_with_grant_option(self, table_type, node=None):
"""Check that user is able to ALTER CONSTRAINT on a table when granted privilege from
a role with grant option
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations():
privileges = alter_constraint_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with When(f"I grant privileges with grant option to a role"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION")
with When("I grant role to a user"):
node.query(f"GRANT {role_name} TO {user0_name}")
with And("I grant privileges to a user via grant option"):
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}",
settings = [("user", user0_name)])
with Then(f"I try to ALTER CONSTRAINT"):
alter_constraint_privilege_handler(permutation, table_name, user1_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Grant("1.0"),
)
def role_with_privileges_from_role_with_grant_option(self, table_type, node=None):
"""Check that a user is able to ALTER CONSTRAINT on a table with a role that was
granted privilege by another role with grant option
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role0_name = f"role0_{getuid()}"
role1_name = f"role1_{getuid()}"
for permutation in permutations():
privileges = alter_constraint_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role0_name), role(node, role1_name):
with When(f"I grant privileges"):
node.query(f"GRANT {privileges} ON {table_name} TO {role0_name} WITH GRANT OPTION")
with And("I grant the role to a user"):
node.query(f"GRANT {role0_name} TO {user0_name}")
with And("I grant privileges to another role via grant option"):
node.query(f"GRANT {privileges} ON {table_name} TO {role1_name}",
settings = [("user", user0_name)])
with And("I grant the second role to another user"):
node.query(f"GRANT {role1_name} TO {user1_name}")
with Then(f"I try to ALTER CONSTRAINT"):
alter_constraint_privilege_handler(permutation, table_name, user1_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_user_via_user_with_grant_option(self, table_type, node=None):
"""Check that user is unable to revoke a privilege they don't have access to from a user.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
for permutation in permutations():
privileges = alter_constraint_privileges(permutation)
with When(f"granted={privileges}"):
# This test does not apply when no privileges are granted (permutation 0)
if permutation == 0:
continue
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with Given(f"I grant privileges with grant option to user0"):
node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And(f"I grant privileges with grant option to user1"):
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name} WITH GRANT OPTION",
settings=[("user", user0_name)])
with When("I revoke privilege from user0 using user1"):
node.query(f"REVOKE {privileges} ON {table_name} FROM {user0_name}",
settings=[("user", user1_name)])
with Then("I verify that user0 has privileges revoked"):
exitcode, message = errors.not_enough_privileges(user0_name)
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
node.query(f"REVOKE {privileges} ON {table_name} FROM {user1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_role_via_user_with_grant_option(self, table_type, node=None):
"""Check that user is unable to revoke a privilege they dont have access to from a role.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations():
privileges = alter_constraint_privileges(permutation)
with When(f"granted={privileges}"):
# This test does not apply when no privileges are granted (permutation 0)
if permutation == 0:
continue
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with Given(f"I grant privileges with grant option to role0"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION")
with And("I grant role0 to user0"):
node.query(f"GRANT {role_name} TO {user0_name}")
with And(f"I grant privileges with grant option to user1"):
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name} WITH GRANT OPTION",
settings=[("user", user0_name)])
with When("I revoke privilege from role0 using user1"):
node.query(f"REVOKE {privileges} ON {table_name} FROM {role_name}",
settings=[("user", user1_name)])
with Then("I verify that role0(user0) has privileges revoked"):
exitcode, message = errors.not_enough_privileges(user0_name)
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
node.query(f"REVOKE {privileges} ON {table_name} FROM {user1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_user_via_role_with_grant_option(self, table_type, node=None):
"""Check that user with a role is unable to revoke a privilege they dont have access to from a user.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations():
privileges = alter_constraint_privileges(permutation)
with When(f"granted={privileges}"):
# This test does not apply when no privileges are granted (permutation 0)
if permutation == 0:
continue
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with Given(f"I grant privileges with grant option to user0"):
node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And(f"I grant privileges with grant option to role1"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION",
settings=[("user", user0_name)])
with When("I grant role1 to user1"):
node.query(f"GRANT {role_name} TO {user1_name}")
with And("I revoke privilege from user0 using role1(user1)"):
node.query(f"REVOKE {privileges} ON {table_name} FROM {user0_name}",
settings=[("user" ,user1_name)])
with Then("I verify that user0 has privileges revoked"):
exitcode, message = errors.not_enough_privileges(user0_name)
node.query(f"GRANT {privileges} ON {table_name} TO {role_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
node.query(f"REVOKE {privileges} ON {table_name} FROM {role_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_role_via_role_with_grant_option(self, table_type, node=None):
"""Check that user with a role is unable to revoke a privilege they dont have access to from a role.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role0_name = f"role0_{getuid()}"
role1_name = f"role1_{getuid()}"
for permutation in permutations():
privileges = alter_constraint_privileges(permutation)
with When(f"granted={privileges}"):
# This test does not apply when no privileges are granted (permutation 0)
if permutation == 0:
continue
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role0_name), role(node, role1_name):
with Given(f"I grant privileges with grant option to role0"):
node.query(f"GRANT {privileges} ON {table_name} TO {role0_name} WITH GRANT OPTION")
with And("I grant role0 to user0"):
node.query(f"GRANT {role0_name} TO {user0_name}")
with And(f"I grant privileges with grant option to role1"):
node.query(f"GRANT {privileges} ON {table_name} TO {role1_name} WITH GRANT OPTION",
settings=[("user", user0_name)])
with When("I grant role1 to user1"):
node.query(f"GRANT {role1_name} TO {user1_name}")
with And("I revoke privilege from role0(user0) using role1(user1)"):
node.query(f"REVOKE {privileges} ON {table_name} FROM {role0_name}",
settings=[("user", user1_name)])
with Then("I verify that role0(user0) has privileges revoked"):
exitcode, message = errors.not_enough_privileges(user0_name)
node.query(f"GRANT {privileges} ON {table_name} TO {role1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
node.query(f"REVOKE {privileges} ON {table_name} FROM {role1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
@TestFeature
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterConstraint("1.0"),
RQ_SRS_006_RBAC_Privileges_AlterConstraint_TableEngines("1.0")
)
@Examples("table_type", [
(key,) for key in table_types.keys()
])
@Name("alter constraint")
def feature(self, node="clickhouse1", parallel=None, stress=None):
self.context.node = self.context.cluster.node(node)
if parallel is not None:
self.context.parallel = parallel
if stress is not None:
self.context.stress = stress
for example in self.examples:
table_type, = example
if table_type != "MergeTree" and not self.context.stress:
continue
with Example(str(example)):
pool = Pool(13)
try:
tasks = []
try:
for scenario in loads(current_module(), Scenario):
run_scenario(pool, tasks, Scenario(test=scenario), {"table_type" : table_type})
finally:
join(tasks)
finally:
pool.close()

View File

@ -0,0 +1,815 @@
import json
import random
from testflows._core.cli.arg.parser import parser
from testflows.core import *
from testflows.asserts import error
from rbac.requirements import *
from rbac.helper.common import *
import rbac.helper.errors as errors
from rbac.helper.tables import table_types
subprivileges = {
"ORDER BY" : 1 << 0,
"SAMPLE BY": 1 << 1,
"ADD INDEX" : 1 << 2,
"MATERIALIZE INDEX" : 1 << 3,
"CLEAR INDEX": 1 << 4,
"DROP INDEX": 1 << 5,
}
aliases = {
"ORDER BY" : ["ALTER ORDER BY", "ALTER MODIFY ORDER BY", "MODIFY ORDER BY"],
"SAMPLE BY": ["ALTER SAMPLE BY", "ALTER MODIFY SAMPLE BY", "MODIFY SAMPLE BY"],
"ADD INDEX" : ["ALTER ADD INDEX", "ADD INDEX"],
"MATERIALIZE INDEX" : ["ALTER MATERIALIZE INDEX", "MATERIALIZE INDEX"],
"CLEAR INDEX": ["ALTER CLEAR INDEX", "CLEAR INDEX"],
"DROP INDEX": ["ALTER DROP INDEX", "DROP INDEX"],
"ALTER INDEX": ["ALTER INDEX", "INDEX"] # super-privilege
}
# Extra permutation is for 'ALTER INDEX' super-privilege
permutation_count = (1 << len(subprivileges))
def permutations(table_type):
"""Uses stress flag and table type, returns list of all permutations to run
Stress test: All permutations, all tables (when stress=True)
Sanity test: All permutations for MergeTree, selected permutations* for other tables
"""
if current().context.stress or table_type == "MergeTree":
return [*range(permutation_count + len(aliases["ALTER INDEX"]))]
else:
# *Selected permutations currently stand as [1,2,4,8,16,32,0,42,63,64,65].
# Testing ["ORDER BY", "SAMPLE BY", "ADD INDEX", "MATERIALIZE INDEX", "CLEAR INDEX",
# "DROP INDEX", "NONE", {"DROP, MATERIALIZE, SAMPLE BY"}, all, "ALTER INDEX", and "INDEX"]
return [1 << index for index in range(len(subprivileges))] + \
[0, int('101010', 2), permutation_count-1, permutation_count, permutation_count+1]
def alter_index_privileges(grants: int):
"""Takes in an integer, and returns the corresponding set of tests to grant and
not grant using the binary string. Each integer corresponds to a unique permutation
of grants.
"""
note(grants)
privileges = []
# Extra iteration for ALTER INDEX
if grants >= permutation_count:
privileges.append(aliases["ALTER INDEX"][grants-permutation_count])
elif grants==0: # No privileges
privileges.append("NONE")
else:
if (grants & subprivileges["ORDER BY"]):
privileges.append(aliases["ORDER BY"][grants % len(aliases["ORDER BY"])])
if (grants & subprivileges["SAMPLE BY"]):
privileges.append(aliases["SAMPLE BY"][grants % len(aliases["SAMPLE BY"])])
if (grants & subprivileges["ADD INDEX"]):
privileges.append(aliases["ADD INDEX"][grants % len(aliases["ADD INDEX"])])
if (grants & subprivileges["MATERIALIZE INDEX"]):
privileges.append(aliases["MATERIALIZE INDEX"][grants % len(aliases["MATERIALIZE INDEX"])])
if (grants & subprivileges["CLEAR INDEX"]):
privileges.append(aliases["CLEAR INDEX"][grants % len(aliases["CLEAR INDEX"])])
if (grants & subprivileges["DROP INDEX"]):
privileges.append(aliases["DROP INDEX"][grants % len(aliases["DROP INDEX"])])
note(f"Testing these privileges: {privileges}")
return ', '.join(privileges)
def alter_index_privilege_handler(grants, table, user, node):
"""For all 5 subprivileges, if the privilege is granted: run test to ensure correct behavior,
and if the privilege is not granted, run test to ensure correct behavior there as well.
"""
# Testing ALTER INDEX and INDEX is the same as testing all subprivileges
if grants > permutation_count-1:
grants = permutation_count-1
if (grants & subprivileges["ORDER BY"]):
with When("I check order by when privilege is granted"):
check_order_by_when_privilege_is_granted(table, user, node)
else:
with When("I check order by when privilege is not granted"):
check_order_by_when_privilege_is_not_granted(table, user, node)
if (grants & subprivileges["SAMPLE BY"]):
with When("I check sample by when privilege is granted"):
check_sample_by_when_privilege_is_granted(table, user, node)
else:
with When("I check sample by when privilege is not granted"):
check_sample_by_when_privilege_is_not_granted(table, user, node)
if (grants & subprivileges["ADD INDEX"]):
with When("I check add index when privilege is granted"):
check_add_index_when_privilege_is_granted(table, user, node)
else:
with When("I check add index when privilege is not granted"):
check_add_index_when_privilege_is_not_granted(table, user, node)
if (grants & subprivileges["MATERIALIZE INDEX"]):
with When("I check materialize index when privilege is granted"):
check_materialize_index_when_privilege_is_granted(table, user, node)
else:
with When("I check materialize index when privilege is not granted"):
check_materialize_index_when_privilege_is_not_granted(table, user, node)
if (grants & subprivileges["CLEAR INDEX"]):
with When("I check clear index when privilege is granted"):
check_clear_index_when_privilege_is_granted(table, user, node)
else:
with When("I check clear index when privilege is not granted"):
check_clear_index_when_privilege_is_not_granted(table, user, node)
if (grants & subprivileges["DROP INDEX"]):
with When("I check drop index when privilege is granted"):
check_drop_index_when_privilege_is_granted(table, user, node)
else:
with When("I check drop index when privilege is not granted"):
check_drop_index_when_privilege_is_not_granted(table, user, node)
def check_order_by_when_privilege_is_granted(table, user, node):
"""Ensures ORDER BY runs as expected when the privilege is granted to the specified user
"""
column = "order"
with Given("I run sanity check"):
node.query(f"ALTER TABLE {table} MODIFY ORDER BY d", settings = [("user", user)])
with And("I add new column and modify order using that column"):
node.query(f"ALTER TABLE {table} ADD COLUMN {column} UInt32, MODIFY ORDER BY (d, {column})")
with When(f"I insert random data into the ordered-by column {column}"):
data = random.sample(range(1,1000),100)
values = ', '.join(f'({datum})' for datum in data)
node.query(f"INSERT INTO {table}({column}) VALUES {values}")
with Then("I synchronize with optimize table"):
node.query(f"OPTIMIZE TABLE {table} final")
with And("I verify that the added data is ordered in the table"):
data.sort()
note(data)
column_data = node.query(f"SELECT {column} FROM {table} FORMAT JSONEachRow").output
column_data = column_data.split('\n')
for row, datum in zip(column_data[:10], data[:10]):
assert json.loads(row) == {column:datum}, error()
with And("I verify that the sorting key is present in the table"):
output = json.loads(node.query(f"SHOW CREATE TABLE {table} FORMAT JSONEachRow").output)
assert f"ORDER BY (d, {column})" in output['statement'], error()
with But(f"I cannot drop the required column {column}"):
exitcode, message = errors.missing_columns(column)
node.query(f"ALTER TABLE {table} DROP COLUMN {column}",
exitcode=exitcode, message=message)
def check_sample_by_when_privilege_is_granted(table, user, node):
"""Ensures SAMPLE BY runs as expected when the privilege is granted to the specified user
"""
column = 'sample'
with Given(f"I add new column {column}"):
node.query(f"ALTER TABLE {table} ADD COLUMN {column} UInt32")
with When(f"I add sample by clause"):
node.query(f"ALTER TABLE {table} MODIFY SAMPLE BY (d, {column})",
settings = [("user", user)])
with Then("I verify that the sample is in the table"):
output = json.loads(node.query(f"SHOW CREATE TABLE {table} FORMAT JSONEachRow").output)
assert f"SAMPLE BY (d, {column})" in output['statement'], error()
with But(f"I cannot drop the required column {column}"):
exitcode, message = errors.missing_columns(column)
node.query(f"ALTER TABLE {table} DROP COLUMN {column}",
exitcode=exitcode, message=message)
def check_add_index_when_privilege_is_granted(table, user, node):
"""Ensures ADD INDEX runs as expected when the privilege is granted to the specified user
"""
index = "add"
with Given(f"I add index '{index}'"): # Column x: String
node.query(f"ALTER TABLE {table} ADD INDEX {index}(x) TYPE set(0) GRANULARITY 1",
settings = [("user", user)])
with Then("I verify that the index is in the table"):
output = json.loads(node.query(f"SHOW CREATE TABLE {table} FORMAT JSONEachRow").output)
assert f"INDEX {index} x TYPE set(0) GRANULARITY 1" in output['statement'], error()
with Finally(f"I drop index {index}"):
node.query(f"ALTER TABLE {table} DROP INDEX {index}")
def check_materialize_index_when_privilege_is_granted(table, user, node):
"""Ensures MATERIALIZE INDEX runs as expected when the privilege is granted to the specified user
"""
index = "materialize"
with Given(f"I add index '{index}'"):
node.query(f"ALTER TABLE {table} ADD INDEX {index}(x) TYPE set(0) GRANULARITY 1")
with When(f"I materialize index '{index}'"):
node.query(f"ALTER TABLE {table} MATERIALIZE INDEX {index} IN PARTITION 1 SETTINGS mutations_sync = 2",
settings = [("user", user)])
with Then("I verify that the index is in the table"):
output = json.loads(node.query(f"SHOW CREATE TABLE {table} FORMAT JSONEachRow").output)
assert f"INDEX {index} x TYPE set(0) GRANULARITY 1" in output['statement'], error()
with Finally(f"I drop index {index}"):
node.query(f"ALTER TABLE {table} DROP INDEX {index}")
def check_clear_index_when_privilege_is_granted(table, user, node):
"""Ensures CLEAR INDEX runs as expected when the privilege is granted to the specified user
"""
index = "clear"
with Given(f"I add index '{index}'"): # Column x: String
node.query(f"ALTER TABLE {table} ADD INDEX {index}(x) TYPE set(0) GRANULARITY 1")
with When(f"I clear index {index}"):
node.query(f"ALTER TABLE {table} CLEAR INDEX {index} IN PARTITION 1")
with Then("I verify that the index is in the table"):
output = json.loads(node.query(f"SHOW CREATE TABLE {table} FORMAT JSONEachRow").output)
assert f"INDEX {index} x TYPE set(0) GRANULARITY 1" in output['statement'], error()
with Finally(f"I drop index {index}"):
node.query(f"ALTER TABLE {table} DROP INDEX {index}")
def check_drop_index_when_privilege_is_granted(table, user, node):
"""Ensures DROP INDEX runs as expected when the privilege is granted to the specified user
"""
with When("I try to drop nonexistent index, throws exception"):
exitcode, message = errors.wrong_index_name("fake_index")
node.query(f"ALTER TABLE {table} DROP INDEX fake_index",
settings = [("user", user)], exitcode=exitcode, message=message)
index = "drop"
with Given(f"I add the index"):
node.query(f"ALTER TABLE {table} ADD INDEX {index}(x) TYPE set(0) GRANULARITY 1")
with Then(f"I drop index {index} which exists"):
node.query(f"ALTER TABLE {table} DROP INDEX {index}",
settings = [("user", user)])
with And("I verify that the index is not in the table"):
output = json.loads(node.query(f"SHOW CREATE TABLE {table} FORMAT JSONEachRow").output)
assert f"INDEX {index} x TYPE set(0) GRANULARITY 1" not in output['statement'], error()
def check_order_by_when_privilege_is_not_granted(table, user, node):
"""Ensures ORDER BY errors as expected without the required privilege for the specified user
"""
with When("I try to use privilege that has not been granted"):
exitcode, message = errors.not_enough_privileges(user)
node.query(f"ALTER TABLE {table} MODIFY ORDER BY d",
settings = [("user", user)], exitcode=exitcode, message=message)
def check_sample_by_when_privilege_is_not_granted(table, user, node):
"""Ensures SAMPLE BY errors as expected without the required privilege for the specified user
"""
with When("I try to use privilege that has not been granted"):
exitcode, message = errors.not_enough_privileges(user)
node.query(f"ALTER TABLE {table} MODIFY SAMPLE BY d",
settings = [("user", user)], exitcode=exitcode, message=message)
def check_add_index_when_privilege_is_not_granted(table, user, node):
"""Ensures ADD INDEX errors as expected without the required privilege for the specified user
"""
with When("I try to use privilege that has not been granted"):
exitcode, message = errors.not_enough_privileges(user)
node.query(f"ALTER TABLE {table} ADD INDEX index1 b * length(x) TYPE set(1000) GRANULARITY 4",
settings = [("user", user)], exitcode=exitcode, message=message)
def check_materialize_index_when_privilege_is_not_granted(table, user, node):
"""Ensures MATERIALIZE INDEX errors as expected without the required privilege for the specified user
"""
with When("I try to use privilege that has not been granted"):
exitcode, message = errors.not_enough_privileges(user)
node.query(f"ALTER TABLE {table} MATERIALIZE INDEX index1",
settings = [("user", user)], exitcode=exitcode, message=message)
def check_clear_index_when_privilege_is_not_granted(table, user, node):
"""Ensures CLEAR INDEX errors as expected without the required privilege for the specified user
"""
with When("I try to use privilege that has not been granted"):
exitcode, message = errors.not_enough_privileges(user)
node.query(f"ALTER TABLE {table} CLEAR INDEX index1 IN PARTITION 1",
settings = [("user", user)], exitcode=exitcode, message=message)
def check_drop_index_when_privilege_is_not_granted(table, user, node):
"""Ensures DROP INDEX errors as expected without the required privilege for the specified user
"""
with When("I try to use privilege that has not been granted"):
exitcode, message = errors.not_enough_privileges(user)
node.query(f"ALTER TABLE {table} DROP INDEX index1",
settings = [("user", user)], exitcode=exitcode, message=message)
@TestScenario
@Flags(TE)
def user_with_some_privileges(self, table_type, node=None):
"""Check that user with any permutation of ALTER INDEX subprivileges is able
to alter the table for privileges granted, and not for privileges not granted.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
for permutation in permutations(table_type):
privileges = alter_index_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name):
with Given("I first grant the privileges needed"):
node.query(f"GRANT {privileges} ON {table_name} TO {user_name}")
with Then(f"I try to ALTER INDEX with given privileges"):
alter_index_privilege_handler(permutation, table_name, user_name, node)
@TestScenario
@Flags(TE)
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterIndex_Revoke("1.0"),
)
def user_with_revoked_privileges(self, table_type, node=None):
"""Check that user is unable to ALTER INDEX on table after ALTER INDEX privilege
on that table has been revoked from the user.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
for permutation in permutations(table_type):
privileges = alter_index_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name):
with Given("I first grant the privileges"):
node.query(f"GRANT {privileges} ON {table_name} TO {user_name}")
with And("I then revoke the privileges"):
node.query(f"REVOKE {privileges} ON {table_name} FROM {user_name}")
with When(f"I try to ALTER INDEX with given privileges"):
# Permutation 0: no privileges
alter_index_privilege_handler(0, table_name, user_name, node)
@TestScenario
@Flags(TE)
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterIndex_Grant("1.0"),
)
def role_with_some_privileges(self, table_type, node=None):
"""Check that user can ALTER INDEX on a table after it is granted a role that
has the ALTER INDEX privilege for that table.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations(table_type):
privileges = alter_index_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name), role(node, role_name):
with Given("I grant the ALTER INDEX privilege to a role"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name}")
with And("I grant role to the user"):
node.query(f"GRANT {role_name} TO {user_name}")
with Then(f"I try to ALTER INDEX with given privileges"):
alter_index_privilege_handler(permutation, table_name, user_name, node)
@TestScenario
@Flags(TE)
def user_with_revoked_role(self, table_type, node=None):
"""Check that user with a role that has ALTER INDEX privilege on a table is unable to
ALTER INDEX from that table after the role with privilege has been revoked from the user.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations(table_type):
privileges = alter_index_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name), role(node, role_name):
with When("I grant privileges to a role"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name}")
with And("I grant the role to a user"):
node.query(f"GRANT {role_name} TO {user_name}")
with And("I revoke the role from the user"):
node.query(f"REVOKE {role_name} FROM {user_name}")
with And("I ALTER INDEX on the table"):
# Permutation 0: no privileges for any permutation on these columns
alter_index_privilege_handler(0, table_name, user_name, node)
@TestScenario
@Flags(TE)
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterIndex_Cluster("1.0"),
)
def user_with_privileges_on_cluster(self, table_type, node=None):
"""Check that user is able to ALTER INDEX on a table with
privilege granted on a cluster.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
for permutation in permutations(table_type):
privileges = alter_index_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type):
try:
with Given("I have a user on a cluster"):
node.query(f"CREATE USER OR REPLACE {user_name} ON CLUSTER sharded_cluster")
with When("I grant ALTER INDEX privileges needed for iteration on a cluster"):
node.query(f"GRANT ON CLUSTER sharded_cluster {privileges} ON {table_name} TO {user_name}")
with Then(f"I try to ALTER INDEX with given privileges"):
alter_index_privilege_handler(permutation, table_name, user_name, node)
finally:
with Finally("I drop the user on cluster"):
node.query(f"DROP USER {user_name} ON CLUSTER sharded_cluster")
@TestScenario
@Flags(TE)
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Grant("1.0"),
)
def user_with_privileges_from_user_with_grant_option(self, table_type, node=None):
"""Check that user is able to ALTER INDEX on a table when granted privilege
from another user with grant option.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
for permutation in permutations(table_type):
privileges = alter_index_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type),user(node, user0_name), user(node, user1_name):
with When("I grant privileges needed for iteration with grant option to user"):
node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And("I grant privileges to another user via grant option"):
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}",
settings = [("user", user0_name)])
with Then(f"I try to ALTER INDEX with given privileges"):
alter_index_privilege_handler(permutation, table_name, user1_name, node)
@TestScenario
@Flags(TE)
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Grant("1.0"),
)
def role_with_privileges_from_user_with_grant_option(self, table_type, node=None):
"""Check that user is able to ALTER INDEX on a table when granted a role with
ALTER INDEX privilege that was granted by another user with grant option.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations(table_type):
privileges = alter_index_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with When("I grant needed subprivileges with grant option to user"):
node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And("I grant privileges to a role via grant option"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name}",
settings = [("user", user0_name)])
with And("I grant the role to another user"):
node.query(f"GRANT {role_name} TO {user1_name}")
with Then(f"I try to ALTER INDEX with given privileges"):
alter_index_privilege_handler(permutation, table_name, user1_name, node)
@TestScenario
@Flags(TE)
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Grant("1.0"),
)
def user_with_privileges_from_role_with_grant_option(self, table_type, node=None):
"""Check that user is able to ALTER INDEX on a table when granted privilege from
a role with grant option
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations(table_type):
privileges = alter_index_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with When(f"I grant privileges with grant option to a role"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION")
with And("I grant role to a user"):
node.query(f"GRANT {role_name} TO {user0_name}")
with And("I grant privileges to a user via grant option"):
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}",
settings = [("user", user0_name)])
with Then(f"I try to ALTER INDEX with given privileges"):
alter_index_privilege_handler(permutation, table_name, user1_name, node)
@TestScenario
@Flags(TE)
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Grant("1.0"),
)
def role_with_privileges_from_role_with_grant_option(self, table_type, node=None):
"""Check that a user is able to ALTER INDEX on a table with a role that was
granted privilege by another role with grant option
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role0_name = f"role0_{getuid()}"
role1_name = f"role1_{getuid()}"
for permutation in permutations(table_type):
privileges = alter_index_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role0_name), role(node, role1_name):
with When(f"I grant privilege with grant option to role"):
node.query(f"GRANT {privileges} ON {table_name} TO {role0_name} WITH GRANT OPTION")
with And("I grant the role to a user"):
node.query(f"GRANT {role0_name} TO {user0_name}")
with And("I grant privileges to another role via grant option"):
node.query(f"GRANT {privileges} ON {table_name} TO {role1_name}",
settings = [("user", user0_name)])
with And("I grant the second role to another user"):
node.query(f"GRANT {role1_name} TO {user1_name}")
with Then(f"I try to ALTER INDEX with given privileges"):
alter_index_privilege_handler(permutation, table_name, user1_name, node)
@TestScenario
@Flags(TE)
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_user_via_user_with_grant_option(self, table_type, node=None):
"""Check that user is unable to revoke a privilege they don't have access to from a user.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
for permutation in permutations(table_type):
privileges = alter_index_privileges(permutation)
with When(f"granted={privileges}"):
# This test does not apply when no privileges are granted (permutation 0)
if permutation == 0:
continue
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with Given(f"I grant privileges with grant option to user0"):
node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And(f"I grant privileges with grant option to user1"):
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name} WITH GRANT OPTION",
settings=[("user", user0_name)])
with When("I revoke privilege from user0 using user1"):
node.query(f"REVOKE {privileges} ON {table_name} FROM {user0_name}",
settings=[("user", user1_name)])
with Then("I verify that user0 has privileges revoked"):
exitcode, message = errors.not_enough_privileges(user0_name)
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
node.query(f"REVOKE {privileges} ON {table_name} FROM {user1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
@TestScenario
@Flags(TE)
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_role_via_user_with_grant_option(self, table_type, node=None):
"""Check that user is unable to revoke a privilege they don't have access to from a role.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations(table_type):
privileges = alter_index_privileges(permutation)
with When(f"granted={privileges}"):
# This test does not apply when no privileges are granted (permutation 0)
if permutation == 0:
continue
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with Given(f"I grant privileges with grant option to role0"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION")
with And("I grant role0 to user0"):
node.query(f"GRANT {role_name} TO {user0_name}")
with And(f"I grant privileges with grant option to user1"):
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name} WITH GRANT OPTION",
settings=[("user", user0_name)])
with When("I revoke privilege from role0 using user1"):
node.query(f"REVOKE {privileges} ON {table_name} FROM {role_name}",
settings=[("user", user1_name)])
with Then("I verify that role0(user0) has privileges revoked"):
exitcode, message = errors.not_enough_privileges(user0_name)
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
node.query(f"REVOKE {privileges} ON {table_name} FROM {user1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
@TestScenario
@Flags(TE)
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_user_via_role_with_grant_option(self, table_type, node=None):
"""Check that user with a role is unable to revoke a privilege they don't have access to from a user.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations(table_type):
privileges = alter_index_privileges(permutation)
with When(f"granted={privileges}"):
# This test does not apply when no privileges are granted (permutation 0)
if permutation == 0:
continue
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with Given(f"I grant privileges with grant option to user0"):
node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And(f"I grant privileges with grant option to role1"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION",
settings=[("user", user0_name)])
with When("I grant role1 to user1"):
node.query(f"GRANT {role_name} TO {user1_name}")
with And("I revoke privilege from user0 using role1(user1)"):
node.query(f"REVOKE {privileges} ON {table_name} FROM {user0_name}",
settings=[("user" ,user1_name)])
with Then("I verify that user0 has privileges revoked"):
exitcode, message = errors.not_enough_privileges(user0_name)
node.query(f"GRANT {privileges} ON {table_name} TO {role_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
node.query(f"REVOKE {privileges} ON {table_name} FROM {role_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
@TestScenario
@Flags(TE)
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_role_via_role_with_grant_option(self, table_type, node=None):
"""Check that user with a role is unable to revoke a privilege they don't have access to from a role.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role0_name = f"role0_{getuid()}"
role1_name = f"role1_{getuid()}"
for permutation in permutations(table_type):
privileges = alter_index_privileges(permutation)
with When(f"granted={privileges}"):
# This test does not apply when no privileges are granted (permutation 0)
if permutation == 0:
continue
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role0_name), role(node, role1_name):
with Given(f"I grant privileges with grant option to role0"):
node.query(f"GRANT {privileges} ON {table_name} TO {role0_name} WITH GRANT OPTION")
with And("I grant role0 to user0"):
node.query(f"GRANT {role0_name} TO {user0_name}")
with And(f"I grant privileges with grant option to role1"):
node.query(f"GRANT {privileges} ON {table_name} TO {role1_name} WITH GRANT OPTION",
settings=[("user", user0_name)])
with When("I grant role1 to user1"):
node.query(f"GRANT {role1_name} TO {user1_name}")
with And("I revoke privilege from role0(user0) using role1(user1)"):
node.query(f"REVOKE {privileges} ON {table_name} FROM {role0_name}",
settings=[("user", user1_name)])
with Then("I verify that role0(user0) has privileges revoked"):
exitcode, message = errors.not_enough_privileges(user0_name)
node.query(f"GRANT {privileges} ON {table_name} TO {role1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
node.query(f"REVOKE {privileges} ON {table_name} FROM {role1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
@TestFeature
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterIndex("1.0"),
RQ_SRS_006_RBAC_Privileges_AlterIndex_TableEngines("1.0")
)
@Examples("table_type", [
(key,) for key in table_types.keys()
])
@Flags(TE)
@Name("alter index")
def feature(self, node="clickhouse1", stress=None, parallel=None):
self.context.node = self.context.cluster.node(node)
if parallel is not None:
self.context.parallel = parallel
if stress is not None:
self.context.stress = stress
for example in self.examples:
table_type, = example
if table_type != "MergeTree" and not self.context.stress:
continue
with Example(str(example)):
pool = Pool(13)
try:
tasks = []
try:
for scenario in loads(current_module(), Scenario):
run_scenario(pool, tasks, Scenario(test=scenario), {"table_type" : table_type})
finally:
join(tasks)
finally:
pool.close()

View File

@ -0,0 +1,476 @@
import json
from multiprocessing.dummy import Pool
from testflows.core import *
from testflows.asserts import error
from rbac.requirements import *
from rbac.helper.common import *
import rbac.helper.errors as errors
from rbac.helper.tables import table_types
aliases = {"ALTER SETTINGS", "ALTER SETTING", "ALTER MODIFY SETTING", "MODIFY SETTING"}
def check_alter_settings_when_privilege_is_granted(table, user, node):
"""Ensures ADD SETTINGS runs as expected when the privilege is granted to the specified user
"""
with Given("I check that the modified setting is not already in the table"):
output = json.loads(node.query(f"SHOW CREATE TABLE {table} FORMAT JSONEachRow").output)
assert "merge_with_ttl_timeout = 5" not in output['statement'], error()
with And(f"I modify settings"):
node.query(f"ALTER TABLE {table} MODIFY SETTING merge_with_ttl_timeout=5",
settings = [("user", user)])
with Then("I verify that the setting is in the table"):
output = json.loads(node.query(f"SHOW CREATE TABLE {table} FORMAT JSONEachRow").output)
assert "SETTINGS index_granularity = 8192, merge_with_ttl_timeout = 5" in output['statement'], error()
def check_alter_settings_when_privilege_is_not_granted(table, user, node):
"""Ensures CLEAR SETTINGS runs as expected when the privilege is granted to the specified user
"""
with When("I try to use ALTER SETTING, has not been granted"):
exitcode, message = errors.not_enough_privileges(user)
node.query(f"ALTER TABLE {table} MODIFY SETTING merge_with_ttl_timeout=5",
settings = [("user", user)], exitcode=exitcode, message=message)
@TestScenario
def user_with_privileges(self, privilege, table_type, node=None):
"""Check that user with ALTER SETTINGS privilege is able
to alter the table
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
with table(node, table_name, table_type), user(node, user_name):
with Given("I first grant the privilege"):
node.query(f"GRANT {privilege} ON {table_name} TO {user_name}")
with Then(f"I try to ALTER SETTINGS"):
check_alter_settings_when_privilege_is_granted(table_name, user_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterSettings_Revoke("1.0"),
)
def user_with_revoked_privileges(self, privilege, table_type, node=None):
"""Check that user is unable to alter settingss on table after ALTER SETTINGS privilege
on that table has been revoked from the user.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
with table(node, table_name, table_type), user(node, user_name):
with Given("I first grant the privilege"):
node.query(f"GRANT {privilege} ON {table_name} TO {user_name}")
with And("I then revoke the privileges"):
node.query(f"REVOKE {privilege} ON {table_name} FROM {user_name}")
with When(f"I try to ALTER SETTINGS"):
check_alter_settings_when_privilege_is_not_granted(table_name, user_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterSettings_Grant("1.0"),
)
def role_with_some_privileges(self, privilege, table_type, node=None):
"""Check that user can alter settings on a table after it is granted a role that
has the alter settings privilege for that table.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
with table(node, table_name, table_type), user(node, user_name), role(node, role_name):
with Given("I grant the alter settings privilege to a role"):
node.query(f"GRANT {privilege} ON {table_name} TO {role_name}")
with And("I grant role to the user"):
node.query(f"GRANT {role_name} TO {user_name}")
with Then(f"I try to ALTER SETTINGS"):
check_alter_settings_when_privilege_is_granted(table_name, user_name, node)
@TestScenario
def user_with_revoked_role(self, privilege, table_type, node=None):
"""Check that user with a role that has alter settings privilege on a table is unable to
alter settings from that table after the role with privilege has been revoked from the user.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
with table(node, table_name, table_type), user(node, user_name), role(node, role_name):
with When("I grant privileges to a role"):
node.query(f"GRANT {privilege} ON {table_name} TO {role_name}")
with And("I grant the role to a user"):
node.query(f"GRANT {role_name} TO {user_name}")
with And("I revoke the role from the user"):
node.query(f"REVOKE {role_name} FROM {user_name}")
with And("I alter settings on the table"):
check_alter_settings_when_privilege_is_not_granted(table_name, user_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterSettings_Cluster("1.0"),
)
def user_with_privileges_on_cluster(self, privilege, table_type, node=None):
"""Check that user is able to alter settings on a table with
privilege granted on a cluster.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
with When(f"granted=ALTER SETTINGS"):
with table(node, table_name, table_type):
try:
with Given("I have a user on a cluster"):
node.query(f"CREATE USER OR REPLACE {user_name} ON CLUSTER sharded_cluster")
with When("I grant alter settings privileges on a cluster"):
node.query(f"GRANT ON CLUSTER sharded_cluster ALTER SETTINGS ON {table_name} TO {user_name}")
with Then(f"I try to ALTER SETTINGS"):
check_alter_settings_when_privilege_is_granted(table_name, user_name, node)
with When("I revoke alter settings privileges on a cluster"):
node.query(f"REVOKE ON CLUSTER sharded_cluster ALTER SETTINGS ON {table_name} FROM {user_name}")
with Then(f"I try to ALTER SETTINGS"):
check_alter_settings_when_privilege_is_not_granted(table_name, user_name, node)
finally:
with Finally("I drop the user on a cluster"):
node.query(f"DROP USER {user_name} ON CLUSTER sharded_cluster")
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Grant("1.0"),
)
def user_with_privileges_from_user_with_grant_option(self, privilege, table_type, node=None):
"""Check that user is able to alter settings on a table when granted privilege
from another user with grant option.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with When("I grant privileges with grant option to user"):
node.query(f"GRANT {privilege} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And("I grant privileges to another user via grant option"):
node.query(f"GRANT {privilege} ON {table_name} TO {user1_name}",
settings = [("user", user0_name)])
with Then(f"I try to ALTER SETTINGS"):
check_alter_settings_when_privilege_is_granted(table_name, user1_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Grant("1.0"),
)
def role_with_privileges_from_user_with_grant_option(self, privilege, table_type, node=None):
"""Check that user is able to alter settings on a table when granted a role with
alter settings privilege that was granted by another user with grant option.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with When("I grant subprivileges with grant option to user"):
node.query(f"GRANT {privilege} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And("I grant privileges to a role via grant option"):
node.query(f"GRANT {privilege} ON {table_name} TO {role_name}",
settings = [("user", user0_name)])
with And("I grant the role to another user"):
node.query(f"GRANT {role_name} TO {user1_name}")
with Then(f"I try to ALTER SETTINGS"):
check_alter_settings_when_privilege_is_granted(table_name, user1_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Grant("1.0"),
)
def user_with_privileges_from_role_with_grant_option(self, privilege, table_type, node=None):
"""Check that user is able to alter settings on a table when granted privilege from
a role with grant option
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with When(f"I grant privileges with grant option to a role"):
node.query(f"GRANT {privilege} ON {table_name} TO {role_name} WITH GRANT OPTION")
with When("I grant role to a user"):
node.query(f"GRANT {role_name} TO {user0_name}")
with And("I grant privileges to a user via grant option"):
node.query(f"GRANT {privilege} ON {table_name} TO {user1_name}",
settings = [("user", user0_name)])
with Then(f"I try to ALTER SETTINGS"):
check_alter_settings_when_privilege_is_granted(table_name, user1_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Grant("1.0"),
)
def role_with_privileges_from_role_with_grant_option(self, privilege, table_type, node=None):
"""Check that a user is able to alter settings on a table with a role that was
granted privilege by another role with grant option
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role0_name = f"role0_{getuid()}"
role1_name = f"role1_{getuid()}"
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role0_name), role(node, role1_name):
with When(f"I grant privilege with grant option to role"):
node.query(f"GRANT {privilege} ON {table_name} TO {role0_name} WITH GRANT OPTION")
with And("I grant the role to a user"):
node.query(f"GRANT {role0_name} TO {user0_name}")
with And("I grant privileges to another role via grant option"):
node.query(f"GRANT {privilege} ON {table_name} TO {role1_name}",
settings = [("user", user0_name)])
with And("I grant the second role to another user"):
node.query(f"GRANT {role1_name} TO {user1_name}")
with Then(f"I try to ALTER SETTINGS"):
check_alter_settings_when_privilege_is_granted(table_name, user1_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_user_via_user_with_grant_option(self, privilege, table_type, node=None):
"""Check that user is unable to revoke a privilege they don't have access to from a user.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with Given(f"I grant privileges with grant option to user0"):
node.query(f"GRANT {privilege} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And(f"I grant privileges with grant option to user1"):
node.query(f"GRANT {privilege} ON {table_name} TO {user1_name} WITH GRANT OPTION",
settings=[("user", user0_name)])
with When("I revoke privilege from user0 using user1"):
node.query(f"REVOKE {privilege} ON {table_name} FROM {user0_name}",
settings=[("user", user1_name)])
with Then("I verify that user0 has privileges revoked"):
exitcode, message = errors.not_enough_privileges(user0_name)
node.query(f"GRANT {privilege} ON {table_name} TO {user1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
node.query(f"REVOKE {privilege} ON {table_name} FROM {user1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_role_via_user_with_grant_option(self, privilege, table_type, node=None):
"""Check that user is unable to revoke a privilege they don't have access to from a role.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user_{getuid()}"
user1_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with Given(f"I grant privileges with grant option to role0"):
node.query(f"GRANT {privilege} ON {table_name} TO {role_name} WITH GRANT OPTION")
with And("I grant role0 to user0"):
node.query(f"GRANT {role_name} TO {user0_name}")
with And(f"I grant privileges with grant option to user1"):
node.query(f"GRANT {privilege} ON {table_name} TO {user1_name} WITH GRANT OPTION",
settings=[("user", user0_name)])
with When("I revoke privilege from role0 using user1"):
node.query(f"REVOKE {privilege} ON {table_name} FROM {role_name}",
settings=[("user", user1_name)])
with Then("I verify that role0(user0) has privileges revoked"):
exitcode, message = errors.not_enough_privileges(user0_name)
node.query(f"GRANT {privilege} ON {table_name} TO {user1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
node.query(f"REVOKE {privilege} ON {table_name} FROM {user1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_user_via_role_with_grant_option(self, privilege, table_type, node=None):
"""Check that user with a role is unable to revoke a privilege they don't have access to from a user.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with Given(f"I grant privileges with grant option to user0"):
node.query(f"GRANT {privilege} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And(f"I grant privileges with grant option to role1"):
node.query(f"GRANT {privilege} ON {table_name} TO {role_name} WITH GRANT OPTION",
settings=[("user", user0_name)])
with When("I grant role1 to user1"):
node.query(f"GRANT {role_name} TO {user1_name}")
with And("I revoke privilege from user0 using role1(user1)"):
node.query(f"REVOKE {privilege} ON {table_name} FROM {user0_name}",
settings=[("user" ,user1_name)])
with Then("I verify that user0 has privileges revoked"):
exitcode, message = errors.not_enough_privileges(user0_name)
node.query(f"GRANT {privilege} ON {table_name} TO {role_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
node.query(f"REVOKE {privilege} ON {table_name} FROM {role_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_role_via_role_with_grant_option(self, privilege, table_type, node=None):
"""Check that user with a role is unable to revoke a privilege they don't have acces to from a role.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user_{getuid()}"
user1_name = f"user_{getuid()}"
role0_name = f"role0_{getuid()}"
role1_name = f"role1_{getuid()}"
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role0_name), role(node, role1_name):
with Given(f"I grant privileges with grant option to role0"):
node.query(f"GRANT {privilege} ON {table_name} TO {role0_name} WITH GRANT OPTION")
with And("I grant role0 to user0"):
node.query(f"GRANT {role0_name} TO {user0_name}")
with And(f"I grant privileges with grant option to role1"):
node.query(f"GRANT {privilege} ON {table_name} TO {role1_name} WITH GRANT OPTION",
settings=[("user", user0_name)])
with When("I grant role1 to user1"):
node.query(f"GRANT {role1_name} TO {user1_name}")
with And("I revoke privilege from role0(user0) using role1(user1)"):
node.query(f"REVOKE {privilege} ON {table_name} FROM {role0_name}",
settings=[("user", user1_name)])
with Then("I verify that role0(user0) has privileges revoked"):
exitcode, message = errors.not_enough_privileges(user0_name)
node.query(f"GRANT {privilege} ON {table_name} TO {role1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
node.query(f"REVOKE {privilege} ON {table_name} FROM {role1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
@TestFeature
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterSettings("1.0"),
RQ_SRS_006_RBAC_Privileges_AlterSettings_TableEngines("1.0")
)
@Examples("table_type", [
(key,) for key in table_types.keys()
])
@Name("alter settings")
def feature(self, node="clickhouse1", stress=None, parallel=None):
"""Runs test suites above which check correctness over scenarios and permutations
"""
self.context.node = self.context.cluster.node(node)
if parallel is not None:
self.context.parallel = parallel
if stress is not None:
self.context.stress = stress
for example in self.examples:
table_type, = example
if table_type != "MergeTree" and not self.context.stress:
continue
with Example(str(example)):
pool = Pool(13)
try:
tasks = []
try:
for alias in aliases:
for scenario in loads(current_module(), Scenario):
with Suite(name=f"{alias}"):
run_scenario(pool, tasks, Scenario(test=scenario), {"table_type": table_type, "privilege": alias})
finally:
join(tasks)
finally:
pool.close()

View File

@ -0,0 +1,605 @@
import json
from testflows.core import *
from testflows.core import threading
from testflows.asserts import error
from rbac.requirements import *
from rbac.helper.common import *
import rbac.helper.errors as errors
from rbac.helper.tables import table_types
subprivileges = {
"TTL" : 1 << 0,
"MATERIALIZE TTL" : 1 << 1,
}
aliases = {
"TTL" : ["ALTER TTL", "ALTER MODIFY TTL", "MODIFY TTL"],
"MATERIALIZE TTL": ["ALTER MATERIALIZE TTL", "MATERIALIZE TTL"],
}
permutation_count = (1 << len(subprivileges))
def permutations():
"""Returns list of all permutations to run.
Currently includes NONE, TTL, MATERIALIZE TTL, and both
"""
return [*range(permutation_count)]
def alter_ttl_privileges(grants: int):
"""Takes in an integer, and returns the corresponding set of tests to grant and
not grant using the binary string. Each integer corresponds to a unique permutation
of grants.
"""
note(grants)
privileges = []
if grants==0: # No privileges
privileges.append("NONE")
else:
if (grants & subprivileges["TTL"]):
privileges.append(f"ALTER TTL")
if (grants & subprivileges["MATERIALIZE TTL"]):
privileges.append(f"ALTER MATERIALIZE TTL")
note(f"Testing these privileges: {privileges}")
return ', '.join(privileges)
def alter_ttl_privilege_handler(grants, table, user, node):
"""For all 2 subprivileges, if the privilege is granted: run test to ensure correct behavior,
and if the privilege is not granted, run test to ensure correct behavior there as well
"""
if (grants & subprivileges["TTL"]):
with When("I check ttl when privilege is granted"):
check_ttl_when_privilege_is_granted(table, user, node)
else:
with When("I check ttl when privilege is not granted"):
check_ttl_when_privilege_is_not_granted(table, user, node)
if (grants & subprivileges["MATERIALIZE TTL"]):
with When("I check materialize ttl when privilege is granted"):
check_materialize_ttl_when_privilege_is_granted(table, user, node)
else:
with When("I check materialize ttl when privilege is not granted"):
check_materialize_ttl_when_privilege_is_not_granted(table, user, node)
def check_ttl_when_privilege_is_granted(table, user, node):
"""Ensures ALTER TTL runs as expected when the privilege is granted to the specified user
"""
with Given(f"I modify TTL"):
node.query(f"ALTER TABLE {table} MODIFY TTL d + INTERVAL 1 DAY;",
settings = [("user", user)])
with Then("I verify that the TTL clause is in the table"):
output = json.loads(node.query(f"SHOW CREATE TABLE {table} FORMAT JSONEachRow").output)
assert "TTL d + toIntervalDay(1)" in output['statement'], error()
def check_materialize_ttl_when_privilege_is_granted(table, user, node):
"""Ensures MATERIALIZE TTL runs as expected when the privilege is granted to the specified user
"""
with Given("I modify TTL so it exists"):
node.query(f"ALTER TABLE {table} MODIFY TTL d + INTERVAL 1 MONTH;")
with Then("I materialize the TTL"):
node.query(f"ALTER TABLE {table} MATERIALIZE TTL IN PARTITION 2",
settings = [("user", user)])
with Then("I verify that the TTL clause is in the table"):
output = json.loads(node.query(f"SHOW CREATE TABLE {table} FORMAT JSONEachRow").output)
assert "TTL d + toIntervalMonth(1)" in output['statement'], error()
def check_ttl_when_privilege_is_not_granted(table, user, node):
"""Ensures ALTER TTL errors as expected without the required privilege for the specified user
"""
with When("I try to use privilege that has not been granted"):
exitcode, message = errors.not_enough_privileges(user)
node.query(f"ALTER TABLE {table} MODIFY TTL d + INTERVAL 1 DAY;",
settings = [("user", user)], exitcode=exitcode, message=message)
def check_materialize_ttl_when_privilege_is_not_granted(table, user, node):
"""Ensures MATERIALIZE TTL errors as expected without the required privilege for the specified user
"""
with When("I try to use privilege that has not been granted"):
exitcode, message = errors.not_enough_privileges(user)
node.query(f"ALTER TABLE {table} MATERIALIZE TTL IN PARTITION 4",
settings = [("user", user)], exitcode=exitcode, message=message)
@TestScenario
def user_with_some_privileges(self, table_type, node=None):
"""Check that user with any permutation of ALTER TTL subprivileges is able
to alter the table for privileges granted, and not for privileges not granted.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
for permutation in permutations():
privileges = alter_ttl_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name):
with Given("I first grant the privileges"):
node.query(f"GRANT {privileges} ON {table_name} TO {user_name}")
with Then(f"I try to ALTER TTL"):
alter_ttl_privilege_handler(permutation, table_name, user_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterTTL_Revoke("1.0"),
)
def user_with_revoked_privileges(self, table_type, node=None):
"""Check that user is unable to ALTER TTLs on table after ALTER TTL privilege
on that table has been revoked from the user.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
for permutation in permutations():
privileges = alter_ttl_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name):
with Given("I first grant the privileges"):
node.query(f"GRANT {privileges} ON {table_name} TO {user_name}")
with And("I then revoke the privileges"):
node.query(f"REVOKE {privileges} ON {table_name} FROM {user_name}")
with When(f"I try to ALTER TTL"):
# Permutation 0: no privileges
alter_ttl_privilege_handler(0, table_name, user_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterTTL_Grant("1.0"),
)
def role_with_some_privileges(self, table_type, node=None):
"""Check that user can ALTER TTL on a table after it is granted a role that
has the ALTER TTL privilege for that table.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations():
privileges = alter_ttl_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name), role(node, role_name):
with Given("I grant the ALTER TTL privilege to a role"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name}")
with And("I grant role to the user"):
node.query(f"GRANT {role_name} TO {user_name}")
with Then(f"I try to ALTER TTL"):
alter_ttl_privilege_handler(permutation, table_name, user_name, node)
@TestScenario
def user_with_revoked_role(self, table_type, node=None):
"""Check that user with a role that has ALTER TTL privilege on a table is unable to
ALTER TTL from that table after the role with privilege has been revoked from the user.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations():
privileges = alter_ttl_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user_name), role(node, role_name):
with When("I grant privileges to a role"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name}")
with And("I grant the role to a user"):
node.query(f"GRANT {role_name} TO {user_name}")
with And("I revoke the role from the user"):
node.query(f"REVOKE {role_name} FROM {user_name}")
with And("I ALTER TTL on the table"):
# Permutation 0: no privileges for any permutation
alter_ttl_privilege_handler(0, table_name, user_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterTTL_Cluster("1.0"),
)
def user_with_privileges_on_cluster(self, table_type, node=None):
"""Check that user is able to ALTER TTL on a table with
privilege granted on a cluster.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user_name = f"user_{getuid()}"
for permutation in permutations():
privileges = alter_ttl_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type):
try:
with Given("I have a user on a cluster"):
node.query(f"CREATE USER OR REPLACE {user_name} ON CLUSTER sharded_cluster")
with When("I grant ALTER TTL privileges on a cluster"):
node.query(f"GRANT ON CLUSTER sharded_cluster {privileges} ON {table_name} TO {user_name}")
with Then(f"I try to ALTER TTL"):
alter_ttl_privilege_handler(permutation, table_name, user_name, node)
finally:
with Finally("I drop the user on a cluster"):
node.query(f"DROP USER {user_name} ON CLUSTER sharded_cluster")
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Grant("1.0"),
)
def user_with_privileges_from_user_with_grant_option(self, table_type, node=None):
"""Check that user is able to ALTER TTL on a table when granted privilege
from another user with grant option.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
for permutation in permutations():
privileges = alter_ttl_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type),user(node, user0_name), user(node, user1_name):
with When("I grant privileges with grant option to user"):
node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And("I grant privileges to another user via grant option"):
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}",
settings = [("user", user0_name)])
with Then(f"I try to ALTER TTL"):
alter_ttl_privilege_handler(permutation, table_name, user1_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Grant("1.0"),
)
def role_with_privileges_from_user_with_grant_option(self, table_type, node=None):
"""Check that user is able to ALTER TTL on a table when granted a role with
ALTER TTL privilege that was granted by another user with grant option.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations():
privileges = alter_ttl_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with When("I grant subprivileges with grant option to user"):
node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And("I grant privileges to a role via grant option"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name}",
settings = [("user", user0_name)])
with And("I grant the role to another user"):
node.query(f"GRANT {role_name} TO {user1_name}")
with Then(f"I try to ALTER TTL"):
alter_ttl_privilege_handler(permutation, table_name, user1_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Grant("1.0"),
)
def user_with_privileges_from_role_with_grant_option(self, table_type, node=None):
"""Check that user is able to ALTER TTL on a table when granted privilege from
a role with grant option
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations():
privileges = alter_ttl_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with When(f"I grant privileges with grant option to a role"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION")
with When("I grant role to a user"):
node.query(f"GRANT {role_name} TO {user0_name}")
with And("I grant privileges to a user via grant option"):
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}",
settings = [("user", user0_name)])
with Then(f"I try to ALTER TTL"):
alter_ttl_privilege_handler(permutation, table_name, user1_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Grant("1.0"),
)
def role_with_privileges_from_role_with_grant_option(self, table_type, node=None):
"""Check that a user is able to ALTER TTL on a table with a role that was
granted privilege by another role with grant option
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role0_name = f"role0_{getuid()}"
role1_name = f"role1_{getuid()}"
for permutation in permutations():
privileges = alter_ttl_privileges(permutation)
with When(f"granted={privileges}"):
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role0_name), role(node, role1_name):
with When(f"I grant privileges"):
node.query(f"GRANT {privileges} ON {table_name} TO {role0_name} WITH GRANT OPTION")
with And("I grant the role to a user"):
node.query(f"GRANT {role0_name} TO {user0_name}")
with And("I grant privileges to another role via grant option"):
node.query(f"GRANT {privileges} ON {table_name} TO {role1_name}",
settings = [("user", user0_name)])
with And("I grant the second role to another user"):
node.query(f"GRANT {role1_name} TO {user1_name}")
with Then(f"I try to ALTER TTL"):
alter_ttl_privilege_handler(permutation, table_name, user1_name, node)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_user_via_user_with_grant_option(self, table_type, node=None):
"""Check that user is unable to revoke a privilege they don't have access to from a user.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
for permutation in permutations():
privileges = alter_ttl_privileges(permutation)
with When(f"granted={privileges}"):
# This test does not apply when no privileges are granted (permutation 0)
if permutation == 0:
continue
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with Given(f"I grant privileges with grant option to user0"):
node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And(f"I grant privileges with grant option to user1"):
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name} WITH GRANT OPTION",
settings=[("user", user0_name)])
with When("I revoke privilege from user0 using user1"):
node.query(f"REVOKE {privileges} ON {table_name} FROM {user0_name}",
settings=[("user", user1_name)])
with Then("I verify that user0 has privileges revoked"):
exitcode, message = errors.not_enough_privileges(user0_name)
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
node.query(f"REVOKE {privileges} ON {table_name} FROM {user1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_role_via_user_with_grant_option(self, table_type, node=None):
"""Check that user is unable to revoke a privilege they dont have access to from a role.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations():
privileges = alter_ttl_privileges(permutation)
with When(f"granted={privileges}"):
# This test does not apply when no privileges are granted (permutation 0)
if permutation == 0:
continue
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with Given(f"I grant privileges with grant option to role0"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION")
with And("I grant role0 to user0"):
node.query(f"GRANT {role_name} TO {user0_name}")
with And(f"I grant privileges with grant option to user1"):
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name} WITH GRANT OPTION",
settings=[("user", user0_name)])
with When("I revoke privilege from role0 using user1"):
node.query(f"REVOKE {privileges} ON {table_name} FROM {role_name}",
settings=[("user", user1_name)])
with Then("I verify that role0(user0) has privileges revoked"):
exitcode, message = errors.not_enough_privileges(user0_name)
node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
node.query(f"REVOKE {privileges} ON {table_name} FROM {user1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_user_via_role_with_grant_option(self, table_type, node=None):
"""Check that user with a role is unable to revoke a privilege they dont have access to from a user.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
for permutation in permutations():
privileges = alter_ttl_privileges(permutation)
with When(f"granted={privileges}"):
# This test does not apply when no privileges are granted (permutation 0)
if permutation == 0:
continue
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role_name):
with Given(f"I grant privileges with grant option to user0"):
node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And(f"I grant privileges with grant option to role1"):
node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION",
settings=[("user", user0_name)])
with When("I grant role1 to user1"):
node.query(f"GRANT {role_name} TO {user1_name}")
with And("I revoke privilege from user0 using role1(user1)"):
node.query(f"REVOKE {privileges} ON {table_name} FROM {user0_name}",
settings=[("user" ,user1_name)])
with Then("I verify that user0 has privileges revoked"):
exitcode, message = errors.not_enough_privileges(user0_name)
node.query(f"GRANT {privileges} ON {table_name} TO {role_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
node.query(f"REVOKE {privileges} ON {table_name} FROM {role_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Revoke("1.0"),
)
def revoke_privileges_from_role_via_role_with_grant_option(self, table_type, node=None):
"""Check that user with a role is unable to revoke a privilege they dont have access to from a role.
"""
if node is None:
node = self.context.node
table_name = f"merge_tree_{getuid()}"
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role0_name = f"role0_{getuid()}"
role1_name = f"role1_{getuid()}"
for permutation in permutations():
privileges = alter_ttl_privileges(permutation)
with When(f"granted={privileges}"):
# This test does not apply when no privileges are granted (permutation 0)
if permutation == 0:
continue
with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name):
with role(node, role0_name), role(node, role1_name):
with Given(f"I grant privileges with grant option to role0"):
node.query(f"GRANT {privileges} ON {table_name} TO {role0_name} WITH GRANT OPTION")
with And("I grant role0 to user0"):
node.query(f"GRANT {role0_name} TO {user0_name}")
with And(f"I grant privileges with grant option to role1"):
node.query(f"GRANT {privileges} ON {table_name} TO {role1_name} WITH GRANT OPTION",
settings=[("user", user0_name)])
with When("I grant role1 to user1"):
node.query(f"GRANT {role1_name} TO {user1_name}")
with And("I revoke privilege from role0(user0) using role1(user1)"):
node.query(f"REVOKE {privileges} ON {table_name} FROM {role0_name}",
settings=[("user", user1_name)])
with Then("I verify that role0(user0) has privileges revoked"):
exitcode, message = errors.not_enough_privileges(user0_name)
node.query(f"GRANT {privileges} ON {table_name} TO {role1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
node.query(f"REVOKE {privileges} ON {table_name} FROM {role1_name}",
settings=[("user", user0_name)], exitcode=exitcode, message=message)
@TestFeature
@Requirements(
RQ_SRS_006_RBAC_Privileges_AlterTTL("1.0"),
RQ_SRS_006_RBAC_Privileges_AlterTTL_TableEngines("1.0")
)
@Examples("table_type", [
(key,) for key in table_types.keys()
])
@Name("alter ttl")
def feature(self, node="clickhouse1", stress=None, parallel=None):
self.context.node = self.context.cluster.node(node)
if parallel is not None:
self.context.parallel = parallel
if stress is not None:
self.context.stress = stress
for example in self.examples:
table_type, = example
if table_type != "MergeTree" and not self.context.stress:
continue
with Example(str(example)):
pool = Pool(13)
try:
tasks = []
try:
for scenario in loads(current_module(), Scenario):
run_scenario(pool, tasks, Scenario(test=scenario), {"table_type" : table_type})
finally:
join(tasks)
finally:
pool.close()

File diff suppressed because it is too large Load Diff

View File

@ -1,7 +1,27 @@
from testflows.core import * from testflows.core import *
from rbac.helper.common import *
@TestFeature @TestFeature
@Name("privileges") @Name("privileges")
def feature(self): def feature(self):
Feature(run=load("rbac.tests.privileges.insert", "feature"), flags=TE)
Feature(run=load("rbac.tests.privileges.select", "feature"), flags=TE) tasks = []
pool = Pool(10)
try:
try:
run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.insert", "feature"), flags=TE), {})
run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.select", "feature"), flags=TE), {})
run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.show_tables", "feature"), flags=TE), {})
run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.public_tables", "feature"), flags=TE), {})
run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.distributed_table", "feature"), flags=TE), {})
run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_column", "feature"), flags=TE), {})
run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_index", "feature"), flags=TE), {})
run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_constraint", "feature"), flags=TE), {})
run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_ttl", "feature"), flags=TE), {})
run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_settings", "feature"), flags=TE), {})
finally:
join(tasks)
finally:
pool.close()

View File

@ -5,101 +5,50 @@ from testflows.core import *
from testflows.asserts import error from testflows.asserts import error
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors from rbac.helper.common import *
import rbac.helper.errors as errors
table_types = { def input_output_equality_check(node, input_columns, input_data, table_name):
"MergeTree": "CREATE TABLE {name} (d DATE, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree(d, (a, b), 111)",
"ReplacingMergeTree": "CREATE TABLE {name} (d DATE, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplacingMergeTree(d, (a, b), 111)",
"SummingMergeTree": "CREATE TABLE {name} (d DATE, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = SummingMergeTree(d, (a, b), 111)",
"AggregatingMergeTree": "CREATE TABLE {name} (d DATE, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = AggregatingMergeTree(d, (a, b), 111)",
"CollapsingMergeTree": "CREATE TABLE {name} (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = CollapsingMergeTree(d, (a, b), 111, y);",
"VersionedCollapsingMergeTree": "CREATE TABLE {name} (d Date, a String, b UInt8, x String, y Int8, z UInt32, version UInt64, sign Int8, INDEX a (b * y, d) TYPE minmax GRANULARITY 3) ENGINE = VersionedCollapsingMergeTree(sign, version) ORDER BY tuple()",
"GraphiteMergeTree": "CREATE TABLE {name} (key UInt32, Path String, Time DateTime, d Date, a String, b UInt8, x String, y Int8, z UInt32, Value Float64, Version UInt32, col UInt64, INDEX a (key * Value, Time) TYPE minmax GRANULARITY 3) ENGINE = GraphiteMergeTree('graphite_rollup_example') ORDER BY tuple()"
}
table_requirements ={
"MergeTree": RQ_SRS_006_RBAC_Privileges_Insert_MergeTree("1.0"),
"ReplacingMergeTree": RQ_SRS_006_RBAC_Privileges_Insert_ReplacingMergeTree("1.0"),
"SummingMergeTree": RQ_SRS_006_RBAC_Privileges_Insert_SummingMergeTree("1.0"),
"AggregatingMergeTree": RQ_SRS_006_RBAC_Privileges_Insert_AggregatingMergeTree("1.0"),
"CollapsingMergeTree": RQ_SRS_006_RBAC_Privileges_Insert_CollapsingMergeTree("1.0"),
"VersionedCollapsingMergeTree": RQ_SRS_006_RBAC_Privileges_Insert_VersionedCollapsingMergeTree("1.0"),
"GraphiteMergeTree": RQ_SRS_006_RBAC_Privileges_Insert_GraphiteMergeTree("1.0"),
}
@contextmanager
def table(node, name, table_type="MergeTree"):
try:
with Given(f"I have a {table_type} table"):
node.query(table_types[table_type].format(name=name))
yield
finally:
with Finally("I drop the table"):
node.query(f"DROP TABLE IF EXISTS {name}")
@contextmanager
def user(node, name):
try:
names = name.split(",")
for i in names:
with Given("I have a user"):
node.query(f"CREATE USER OR REPLACE {i}")
yield
finally:
for i in names:
with Finally("I drop the user"):
node.query(f"DROP USER IF EXISTS {name}")
@contextmanager
def role(node, role):
try:
roles = role.split(",")
for j in roles:
with Given("I have a role"):
node.query(f"CREATE ROLE OR REPLACE {j}")
yield
finally:
for j in roles:
with Finally("I drop the role"):
node.query(f"DROP ROLE IF EXISTS {role}")
def input_output_equality_check(node, input_columns, input_data):
data_list = [x.strip("'") for x in input_data.split(",")] data_list = [x.strip("'") for x in input_data.split(",")]
input_dict = dict(list(zip(input_columns.split(","), data_list))) input_dict = dict(zip(input_columns.split(","), data_list))
output_dict = json.loads(node.query(f"select {input_columns} from merge_tree format JSONEachRow").output) output_dict = json.loads(node.query(f"select {input_columns} from {table_name} format JSONEachRow").output)
output_dict = {k:str(v) for (k,v) in list(output_dict.items())} output_dict = {k:str(v) for (k,v) in output_dict.items()}
return input_dict == output_dict return input_dict == output_dict
@TestScenario @TestScenario
def without_privilege(self, table_type, node=None): def without_privilege(self, table_type, node=None):
"""Check that user without insert privilege on a table is not able to insert on that table. """Check that user without insert privilege on a table is not able to insert on that table.
""" """
user_name = f"user_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"): with user(node, user_name):
with When("I run INSERT without privilege"): with When("I run INSERT without privilege"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')", settings = [("user","user0")], node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')", settings = [("user", user_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
@TestScenario @TestScenario
@Requirements( @Requirements(
RQ_SRS_006_RBAC_Privileges_Insert_Grant("1.0"), RQ_SRS_006_RBAC_Privileges_Insert_Grant("1.0"),
) )
def user_with_privilege(self, table_type, node=None): def user_with_privilege(self, table_type, node=None):
"""Check that user can insert into a table on which they have insert privilege and the inserted data is correct. """Check that user can insert into a table on which they have insert privilege.
""" """
user_name = f"user_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"): with user(node, user_name):
with When("I grant privilege"): with When("I grant insert privilege"):
node.query("GRANT INSERT ON merge_tree TO user0") node.query(f"GRANT INSERT ON {table_name} TO {user_name}")
with And("I use INSERT"): with And("I use INSERT"):
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')", settings=[("user","user0")]) node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')", settings=[("user",user_name)])
with Then("I check the insert functioned"): with Then("I check the insert functioned"):
output = node.query("SELECT d FROM merge_tree FORMAT JSONEachRow").output output = node.query(f"SELECT d FROM {table_name} FORMAT JSONEachRow").output
assert output == '{"d":"2020-01-01"}', error() assert output == '{"d":"2020-01-01"}', error()
@TestScenario @TestScenario
@ -109,18 +58,20 @@ def user_with_privilege(self, table_type, node=None):
def user_with_revoked_privilege(self, table_type, node=None): def user_with_revoked_privilege(self, table_type, node=None):
"""Check that user is unable to insert into a table after insert privilege on that table has been revoked from user. """Check that user is unable to insert into a table after insert privilege on that table has been revoked from user.
""" """
user_name = f"user_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"): with user(node, user_name):
with When("I grant privilege"): with When("I grant insert privilege"):
node.query("GRANT INSERT ON merge_tree TO user0") node.query(f"GRANT INSERT ON {table_name} TO {user_name}")
with And("I revoke privilege"): with And("I revoke insert privilege"):
node.query("REVOKE INSERT ON merge_tree FROM user0") node.query(f"REVOKE INSERT ON {table_name} FROM {user_name}")
with And("I use INSERT"): with And("I use INSERT"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')", node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')",
settings=[("user","user0")], exitcode=exitcode, message=message) settings=[("user",user_name)], exitcode=exitcode, message=message)
@TestScenario @TestScenario
def user_with_privilege_on_columns(self, table_type): def user_with_privilege_on_columns(self, table_type):
@ -129,7 +80,7 @@ def user_with_privilege_on_columns(self, table_type):
[tuple(list(row)+[table_type]) for row in user_column_privileges.examples])) [tuple(list(row)+[table_type]) for row in user_column_privileges.examples]))
@TestOutline(Scenario) @TestOutline
@Requirements( @Requirements(
RQ_SRS_006_RBAC_Privileges_Insert_Column("1.0"), RQ_SRS_006_RBAC_Privileges_Insert_Column("1.0"),
) )
@ -140,55 +91,60 @@ def user_with_privilege_on_columns(self, table_type):
("d,a,b", "b", "y", "d,a,b", '9', '\'2020-01-01\',\'woo\',9') ("d,a,b", "b", "y", "d,a,b", '9', '\'2020-01-01\',\'woo\',9')
]) ])
def user_column_privileges(self, grant_columns, insert_columns_pass, data_fail, data_pass, table_type, def user_column_privileges(self, grant_columns, insert_columns_pass, data_fail, data_pass, table_type,
revoke_columns=None, insert_columns_fail=None, node=None): revoke_columns=None, insert_columns_fail=None, node=None):
"""Check that user is able to insert on granted columns """Check that user is able to insert on columns where insert privilege is granted
and unable to insert on not granted or revoked columns. and unable to insert on columns where insert privilege has not been granted or has been revoked.
""" """
user_name = f"user_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"): with user(node, user_name):
with When("I grant insert privilege"): with When("I grant insert privilege"):
node.query(f"GRANT INSERT({grant_columns}) ON merge_tree TO user0") node.query(f"GRANT INSERT({grant_columns}) ON {table_name} TO {user_name}")
if insert_columns_fail is not None: if insert_columns_fail is not None:
with And("I insert into not granted column"): with And("I insert into a column without insert privilege"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query(f"INSERT INTO merge_tree ({insert_columns_fail}) VALUES ({data_fail})", node.query(f"INSERT INTO {table_name} ({insert_columns_fail}) VALUES ({data_fail})",
settings=[("user","user0")], exitcode=exitcode, message=message) settings=[("user",user_name)], exitcode=exitcode, message=message)
with And("I insert into granted column"): with And("I insert into granted column"):
node.query(f"INSERT INTO merge_tree ({insert_columns_pass}) VALUES ({data_pass})", node.query(f"INSERT INTO {table_name} ({insert_columns_pass}) VALUES ({data_pass})",
settings=[("user","user0")]) settings=[("user",user_name)])
with Then("I check the insert functioned"): with Then("I check the insert functioned"):
input_equals_output = input_output_equality_check(node, insert_columns_pass, data_pass) input_equals_output = input_output_equality_check(node, insert_columns_pass, data_pass, table_name)
assert input_equals_output, error() assert input_equals_output, error()
if revoke_columns is not None: if revoke_columns is not None:
with When("I revoke insert privilege from columns"): with When("I revoke insert privilege from columns"):
node.query(f"REVOKE INSERT({revoke_columns}) ON merge_tree FROM user0") node.query(f"REVOKE INSERT({revoke_columns}) ON {table_name} FROM {user_name}")
with And("I insert into revoked columns"): with And("I insert into revoked columns"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query(f"INSERT INTO merge_tree ({insert_columns_pass}) VALUES ({data_pass})", node.query(f"INSERT INTO {table_name} ({insert_columns_pass}) VALUES ({data_pass})",
settings=[("user","user0")], exitcode=exitcode, message=message) settings=[("user",user_name)], exitcode=exitcode, message=message)
@TestScenario @TestScenario
@Requirements( @Requirements(
RQ_SRS_006_RBAC_Privileges_Insert_Grant("1.0"), RQ_SRS_006_RBAC_Privileges_Insert_Grant("1.0"),
) )
def role_with_privilege(self, table_type, node=None): def role_with_privilege(self, table_type, node=None):
"""Check that user can insert into a table after it is granted a role that """Check that user can insert into a table after being granted a role that
has the insert privilege for that table. has the insert privilege for that table.
""" """
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"), role(node, "role0"): with user(node, user_name), role(node, role_name):
with When("I grant insert privilege to a role"): with When("I grant insert privilege to a role"):
node.query("GRANT INSERT ON merge_tree TO role0") node.query(f"GRANT INSERT ON {table_name} TO {role_name}")
with And("I grant role to the user"): with And("I grant the role to a user"):
node.query("GRANT role0 TO user0") node.query(f"GRANT {role_name} TO {user_name}")
with And("I insert into a table"): with And("I insert into the table"):
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')", settings=[("user","user0")]) node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')", settings=[("user",user_name)])
with Then("I check that I can read inserted data"): with Then("I check the data matches the input"):
output = node.query("SELECT d FROM merge_tree FORMAT JSONEachRow").output output = node.query(f"SELECT d FROM {table_name} FORMAT JSONEachRow").output
assert output == '{"d":"2020-01-01"}', error() assert output == '{"d":"2020-01-01"}', error()
@TestScenario @TestScenario
@ -200,41 +156,46 @@ def role_with_revoked_privilege(self, table_type, node=None):
is unable to insert into that table after insert privilege is unable to insert into that table after insert privilege
has been revoked from the role. has been revoked from the role.
""" """
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"), role(node, "role0"): with user(node, user_name), role(node, role_name):
with When("I grant privilege to a role"): with When("I grant privilege to a role"):
node.query("GRANT INSERT ON merge_tree TO role0") node.query(f"GRANT INSERT ON {table_name} TO {role_name}")
with And("I grant the role to a user"): with And("I grant the role to a user"):
node.query("GRANT role0 TO user0") node.query(f"GRANT {role_name} TO {user_name}")
with And("I revoke privilege from the role"): with And("I revoke privilege from the role"):
node.query("REVOKE INSERT ON merge_tree FROM role0") node.query(f"REVOKE INSERT ON {table_name} FROM {role_name}")
with And("I insert into the table"): with And("I insert into the table"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')", node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')",
settings=[("user","user0")], exitcode=exitcode, message=message) settings=[("user",user_name)], exitcode=exitcode, message=message)
@TestScenario @TestScenario
def user_with_revoked_role(self, table_type, node=None): def user_with_revoked_role(self, table_type, node=None):
"""Check that user with a role that has insert privilege on a table """Check that user with a role that has insert privilege on a table
is unable to insert into that table after the role with insert is unable to insert into that table after the role has been revoked from the user.
privilege has been revoked from the user.
""" """
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"), role(node, "role0"): with user(node, user_name), role(node, role_name):
with When("I grant privilege to a role"): with When("I grant privilege to a role"):
node.query("GRANT INSERT ON merge_tree TO role0") node.query(f"GRANT INSERT ON {table_name} TO {role_name}")
with And("I grant the role to a user"): with And("I grant the role to a user"):
node.query("GRANT role0 TO user0") node.query(f"GRANT {role_name} TO {user_name}")
with And("I revoke the role from the user"): with And("I revoke the role from the user"):
node.query("REVOKE role0 FROM user0") node.query(f"REVOKE {role_name} FROM {user_name}")
with And("I insert into the table"): with And("I insert into the table"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')", node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')",
settings=[("user","user0")], exitcode=exitcode, message=message) settings=[("user",user_name)], exitcode=exitcode, message=message)
@TestScenario @TestScenario
def role_with_privilege_on_columns(self, table_type): def role_with_privilege_on_columns(self, table_type):
@ -242,7 +203,7 @@ def role_with_privilege_on_columns(self, table_type):
examples=Examples("grant_columns revoke_columns insert_columns_fail insert_columns_pass data_fail data_pass table_type", examples=Examples("grant_columns revoke_columns insert_columns_fail insert_columns_pass data_fail data_pass table_type",
[tuple(list(row)+[table_type]) for row in role_column_privileges.examples])) [tuple(list(row)+[table_type]) for row in role_column_privileges.examples]))
@TestOutline(Scenario) @TestOutline
@Requirements( @Requirements(
RQ_SRS_006_RBAC_Privileges_Insert_Column("1.0"), RQ_SRS_006_RBAC_Privileges_Insert_Column("1.0"),
) )
@ -253,94 +214,154 @@ def role_with_privilege_on_columns(self, table_type):
("d,a,b", "b", "y", "d,a,b", '9', '\'2020-01-01\',\'woo\',9') ("d,a,b", "b", "y", "d,a,b", '9', '\'2020-01-01\',\'woo\',9')
]) ])
def role_column_privileges(self, grant_columns, insert_columns_pass, data_fail, data_pass, def role_column_privileges(self, grant_columns, insert_columns_pass, data_fail, data_pass,
table_type, revoke_columns=None, insert_columns_fail=None, node=None): table_type, revoke_columns=None, insert_columns_fail=None, node=None):
"""Check that user with a role is able to insert on granted columns and unable """Check that user with a role is able to insert on columns where insert privilege is granted to the role
to insert on not granted or revoked columns. and unable to insert on columns where insert privilege has not been granted or has been revoked from the role.
""" """
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"), role(node, "role0"): with user(node, user_name), role(node, role_name):
with When("I grant insert privilege"): with When("I grant insert privilege"):
node.query(f"GRANT INSERT({grant_columns}) ON merge_tree TO role0") node.query(f"GRANT INSERT({grant_columns}) ON {table_name} TO {role_name}")
with And("I grant the role to a user"): with And("I grant the role to a user"):
node.query("GRANT role0 TO user0") node.query(f"GRANT {role_name} TO {user_name}")
if insert_columns_fail is not None: if insert_columns_fail is not None:
with And("I insert into not granted column"): with And("I insert into columns without insert privilege"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query(f"INSERT INTO merge_tree ({insert_columns_fail}) VALUES ({data_fail})", node.query(f"INSERT INTO {table_name} ({insert_columns_fail}) VALUES ({data_fail})",
settings=[("user","user0")], exitcode=exitcode, message=message) settings=[("user",user_name)], exitcode=exitcode, message=message)
with And("I insert into granted column"): with And("I insert into granted column"):
node.query(f"INSERT INTO merge_tree ({insert_columns_pass}) VALUES ({data_pass})", node.query(f"INSERT INTO {table_name} ({insert_columns_pass}) VALUES ({data_pass})",
settings=[("user","user0")]) settings=[("user",user_name)])
with Then("I check the insert functioned"): with Then("I check the insert functioned"):
input_equals_output = input_output_equality_check(node, insert_columns_pass, data_pass) input_equals_output = input_output_equality_check(node, insert_columns_pass, data_pass, table_name)
assert input_equals_output, error() assert input_equals_output, error()
if revoke_columns is not None: if revoke_columns is not None:
with When("I revoke insert privilege from columns"): with When("I revoke insert privilege from columns"):
node.query(f"REVOKE INSERT({revoke_columns}) ON merge_tree FROM role0") node.query(f"REVOKE INSERT({revoke_columns}) ON {table_name} FROM {role_name}")
with And("I insert into revoked columns"): with And("I insert into revoked columns"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query(f"INSERT INTO merge_tree ({insert_columns_pass}) VALUES ({data_pass})", node.query(f"INSERT INTO {table_name} ({insert_columns_pass}) VALUES ({data_pass})",
settings=[("user","user0")], exitcode=exitcode, message=message) settings=[("user",user_name)], exitcode=exitcode, message=message)
@TestScenario @TestScenario
@Requirements( @Requirements(
RQ_SRS_006_RBAC_Privileges_Insert_Cluster("1.0"), RQ_SRS_006_RBAC_Privileges_Insert_Cluster("1.0"),
) )
def user_with_privilege_on_cluster(self, table_type, node=None): def user_with_privilege_on_cluster(self, table_type, node=None):
"""Check that user is able to insert on a table with """Check that user is able or unable to insert into a table
privilege granted on a cluster. depending whether insert privilege is granted or revoked on a cluster.
""" """
user_name = f"user_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
try: try:
with Given("I have a user on a cluster"): with Given("I have a user on a cluster"):
node.query("CREATE USER OR REPLACE user0 ON CLUSTER sharded_cluster") node.query(f"CREATE USER OR REPLACE {user_name} ON CLUSTER sharded_cluster")
with When("I grant insert privilege on a cluster without the node with the table"): with When("I grant insert privilege on a cluster without the node with the table"):
node.query("GRANT ON CLUSTER cluster23 INSERT ON merge_tree TO user0") node.query(f"GRANT ON CLUSTER sharded_cluster23 INSERT ON {table_name} TO {user_name}")
with And("I insert into the table expecting a fail"): with And("I insert into the table expecting a fail"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')", settings=[("user","user0")], node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')", settings=[("user",user_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
with And("I grant insert privilege on cluster including all nodes"): with And("I grant insert privilege on cluster including all nodes"):
node.query("GRANT ON CLUSTER sharded_cluster INSERT ON merge_tree TO user0") node.query(f"GRANT ON CLUSTER sharded_cluster INSERT ON {table_name} TO {user_name}")
with And("I revoke insert privilege on cluster without the table node"): with And("I revoke insert privilege on cluster without the node with the table"):
node.query("REVOKE ON CLUSTER cluster23 INSERT ON merge_tree FROM user0") node.query(f"REVOKE ON CLUSTER sharded_cluster23 INSERT ON {table_name} FROM {user_name}")
with And("I insert into the table"): with And("I insert into the table"):
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')", settings=[("user","user0")]) node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')", settings=[("user",user_name)])
with Then("I check that I can read inserted data"): with And("I check that I can read inserted data"):
output = node.query("SELECT d FROM merge_tree FORMAT JSONEachRow").output output = node.query(f"SELECT d FROM {table_name} FORMAT JSONEachRow").output
assert output == '{"d":"2020-01-01"}', error() assert output == '{"d":"2020-01-01"}', error()
with And("I revoke insert privilege on cluster with all nodes"):
node.query(f"REVOKE ON CLUSTER sharded_cluster INSERT ON {table_name} FROM {user_name}")
with Then("I insert into table expecting fail"):
exitcode, message = errors.not_enough_privileges(name=user_name)
node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')", settings=[("user",user_name)],
exitcode=exitcode, message=message)
finally: finally:
with Finally("I drop the user"): with Finally("I drop the user"):
node.query("DROP USER user0 ON CLUSTER sharded_cluster") node.query(f"DROP USER {user_name} ON CLUSTER sharded_cluster")
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Privileges_Insert_Cluster("1.0"),
)
def role_with_privilege_on_cluster(self, table_type, node=None):
"""Check that user with role is able to insert into a table
depending whether insert privilege granted or revoked from the role on the cluster.
"""
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None:
node = self.context.node
with table(node, table_name, table_type):
try:
with Given("I have a user on a cluster"):
node.query(f"CREATE USER OR REPLACE {user_name} ON CLUSTER sharded_cluster")
with And("I have a role on a cluster"):
node.query(f"CREATE ROLE OR REPLACE {role_name} ON CLUSTER sharded_cluster")
with When("I grant the role to the user"):
node.query(f"GRANT {role_name} TO {user_name}")
with And("I grant insert privilege on a cluster without the node with the table"):
node.query(f"GRANT ON CLUSTER sharded_cluster23 INSERT ON {table_name} TO {role_name}")
with And("I insert into the table expecting a fail"):
exitcode, message = errors.not_enough_privileges(name=user_name)
node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')", settings=[("user",user_name)],
exitcode=exitcode, message=message)
with And("I grant insert privilege on cluster including all nodes"):
node.query(f"GRANT ON CLUSTER sharded_cluster INSERT ON {table_name} TO {role_name}")
with And("I revoke insert privilege on cluster without the table node"):
node.query(f"REVOKE ON CLUSTER sharded_cluster23 INSERT ON {table_name} FROM {role_name}")
with And("I insert into the table"):
node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')", settings=[("user",user_name)])
with And("I check that I can read inserted data"):
output = node.query(f"SELECT d FROM {table_name} FORMAT JSONEachRow").output
assert output == '{"d":"2020-01-01"}', error()
with And("I revoke insert privilege on cluster with all nodes"):
node.query(f"REVOKE ON CLUSTER sharded_cluster INSERT ON {table_name} FROM {role_name}")
with Then("I insert into table expecting fail"):
exitcode, message = errors.not_enough_privileges(name=user_name)
node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')", settings=[("user",user_name)],
exitcode=exitcode, message=message)
finally:
with Finally("I drop the user"):
node.query(f"DROP USER {user_name} ON CLUSTER sharded_cluster")
@TestScenario @TestScenario
@Requirements( @Requirements(
RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Grant("1.0"), RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Grant("1.0"),
) )
def user_with_privilege_from_user_with_grant_option(self, table_type, node=None): def user_with_privilege_from_user_with_grant_option(self, table_type, node=None):
"""Check that user is able to insert on a table when granted privilege """Check that user is able to insert into a table when granted privilege
from another user with grant option. from another user with grant option.
""" """
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0,user1"): with user(node, f"{user0_name},{user1_name}"):
with When("I grant privilege with grant option to user"): with When("I grant privilege with grant option to user"):
node.query("GRANT INSERT(d) ON merge_tree TO user0 WITH GRANT OPTION") node.query(f"GRANT INSERT(d) ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And("I grant privilege on a column I dont have permission on"): with And("I grant privilege on a column I don't have permission on"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user0_name)
node.query("GRANT INSERT(b) ON merge_tree TO user1", settings=[("user","user0")], node.query(f"GRANT INSERT(b) ON {table_name} TO {user1_name}", settings=[("user",user0_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
with And("I grant privilege to another user via grant option"): with And("I grant privilege to another user via grant option"):
node.query("GRANT INSERT(d) ON merge_tree TO user1", settings=[("user","user0")]) node.query(f"GRANT INSERT(d) ON {table_name} TO {user1_name}", settings=[("user",user0_name)])
with And("I insert into a table"): with And("I insert into a table"):
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')", settings=[("user","user1")]) node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')", settings=[("user",user1_name)])
with Then("I check that I can read inserted data"): with Then("I check that I can read inserted data"):
output = node.query("SELECT d FROM merge_tree FORMAT JSONEachRow").output output = node.query(f"SELECT d FROM {table_name} FORMAT JSONEachRow").output
assert output == '{"d":"2020-01-01"}', error() assert output == '{"d":"2020-01-01"}', error()
@TestScenario @TestScenario
@ -348,27 +369,31 @@ def user_with_privilege_from_user_with_grant_option(self, table_type, node=None)
RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Grant("1.0"), RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Grant("1.0"),
) )
def role_with_privilege_from_user_with_grant_option(self, table_type, node=None): def role_with_privilege_from_user_with_grant_option(self, table_type, node=None):
"""Check that user is able to insert on a table when granted a role with """Check that user is able to insert into a table when granted a role with
insert privilege that was granted by another user with grant option. insert privilege that was granted by another user with grant option.
""" """
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0,user1"), role(node, "role0"): with user(node, f"{user0_name},{user1_name}"), role(node, role_name):
with When("I grant privilege with grant option to user"): with When("I grant privilege with grant option to user"):
node.query("GRANT INSERT(d) ON merge_tree TO user0 WITH GRANT OPTION") node.query(f"GRANT INSERT(d) ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And("I grant privilege on a column I dont have permission on"): with And("I grant privilege on a column I don't have permission on"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user0_name)
node.query("GRANT INSERT(b) ON merge_tree TO role0", settings=[("user","user0")], node.query(f"GRANT INSERT(b) ON {table_name} TO {role_name}", settings=[("user",user0_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
with And("I grant privilege to a role via grant option"): with And("I grant privilege to a role via grant option"):
node.query("GRANT INSERT(d) ON merge_tree TO role0", settings=[("user","user0")]) node.query(f"GRANT INSERT(d) ON {table_name} TO {role_name}", settings=[("user",user0_name)])
with And("I grant the role to another user"): with And("I grant the role to another user"):
node.query("GRANT role0 TO user1") node.query(f"GRANT {role_name} TO {user1_name}")
with And("I insert into a table"): with And("I insert into a table"):
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')", settings=[("user","user1")]) node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')", settings=[("user",user1_name)])
with Then("I check that I can read inserted data"): with Then("I check that I can read inserted data"):
output = node.query("SELECT d FROM merge_tree FORMAT JSONEachRow").output output = node.query(f"SELECT d FROM {table_name} FORMAT JSONEachRow").output
assert output == '{"d":"2020-01-01"}', error() assert output == '{"d":"2020-01-01"}', error()
@TestScenario @TestScenario
@ -376,26 +401,30 @@ def role_with_privilege_from_user_with_grant_option(self, table_type, node=None)
RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Grant("1.0"), RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Grant("1.0"),
) )
def user_with_privilege_from_role_with_grant_option(self, table_type, node=None): def user_with_privilege_from_role_with_grant_option(self, table_type, node=None):
"""Check that user is able to insert on a table when granted privilege from a role with grant option """Check that user is able to insert into a table when granted privilege from a role with grant option.
""" """
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0,user1"), role(node, "role0"): with user(node, f"{user0_name},{user1_name}"), role(node, role_name):
with When("I grant privilege with grant option to a role"): with When("I grant privilege with grant option to a role"):
node.query("GRANT INSERT(d) ON merge_tree TO role0 WITH GRANT OPTION") node.query(f"GRANT INSERT(d) ON {table_name} TO {role_name} WITH GRANT OPTION")
with When("I grant role to a user"): with When("I grant role to a user"):
node.query("GRANT role0 TO user0") node.query(f"GRANT {role_name} TO {user0_name}")
with And("I grant privilege on a column I dont have permission on"): with And("I grant privilege on a column I don't have permission on"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user0_name)
node.query("GRANT INSERT(b) ON merge_tree TO user1", settings=[("user","user0")], node.query(f"GRANT INSERT(b) ON {table_name} TO {user1_name}", settings=[("user",user0_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
with And("I grant privilege to a user via grant option"): with And("I grant privilege to a user via grant option"):
node.query("GRANT INSERT(d) ON merge_tree TO user1", settings=[("user","user0")]) node.query(f"GRANT INSERT(d) ON {table_name} TO {user1_name}", settings=[("user",user0_name)])
with And("I insert into a table"): with And("I insert into a table"):
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')", settings=[("user","user1")]) node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')", settings=[("user",user1_name)])
with Then("I check that I can read inserted data"): with Then("I check that I can read inserted data"):
output = node.query("SELECT d FROM merge_tree FORMAT JSONEachRow").output output = node.query(f"SELECT d FROM {table_name} FORMAT JSONEachRow").output
assert output == '{"d":"2020-01-01"}', error() assert output == '{"d":"2020-01-01"}', error()
@TestScenario @TestScenario
@ -403,29 +432,34 @@ def user_with_privilege_from_role_with_grant_option(self, table_type, node=None)
RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Grant("1.0"), RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Grant("1.0"),
) )
def role_with_privilege_from_role_with_grant_option(self, table_type, node=None): def role_with_privilege_from_role_with_grant_option(self, table_type, node=None):
"""Check that a user is able to insert on a table with a role that was granted privilege """Check that a user is able to insert into a table with a role that was granted privilege
by another role with grant option by another role with grant option.
""" """
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role0_name = f"role0_{getuid()}"
role1_name = f"role1_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0,user1"), role(node, "role0,role1"): with user(node, f"{user0_name},{user1_name}"), role(node, f"{role0_name},{role1_name}"):
with When("I grant privilege with grant option to role"): with When("I grant privilege with grant option to role"):
node.query("GRANT INSERT(d) ON merge_tree TO role0 WITH GRANT OPTION") node.query(f"GRANT INSERT(d) ON {table_name} TO {role0_name} WITH GRANT OPTION")
with And("I grant the role to a user"): with And("I grant the role to a user"):
node.query("GRANT role0 TO user0") node.query(f"GRANT {role0_name} TO {user0_name}")
with And("I grant privilege on a column I dont have permission on"): with And("I grant privilege on a column I don't have permission on"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user0_name)
node.query("GRANT INSERT(b) ON merge_tree TO role1", settings=[("user","user0")], node.query(f"GRANT INSERT(b) ON {table_name} TO {role1_name}", settings=[("user",user0_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
with And("I grant privilege to another role via grant option"): with And("I grant privilege to another role via grant option"):
node.query("GRANT INSERT(d) ON merge_tree TO role1", settings=[("user","user0")]) node.query(f"GRANT INSERT(d) ON {table_name} TO {role1_name}", settings=[("user",user0_name)])
with And("I grant the second role to another user"): with And("I grant the second role to another user"):
node.query("GRANT role1 TO user1") node.query(f"GRANT {role1_name} TO {user1_name}")
with And("I insert into a table"): with And("I insert into a table"):
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')", settings=[("user","user1")]) node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')", settings=[("user",user1_name)])
with Then("I check that I can read inserted data"): with Then("I check that I can read inserted data"):
output = node.query("SELECT d FROM merge_tree FORMAT JSONEachRow").output output = node.query(f"SELECT d FROM {table_name} FORMAT JSONEachRow").output
assert output == '{"d":"2020-01-01"}', error() assert output == '{"d":"2020-01-01"}', error()
@TestScenario @TestScenario
@ -433,106 +467,122 @@ def role_with_privilege_from_role_with_grant_option(self, table_type, node=None)
RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Revoke("1.0"), RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Revoke("1.0"),
) )
def revoke_privilege_from_user_via_user_with_grant_option(self, table_type, node=None): def revoke_privilege_from_user_via_user_with_grant_option(self, table_type, node=None):
"""Check that user is unable to revoke a column they don't have access to from a user. """Check that user is unable to revoke insert privilege from another user
on a column they the original user doesn't have access to.
""" """
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0,user1"): with user(node, f"{user0_name},{user1_name}"):
with When("I grant privilege with grant option to user"): with When("I grant privilege with grant option to user"):
node.query("GRANT INSERT(d) ON merge_tree TO user0 WITH GRANT OPTION") node.query(f"GRANT INSERT(d) ON {table_name} TO {user0_name} WITH GRANT OPTION")
with Then("I revoke privilege on a column the user with grant option does not have access to"): with Then("I revoke privilege on a column the user with grant option does not have access to"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user0_name)
node.query("REVOKE INSERT(b) ON merge_tree FROM user1", settings=[("user","user0")], node.query(f"REVOKE INSERT(b) ON {table_name} FROM {user1_name}", settings=[("user",user0_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
@TestScenario @TestScenario
@Requirements( @Requirements(
RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Revoke("1.0"), RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Revoke("1.0"),
) )
def revoke_privilege_from_role_via_user_with_grant_option(self, table_type, node=None): def revoke_privilege_from_role_via_user_with_grant_option(self, table_type, node=None):
"""Check that user is unable to revoke a column they dont have acces to from a role. """Check that user is unable to revoke insert privilege from a role
on a column the original user doesn't have access to.
""" """
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"), role(node, "role0"): with user(node, user_name), role(node, role_name):
with When("I grant privilege with grant option to user"): with When("I grant privilege with grant option to user"):
node.query("GRANT INSERT(d) ON merge_tree TO user0 WITH GRANT OPTION") node.query(f"GRANT INSERT(d) ON {table_name} TO {user_name} WITH GRANT OPTION")
with Then("I revoke privilege on a column the user with grant option does not have access to"): with Then("I revoke privilege on a column the user with grant option does not have access to"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query("REVOKE INSERT(b) ON merge_tree FROM role0", settings=[("user","user0")], node.query(f"REVOKE INSERT(b) ON {table_name} FROM {role_name}", settings=[("user",user_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
@TestScenario @TestScenario
@Requirements( @Requirements(
RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Revoke("1.0"), RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Revoke("1.0"),
) )
def revoke_privilege_from_user_via_role_with_grant_option(self, table_type, node=None): def revoke_privilege_from_user_via_role_with_grant_option(self, table_type, node=None):
"""Check that user with a role is unable to revoke a column they dont have acces to from a user. """Check that user with a role is unable to revoke insert privilege from a user
on a column the original user doesn't have access to.
""" """
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0,user1"), role(node, "role0"): with user(node, f"{user0_name},{user1_name}"), role(node, role_name):
with When("I grant privilege with grant option to a role"): with When("I grant privilege with grant option to a role"):
node.query("GRANT INSERT(d) ON merge_tree TO role0 WITH GRANT OPTION") node.query(f"GRANT INSERT(d) ON {table_name} TO {role_name} WITH GRANT OPTION")
with And("I grant the role to a user"): with And("I grant the role to a user"):
node.query("GRANT role0 TO user0") node.query(f"GRANT {role_name} TO {user0_name}")
with Then("I revoke privilege on a column the user with grant option does not have access to"): with Then("I revoke privilege on a column the user with grant option does not have access to"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user0_name)
node.query("REVOKE INSERT(b) ON merge_tree FROM user1", settings=[("user","user0")], node.query(f"REVOKE INSERT(b) ON {table_name} FROM {user1_name}", settings=[("user",user0_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
@TestScenario @TestScenario
@Requirements( @Requirements(
RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Revoke("1.0"), RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Revoke("1.0"),
) )
def revoke_privilege_from_role_via_role_with_grant_option(self, table_type, node=None): def revoke_privilege_from_role_via_role_with_grant_option(self, table_type, node=None):
"""Check that user with a role is unable to revoke a column they dont have acces to from a role. """Check that user with a role is unable to revoke insert privilege from a role
on a column the original user doesn't have access to.
""" """
user_name = f"user_{getuid()}"
role0_name = f"role0_{getuid()}"
role1_name = f"role1_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"), role(node, "role0,role1"): with user(node, user_name), role(node, f"{role0_name},{role1_name}"):
with When("I grant privilege with grant option to a role"): with When("I grant privilege with grant option to a role"):
node.query("GRANT INSERT(d) ON merge_tree TO user0 WITH GRANT OPTION") node.query(f"GRANT INSERT(d) ON {table_name} TO {user_name} WITH GRANT OPTION")
with And("I grant the role to a user"): with And("I grant the role to a user"):
node.query("GRANT role0 TO user0") node.query(f"GRANT {role0_name} TO {user_name}")
with Then("I revoke privilege on a column the user with grant option does not have access to"): with Then("I revoke privilege on a column the user with grant option does not have access to"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query("REVOKE INSERT(b) ON merge_tree FROM role1", settings=[("user","user0")], node.query(f"REVOKE INSERT(b) ON {table_name} FROM {role1_name}", settings=[("user",user_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
@TestOutline(Feature) @TestOutline(Feature)
@Requirements( @Requirements(
RQ_SRS_006_RBAC_Privileges_Insert("1.0"), RQ_SRS_006_RBAC_Privileges_Insert("1.0"),
RQ_SRS_006_RBAC_Privileges_Insert_TableEngines("1.0")
) )
@Examples("table_type", [ @Examples("table_type", [
(table_type, Requirements(requirement)) for table_type, requirement in list(table_requirements.items()) (key,) for key in table_types.keys()
]) ])
@Flags(TE)
@Name("insert") @Name("insert")
def feature(self, table_type, node="clickhouse1"): def feature(self, table_type, parallel=None, stress=None, node="clickhouse1"):
self.context.node = self.context.cluster.node(node) self.context.node = self.context.cluster.node(node)
self.context.node1 = self.context.cluster.node("clickhouse1") self.context.node1 = self.context.cluster.node("clickhouse1")
self.context.node2 = self.context.cluster.node("clickhouse2") self.context.node2 = self.context.cluster.node("clickhouse2")
self.context.node3 = self.context.cluster.node("clickhouse3") self.context.node3 = self.context.cluster.node("clickhouse3")
Scenario(test=without_privilege)(table_type=table_type) if stress is not None:
Scenario(test=user_with_privilege)(table_type=table_type) self.context.stress = stress
Scenario(test=user_with_revoked_privilege)(table_type=table_type) if parallel is not None:
Scenario(test=user_with_privilege_on_columns)(table_type=table_type) self.context.stress = parallel
Scenario(test=role_with_privilege)(table_type=table_type)
Scenario(test=role_with_revoked_privilege)(table_type=table_type) tasks = []
Scenario(test=user_with_revoked_role)(table_type=table_type) pool = Pool(3)
Scenario(test=role_with_privilege_on_columns)(table_type=table_type)
Scenario(test=user_with_privilege_on_cluster)(table_type=table_type) try:
Scenario(test=user_with_privilege_from_user_with_grant_option)(table_type=table_type) for scenario in loads(current_module(), Scenario):
Scenario(test=role_with_privilege_from_user_with_grant_option)(table_type=table_type) run_scenario(pool, tasks, scenario, {"table_type" : table_type})
Scenario(test=user_with_privilege_from_role_with_grant_option)(table_type=table_type) finally:
Scenario(test=role_with_privilege_from_role_with_grant_option)(table_type=table_type) join(tasks)
Scenario(test=revoke_privilege_from_user_via_user_with_grant_option)(table_type=table_type)
Scenario(test=revoke_privilege_from_role_via_user_with_grant_option)(table_type=table_type)
Scenario(test=revoke_privilege_from_user_via_role_with_grant_option)(table_type=table_type)
Scenario(test=revoke_privilege_from_role_via_role_with_grant_option)(table_type=table_type)

View File

@ -0,0 +1,38 @@
from contextlib import contextmanager
from testflows.core import *
from testflows.asserts import error
from rbac.requirements import *
from rbac.helper.common import *
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Table_PublicTables("1.0"),
)
def public_tables(self, node=None):
"""Check that a user with no privilege is able to select from public tables.
"""
user_name = f"user_{getuid()}"
if node is None:
node = self.context.node
with user(node, f"{user_name}"):
with Then("I check the user is able to select on system.one"):
node.query("SELECT count(*) FROM system.one", settings = [("user",user_name)])
with And("I check the user is able to select on system.numbers"):
node.query("SELECT * FROM system.numbers LIMIT 1", settings = [("user",user_name)])
with And("I check the user is able to select on system.contributors"):
node.query("SELECT count(*) FROM system.contributors", settings = [("user",user_name)])
with And("I check the user is able to select on system.functions"):
node.query("SELECT count(*) FROM system.functions", settings = [("user",user_name)])
@TestFeature
@Name("public tables")
def feature(self, node="clickhouse1"):
self.context.node = self.context.cluster.node(node)
Scenario(run=public_tables, flags=TE)

419
tests/testflows/rbac/tests/privileges/select.py Normal file → Executable file
View File

@ -5,69 +5,22 @@ from testflows.core import *
from testflows.asserts import error from testflows.asserts import error
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors from rbac.helper.common import *
import rbac.helper.errors as errors
table_types = {
"MergeTree": "CREATE TABLE {name} (d DATE, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree(d, (a, b), 111)",
"ReplacingMergeTree": "CREATE TABLE {name} (d DATE, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = ReplacingMergeTree(d, (a, b), 111)",
"SummingMergeTree": "CREATE TABLE {name} (d DATE, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = SummingMergeTree(d, (a, b), 111)",
"AggregatingMergeTree": "CREATE TABLE {name} (d DATE, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = AggregatingMergeTree(d, (a, b), 111)",
"CollapsingMergeTree": "CREATE TABLE {name} (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = CollapsingMergeTree(d, (a, b), 111, y);",
"VersionedCollapsingMergeTree": "CREATE TABLE {name} (d Date, a String, b UInt8, x String, y Int8, z UInt32, version UInt64, sign Int8, INDEX a (b * y, d) TYPE minmax GRANULARITY 3) ENGINE = VersionedCollapsingMergeTree(sign, version) ORDER BY tuple()",
"GraphiteMergeTree": "CREATE TABLE {name} (key UInt32, Path String, Time DateTime, d Date, a String, b UInt8, x String, y Int8, z UInt32, Value Float64, Version UInt32, col UInt64, INDEX a (key * Value, Time) TYPE minmax GRANULARITY 3) ENGINE = GraphiteMergeTree('graphite_rollup_example') ORDER BY tuple()"
}
table_requirements ={
"MergeTree": RQ_SRS_006_RBAC_Privileges_Select_MergeTree("1.0"),
"ReplacingMergeTree": RQ_SRS_006_RBAC_Privileges_Select_ReplacingMergeTree("1.0"),
"SummingMergeTree": RQ_SRS_006_RBAC_Privileges_Select_SummingMergeTree("1.0"),
"AggregatingMergeTree": RQ_SRS_006_RBAC_Privileges_Select_AggregatingMergeTree("1.0"),
"CollapsingMergeTree": RQ_SRS_006_RBAC_Privileges_Select_CollapsingMergeTree("1.0"),
"VersionedCollapsingMergeTree": RQ_SRS_006_RBAC_Privileges_Select_VersionedCollapsingMergeTree("1.0"),
"GraphiteMergeTree": RQ_SRS_006_RBAC_Privileges_Select_GraphiteMergeTree("1.0"),
}
@contextmanager
def table(node, name, table_type="MergeTree"):
try:
with Given(f"I have a {table_type} table"):
node.query(table_types[table_type].format(name=name))
yield
finally:
with Finally("I drop the table"):
node.query(f"DROP TABLE IF EXISTS {name}")
@contextmanager
def user(node, name):
try:
with Given("I have a user"):
node.query(f"CREATE USER OR REPLACE {name}")
yield
finally:
with Finally("I drop the user"):
node.query(f"DROP USER IF EXISTS {name}")
@contextmanager
def role(node, role):
try:
with Given("I have a role"):
node.query(f"CREATE ROLE OR REPLACE {role}")
yield
finally:
with Finally("I drop the role"):
node.query(f"DROP ROLE IF EXISTS {role}")
@TestScenario @TestScenario
def without_privilege(self, table_type, node=None): def without_privilege(self, table_type, node=None):
"""Check that user without select privilege on a table is not able to select on that table. """Check that user without select privilege on a table is not able to select on that table.
""" """
user_name = f"user_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"): with user(node, user_name):
with When("I run SELECT without privilege"): with When("I run SELECT without privilege"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query("SELECT * FROM merge_tree", settings = [("user","user0")], node.query(f"SELECT * FROM {table_name}", settings = [("user",user_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
@TestScenario @TestScenario
@ -77,19 +30,19 @@ def without_privilege(self, table_type, node=None):
def user_with_privilege(self, table_type, node=None): def user_with_privilege(self, table_type, node=None):
"""Check that user can select from a table on which they have select privilege. """Check that user can select from a table on which they have select privilege.
""" """
user_name = f"user_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with Given("I have some data inserted into table"): with Given("I have some data inserted into table"):
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')") node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')")
with user(node, "user88"): with user(node, user_name):
pass
with user(node, "user0"):
with When("I grant privilege"): with When("I grant privilege"):
node.query("GRANT SELECT ON merge_tree TO user0") node.query(f"GRANT SELECT ON {table_name} TO {user_name}")
with Then("I verify SELECT command"): with Then("I verify SELECT command"):
user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user0")]) user_select = node.query(f"SELECT d FROM {table_name}", settings = [("user",user_name)])
default = node.query("SELECT d FROM merge_tree") default = node.query(f"SELECT d FROM {table_name}")
assert user_select.output == default.output, error() assert user_select.output == default.output, error()
@TestScenario @TestScenario
@ -100,26 +53,28 @@ def user_with_revoked_privilege(self, table_type, node=None):
"""Check that user is unable to select from a table after select privilege """Check that user is unable to select from a table after select privilege
on that table has been revoked from the user. on that table has been revoked from the user.
""" """
user_name = f"user_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"): with user(node, user_name):
with When("I grant privilege"): with When("I grant privilege"):
node.query("GRANT SELECT ON merge_tree TO user0") node.query(f"GRANT SELECT ON {table_name} TO {user_name}")
with And("I revoke privilege"): with And("I revoke privilege"):
node.query("REVOKE SELECT ON merge_tree FROM user0") node.query(f"REVOKE SELECT ON {table_name} FROM {user_name}")
with And("I use SELECT, throws exception"): with And("I use SELECT, throws exception"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query("SELECT * FROM merge_tree", settings = [("user","user0")], node.query(f"SELECT * FROM {table_name}", settings = [("user",user_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
@TestScenario @TestScenario
def user_with_privilege_on_columns(self, table_type): def user_with_privilege_on_columns(self, table_type):
Scenario(run=user_column_privileges, Scenario(run=user_column_privileges,
examples=Examples("grant_columns revoke_columns select_columns_fail select_columns_pass data_pass table_type", examples=Examples("grant_columns revoke_columns select_columns_fail select_columns_pass data_pass table_type",
[tuple(list(row)+[table_type]) for row in user_column_privileges.examples])) [tuple(list(row)+[table_type]) for row in user_column_privileges.examples]))
@TestOutline(Scenario) @TestOutline
@Requirements( @Requirements(
RQ_SRS_006_RBAC_Privileges_Select_Column("1.0"), RQ_SRS_006_RBAC_Privileges_Select_Column("1.0"),
) )
@ -133,28 +88,30 @@ def user_column_privileges(self, grant_columns, select_columns_pass, data_pass,
"""Check that user is able to select on granted columns """Check that user is able to select on granted columns
and unable to select on not granted or revoked columns. and unable to select on not granted or revoked columns.
""" """
user_name = f"user_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type), user(node, "user0"): with table(node, table_name, table_type), user(node, user_name):
with Given("The table has some data on some columns"): with Given("The table has some data on some columns"):
node.query(f"INSERT INTO merge_tree ({select_columns_pass}) VALUES ({data_pass})") node.query(f"INSERT INTO {table_name} ({select_columns_pass}) VALUES ({data_pass})")
with When("I grant select privilege"): with When("I grant select privilege"):
node.query(f"GRANT SELECT({grant_columns}) ON merge_tree TO user0") node.query(f"GRANT SELECT({grant_columns}) ON {table_name} TO {user_name}")
if select_columns_fail is not None: if select_columns_fail is not None:
with And("I select from not granted column"): with And("I select from not granted column"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query(f"SELECT ({select_columns_fail}) FROM merge_tree", node.query(f"SELECT ({select_columns_fail}) FROM {table_name}",
settings = [("user","user0")], exitcode=exitcode, message=message) settings = [("user",user_name)], exitcode=exitcode, message=message)
with Then("I select from granted column, verify correct result"): with Then("I select from granted column, verify correct result"):
user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user0")]) user_select = node.query(f"SELECT ({select_columns_pass}) FROM {table_name}", settings = [("user",user_name)])
default = node.query("SELECT d FROM merge_tree") default = node.query(f"SELECT ({select_columns_pass}) FROM {table_name}")
assert user_select.output == default.output assert user_select.output == default.output
if revoke_columns is not None: if revoke_columns is not None:
with When("I revoke select privilege for columns from user"): with When("I revoke select privilege for columns from user"):
node.query(f"REVOKE SELECT({revoke_columns}) ON merge_tree FROM user0") node.query(f"REVOKE SELECT({revoke_columns}) ON {table_name} FROM {user_name}")
with And("I select from revoked columns"): with And("I select from revoked columns"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query(f"SELECT ({select_columns_pass}) FROM merge_tree", settings = [("user","user0")], exitcode=exitcode, message=message) node.query(f"SELECT ({select_columns_pass}) FROM {table_name}", settings = [("user",user_name)], exitcode=exitcode, message=message)
@TestScenario @TestScenario
@Requirements( @Requirements(
@ -164,20 +121,23 @@ def role_with_privilege(self, table_type, node=None):
"""Check that user can select from a table after it is granted a role that """Check that user can select from a table after it is granted a role that
has the select privilege for that table. has the select privilege for that table.
""" """
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with Given("I have some data inserted into table"): with Given("I have some data inserted into table"):
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')") node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')")
with user(node, "user0"): with user(node, user_name):
with role(node, "role0"): with role(node, role_name):
with When("I grant select privilege to a role"): with When("I grant select privilege to a role"):
node.query("GRANT SELECT ON merge_tree TO role0") node.query(f"GRANT SELECT ON {table_name} TO {role_name}")
with And("I grant role to the user"): with And("I grant role to the user"):
node.query("GRANT role0 TO user0") node.query(f"GRANT {role_name} TO {user_name}")
with Then("I verify SELECT command"): with Then("I verify SELECT command"):
user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user0")]) user_select = node.query(f"SELECT d FROM {table_name}", settings = [("user",user_name)])
default = node.query("SELECT d FROM merge_tree") default = node.query(f"SELECT d FROM {table_name}")
assert user_select.output == default.output, error() assert user_select.output == default.output, error()
@TestScenario @TestScenario
@ -188,48 +148,54 @@ def role_with_revoked_privilege(self, table_type, node=None):
"""Check that user with a role that has select privilege on a table is unable """Check that user with a role that has select privilege on a table is unable
to select from that table after select privilege has been revoked from the role. to select from that table after select privilege has been revoked from the role.
""" """
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"), role(node, "role0"): with user(node, user_name), role(node, role_name):
with When("I grant privilege to a role"): with When("I grant privilege to a role"):
node.query("GRANT SELECT ON merge_tree TO role0") node.query(f"GRANT SELECT ON {table_name} TO {role_name}")
with And("I grant the role to a user"): with And("I grant the role to a user"):
node.query("GRANT role0 TO user0") node.query(f"GRANT {role_name} TO {user_name}")
with And("I revoke privilege from the role"): with And("I revoke privilege from the role"):
node.query("REVOKE SELECT ON merge_tree FROM role0") node.query(f"REVOKE SELECT ON {table_name} FROM {role_name}")
with And("I select from the table"): with And("I select from the table"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query("SELECT * FROM merge_tree", settings = [("user","user0")], node.query(f"SELECT * FROM {table_name}", settings = [("user",user_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
@TestScenario @TestScenario
def user_with_revoked_role(self, table_type, node=None): def user_with_revoked_role(self, table_type, node=None):
"""Check that user with a role that has select privilege on a table is unable to """Check that user with a role that has select privilege on a table is unable to
select from that table after the role with select privilege has been revoked from the user. select from that table after the role with select privilege has been revoked from the user.
""" """
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"), role(node, "role0"): with user(node, user_name), role(node, role_name):
with When("I grant privilege to a role"): with When("I grant privilege to a role"):
node.query("GRANT SELECT ON merge_tree TO role0") node.query(f"GRANT SELECT ON {table_name} TO {role_name}")
with And("I grant the role to a user"): with And("I grant the role to a user"):
node.query("GRANT role0 TO user0") node.query(f"GRANT {role_name} TO {user_name}")
with And("I revoke the role from the user"): with And("I revoke the role from the user"):
node.query("REVOKE role0 FROM user0") node.query(f"REVOKE {role_name} FROM {user_name}")
with And("I select from the table"): with And("I select from the table"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query("SELECT * FROM merge_tree", settings = [("user","user0")], node.query(f"SELECT * FROM {table_name}", settings = [("user",user_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
@TestScenario @TestScenario
def role_with_privilege_on_columns(self, table_type): def role_with_privilege_on_columns(self, table_type):
Scenario(run=role_column_privileges, Scenario(run=role_column_privileges,
examples=Examples("grant_columns revoke_columns select_columns_fail select_columns_pass data_pass table_type", examples=Examples("grant_columns revoke_columns select_columns_fail select_columns_pass data_pass table_type",
[tuple(list(row)+[table_type]) for row in role_column_privileges.examples])) [tuple(list(row)+[table_type]) for row in role_column_privileges.examples]))
@TestOutline(Scenario) @TestOutline
@Requirements( @Requirements(
RQ_SRS_006_RBAC_Privileges_Select_Column("1.0"), RQ_SRS_006_RBAC_Privileges_Select_Column("1.0"),
) )
@ -243,32 +209,35 @@ def role_column_privileges(self, grant_columns, select_columns_pass, data_pass,
"""Check that user is able to select from granted columns and unable """Check that user is able to select from granted columns and unable
to select from not granted or revoked columns. to select from not granted or revoked columns.
""" """
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with Given("The table has some data on some columns"): with Given("The table has some data on some columns"):
node.query(f"INSERT INTO merge_tree ({select_columns_pass}) VALUES ({data_pass})") node.query(f"INSERT INTO {table_name} ({select_columns_pass}) VALUES ({data_pass})")
with user(node, "user0"), role(node, "role0"): with user(node, user_name), role(node, role_name):
with When("I grant select privilege"): with When("I grant select privilege"):
node.query(f"GRANT SELECT({grant_columns}) ON merge_tree TO role0") node.query(f"GRANT SELECT({grant_columns}) ON {table_name} TO {role_name}")
with And("I grant the role to a user"): with And("I grant the role to a user"):
node.query("GRANT role0 TO user0") node.query(f"GRANT {role_name} TO {user_name}")
if select_columns_fail is not None: if select_columns_fail is not None:
with And("I select from not granted column"): with And("I select from not granted column"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query(f"SELECT ({select_columns_fail}) FROM merge_tree", node.query(f"SELECT ({select_columns_fail}) FROM {table_name}",
settings = [("user","user0")], exitcode=exitcode, message=message) settings = [("user",user_name)], exitcode=exitcode, message=message)
with Then("I verify SELECT command"): with Then("I verify SELECT command"):
user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user0")]) user_select = node.query(f"SELECT d FROM {table_name}", settings = [("user",user_name)])
default = node.query("SELECT d FROM merge_tree") default = node.query(f"SELECT d FROM {table_name}")
assert user_select.output == default.output, error() assert user_select.output == default.output, error()
if revoke_columns is not None: if revoke_columns is not None:
with When("I revoke select privilege for columns from role"): with When("I revoke select privilege for columns from role"):
node.query(f"REVOKE SELECT({revoke_columns}) ON merge_tree FROM role0") node.query(f"REVOKE SELECT({revoke_columns}) ON {table_name} FROM {role_name}")
with And("I select from revoked columns"): with And("I select from revoked columns"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query(f"SELECT ({select_columns_pass}) FROM merge_tree", node.query(f"SELECT ({select_columns_pass}) FROM {table_name}",
settings = [("user","user0")], exitcode=exitcode, message=message) settings = [("user",user_name)], exitcode=exitcode, message=message)
@TestScenario @TestScenario
@Requirements( @Requirements(
@ -278,23 +247,26 @@ def user_with_privilege_on_cluster(self, table_type, node=None):
"""Check that user is able to select from a table with """Check that user is able to select from a table with
privilege granted on a cluster. privilege granted on a cluster.
""" """
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
try: try:
with Given("I have some data inserted into table"): with Given("I have some data inserted into table"):
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')") node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')")
with Given("I have a user on a cluster"): with Given("I have a user on a cluster"):
node.query("CREATE USER OR REPLACE user0 ON CLUSTER sharded_cluster") node.query(f"CREATE USER OR REPLACE {user_name} ON CLUSTER sharded_cluster")
with When("I grant select privilege on a cluster"): with When("I grant select privilege on a cluster"):
node.query("GRANT ON CLUSTER sharded_cluster SELECT ON merge_tree TO user0") node.query(f"GRANT ON CLUSTER sharded_cluster SELECT ON {table_name} TO {user_name}")
with Then("I verify SELECT command"): with Then("I verify SELECT command"):
user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user0")]) user_select = node.query(f"SELECT d FROM {table_name}", settings = [("user",user_name)])
default = node.query("SELECT d FROM merge_tree") default = node.query(f"SELECT d FROM {table_name}")
assert user_select.output == default.output, error() assert user_select.output == default.output, error()
finally: finally:
with Finally("I drop the user"): with Finally("I drop the user"):
node.query("DROP USER user0 ON CLUSTER sharded_cluster") node.query(f"DROP USER {user_name} ON CLUSTER sharded_cluster")
@TestScenario @TestScenario
@Requirements( @Requirements(
@ -304,19 +276,22 @@ def user_with_privilege_from_user_with_grant_option(self, table_type, node=None)
"""Check that user is able to select from a table when granted privilege """Check that user is able to select from a table when granted privilege
from another user with grant option. from another user with grant option.
""" """
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with Given("I have some data inserted into table"): with Given("I have some data inserted into table"):
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')") node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')")
with user(node, "user0"), user(node, "user1"): with user(node, f"{user0_name},{user1_name}"):
with When("I grant privilege with grant option to user"): with When("I grant privilege with grant option to user"):
node.query("GRANT SELECT ON merge_tree TO user0 WITH GRANT OPTION") node.query(f"GRANT SELECT ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And("I grant privilege to another user via grant option"): with And("I grant privilege to another user via grant option"):
node.query("GRANT SELECT ON merge_tree TO user1", settings = [("user","user0")]) node.query(f"GRANT SELECT ON {table_name} TO {user1_name}", settings = [("user",user0_name)])
with Then("I verify SELECT command"): with Then("I verify SELECT command"):
user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user1")]) user_select = node.query(f"SELECT d FROM {table_name}", settings = [("user",user1_name)])
default = node.query("SELECT d FROM merge_tree") default = node.query(f"SELECT d FROM {table_name}")
assert user_select.output == default.output, error() assert user_select.output == default.output, error()
@TestScenario @TestScenario
@ -327,21 +302,25 @@ def role_with_privilege_from_user_with_grant_option(self, table_type, node=None)
"""Check that user is able to select from a table when granted a role with """Check that user is able to select from a table when granted a role with
select privilege that was granted by another user with grant option. select privilege that was granted by another user with grant option.
""" """
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with Given("I have some data inserted into table"): with Given("I have some data inserted into table"):
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')") node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')")
with user(node, "user0"), user(node, "user1"), role(node, "role0"): with user(node, f"{user0_name},{user1_name}"), role(node, role_name):
with When("I grant privilege with grant option to user"): with When("I grant privilege with grant option to user"):
node.query("GRANT SELECT ON merge_tree TO user0 WITH GRANT OPTION") node.query(f"GRANT SELECT ON {table_name} TO {user0_name} WITH GRANT OPTION")
with And("I grant privilege to a role via grant option"): with And("I grant privilege to a role via grant option"):
node.query("GRANT SELECT ON merge_tree TO role0", settings = [("user","user0")]) node.query(f"GRANT SELECT ON {table_name} TO {role_name}", settings = [("user",user0_name)])
with And("I grant the role to another user"): with And("I grant the role to another user"):
node.query("GRANT role0 TO user1") node.query(f"GRANT {role_name} TO {user1_name}")
with Then("I verify SELECT command"): with Then("I verify SELECT command"):
user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user1")]) user_select = node.query(f"SELECT d FROM {table_name}", settings = [("user",user1_name)])
default = node.query("SELECT d FROM merge_tree") default = node.query(f"SELECT d FROM {table_name}")
assert user_select.output == default.output, error() assert user_select.output == default.output, error()
@TestScenario @TestScenario
@ -352,21 +331,25 @@ def user_with_privilege_from_role_with_grant_option(self, table_type, node=None)
"""Check that user is able to select from a table when granted privilege from """Check that user is able to select from a table when granted privilege from
a role with grant option a role with grant option
""" """
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with Given("I have some data inserted into table"): with Given("I have some data inserted into table"):
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')") node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')")
with user(node, "user0"), user(node, "user1"), role(node, "role0"): with user(node, f"{user0_name},{user1_name}"), role(node, role_name):
with When("I grant privilege with grant option to a role"): with When("I grant privilege with grant option to a role"):
node.query("GRANT SELECT ON merge_tree TO role0 WITH GRANT OPTION") node.query(f"GRANT SELECT ON {table_name} TO {role_name} WITH GRANT OPTION")
with When("I grant role to a user"): with When("I grant role to a user"):
node.query("GRANT role0 TO user0") node.query(f"GRANT {role_name} TO {user0_name}")
with And("I grant privilege to a user via grant option"): with And("I grant privilege to a user via grant option"):
node.query("GRANT SELECT ON merge_tree TO user1", settings = [("user","user0")]) node.query(f"GRANT SELECT ON {table_name} TO {user1_name}", settings = [("user",user0_name)])
with Then("I verify SELECT command"): with Then("I verify SELECT command"):
user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user1")]) user_select = node.query(f"SELECT d FROM {table_name}", settings = [("user",user1_name)])
default = node.query("SELECT d FROM merge_tree") default = node.query(f"SELECT d FROM {table_name}")
assert user_select.output == default.output, error() assert user_select.output == default.output, error()
@TestScenario @TestScenario
@ -377,23 +360,28 @@ def role_with_privilege_from_role_with_grant_option(self, table_type, node=None)
"""Check that a user is able to select from a table with a role that was """Check that a user is able to select from a table with a role that was
granted privilege by another role with grant option granted privilege by another role with grant option
""" """
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role0_name = f"role0_{getuid()}"
role1_name = f"role1_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with Given("I have some data inserted into table"): with Given("I have some data inserted into table"):
node.query("INSERT INTO merge_tree (d) VALUES ('2020-01-01')") node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')")
with user(node, "user0"), user(node, "user1"), role(node, "role0"), role(node, "role1"): with user(node, f"{user0_name},{user1_name}"), role(node, f"{role0_name},{role1_name}"):
with When("I grant privilege with grant option to role"): with When("I grant privilege with grant option to role"):
node.query("GRANT SELECT ON merge_tree TO role0 WITH GRANT OPTION") node.query(f"GRANT SELECT ON {table_name} TO {role0_name} WITH GRANT OPTION")
with And("I grant the role to a user"): with And("I grant the role to a user"):
node.query("GRANT role0 TO user0") node.query(f"GRANT {role0_name} TO {user0_name}")
with And("I grant privilege to another role via grant option"): with And("I grant privilege to another role via grant option"):
node.query("GRANT SELECT ON merge_tree TO role1", settings = [("user","user0")]) node.query(f"GRANT SELECT ON {table_name} TO {role1_name}", settings = [("user",user0_name)])
with And("I grant the second role to another user"): with And("I grant the second role to another user"):
node.query("GRANT role1 TO user1") node.query(f"GRANT {role1_name} TO {user1_name}")
with Then("I verify SELECT command"): with Then("I verify SELECT command"):
user_select = node.query("SELECT d FROM merge_tree", settings = [("user","user1")]) user_select = node.query(f"SELECT d FROM {table_name}", settings = [("user",user1_name)])
default = node.query("SELECT d FROM merge_tree") default = node.query(f"SELECT d FROM {table_name}")
assert user_select.output == default.output, error() assert user_select.output == default.output, error()
@TestScenario @TestScenario
@ -403,16 +391,19 @@ def role_with_privilege_from_role_with_grant_option(self, table_type, node=None)
def revoke_privilege_from_user_via_user_with_grant_option(self, table_type, node=None): def revoke_privilege_from_user_via_user_with_grant_option(self, table_type, node=None):
"""Check that user is unable to revoke a column they don't have access to from a user. """Check that user is unable to revoke a column they don't have access to from a user.
""" """
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"), user(node, "user1"): with user(node, f"{user0_name},{user1_name}"):
with When("I grant privilege with grant option to user"): with When("I grant privilege with grant option to user"):
node.query("GRANT SELECT(d) ON merge_tree TO user0 WITH GRANT OPTION") node.query(f"GRANT SELECT(d) ON {table_name} TO {user0_name} WITH GRANT OPTION")
with Then("I revoke privilege on a column the user with grant option does not have access to"): with Then("I revoke privilege on a column the user with grant option does not have access to"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user0_name)
node.query("REVOKE SELECT(b) ON merge_tree FROM user1", settings=[("user","user0")], node.query(f"REVOKE SELECT(b) ON {table_name} FROM {user1_name}", settings=[("user",user0_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
@TestScenario @TestScenario
@Requirements( @Requirements(
@ -421,16 +412,19 @@ def revoke_privilege_from_user_via_user_with_grant_option(self, table_type, node
def revoke_privilege_from_role_via_user_with_grant_option(self, table_type, node=None): def revoke_privilege_from_role_via_user_with_grant_option(self, table_type, node=None):
"""Check that user is unable to revoke a column they dont have acces to from a role. """Check that user is unable to revoke a column they dont have acces to from a role.
""" """
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"), role(node, "role0"): with user(node, user_name), role(node, role_name):
with When("I grant privilege with grant option to user"): with When("I grant privilege with grant option to user"):
node.query("GRANT SELECT(d) ON merge_tree TO user0 WITH GRANT OPTION") node.query(f"GRANT SELECT(d) ON {table_name} TO {user_name} WITH GRANT OPTION")
with Then("I revoke privilege on a column the user with grant option does not have access to"): with Then("I revoke privilege on a column the user with grant option does not have access to"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query("REVOKE SELECT(b) ON merge_tree FROM role0", settings=[("user","user0")], node.query(f"REVOKE SELECT(b) ON {table_name} FROM {role_name}", settings=[("user",user_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
@TestScenario @TestScenario
@Requirements( @Requirements(
@ -439,18 +433,22 @@ def revoke_privilege_from_role_via_user_with_grant_option(self, table_type, node
def revoke_privilege_from_user_via_role_with_grant_option(self, table_type, node=None): def revoke_privilege_from_user_via_role_with_grant_option(self, table_type, node=None):
"""Check that user with a role is unable to revoke a column they dont have acces to from a user. """Check that user with a role is unable to revoke a column they dont have acces to from a user.
""" """
user0_name = f"user0_{getuid()}"
user1_name = f"user1_{getuid()}"
role_name = f"role_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"), user(node,"user1"), role(node, "role0"): with user(node, f"{user0_name},{user1_name}"), role(node, role_name):
with When("I grant privilege with grant option to a role"): with When("I grant privilege with grant option to a role"):
node.query("GRANT SELECT(d) ON merge_tree TO role0 WITH GRANT OPTION") node.query(f"GRANT SELECT(d) ON {table_name} TO {role_name} WITH GRANT OPTION")
with And("I grant the role to a user"): with And("I grant the role to a user"):
node.query("GRANT role0 TO user0") node.query(f"GRANT {role_name} TO {user0_name}")
with Then("I revoke privilege on a column the user with grant option does not have access to"): with Then("I revoke privilege on a column the user with grant option does not have access to"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user0_name)
node.query("REVOKE SELECT(b) ON merge_tree FROM user1", settings=[("user","user0")], node.query(f"REVOKE SELECT(b) ON {table_name} FROM {user1_name}", settings=[("user",user0_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
@TestScenario @TestScenario
@Requirements( @Requirements(
@ -459,44 +457,45 @@ def revoke_privilege_from_user_via_role_with_grant_option(self, table_type, node
def revoke_privilege_from_role_via_role_with_grant_option(self, table_type, node=None): def revoke_privilege_from_role_via_role_with_grant_option(self, table_type, node=None):
"""Check that user with a role is unable to revoke a column they dont have acces to from a role. """Check that user with a role is unable to revoke a column they dont have acces to from a role.
""" """
user_name = f"user_{getuid()}"
role0_name = f"role0_{getuid()}"
role1_name = f"role1_{getuid()}"
table_name = f"table_{getuid()}"
if node is None: if node is None:
node = self.context.node node = self.context.node
with table(node, "merge_tree", table_type): with table(node, table_name, table_type):
with user(node, "user0"), role(node, "role0"), role(node, "role1"): with user(node, user_name), role(node, f"{role0_name},{role1_name}"):
with When("I grant privilege with grant option to a role"): with When("I grant privilege with grant option to a role"):
node.query("GRANT SELECT(d) ON merge_tree TO user0 WITH GRANT OPTION") node.query(f"GRANT SELECT(d) ON {table_name} TO {role0_name} WITH GRANT OPTION")
with And("I grant the role to a user"): with And("I grant the role to a user"):
node.query("GRANT role0 TO user0") node.query(f"GRANT {role0_name} TO {user_name}")
with Then("I revoke privilege on a column the user with grant option does not have access to"): with Then("I revoke privilege on a column the user with grant option does not have access to"):
exitcode, message = errors.not_enough_privileges(name="user0") exitcode, message = errors.not_enough_privileges(name=user_name)
node.query("REVOKE SELECT(b) ON merge_tree FROM role1", settings=[("user","user0")], node.query(f"REVOKE SELECT(b) ON {table_name} FROM {role1_name}", settings=[("user",user_name)],
exitcode=exitcode, message=message) exitcode=exitcode, message=message)
@TestOutline(Feature) @TestOutline(Feature)
@Requirements( @Requirements(
RQ_SRS_006_RBAC_Privileges_Select("1.0"), RQ_SRS_006_RBAC_Privileges_Select("1.0"),
RQ_SRS_006_RBAC_Privileges_Select_TableEngines("1.0")
) )
@Examples("table_type", [ @Examples("table_type", [
(table_type, Requirements(requirement)) for table_type, requirement in list(table_requirements.items()) (key,) for key in table_types.keys()
]) ])
@Name("select") @Name("select")
def feature(self, table_type, node="clickhouse1"): def feature(self, table_type, parallel=None, stress=None, node="clickhouse1"):
self.context.node = self.context.cluster.node(node) self.context.node = self.context.cluster.node(node)
Scenario(test=without_privilege)(table_type=table_type) if stress is not None:
Scenario(test=user_with_privilege)(table_type=table_type) self.context.stress = stress
Scenario(test=user_with_revoked_privilege)(table_type=table_type) if parallel is not None:
Scenario(test=user_with_privilege_on_columns)(table_type=table_type) self.context.stress = parallel
Scenario(test=role_with_privilege)(table_type=table_type)
Scenario(test=role_with_revoked_privilege)(table_type=table_type) tasks = []
Scenario(test=user_with_revoked_role)(table_type=table_type) pool = Pool(3)
Scenario(test=role_with_privilege_on_columns)(table_type=table_type)
Scenario(test=user_with_privilege_on_cluster)(table_type=table_type) try:
Scenario(test=user_with_privilege_from_user_with_grant_option)(table_type=table_type) for scenario in loads(current_module(), Scenario):
Scenario(test=role_with_privilege_from_user_with_grant_option)(table_type=table_type) run_scenario(pool, tasks, scenario, {"table_type" : table_type})
Scenario(test=user_with_privilege_from_role_with_grant_option)(table_type=table_type) finally:
Scenario(test=role_with_privilege_from_role_with_grant_option)(table_type=table_type) join(tasks)
Scenario(test=revoke_privilege_from_user_via_user_with_grant_option)(table_type=table_type)
Scenario(test=revoke_privilege_from_role_via_user_with_grant_option)(table_type=table_type)
Scenario(test=revoke_privilege_from_user_via_role_with_grant_option)(table_type=table_type)
Scenario(test=revoke_privilege_from_role_via_role_with_grant_option)(table_type=table_type)

View File

@ -0,0 +1,62 @@
from contextlib import contextmanager
from testflows.core import *
from testflows.asserts import error
from rbac.requirements import *
from rbac.helper.common import *
import rbac.helper.errors as errors
@TestScenario
@Requirements(
RQ_SRS_006_RBAC_Table_ShowTables("1.0"),
)
def show_tables(self, node=None):
"""Check that a user is able to see a table in `SHOW TABLES` if and only if the user has privilege on that table,
either granted directly or through a role.
"""
user_name = f"user_{getuid()}"
role_name = f"role_{getuid()}"
if node is None:
node = self.context.node
with user(node, f"{user_name}"):
Scenario(test=show_tables_general, flags=TE,
name="create with create view and select privilege granted directly")(grant_target_name=user_name, user_name=user_name)
with user(node, f"{user_name}"), role(node, f"{role_name}"):
with When("I grant the role to the user"):
node.query(f"GRANT {role_name} TO {user_name}")
Scenario(test=show_tables_general, flags=TE,
name="create with create view and select privilege granted through a role")(grant_target_name=role_name, user_name=user_name)
@TestScenario
def show_tables_general(self, grant_target_name, user_name, node=None):
table0_name = f"table0_{getuid()}"
if node is None:
node = self.context.node
try:
with Given("I have a table"):
node.query(f"DROP TABLE IF EXISTS {table0_name}")
node.query(f"CREATE TABLE {table0_name} (a String, b Int8, d Date) Engine = Memory")
with Then("I check user does not see any tables"):
output = node.query("SHOW TABLES", settings = [("user", f"{user_name}")]).output
assert output == '', error()
with When("I grant select privilege on the table"):
node.query(f"GRANT SELECT(a) ON {table0_name} TO {grant_target_name}")
with Then("I check the user does see a table"):
output = node.query("SHOW TABLES", settings = [("user", f"{user_name}")]).output
assert output == f'{table0_name}', error()
finally:
with Finally("I drop the table"):
node.query(f"DROP TABLE IF EXISTS {table0_name}")
@TestFeature
@Name("show tables")
def feature(self, node="clickhouse1"):
self.context.node = self.context.cluster.node(node)
Scenario(run=show_tables, flags=TE)

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("alter quota") @Name("alter quota")
@ -15,7 +15,7 @@ def feature(self, node="clickhouse1"):
ALTER QUOTA [IF EXISTS] name [ON CLUSTER cluster_name] ALTER QUOTA [IF EXISTS] name [ON CLUSTER cluster_name]
[RENAME TO new_name] [RENAME TO new_name]
[KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}]
[FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | MONTH}
{MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] |
NO LIMITS | TRACKING ONLY} [,...]] NO LIMITS | TRACKING ONLY} [,...]]
[TO {role [,...] | ALL | ALL EXCEPT role [,...]}] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
@ -93,7 +93,7 @@ def feature(self, node="clickhouse1"):
with When("I alter quota on a randomized interval"): with When("I alter quota on a randomized interval"):
node.query("ALTER QUOTA quota0 FOR RANDOMIZED INTERVAL 1 DAY NO LIMITS") node.query("ALTER QUOTA quota0 FOR RANDOMIZED INTERVAL 1 DAY NO LIMITS")
intervals = ['SECOND', 'MINUTE', 'HOUR', 'DAY', 'WEEK', 'MONTH', 'QUARTER', 'YEAR'] intervals = ['SECOND', 'MINUTE', 'HOUR', 'DAY', 'MONTH']
for i, interval in enumerate(intervals): for i, interval in enumerate(intervals):
with Scenario(f"I alter quota for interval {interval}", flags=TE, requirements=[ with Scenario(f"I alter quota for interval {interval}", flags=TE, requirements=[
RQ_SRS_006_RBAC_Quota_Alter_Interval("1.0")]): RQ_SRS_006_RBAC_Quota_Alter_Interval("1.0")]):
@ -203,4 +203,4 @@ def feature(self, node="clickhouse1"):
with Finally("I drop the quota and all the users and roles"): with Finally("I drop the quota and all the users and roles"):
node.query(f"DROP QUOTA IF EXISTS quota0") node.query(f"DROP QUOTA IF EXISTS quota0")
node.query(f"DROP USER IF EXISTS user0") node.query(f"DROP USER IF EXISTS user0")
node.query(f"DROP ROLE IF EXISTS role0") node.query(f"DROP ROLE IF EXISTS role0")

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("alter role") @Name("alter role")

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("alter row policy") @Name("alter row policy")

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("alter settings profile") @Name("alter settings profile")
@ -82,13 +82,13 @@ def feature(self, node="clickhouse1"):
del new_profile del new_profile
with Scenario("I alter settings profile with a setting value", flags=TE, requirements=[ with Scenario("I alter settings profile with a setting value", flags=TE, requirements=[
RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables("1.0"), RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables("1.0"),
RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables_Value("1.0")]): RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables_Value("1.0")]):
with When("I alter settings profile using settings"): with When("I alter settings profile using settings"):
node.query("ALTER SETTINGS PROFILE profile0 SETTINGS max_memory_usage = 100000001") node.query("ALTER SETTINGS PROFILE profile0 SETTINGS max_memory_usage = 100000001")
with Scenario("I alter settings profile with a setting value, does not exist, throws exception", flags=TE, requirements=[ with Scenario("I alter settings profile with a setting value, does not exist, throws exception", flags=TE, requirements=[
RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables("1.0"), RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables("1.0"),
RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables_Value("1.0")]): RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables_Value("1.0")]):
with When("I alter settings profile using settings and nonexistent value"): with When("I alter settings profile using settings and nonexistent value"):
exitcode, message = errors.unknown_setting("fake_setting") exitcode, message = errors.unknown_setting("fake_setting")
@ -137,7 +137,7 @@ def feature(self, node="clickhouse1"):
del profile del profile
with Scenario("I alter settings profile with multiple settings", flags=TE, requirements=[ with Scenario("I alter settings profile with multiple settings", flags=TE, requirements=[
RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables("1.0"), RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables("1.0"),
RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables_Value("1.0")]): RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables_Value("1.0")]):
with When("I alter settings profile with multiple settings"): with When("I alter settings profile with multiple settings"):
node.query("ALTER SETTINGS PROFILE profile0" node.query("ALTER SETTINGS PROFILE profile0"
@ -145,7 +145,7 @@ def feature(self, node="clickhouse1"):
" SETTINGS max_memory_usage_for_user = 100000001") " SETTINGS max_memory_usage_for_user = 100000001")
with Scenario("I alter settings profile with multiple settings short form", flags=TE, requirements=[ with Scenario("I alter settings profile with multiple settings short form", flags=TE, requirements=[
RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables("1.0"), RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables("1.0"),
RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables_Value("1.0")]): RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables_Value("1.0")]):
with When("I alter settings profile with short form multiple settings"): with When("I alter settings profile with short form multiple settings"):
node.query("ALTER SETTINGS PROFILE profile0" node.query("ALTER SETTINGS PROFILE profile0"

View File

@ -3,8 +3,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("alter user") @Name("alter user")

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("create quota") @Name("create quota")
@ -14,7 +14,7 @@ def feature(self, node="clickhouse1"):
```sql ```sql
CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name]
[KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}]
[FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR} [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}
{MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] |
NO LIMITS | TRACKING ONLY} [,...]] NO LIMITS | TRACKING ONLY} [,...]]
[TO {role [,...] | ALL | ALL EXCEPT role [,...]}] [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
@ -107,7 +107,7 @@ def feature(self, node="clickhouse1"):
with When("I create a quota for randomized interval"): with When("I create a quota for randomized interval"):
node.query("CREATE QUOTA quota9 FOR RANDOMIZED INTERVAL 1 DAY NO LIMITS") node.query("CREATE QUOTA quota9 FOR RANDOMIZED INTERVAL 1 DAY NO LIMITS")
intervals = ['SECOND', 'MINUTE', 'HOUR', 'DAY', 'WEEK', 'MONTH', 'QUARTER', 'YEAR'] intervals = ['SECOND', 'MINUTE', 'HOUR', 'DAY', 'MONTH']
for i, interval in enumerate(intervals): for i, interval in enumerate(intervals):
with Scenario(f"I create quota for interval {interval}", flags=TE, requirements=[ with Scenario(f"I create quota for interval {interval}", flags=TE, requirements=[
RQ_SRS_006_RBAC_Quota_Create_Interval("1.0")]): RQ_SRS_006_RBAC_Quota_Create_Interval("1.0")]):

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("create role") @Name("create role")
@ -42,7 +42,9 @@ def feature(self, node="clickhouse1"):
RQ_SRS_006_RBAC_Role_Create("1.0")]): RQ_SRS_006_RBAC_Role_Create("1.0")]):
role = "role0" role = "role0"
with cleanup(role): with cleanup(role):
with When(f"I create role {role}"): with Given(f"I have role {role}"):
node.query(f"CREATE ROLE {role}")
with When(f"I create role {role}, throws exception"):
exitcode, message = errors.cannot_insert_role(name=role) exitcode, message = errors.cannot_insert_role(name=role)
node.query(f"CREATE ROLE {role}", exitcode=exitcode, message=message) node.query(f"CREATE ROLE {role}", exitcode=exitcode, message=message)
del role del role

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("create row policy") @Name("create row policy")
@ -158,14 +158,14 @@ def feature(self, node="clickhouse1"):
node.query(f"CREATE ROW POLICY policy8a ON default.foo TO ALL EXCEPT {role}", exitcode=exitcode, message=message) node.query(f"CREATE ROW POLICY policy8a ON default.foo TO ALL EXCEPT {role}", exitcode=exitcode, message=message)
del role del role
with Scenario("I create row policy assigned to multiple roles", flags=TE, requirements=[ with Scenario("I create row policy assigned to multiple roles", flags=TE, requirements=[
RQ_SRS_006_RBAC_RowPolicy_Create_Assignment("1.0"), RQ_SRS_006_RBAC_RowPolicy_Create_Assignment("1.0"),
RQ_SRS_006_RBAC_RowPolicy_Create_On("1.0")]): RQ_SRS_006_RBAC_RowPolicy_Create_On("1.0")]):
with cleanup("policy8b"): with cleanup("policy8b"):
with When("I create row policy for multiple roles"): with When("I create row policy for multiple roles"):
node.query("CREATE ROW POLICY policy8b ON default.foo TO role0, role1") node.query("CREATE ROW POLICY policy8b ON default.foo TO role0, role1")
with Scenario("I create row policy assigned to all", flags=TE, requirements=[ with Scenario("I create row policy assigned to all", flags=TE, requirements=[
RQ_SRS_006_RBAC_RowPolicy_Create_Assignment_All("1.0"), RQ_SRS_006_RBAC_RowPolicy_Create_Assignment_All("1.0"),
RQ_SRS_006_RBAC_RowPolicy_Create_On("1.0")]): RQ_SRS_006_RBAC_RowPolicy_Create_On("1.0")]):
with cleanup("policy9"): with cleanup("policy9"):
@ -186,7 +186,7 @@ def feature(self, node="clickhouse1"):
with When("I create row policy for all except multiple roles"): with When("I create row policy for all except multiple roles"):
node.query("CREATE ROW POLICY policy11 ON default.foo TO ALL EXCEPT role0, role1") node.query("CREATE ROW POLICY policy11 ON default.foo TO ALL EXCEPT role0, role1")
with Scenario("I create row policy assigned to none", flags=TE, requirements=[ with Scenario("I create row policy assigned to none", flags=TE, requirements=[
RQ_SRS_006_RBAC_RowPolicy_Create_Assignment_None("1.0"), RQ_SRS_006_RBAC_RowPolicy_Create_Assignment_None("1.0"),
RQ_SRS_006_RBAC_RowPolicy_Create_On("1.0")]): RQ_SRS_006_RBAC_RowPolicy_Create_On("1.0")]):
with cleanup("policy11"): with cleanup("policy11"):

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("create settings profile") @Name("create settings profile")
@ -97,7 +97,7 @@ def feature(self, node="clickhouse1"):
node.query("CREATE SETTINGS PROFILE profile4 SETTINGS max_memory_usage = 100000001") node.query("CREATE SETTINGS PROFILE profile4 SETTINGS max_memory_usage = 100000001")
with Scenario("I create settings profile with a setting value, does not exist, throws exception", flags=TE, requirements=[ with Scenario("I create settings profile with a setting value, does not exist, throws exception", flags=TE, requirements=[
RQ_SRS_006_RBAC_SettingsProfile_Create_Variables("1.0"), RQ_SRS_006_RBAC_SettingsProfile_Create_Variables("1.0"),
RQ_SRS_006_RBAC_SettingsProfile_Create_Variables_Value("1.0")]): RQ_SRS_006_RBAC_SettingsProfile_Create_Variables_Value("1.0")]):
with When("I create settings profile using settings and nonexistent value"): with When("I create settings profile using settings and nonexistent value"):
exitcode, message = errors.unknown_setting("fake_setting") exitcode, message = errors.unknown_setting("fake_setting")

View File

@ -3,8 +3,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("create user") @Name("create user")

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("drop quota") @Name("drop quota")

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("drop role") @Name("drop role")
@ -73,9 +73,9 @@ def feature(self, node="clickhouse1"):
with Scenario("I drop role on cluster", flags=TE, requirements=[ with Scenario("I drop role on cluster", flags=TE, requirements=[
RQ_SRS_006_RBAC_Role_Drop_Cluster("1.0")]): RQ_SRS_006_RBAC_Role_Drop_Cluster("1.0")]):
with Given("I have a role on cluster"): with Given("I have a role on cluster"):
node.query("CREATE ROLE role0 ON CLUSTER sharded_cluster") node.query("CREATE ROLE OR REPLACE role0 ON CLUSTER sharded_cluster")
with When("I drop the role from the cluster"): with When("I drop the role from the cluster"):
node.query("DROP ROLE role0 ON CLUSTER sharded_cluster") node.query("DROP ROLE IF EXISTS role0 ON CLUSTER sharded_cluster")
with Scenario("I drop role on fake cluster", flags=TE, requirements=[ with Scenario("I drop role on fake cluster", flags=TE, requirements=[
RQ_SRS_006_RBAC_Role_Drop_Cluster("1.0")]): RQ_SRS_006_RBAC_Role_Drop_Cluster("1.0")]):

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("drop row policy") @Name("drop row policy")

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("drop settings profile") @Name("drop settings profile")

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("drop user") @Name("drop user")

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@contextmanager @contextmanager
def setup(node): def setup(node):
@ -59,7 +59,7 @@ def grant_privilege(self, privilege, on, allow_column, allow_introspection, node
node.query(f"GRANT {privilege} ON {on_} TO user1 WITH GRANT OPTION", settings=settings) node.query(f"GRANT {privilege} ON {on_} TO user1 WITH GRANT OPTION", settings=settings)
if allow_column and ('*' not in on_): if allow_column and ('*' not in on_):
#grant column specific for some column 'x' # Grant column specific for some column 'x'
with When("I grant privilege with columns"): with When("I grant privilege with columns"):
node.query(f"GRANT {privilege}(x) ON {on_} TO user0", settings=settings) node.query(f"GRANT {privilege}(x) ON {on_} TO user0", settings=settings)
@ -131,4 +131,4 @@ def feature(self, node="clickhouse1"):
RQ_SRS_006_RBAC_Grant_Privilege_None("1.0")]): RQ_SRS_006_RBAC_Grant_Privilege_None("1.0")]):
with setup(node): with setup(node):
with When("I grant privilege with grant option"): with When("I grant privilege with grant option"):
node.query("GRANT NONE ON *.* TO user0 WITH GRANT OPTION") node.query("GRANT NONE ON *.* TO user0 WITH GRANT OPTION")

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("grant role") @Name("grant role")
@ -98,14 +98,14 @@ def feature(self, node="clickhouse1"):
RQ_SRS_006_RBAC_Grant_Role_OnCluster("1.0")]): RQ_SRS_006_RBAC_Grant_Role_OnCluster("1.0")]):
try: try:
with Given("I have a user and a role on a cluster"): with Given("I have a user and a role on a cluster"):
node.query("CREATE USER user0 ON CLUSTER sharded_cluster") node.query("CREATE USER OR REPLACE user0 ON CLUSTER sharded_cluster")
node.query("CREATE ROLE role0 ON CLUSTER sharded_cluster") node.query("CREATE ROLE OR REPLACE role0 ON CLUSTER sharded_cluster")
with When("I grant the role to the user"): with When("I grant the role to the user"):
node.query("GRANT ON CLUSTER sharded_cluster role0 TO user0") node.query("GRANT ON CLUSTER sharded_cluster role0 TO user0")
finally: finally:
with Finally("I drop the user and role"): with Finally("I drop the user and role"):
node.query("DROP USER user0 ON CLUSTER sharded_cluster") node.query("DROP USER IF EXISTS user0 ON CLUSTER sharded_cluster")
node.query("DROP ROLE role0 ON CLUSTER sharded_cluster") node.query("DROP ROLE IF EXISTS role0 ON CLUSTER sharded_cluster")
with Scenario("I grant role to user on fake cluster, throws exception", flags=TE, requirements=[ with Scenario("I grant role to user on fake cluster, throws exception", flags=TE, requirements=[
RQ_SRS_006_RBAC_Grant_Role_OnCluster("1.0")]): RQ_SRS_006_RBAC_Grant_Role_OnCluster("1.0")]):

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@contextmanager @contextmanager
def setup(node): def setup(node):
@ -19,31 +19,32 @@ def setup(node):
node.query("DROP USER IF EXISTS user1") node.query("DROP USER IF EXISTS user1")
node.query("DROP ROLE IF EXISTS role1") node.query("DROP ROLE IF EXISTS role1")
@TestOutline(Scenario) @TestOutline(Scenario)
@Examples("privilege on allow_introspection", [ @Examples("privilege on allow_column allow_introspection", [
("dictGet", ("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_DictGet("1.0"))), ("dictGet", ("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_DictGet("1.0"))),
("INTROSPECTION", ("*.*",), True, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Introspection("1.0"))), ("INTROSPECTION", ("*.*",), False, True, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Introspection("1.0"))),
("SELECT", ("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Select("1.0"))), ("SELECT", ("db0.table0","db0.*","*.*","tb0","*"), True, False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Select("1.0"))),
("INSERT",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Insert("1.0"))), ("INSERT",("db0.table0","db0.*","*.*","tb0","*"), True, False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Insert("1.0"))),
("ALTER",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Alter("1.0"))), ("ALTER",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Alter("1.0"))),
("CREATE",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Create("1.0"))), ("CREATE",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Create("1.0"))),
("DROP",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Drop("1.0"))), ("DROP",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Drop("1.0"))),
("TRUNCATE",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Truncate("1.0"))), ("TRUNCATE",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Truncate("1.0"))),
("OPTIMIZE",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Optimize("1.0"))), ("OPTIMIZE",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Optimize("1.0"))),
("SHOW",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Show("1.0"))), ("SHOW",("db0.table0","db0.*","*.*","tb0","*"), True, False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Show("1.0"))),
("KILL QUERY",("*.*",), False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_KillQuery("1.0"))), ("KILL QUERY",("*.*",), False, False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_KillQuery("1.0"))),
("ACCESS MANAGEMENT",("*.*",), False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_AccessManagement("1.0"))), ("ACCESS MANAGEMENT",("*.*",), False, False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_AccessManagement("1.0"))),
("SYSTEM",("db0.table0","db0.*","*.*","tb0","*"), False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_System("1.0"))), ("SYSTEM",("db0.table0","db0.*","*.*","tb0","*"), False, False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_System("1.0"))),
("SOURCES",("*.*",), False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Sources("1.0"))), ("SOURCES",("*.*",), False, False, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_Sources("1.0"))),
("ALL",("*.*",), True, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_All("1.0"))), ("ALL",("*.*",), True, True, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_All("1.0"))),
("ALL PRIVILEGES",("*.*",), True, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_All("1.0"))), #alias for all ("ALL PRIVILEGES",("*.*",), True, True, Requirements(RQ_SRS_006_RBAC_Revoke_Privilege_All("1.0"))), #alias for all
],) ],)
def revoke_privileges(self, privilege, on, allow_introspection, node="clickhouse1"): def revoke_privileges(self, privilege, on, allow_column, allow_introspection, node="clickhouse1"):
revoke_privilege(privilege=privilege, on=on, allow_introspection=allow_introspection, node=node) revoke_privilege(privilege=privilege, on=on, allow_column=allow_column, allow_introspection=allow_introspection, node=node)
@TestOutline(Scenario) @TestOutline(Scenario)
@Requirements([RQ_SRS_006_RBAC_Revoke_Privilege_Any("1.0") , RQ_SRS_006_RBAC_Revoke_Privilege_PrivelegeColumns("1.0")]) @Requirements([RQ_SRS_006_RBAC_Revoke_Privilege_Any("1.0") , RQ_SRS_006_RBAC_Revoke_Privilege_PrivelegeColumns("1.0")])
def revoke_privilege(self, privilege, on, allow_introspection, node="clickhouse1"): def revoke_privilege(self, privilege, on, allow_column, allow_introspection, node="clickhouse1"):
node = self.context.cluster.node(node) node = self.context.cluster.node(node)
for on_ in on: for on_ in on:
with When(f"I revoke {privilege} privilege from user on {on_}"): with When(f"I revoke {privilege} privilege from user on {on_}"):
@ -55,9 +56,10 @@ def revoke_privilege(self, privilege, on, allow_introspection, node="clickhouse1
with When("I revoke privilege without columns"): with When("I revoke privilege without columns"):
node.query(f"REVOKE {privilege} ON {on_} FROM user0", settings=settings) node.query(f"REVOKE {privilege} ON {on_} FROM user0", settings=settings)
#revoke column specific for some column 'x' if allow_column and ('*' not in on_):
with When("I revoke privilege with columns"): # Revoke column specific for some column 'x'
node.query(f"REVOKE {privilege}(x) ON {on_} FROM user0", settings=settings) with When("I revoke privilege with columns"):
node.query(f"REVOKE {privilege}(x) ON {on_} FROM user0", settings=settings)
@TestFeature @TestFeature
@Name("revoke privilege") @Name("revoke privilege")
@ -122,7 +124,7 @@ def feature(self, node="clickhouse1"):
exitcode, message = errors.cannot_update_default() exitcode, message = errors.cannot_update_default()
node.query("REVOKE NONE FROM default", exitcode=exitcode,message=message) node.query("REVOKE NONE FROM default", exitcode=exitcode,message=message)
# By default, ClickHouse treats unnamed object as role #By default, ClickHouse treats unnamed object as role
with Scenario("I revoke privilege from nonexistent role, throws exception", flags=TE, requirements=[ with Scenario("I revoke privilege from nonexistent role, throws exception", flags=TE, requirements=[
RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"), RQ_SRS_006_RBAC_Revoke_Privilege_From("1.0"),
RQ_SRS_006_RBAC_Revoke_Privilege_None("1.0")]): RQ_SRS_006_RBAC_Revoke_Privilege_None("1.0")]):

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("revoke role") @Name("revoke role")
@ -50,7 +50,7 @@ def feature(self, node="clickhouse1"):
exitcode, message = errors.role_not_found_in_disk(name="role0") exitcode, message = errors.role_not_found_in_disk(name="role0")
node.query("REVOKE role0 FROM user0", exitcode=exitcode, message=message) node.query("REVOKE role0 FROM user0", exitcode=exitcode, message=message)
# With nonexistent object name, REVOKE assumes type role (treats user0 as role) # with nonexistent object name, REVOKE assumes type role (treats user0 as role)
with Scenario("I revoke a role from a nonexistent user", requirements=[ with Scenario("I revoke a role from a nonexistent user", requirements=[
RQ_SRS_006_RBAC_Revoke_Role("1.0")]): RQ_SRS_006_RBAC_Revoke_Role("1.0")]):
with setup(0,1): with setup(0,1):
@ -58,7 +58,7 @@ def feature(self, node="clickhouse1"):
exitcode, message = errors.role_not_found_in_disk(name="user0") exitcode, message = errors.role_not_found_in_disk(name="user0")
node.query("REVOKE role0 FROM user0", exitcode=exitcode, message=message) node.query("REVOKE role0 FROM user0", exitcode=exitcode, message=message)
# With nonexistent object name, REVOKE assumes type role (treats user0 as role) # with nonexistent object name, REVOKE assumes type role (treats user0 as role)
with Scenario("I revoke a role from ALL EXCEPT nonexistent user", requirements=[ with Scenario("I revoke a role from ALL EXCEPT nonexistent user", requirements=[
RQ_SRS_006_RBAC_Revoke_Role("1.0")]): RQ_SRS_006_RBAC_Revoke_Role("1.0")]):
with setup(0,1): with setup(0,1):

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("set default role") @Name("set default role")
@ -56,7 +56,7 @@ def feature(self, node="clickhouse1"):
exitcode, message = errors.user_not_found_in_disk(name="user0") exitcode, message = errors.user_not_found_in_disk(name="user0")
node.query("SET DEFAULT ROLE role0 TO user0", exitcode=exitcode, message=message) node.query("SET DEFAULT ROLE role0 TO user0", exitcode=exitcode, message=message)
# In SET DEFAULT ROLE, the nonexistent user is noticed first and becomes the thrown exception #in SET DEFAULT ROLE, the nonexistent user is noticed first and becomes the thrown exception
with Scenario("I set default a nonexistent role to a nonexistent user", requirements=[ with Scenario("I set default a nonexistent role to a nonexistent user", requirements=[
RQ_SRS_006_RBAC_SetDefaultRole("1.0")]): RQ_SRS_006_RBAC_SetDefaultRole("1.0")]):
with setup(0,0): with setup(0,0):

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("set role") @Name("set role")

View File

@ -2,8 +2,8 @@ from contextlib import contextmanager
from testflows.core import * from testflows.core import *
import rbac.helper.errors as errors
from rbac.requirements import * from rbac.requirements import *
import rbac.tests.errors as errors
@TestFeature @TestFeature
@Name("show create role") @Name("show create role")

View File

View File

@ -0,0 +1,20 @@
from testflows.core import *
from rbac.helper.common import *
@TestFeature
@Name("views")
def feature(self):
tasks = []
pool = Pool(3)
try:
try:
run_scenario(pool, tasks, Feature(test=load("rbac.tests.views.view", "feature")), {})
run_scenario(pool, tasks, Feature(test=load("rbac.tests.views.live_view", "feature")), {})
run_scenario(pool, tasks, Feature(test=load("rbac.tests.views.materialized_view", "feature")), {})
finally:
join(tasks)
finally:
pool.close()

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -9,10 +9,10 @@ from helpers.argparser import argparser
@TestModule @TestModule
@Name("clickhouse") @Name("clickhouse")
@ArgumentParser(argparser) @ArgumentParser(argparser)
def regression(self, local, clickhouse_binary_path): def regression(self, local, clickhouse_binary_path, stress=None, parallel=None):
"""ClickHouse regression. """ClickHouse regression.
""" """
args = {"local": local, "clickhouse_binary_path": clickhouse_binary_path} args = {"local": local, "clickhouse_binary_path": clickhouse_binary_path, "stress": stress, "parallel": parallel}
Feature(test=load("example.regression", "regression"))(**args) Feature(test=load("example.regression", "regression"))(**args)
Feature(test=load("ldap.regression", "regression"))(**args) Feature(test=load("ldap.regression", "regression"))(**args)