ClickHouse/tests/integration/test_settings_constraints_distributed/test.py
bharatnc b75963d370 Format and optimize imports in integration test files
This PR formats all the `*.py` files found under the `tests/integration`
folder. It also reorders the imports and cleans up a bunch of unused
imports.

The formatting also takes care of other things like wrapping lines and
fixing spaces and indents such that the tests look more readable.
2020-09-15 23:10:54 -07:00

113 lines
6.6 KiB
Python

import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=["configs/config.d/remote_servers.xml"],
user_configs=["configs/users.d/allow_introspection_functions.xml"])
node2 = cluster.add_instance('node2', main_configs=["configs/config.d/remote_servers.xml"],
user_configs=["configs/users.d/allow_introspection_functions.xml"])
distributed = cluster.add_instance('distributed', main_configs=["configs/config.d/remote_servers.xml"],
user_configs=["configs/users.d/allow_introspection_functions.xml"], stay_alive=True)
@pytest.fixture(scope="module", autouse=True)
def started_cluster():
try:
cluster.start()
for node in [node1, node2]:
node.query("CREATE TABLE sometable (date Date, id UInt32, value Int32) ENGINE = MergeTree() ORDER BY id;")
node.query("INSERT INTO sometable VALUES (toDate('2010-01-10'), 1, 1)")
node.query("CREATE USER shard")
node.query("GRANT ALL ON *.* TO shard")
distributed.query(
"CREATE TABLE proxy (date Date, id UInt32, value Int32) ENGINE = Distributed(test_cluster, default, sometable, toUInt64(date));")
distributed.query(
"CREATE TABLE shard_settings (name String, value String) ENGINE = Distributed(test_cluster, system, settings);")
distributed.query("CREATE ROLE admin")
distributed.query("GRANT ALL ON *.* TO admin")
yield cluster
finally:
cluster.shutdown()
@pytest.fixture(autouse=True)
def restart_distributed():
# Magic: Distributed table tries to keep connections to shards open, and after changing shards' default settings
# we need to reset connections to force the shards to reset sessions and therefore to reset current settings
# to their new defaults.
distributed.restart_clickhouse()
def test_select_clamps_settings():
distributed.query("CREATE USER normal DEFAULT ROLE admin SETTINGS max_memory_usage = 80000000")
distributed.query("CREATE USER wasteful DEFAULT ROLE admin SETTINGS max_memory_usage = 2000000000")
distributed.query("CREATE USER readonly DEFAULT ROLE admin SETTINGS readonly = 1")
node1.query("ALTER USER shard SETTINGS max_memory_usage = 50000000 MIN 11111111 MAX 99999999")
node2.query("ALTER USER shard SETTINGS readonly = 1")
# Check that shards doesn't throw exceptions on constraints violation
query = "SELECT COUNT() FROM proxy"
assert distributed.query(query) == '2\n'
assert distributed.query(query, user='normal') == '2\n'
assert distributed.query(query, user='wasteful') == '2\n'
assert distributed.query(query, user='readonly') == '2\n'
assert distributed.query(query, settings={"max_memory_usage": 40000000, "readonly": 2}) == '2\n'
assert distributed.query(query, settings={"max_memory_usage": 3000000000, "readonly": 2}) == '2\n'
query = "SELECT COUNT() FROM remote('node{1,2}', 'default', 'sometable')"
assert distributed.query(query) == '2\n'
assert distributed.query(query, user='normal') == '2\n'
assert distributed.query(query, user='wasteful') == '2\n'
# Check that shards clamp passed settings.
query = "SELECT hostName() as host, name, value FROM shard_settings WHERE name = 'max_memory_usage' OR name = 'readonly' ORDER BY host, name, value"
assert distributed.query(query) == 'node1\tmax_memory_usage\t99999999\n' \
'node1\treadonly\t0\n' \
'node2\tmax_memory_usage\t10000000000\n' \
'node2\treadonly\t1\n'
assert distributed.query(query, user='normal') == 'node1\tmax_memory_usage\t80000000\n' \
'node1\treadonly\t0\n' \
'node2\tmax_memory_usage\t10000000000\n' \
'node2\treadonly\t1\n'
assert distributed.query(query, user='wasteful') == 'node1\tmax_memory_usage\t99999999\n' \
'node1\treadonly\t0\n' \
'node2\tmax_memory_usage\t10000000000\n' \
'node2\treadonly\t1\n'
assert distributed.query(query, user='readonly') == 'node1\tmax_memory_usage\t99999999\n' \
'node1\treadonly\t1\n' \
'node2\tmax_memory_usage\t10000000000\n' \
'node2\treadonly\t1\n'
assert distributed.query(query, settings={"max_memory_usage": 1}) == 'node1\tmax_memory_usage\t11111111\n' \
'node1\treadonly\t0\n' \
'node2\tmax_memory_usage\t10000000000\n' \
'node2\treadonly\t1\n'
assert distributed.query(query, settings={"max_memory_usage": 40000000,
"readonly": 2}) == 'node1\tmax_memory_usage\t40000000\n' \
'node1\treadonly\t2\n' \
'node2\tmax_memory_usage\t10000000000\n' \
'node2\treadonly\t1\n'
assert distributed.query(query, settings={"max_memory_usage": 3000000000,
"readonly": 2}) == 'node1\tmax_memory_usage\t99999999\n' \
'node1\treadonly\t2\n' \
'node2\tmax_memory_usage\t10000000000\n' \
'node2\treadonly\t1\n'
def test_insert_clamps_settings():
node1.query("ALTER USER shard SETTINGS max_memory_usage = 50000000 MIN 11111111 MAX 99999999")
node2.query("ALTER USER shard SETTINGS max_memory_usage = 50000000 MIN 11111111 MAX 99999999")
distributed.query("INSERT INTO proxy VALUES (toDate('2020-02-20'), 2, 2)")
distributed.query("INSERT INTO proxy VALUES (toDate('2020-02-21'), 2, 2)", settings={"max_memory_usage": 5000000})
distributed.query("SYSTEM FLUSH DISTRIBUTED proxy")
assert_eq_with_retry(distributed, "SELECT COUNT() FROM proxy", "4")