ClickHouse/tests/integration/test_keeper_auth/test.py

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

694 lines
20 KiB
Python
Raw Normal View History

2021-05-24 12:18:04 +00:00
import pytest
import time
2021-05-24 12:18:04 +00:00
from helpers.cluster import ClickHouseCluster
from kazoo.client import KazooClient, KazooState
from kazoo.security import ACL, make_digest_acl, make_acl
from kazoo.exceptions import (
AuthFailedError,
InvalidACLError,
NoAuthError,
KazooException,
)
2021-05-24 12:18:04 +00:00
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance(
"node",
main_configs=["configs/keeper_config.xml"],
with_zookeeper=True,
use_keeper=False,
stay_alive=True,
)
2021-05-24 12:18:04 +00:00
SUPERAUTH = "super:admin"
2021-05-24 12:18:04 +00:00
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
2021-05-24 12:18:04 +00:00
def get_fake_zk(timeout=30.0):
_fake_zk_instance = KazooClient(
hosts=cluster.get_instance_ip("node") + ":9181", timeout=timeout
)
2021-05-24 12:18:04 +00:00
_fake_zk_instance.start()
return _fake_zk_instance
2021-05-24 12:18:04 +00:00
def get_genuine_zk():
print("Zoo1", cluster.get_instance_ip("zoo1"))
return cluster.get_kazoo_client("zoo1")
2022-01-07 09:45:30 +00:00
# FIXME: this sleep is a workaround for the bug that is fixed by this patch [1].
#
# The problem is that after AUTH_FAILED (that is caused by the line above)
# there can be a race, because of which, stop() will hang indefinitely.
#
# [1]: https://github.com/python-zk/kazoo/pull/688
def zk_auth_failure_workaround():
time.sleep(2)
def zk_stop_and_close(zk):
if zk:
zk.stop()
zk.close()
@pytest.mark.parametrize(("get_zk"), [get_genuine_zk, get_fake_zk])
2022-01-07 09:45:30 +00:00
def test_remove_acl(started_cluster, get_zk):
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
auth_connection = None
2022-01-07 09:45:30 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
try:
auth_connection = get_zk()
2022-01-07 09:45:30 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
auth_connection.add_auth("digest", "user1:password1")
2022-01-07 09:45:30 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
# Consistent with zookeeper, accept generated digest
auth_connection.create(
"/test_remove_acl1",
b"dataX",
acl=[
make_acl(
"digest",
"user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=",
read=True,
write=False,
create=False,
delete=False,
admin=False,
)
],
)
auth_connection.create(
"/test_remove_acl2",
b"dataX",
acl=[
make_acl(
"digest",
"user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=",
read=True,
write=True,
create=False,
delete=False,
admin=False,
)
],
)
auth_connection.create(
"/test_remove_acl3",
b"dataX",
acl=[make_acl("digest", "user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=", all=True)],
)
2022-01-07 09:45:30 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
auth_connection.delete("/test_remove_acl2")
auth_connection.create(
"/test_remove_acl4",
b"dataX",
acl=[
make_acl(
"digest",
"user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=",
read=True,
write=True,
create=True,
delete=False,
admin=False,
)
],
)
2022-01-07 09:45:30 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
acls, stat = auth_connection.get_acls("/test_remove_acl3")
2022-01-07 09:45:30 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
assert stat.aversion == 0
assert len(acls) == 1
for acl in acls:
assert acl.acl_list == ["ALL"]
assert acl.perms == 31
finally:
zk_stop_and_close(auth_connection)
2022-01-07 09:45:30 +00:00
@pytest.mark.parametrize(("get_zk"), [get_genuine_zk, get_fake_zk])
2021-05-24 12:18:04 +00:00
def test_digest_auth_basic(started_cluster, get_zk):
try:
auth_connection = None
no_auth_connection = None
2021-05-24 12:18:04 +00:00
auth_connection = get_zk()
auth_connection.add_auth("digest", "user1:password1")
2021-05-24 12:18:04 +00:00
auth_connection.create("/test_no_acl", b"")
auth_connection.create(
"/test_all_acl", b"data", acl=[make_acl("auth", "", all=True)]
)
# Consistent with zookeeper, accept generated digest
auth_connection.create(
"/test_all_digest_acl",
b"dataX",
acl=[make_acl("digest", "user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=", all=True)],
)
2021-05-24 12:18:04 +00:00
assert auth_connection.get("/test_all_acl")[0] == b"data"
assert auth_connection.get("/test_all_digest_acl")[0] == b"dataX"
2021-05-24 12:18:04 +00:00
no_auth_connection = get_zk()
no_auth_connection.set("/test_no_acl", b"hello")
assert no_auth_connection.get("/test_no_acl")[0] == b"hello"
2021-05-24 12:18:04 +00:00
# no ACL, so cannot access these nodes
with pytest.raises(NoAuthError):
no_auth_connection.set("/test_all_acl", b"hello")
2021-05-24 12:18:04 +00:00
with pytest.raises(NoAuthError):
no_auth_connection.get("/test_all_acl")
2021-05-24 12:18:04 +00:00
with pytest.raises(NoAuthError):
no_auth_connection.get("/test_all_digest_acl")
2021-05-24 12:18:04 +00:00
# still doesn't help
with pytest.raises(AuthFailedError):
no_auth_connection.add_auth("world", "anyone")
2021-05-24 12:18:04 +00:00
zk_auth_failure_workaround()
zk_stop_and_close(no_auth_connection)
# session became broken, reconnect
no_auth_connection = get_zk()
2021-05-24 12:18:04 +00:00
# wrong auth
no_auth_connection.add_auth("digest", "user2:password2")
2021-05-24 12:18:04 +00:00
with pytest.raises(NoAuthError):
no_auth_connection.set("/test_all_acl", b"hello")
2021-05-24 12:18:04 +00:00
with pytest.raises(NoAuthError):
no_auth_connection.set("/test_all_acl", b"hello")
2021-05-24 12:18:04 +00:00
with pytest.raises(NoAuthError):
no_auth_connection.get("/test_all_acl")
2021-05-24 12:18:04 +00:00
with pytest.raises(NoAuthError):
no_auth_connection.get("/test_all_digest_acl")
2021-05-24 12:18:04 +00:00
# but can access some non restricted nodes
no_auth_connection.create("/some_allowed_node", b"data")
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
# auth added, go on
no_auth_connection.add_auth("digest", "user1:password1")
for path in ["/test_no_acl", "/test_all_acl"]:
no_auth_connection.set(path, b"auth_added")
assert no_auth_connection.get(path)[0] == b"auth_added"
finally:
zk_stop_and_close(auth_connection)
zk_stop_and_close(no_auth_connection)
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
2021-05-24 12:18:04 +00:00
def test_super_auth(started_cluster):
auth_connection = get_fake_zk()
try:
auth_connection.add_auth("digest", "user1:password1")
auth_connection.create("/test_super_no_acl", b"")
auth_connection.create(
"/test_super_all_acl", b"data", acl=[make_acl("auth", "", all=True)]
)
finally:
zk_stop_and_close(auth_connection)
2021-05-24 12:18:04 +00:00
super_connection = get_fake_zk()
try:
super_connection.add_auth("digest", "super:admin")
for path in ["/test_super_no_acl", "/test_super_all_acl"]:
super_connection.set(path, b"value")
assert super_connection.get(path)[0] == b"value"
finally:
zk_stop_and_close(super_connection)
2021-05-24 12:18:04 +00:00
@pytest.mark.parametrize(("get_zk"), [get_genuine_zk, get_fake_zk])
2021-05-24 12:18:04 +00:00
def test_digest_auth_multiple(started_cluster, get_zk):
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
auth_connection = None
one_auth_connection = None
other_auth_connection = None
2021-05-24 12:18:04 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
try:
auth_connection = get_zk()
auth_connection.add_auth("digest", "user1:password1")
auth_connection.add_auth("digest", "user2:password2")
auth_connection.add_auth("digest", "user3:password3")
2021-05-24 12:18:04 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
auth_connection.create(
"/test_multi_all_acl", b"data", acl=[make_acl("auth", "", all=True)]
)
2021-05-24 12:18:04 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
one_auth_connection = get_zk()
one_auth_connection.add_auth("digest", "user1:password1")
2021-05-24 12:18:04 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
one_auth_connection.set("/test_multi_all_acl", b"X")
assert one_auth_connection.get("/test_multi_all_acl")[0] == b"X"
2021-05-24 12:18:04 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
other_auth_connection = get_zk()
other_auth_connection.add_auth("digest", "user2:password2")
2021-05-24 12:18:04 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
other_auth_connection.set("/test_multi_all_acl", b"Y")
assert other_auth_connection.get("/test_multi_all_acl")[0] == b"Y"
finally:
zk_stop_and_close(auth_connection)
zk_stop_and_close(one_auth_connection)
zk_stop_and_close(other_auth_connection)
2021-05-24 12:18:04 +00:00
@pytest.mark.parametrize(("get_zk"), [get_genuine_zk, get_fake_zk])
2021-05-24 12:18:04 +00:00
def test_partial_auth(started_cluster, get_zk):
auth_connection = get_zk()
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
try:
auth_connection.add_auth("digest", "user1:password1")
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
auth_connection.create(
"/test_partial_acl",
b"data",
acl=[
make_acl(
"auth",
"",
read=False,
write=True,
create=True,
delete=True,
admin=True,
)
],
)
2021-05-24 12:18:04 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
auth_connection.set("/test_partial_acl", b"X")
auth_connection.create(
"/test_partial_acl/subnode",
b"X",
acl=[
make_acl(
"auth",
"",
read=False,
write=True,
create=True,
delete=True,
admin=True,
)
],
)
2021-05-24 12:18:04 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
with pytest.raises(NoAuthError):
auth_connection.get("/test_partial_acl")
2021-05-24 12:18:04 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
with pytest.raises(NoAuthError):
auth_connection.get_children("/test_partial_acl")
2021-05-24 12:18:04 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
# exists works without read perm
assert auth_connection.exists("/test_partial_acl") is not None
2021-05-24 12:18:04 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
auth_connection.create(
"/test_partial_acl_create",
b"data",
acl=[
make_acl(
"auth",
"",
read=True,
write=True,
create=False,
delete=True,
admin=True,
)
],
)
with pytest.raises(NoAuthError):
auth_connection.create("/test_partial_acl_create/subnode")
2021-05-24 12:18:04 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
auth_connection.create(
"/test_partial_acl_set",
b"data",
acl=[
make_acl(
"auth",
"",
read=True,
write=False,
create=True,
delete=True,
admin=True,
)
],
)
with pytest.raises(NoAuthError):
auth_connection.set("/test_partial_acl_set", b"X")
2021-05-24 12:18:04 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
# not allowed to delete child node
auth_connection.create(
"/test_partial_acl_delete",
b"data",
acl=[
make_acl(
"auth",
"",
read=True,
write=True,
create=True,
delete=False,
admin=True,
)
],
)
auth_connection.create("/test_partial_acl_delete/subnode")
with pytest.raises(NoAuthError):
auth_connection.delete("/test_partial_acl_delete/subnode")
finally:
zk_stop_and_close(auth_connection)
2021-05-24 12:18:04 +00:00
def test_bad_auth_1(started_cluster):
2021-05-24 12:18:04 +00:00
auth_connection = get_fake_zk()
with pytest.raises(AuthFailedError):
auth_connection.add_auth("world", "anyone")
zk_auth_failure_workaround()
zk_stop_and_close(auth_connection)
2021-05-24 12:18:04 +00:00
def test_bad_auth_2(started_cluster):
2021-05-24 12:18:04 +00:00
auth_connection = get_fake_zk()
with pytest.raises(AuthFailedError):
print("Sending 1")
auth_connection.add_auth("adssagf", "user1:password1")
zk_auth_failure_workaround()
zk_stop_and_close(auth_connection)
2021-05-24 12:18:04 +00:00
def test_bad_auth_3(started_cluster):
2021-05-24 12:18:04 +00:00
auth_connection = get_fake_zk()
with pytest.raises(AuthFailedError):
print("Sending 2")
auth_connection.add_auth("digest", "")
zk_auth_failure_workaround()
zk_stop_and_close(auth_connection)
2021-05-24 12:18:04 +00:00
def test_bad_auth_4(started_cluster):
2021-05-24 12:18:04 +00:00
auth_connection = get_fake_zk()
with pytest.raises(AuthFailedError):
print("Sending 3")
auth_connection.add_auth("", "user1:password1")
zk_auth_failure_workaround()
zk_stop_and_close(auth_connection)
2021-05-24 12:18:04 +00:00
def test_bad_auth_5(started_cluster):
2021-05-24 12:18:04 +00:00
auth_connection = get_fake_zk()
with pytest.raises(AuthFailedError):
print("Sending 4")
auth_connection.add_auth("digest", "user1")
zk_auth_failure_workaround()
zk_stop_and_close(auth_connection)
2021-05-24 12:18:04 +00:00
def test_bad_auth_6(started_cluster):
2021-05-24 12:18:04 +00:00
auth_connection = get_fake_zk()
with pytest.raises(AuthFailedError):
print("Sending 5")
auth_connection.add_auth("digest", "user1:password:otherpassword")
zk_auth_failure_workaround()
zk_stop_and_close(auth_connection)
2021-05-24 12:18:04 +00:00
def test_bad_auth_7(started_cluster):
2021-05-24 12:18:04 +00:00
auth_connection = get_fake_zk()
with pytest.raises(AuthFailedError):
print("Sending 6")
auth_connection.add_auth("auth", "user1:password")
zk_auth_failure_workaround()
zk_stop_and_close(auth_connection)
2021-05-24 12:18:04 +00:00
def test_bad_auth_8(started_cluster):
2021-05-24 12:18:04 +00:00
auth_connection = get_fake_zk()
with pytest.raises(AuthFailedError):
print("Sending 7")
auth_connection.add_auth("world", "somebody")
zk_auth_failure_workaround()
zk_stop_and_close(auth_connection)
2021-05-24 12:18:04 +00:00
def test_bad_auth_9(started_cluster):
2021-05-24 12:18:04 +00:00
auth_connection = get_fake_zk()
with pytest.raises(InvalidACLError):
print("Sending 8")
auth_connection.create(
"/test_bad_acl",
b"data",
acl=[
make_acl(
"dasd",
"",
read=True,
write=False,
create=True,
delete=True,
admin=True,
)
],
)
zk_auth_failure_workaround()
zk_stop_and_close(auth_connection)
2021-05-24 12:18:04 +00:00
def test_bad_auth_10(started_cluster):
2021-05-24 12:18:04 +00:00
auth_connection = get_fake_zk()
with pytest.raises(InvalidACLError):
print("Sending 9")
auth_connection.create(
"/test_bad_acl",
b"data",
acl=[
make_acl(
"digest",
"",
read=True,
write=False,
create=True,
delete=True,
admin=True,
)
],
)
zk_auth_failure_workaround()
zk_stop_and_close(auth_connection)
2021-05-24 12:18:04 +00:00
def test_bad_auth_11(started_cluster):
2021-05-24 12:18:04 +00:00
auth_connection = get_fake_zk()
with pytest.raises(InvalidACLError):
print("Sending 10")
auth_connection.create(
"/test_bad_acl",
b"data",
acl=[
make_acl(
"", "", read=True, write=False, create=True, delete=True, admin=True
)
],
)
zk_auth_failure_workaround()
zk_stop_and_close(auth_connection)
2021-05-24 12:18:04 +00:00
def test_bad_auth_12(started_cluster):
2021-05-24 12:18:04 +00:00
auth_connection = get_fake_zk()
with pytest.raises(InvalidACLError):
print("Sending 11")
auth_connection.create(
"/test_bad_acl",
b"data",
acl=[
make_acl(
"digest",
"dsdasda",
read=True,
write=False,
create=True,
delete=True,
admin=True,
)
],
)
zk_auth_failure_workaround()
zk_stop_and_close(auth_connection)
2021-05-24 12:18:04 +00:00
def test_bad_auth_13(started_cluster):
2021-05-24 12:18:04 +00:00
auth_connection = get_fake_zk()
with pytest.raises(InvalidACLError):
print("Sending 12")
auth_connection.create(
"/test_bad_acl",
b"data",
acl=[
make_acl(
"digest",
"dsad:DSAa:d",
read=True,
write=False,
create=True,
delete=True,
admin=True,
)
],
)
zk_auth_failure_workaround()
zk_stop_and_close(auth_connection)
def test_auth_snapshot(started_cluster):
connection = None
connection1 = None
connection2 = None
try:
connection = get_fake_zk()
connection.add_auth("digest", "user1:password1")
connection.create(
"/test_snapshot_acl", b"data", acl=[make_acl("auth", "", all=True)]
)
connection1 = get_fake_zk()
connection1.add_auth("digest", "user2:password2")
connection1.create(
"/test_snapshot_acl1", b"data", acl=[make_acl("auth", "", all=True)]
)
connection2 = get_fake_zk()
connection2.create("/test_snapshot_acl2", b"data")
for i in range(100):
connection.create(
f"/test_snapshot_acl/path{i}",
b"data",
acl=[make_acl("auth", "", all=True)],
)
node.restart_clickhouse()
zk_stop_and_close(connection)
connection = get_fake_zk()
with pytest.raises(NoAuthError):
connection.get("/test_snapshot_acl")
connection.add_auth("digest", "user1:password1")
assert connection.get("/test_snapshot_acl")[0] == b"data"
with pytest.raises(NoAuthError):
connection.get("/test_snapshot_acl1")
assert connection.get("/test_snapshot_acl2")[0] == b"data"
for i in range(100):
assert connection.get(f"/test_snapshot_acl/path{i}")[0] == b"data"
zk_stop_and_close(connection1)
connection1 = get_fake_zk()
connection1.add_auth("digest", "user2:password2")
assert connection1.get("/test_snapshot_acl1")[0] == b"data"
with pytest.raises(NoAuthError):
connection1.get("/test_snapshot_acl")
zk_stop_and_close(connection2)
connection2 = get_fake_zk()
assert connection2.get("/test_snapshot_acl2")[0] == b"data"
with pytest.raises(NoAuthError):
connection2.get("/test_snapshot_acl")
with pytest.raises(NoAuthError):
connection2.get("/test_snapshot_acl1")
finally:
zk_stop_and_close(connection)
zk_stop_and_close(connection1)
zk_stop_and_close(connection2)
2021-06-22 10:49:35 +00:00
@pytest.mark.parametrize(("get_zk"), [get_genuine_zk, get_fake_zk])
2021-06-22 10:49:35 +00:00
def test_get_set_acl(started_cluster, get_zk):
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
auth_connection = None
other_auth_connection = None
try:
auth_connection = get_zk()
auth_connection.add_auth("digest", "username1:secret1")
auth_connection.add_auth("digest", "username2:secret2")
2021-06-22 10:49:35 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
auth_connection.create(
"/test_set_get_acl", b"data", acl=[make_acl("auth", "", all=True)]
)
2021-06-22 10:49:35 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
acls, stat = auth_connection.get_acls("/test_set_get_acl")
assert stat.aversion == 0
assert len(acls) == 2
for acl in acls:
assert acl.acl_list == ["ALL"]
assert acl.id.scheme == "digest"
assert acl.perms == 31
assert acl.id.id in (
"username1:eGncMdBgOfGS/TCojt51xWsWv/Y=",
"username2:qgSSumukVlhftkVycylbHNvxhFU=",
)
2021-06-22 10:49:35 +00:00
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
other_auth_connection = get_zk()
other_auth_connection.add_auth("digest", "username1:secret1")
other_auth_connection.add_auth("digest", "username3:secret3")
other_auth_connection.set_acls(
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
"/test_set_get_acl",
acls=[
make_acl(
"auth",
"",
read=True,
write=False,
create=True,
delete=True,
admin=True,
)
],
)
tests/integration: add missing kazoo client termination pytest play games with logging output [1]. [1]: https://github.com/pytest-dev/pytest/issues/5502 But this does not actually affect ClickHouse tests, the only reason is that the kazoo client is not stopped correctly without calling kazoo.client.KazooClient.stop(), and that's why you can see the following messages: <details> test_storage_rabbitmq/test.py::test_rabbitmq_big_message --- Logging error --- Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 179, in _socket_error_handling yield File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/threading.py", line 178, in create_connection return utils.create_tcp_connection(socket, *args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/handlers/utils.py", line 265, in create_tcp_connection sock = module.create_connection(address, timeout_at) File "/usr/lib/python3.8/socket.py", line 808, in create_connection raise err File "/usr/lib/python3.8/socket.py", line 796, in create_connection sock.connect(sa) socket.timeout: timed out During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 579, in _connect_attempt read_timeout, connect_timeout = self._connect(host, hostip, port) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 659, in _connect self._socket = self.handler.create_connection( File "/usr/lib/python3.8/contextlib.py", line 131, in __exit__ self.gen.throw(type, value, traceback) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 182, in _socket_error_handling raise ConnectionDropped("socket connection error: %s" % (err,)) kazoo.exceptions.ConnectionDropped: socket connection error: None During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/usr/lib/python3.8/logging/__init__.py", line 1088, in emit stream.write(msg + self.terminator) ValueError: I/O operation on closed file. Call stack: File "/usr/lib/python3.8/threading.py", line 890, in _bootstrap self._bootstrap_inner() File "/usr/lib/python3.8/threading.py", line 932, in _bootstrap_inner self.run() File "/usr/lib/python3.8/threading.py", line 870, in run self._target(*self._args, **self._kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 512, in zk_loop if retry(self._connect_loop, retry) is STOP_CONNECTING: File "/usr/local/lib/python3.8/dist-packages/kazoo/retry.py", line 126, in __call__ return func(*args, **kwargs) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 552, in _connect_loop status = self._connect_attempt(host, hostip, port, retry) File "/usr/local/lib/python3.8/dist-packages/kazoo/protocol/connection.py", line 622, in _connect_attempt self.logger.warning('Connection dropped: %s', e) Message: 'Connection dropped: %s' Arguments: (ConnectionDropped('socket connection error: None'),) </details> Which eventually leads to incorrectly parsed report. This patch is an addition to the `logging.raiseExceptions=False` - #44618 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-12-27 12:53:14 +00:00
acls, stat = other_auth_connection.get_acls("/test_set_get_acl")
assert stat.aversion == 1
assert len(acls) == 2
for acl in acls:
assert acl.acl_list == ["READ", "CREATE", "DELETE", "ADMIN"]
assert acl.id.scheme == "digest"
assert acl.perms == 29
assert acl.id.id in (
"username1:eGncMdBgOfGS/TCojt51xWsWv/Y=",
"username3:CvWITOxxTwk+u6S5PoGlQ4hNoWI=",
)
with pytest.raises(KazooException):
other_auth_connection.set_acls(
"/test_set_get_acl", acls=[make_acl("auth", "", all=True)], version=0
)
finally:
zk_stop_and_close(auth_connection)
zk_stop_and_close(other_auth_connection)