2021-05-24 12:18:04 +00:00
|
|
|
import pytest
|
2022-12-28 09:40:01 +00:00
|
|
|
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
|
2022-03-22 16:39:58 +00:00
|
|
|
from kazoo.exceptions import (
|
|
|
|
AuthFailedError,
|
|
|
|
InvalidACLError,
|
|
|
|
NoAuthError,
|
|
|
|
KazooException,
|
|
|
|
)
|
2021-05-24 12:18:04 +00:00
|
|
|
|
|
|
|
cluster = ClickHouseCluster(__file__)
|
2022-03-22 16:39:58 +00:00
|
|
|
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"
|
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
|
2021-05-24 12:18:04 +00:00
|
|
|
@pytest.fixture(scope="module")
|
|
|
|
def started_cluster():
|
|
|
|
try:
|
|
|
|
cluster.start()
|
|
|
|
|
|
|
|
yield cluster
|
|
|
|
|
|
|
|
finally:
|
|
|
|
cluster.shutdown()
|
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
|
2021-05-24 12:18:04 +00:00
|
|
|
def get_fake_zk(timeout=30.0):
|
2022-03-22 16:39:58 +00:00
|
|
|
_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
|
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
|
2021-05-24 12:18:04 +00:00
|
|
|
def get_genuine_zk():
|
|
|
|
print("Zoo1", cluster.get_instance_ip("zoo1"))
|
2022-03-22 16:39:58 +00:00
|
|
|
return cluster.get_kazoo_client("zoo1")
|
2022-01-07 09:45:30 +00:00
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
|
2022-12-29 08:22:50 +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()
|
|
|
|
|
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
@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:
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_stop_and_close(auth_connection)
|
2022-01-07 09:45:30 +00:00
|
|
|
|
|
|
|
|
2022-03-22 16:39:58 +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):
|
2022-12-29 08:22:50 +00:00
|
|
|
try:
|
|
|
|
auth_connection = None
|
|
|
|
no_auth_connection = None
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
auth_connection = get_zk()
|
|
|
|
auth_connection.add_auth("digest", "user1:password1")
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +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
|
|
|
|
2022-12-29 08:22:50 +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
|
|
|
|
2022-12-29 08:22:50 +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
|
|
|
|
2022-12-29 08:22:50 +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
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
with pytest.raises(NoAuthError):
|
|
|
|
no_auth_connection.get("/test_all_acl")
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
with pytest.raises(NoAuthError):
|
|
|
|
no_auth_connection.get("/test_all_digest_acl")
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +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
|
|
|
|
2022-12-29 08:22:50 +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
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
# wrong auth
|
|
|
|
no_auth_connection.add_auth("digest", "user2:password2")
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
with pytest.raises(NoAuthError):
|
|
|
|
no_auth_connection.set("/test_all_acl", b"hello")
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
with pytest.raises(NoAuthError):
|
|
|
|
no_auth_connection.set("/test_all_acl", b"hello")
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
with pytest.raises(NoAuthError):
|
|
|
|
no_auth_connection.get("/test_all_acl")
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
with pytest.raises(NoAuthError):
|
|
|
|
no_auth_connection.get("/test_all_digest_acl")
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +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
|
|
|
|
2022-12-29 08:22:50 +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()
|
2022-12-29 08:22:50 +00:00
|
|
|
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()
|
2022-12-29 08:22:50 +00:00
|
|
|
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
|
|
|
|
|
|
|
|
2022-03-22 16:39:58 +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:
|
2022-12-29 08:22:50 +00:00
|
|
|
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
|
|
|
|
2022-03-22 16:39:58 +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):
|
2022-12-29 08:22:50 +00:00
|
|
|
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")
|
2022-03-22 16:39:58 +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",
|
|
|
|
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:
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_stop_and_close(auth_connection)
|
2021-05-24 12:18:04 +00:00
|
|
|
|
|
|
|
|
2022-12-29 08:22:50 +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):
|
2022-03-22 16:39:58 +00:00
|
|
|
auth_connection.add_auth("world", "anyone")
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_auth_failure_workaround()
|
|
|
|
zk_stop_and_close(auth_connection)
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +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")
|
2022-03-22 16:39:58 +00:00
|
|
|
auth_connection.add_auth("adssagf", "user1:password1")
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_auth_failure_workaround()
|
|
|
|
zk_stop_and_close(auth_connection)
|
|
|
|
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +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")
|
2022-03-22 16:39:58 +00:00
|
|
|
auth_connection.add_auth("digest", "")
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_auth_failure_workaround()
|
|
|
|
zk_stop_and_close(auth_connection)
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +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")
|
2022-03-22 16:39:58 +00:00
|
|
|
auth_connection.add_auth("", "user1:password1")
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_auth_failure_workaround()
|
|
|
|
zk_stop_and_close(auth_connection)
|
|
|
|
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +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")
|
2022-03-22 16:39:58 +00:00
|
|
|
auth_connection.add_auth("digest", "user1")
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_auth_failure_workaround()
|
|
|
|
zk_stop_and_close(auth_connection)
|
|
|
|
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +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")
|
2022-03-22 16:39:58 +00:00
|
|
|
auth_connection.add_auth("digest", "user1:password:otherpassword")
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_auth_failure_workaround()
|
|
|
|
zk_stop_and_close(auth_connection)
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +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")
|
2022-03-22 16:39:58 +00:00
|
|
|
auth_connection.add_auth("auth", "user1:password")
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_auth_failure_workaround()
|
|
|
|
zk_stop_and_close(auth_connection)
|
|
|
|
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +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")
|
2022-03-22 16:39:58 +00:00
|
|
|
auth_connection.add_auth("world", "somebody")
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_auth_failure_workaround()
|
|
|
|
zk_stop_and_close(auth_connection)
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +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")
|
2022-03-22 16:39:58 +00:00
|
|
|
auth_connection.create(
|
|
|
|
"/test_bad_acl",
|
|
|
|
b"data",
|
|
|
|
acl=[
|
|
|
|
make_acl(
|
|
|
|
"dasd",
|
|
|
|
"",
|
|
|
|
read=True,
|
|
|
|
write=False,
|
|
|
|
create=True,
|
|
|
|
delete=True,
|
|
|
|
admin=True,
|
|
|
|
)
|
|
|
|
],
|
|
|
|
)
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_auth_failure_workaround()
|
|
|
|
zk_stop_and_close(auth_connection)
|
|
|
|
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +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")
|
2022-03-22 16:39:58 +00:00
|
|
|
auth_connection.create(
|
|
|
|
"/test_bad_acl",
|
|
|
|
b"data",
|
|
|
|
acl=[
|
|
|
|
make_acl(
|
|
|
|
"digest",
|
|
|
|
"",
|
|
|
|
read=True,
|
|
|
|
write=False,
|
|
|
|
create=True,
|
|
|
|
delete=True,
|
|
|
|
admin=True,
|
|
|
|
)
|
|
|
|
],
|
|
|
|
)
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_auth_failure_workaround()
|
|
|
|
zk_stop_and_close(auth_connection)
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +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")
|
2022-03-22 16:39:58 +00:00
|
|
|
auth_connection.create(
|
|
|
|
"/test_bad_acl",
|
|
|
|
b"data",
|
|
|
|
acl=[
|
|
|
|
make_acl(
|
|
|
|
"", "", read=True, write=False, create=True, delete=True, admin=True
|
|
|
|
)
|
|
|
|
],
|
|
|
|
)
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_auth_failure_workaround()
|
|
|
|
zk_stop_and_close(auth_connection)
|
|
|
|
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +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")
|
2022-03-22 16:39:58 +00:00
|
|
|
auth_connection.create(
|
|
|
|
"/test_bad_acl",
|
|
|
|
b"data",
|
|
|
|
acl=[
|
|
|
|
make_acl(
|
|
|
|
"digest",
|
|
|
|
"dsdasda",
|
|
|
|
read=True,
|
|
|
|
write=False,
|
|
|
|
create=True,
|
|
|
|
delete=True,
|
|
|
|
admin=True,
|
|
|
|
)
|
|
|
|
],
|
|
|
|
)
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_auth_failure_workaround()
|
|
|
|
zk_stop_and_close(auth_connection)
|
2021-05-24 12:18:04 +00:00
|
|
|
|
2022-12-29 08:22:50 +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")
|
2022-03-22 16:39:58 +00:00
|
|
|
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,
|
|
|
|
)
|
|
|
|
],
|
|
|
|
)
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_auth_failure_workaround()
|
|
|
|
zk_stop_and_close(auth_connection)
|
2022-03-22 16:39:58 +00:00
|
|
|
|
2021-05-28 11:52:19 +00:00
|
|
|
|
|
|
|
def test_auth_snapshot(started_cluster):
|
2022-12-29 08:22:50 +00:00
|
|
|
connection = None
|
|
|
|
connection1 = None
|
|
|
|
connection2 = None
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
try:
|
|
|
|
connection = get_fake_zk()
|
|
|
|
connection.add_auth("digest", "user1:password1")
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
connection.create(
|
|
|
|
"/test_snapshot_acl", b"data", acl=[make_acl("auth", "", all=True)]
|
|
|
|
)
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
connection1 = get_fake_zk()
|
|
|
|
connection1.add_auth("digest", "user2:password2")
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
connection1.create(
|
|
|
|
"/test_snapshot_acl1", b"data", acl=[make_acl("auth", "", all=True)]
|
|
|
|
)
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
connection2 = get_fake_zk()
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
connection2.create("/test_snapshot_acl2", b"data")
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
for i in range(100):
|
|
|
|
connection.create(
|
|
|
|
f"/test_snapshot_acl/path{i}",
|
|
|
|
b"data",
|
|
|
|
acl=[make_acl("auth", "", all=True)],
|
|
|
|
)
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
node.restart_clickhouse()
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_stop_and_close(connection)
|
|
|
|
connection = get_fake_zk()
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
with pytest.raises(NoAuthError):
|
|
|
|
connection.get("/test_snapshot_acl")
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
connection.add_auth("digest", "user1:password1")
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
assert connection.get("/test_snapshot_acl")[0] == b"data"
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
with pytest.raises(NoAuthError):
|
|
|
|
connection.get("/test_snapshot_acl1")
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
assert connection.get("/test_snapshot_acl2")[0] == b"data"
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
for i in range(100):
|
|
|
|
assert connection.get(f"/test_snapshot_acl/path{i}")[0] == b"data"
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_stop_and_close(connection1)
|
|
|
|
connection1 = get_fake_zk()
|
|
|
|
connection1.add_auth("digest", "user2:password2")
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
assert connection1.get("/test_snapshot_acl1")[0] == b"data"
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
with pytest.raises(NoAuthError):
|
|
|
|
connection1.get("/test_snapshot_acl")
|
2021-05-28 11:52:19 +00:00
|
|
|
|
2022-12-29 08:22:50 +00:00
|
|
|
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
|
|
|
|
|
|
|
|
2022-03-22 16:39:58 +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)]
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
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=",
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
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")
|
2022-03-22 16:39:58 +00:00
|
|
|
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,
|
|
|
|
)
|
|
|
|
],
|
2022-03-22 16:39:58 +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 = 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:
|
2022-12-29 08:22:50 +00:00
|
|
|
zk_stop_and_close(auth_connection)
|
|
|
|
zk_stop_and_close(other_auth_connection)
|