2021-10-14 10:21:41 +00:00
|
|
|
#!/usr/bin/env python3
|
|
|
|
import pytest
|
|
|
|
from helpers.cluster import ClickHouseCluster
|
2022-09-06 10:58:14 +00:00
|
|
|
import helpers.keeper_utils as keeper_utils
|
2021-10-14 10:21:41 +00:00
|
|
|
import random
|
|
|
|
import string
|
|
|
|
import os
|
|
|
|
import time
|
|
|
|
from multiprocessing.dummy import Pool
|
|
|
|
from helpers.test_tools import assert_eq_with_retry
|
|
|
|
from kazoo.client import KazooClient, KazooState
|
|
|
|
|
|
|
|
cluster = ClickHouseCluster(__file__)
|
2022-03-22 16:39:58 +00:00
|
|
|
node1 = cluster.add_instance(
|
|
|
|
"node1",
|
|
|
|
main_configs=["configs/enable_keeper1.xml", "configs/keeper_conf.xml"],
|
|
|
|
stay_alive=True,
|
|
|
|
)
|
|
|
|
node2 = cluster.add_instance(
|
|
|
|
"node2",
|
|
|
|
main_configs=["configs/enable_keeper2.xml", "configs/keeper_conf.xml"],
|
|
|
|
stay_alive=True,
|
|
|
|
)
|
|
|
|
node3 = cluster.add_instance(
|
|
|
|
"node3",
|
|
|
|
main_configs=["configs/enable_keeper3.xml", "configs/keeper_conf.xml"],
|
|
|
|
stay_alive=True,
|
|
|
|
)
|
2021-10-14 10:21:41 +00:00
|
|
|
|
|
|
|
|
|
|
|
def get_fake_zk(nodename, timeout=30.0):
|
2022-03-22 16:39:58 +00:00
|
|
|
_fake_zk_instance = KazooClient(
|
|
|
|
hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout
|
|
|
|
)
|
2021-10-14 10:21:41 +00:00
|
|
|
_fake_zk_instance.start()
|
|
|
|
return _fake_zk_instance
|
|
|
|
|
|
|
|
|
|
|
|
@pytest.fixture(scope="module")
|
|
|
|
def started_cluster():
|
|
|
|
try:
|
|
|
|
cluster.start()
|
|
|
|
|
|
|
|
yield cluster
|
|
|
|
|
|
|
|
finally:
|
|
|
|
cluster.shutdown()
|
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
|
2021-10-14 10:21:41 +00:00
|
|
|
def start(node):
|
2022-03-22 16:39:58 +00:00
|
|
|
node.start_clickhouse()
|
2022-09-06 10:58:14 +00:00
|
|
|
keeper_utils.wait_until_connected(cluster, node)
|
2021-10-14 10:21:41 +00:00
|
|
|
|
|
|
|
|
2021-12-02 10:41:04 +00:00
|
|
|
def delete_with_retry(node_name, path):
|
|
|
|
for _ in range(30):
|
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
|
|
|
zk = None
|
2021-12-02 10:41:04 +00:00
|
|
|
try:
|
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
|
|
|
zk = get_fake_zk(node_name)
|
|
|
|
zk.delete(path)
|
2021-12-02 10:41:04 +00:00
|
|
|
return
|
|
|
|
except:
|
|
|
|
time.sleep(0.5)
|
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
|
|
|
finally:
|
|
|
|
zk.stop()
|
|
|
|
zk.close()
|
2021-12-02 10:41:04 +00:00
|
|
|
raise Exception(f"Cannot delete {path} from node {node_name}")
|
|
|
|
|
|
|
|
|
2021-10-14 10:21:41 +00:00
|
|
|
def test_start_offline(started_cluster):
|
|
|
|
p = Pool(3)
|
|
|
|
try:
|
|
|
|
node1_zk = get_fake_zk("node1")
|
|
|
|
node1_zk.create("/test_alive", b"aaaa")
|
|
|
|
|
|
|
|
node1.stop_clickhouse()
|
|
|
|
node2.stop_clickhouse()
|
|
|
|
node3.stop_clickhouse()
|
|
|
|
|
|
|
|
time.sleep(3)
|
|
|
|
p.map(start, [node2, node3])
|
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
assert node2.contains_in_log(
|
2022-09-21 15:12:16 +00:00
|
|
|
"Cannot connect to ZooKeeper (or Keeper) before internal Keeper start"
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
|
|
|
assert node3.contains_in_log(
|
2022-09-21 15:12:16 +00:00
|
|
|
"Cannot connect to ZooKeeper (or Keeper) before internal Keeper start"
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
2021-10-18 09:13:24 +00:00
|
|
|
|
2021-10-14 10:21:41 +00:00
|
|
|
node2_zk = get_fake_zk("node2")
|
2021-11-11 13:28:50 +00:00
|
|
|
node2_zk.create("/c", b"data")
|
|
|
|
|
2021-10-14 10:21:41 +00:00
|
|
|
finally:
|
|
|
|
p.map(start, [node1, node2, node3])
|
2021-12-02 10:41:04 +00:00
|
|
|
delete_with_retry("node1", "/test_alive")
|
2021-10-14 10:21:41 +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
|
|
|
node1_zk.stop()
|
|
|
|
node1_zk.close()
|
|
|
|
|
2021-10-14 10:21:41 +00:00
|
|
|
|
|
|
|
def test_start_non_existing(started_cluster):
|
|
|
|
p = Pool(3)
|
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
|
|
|
node2_zk = None
|
|
|
|
|
2021-10-14 10:21:41 +00:00
|
|
|
try:
|
|
|
|
node1.stop_clickhouse()
|
|
|
|
node2.stop_clickhouse()
|
|
|
|
node3.stop_clickhouse()
|
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
node1.replace_in_config(
|
|
|
|
"/etc/clickhouse-server/config.d/enable_keeper1.xml",
|
|
|
|
"node3",
|
|
|
|
"non_existing_node",
|
|
|
|
)
|
|
|
|
node2.replace_in_config(
|
|
|
|
"/etc/clickhouse-server/config.d/enable_keeper2.xml",
|
|
|
|
"node3",
|
|
|
|
"non_existing_node",
|
|
|
|
)
|
2021-10-14 10:21:41 +00:00
|
|
|
|
|
|
|
time.sleep(3)
|
|
|
|
p.map(start, [node2, node1])
|
|
|
|
|
2022-03-22 16:39:58 +00:00
|
|
|
assert node1.contains_in_log(
|
2022-09-21 15:12:16 +00:00
|
|
|
"Cannot connect to ZooKeeper (or Keeper) before internal Keeper start"
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
|
|
|
assert node2.contains_in_log(
|
2022-09-21 15:12:16 +00:00
|
|
|
"Cannot connect to ZooKeeper (or Keeper) before internal Keeper start"
|
2022-03-22 16:39:58 +00:00
|
|
|
)
|
2021-10-18 09:13:24 +00:00
|
|
|
|
2021-10-14 10:21:41 +00:00
|
|
|
node2_zk = get_fake_zk("node2")
|
|
|
|
node2_zk.create("/test_non_exising", b"data")
|
|
|
|
finally:
|
2022-03-22 16:39:58 +00:00
|
|
|
node1.replace_in_config(
|
|
|
|
"/etc/clickhouse-server/config.d/enable_keeper1.xml",
|
|
|
|
"non_existing_node",
|
|
|
|
"node3",
|
|
|
|
)
|
|
|
|
node2.replace_in_config(
|
|
|
|
"/etc/clickhouse-server/config.d/enable_keeper2.xml",
|
|
|
|
"non_existing_node",
|
|
|
|
"node3",
|
|
|
|
)
|
2021-10-14 10:21:41 +00:00
|
|
|
p.map(start, [node1, node2, node3])
|
2021-12-02 10:41:04 +00:00
|
|
|
delete_with_retry("node2", "/test_non_exising")
|
2021-11-11 13:28:50 +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
|
|
|
if node2_zk:
|
|
|
|
node2_zk.stop()
|
|
|
|
node2_zk.close()
|
|
|
|
|
2021-10-14 10:21:41 +00:00
|
|
|
|
|
|
|
def test_restart_third_node(started_cluster):
|
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:
|
|
|
|
node1_zk = get_fake_zk("node1")
|
|
|
|
node1_zk.create("/test_restart", b"aaaa")
|
2021-10-14 10:21:41 +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
|
|
|
node3.restart_clickhouse()
|
|
|
|
keeper_utils.wait_until_connected(cluster, node3)
|
2021-10-14 10:21:41 +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 node3.contains_in_log(
|
|
|
|
"Connected to ZooKeeper (or Keeper) before internal Keeper start"
|
|
|
|
)
|
|
|
|
node1_zk.delete("/test_restart")
|
|
|
|
finally:
|
|
|
|
node1_zk.stop()
|
|
|
|
node1_zk.close()
|