ClickHouse/tests/integration/test_keeper_nodes_move/test.py

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

111 lines
3.1 KiB
Python
Raw Normal View History

2021-10-19 12:00:26 +00:00
#!/usr/bin/env python3
import pytest
from helpers.cluster import ClickHouseCluster
import os
import time
from multiprocessing.dummy import Pool
from helpers.test_tools import assert_eq_with_retry
2022-09-06 10:58:14 +00:00
import helpers.keeper_utils as keeper_utils
2021-10-19 12:00:26 +00:00
from kazoo.client import KazooClient, KazooState
cluster = ClickHouseCluster(__file__)
CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs")
node1 = cluster.add_instance(
"node1", main_configs=["configs/enable_keeper1.xml"], stay_alive=True
)
node2 = cluster.add_instance(
"node2", main_configs=["configs/enable_keeper2.xml"], stay_alive=True
)
node3 = cluster.add_instance(
"node3", main_configs=["configs/enable_keeper3.xml"], stay_alive=True
)
2021-10-19 12:00:26 +00:00
node4 = cluster.add_instance("node4", stay_alive=True)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def start(node):
node.start_clickhouse()
2022-09-06 10:58:14 +00:00
keeper_utils.wait_until_connected(cluster, node)
2021-10-19 12:00:26 +00:00
2021-10-19 12:00:26 +00:00
def get_fake_zk(node, timeout=30.0):
_fake_zk_instance = KazooClient(
hosts=cluster.get_instance_ip(node.name) + ":9181", timeout=timeout
)
_fake_zk_instance.start()
return _fake_zk_instance
def test_node_move(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
zk_conn = None
zk_conn2 = None
zk_conn3 = None
zk_conn4 = None
2021-10-19 12:00:26 +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:
zk_conn = get_fake_zk(node1)
for i in range(100):
zk_conn.create("/test_four_" + str(i), b"somedata")
zk_conn2 = get_fake_zk(node2)
zk_conn2.sync("/test_four_0")
zk_conn3 = get_fake_zk(node3)
zk_conn3.sync("/test_four_0")
for i in range(100):
assert zk_conn2.exists("test_four_" + str(i)) is not None
assert zk_conn3.exists("test_four_" + str(i)) is not None
node4.stop_clickhouse()
node4.copy_file_to_container(
os.path.join(CONFIG_DIR, "enable_keeper_node4_4.xml"),
"/etc/clickhouse-server/config.d/enable_keeper4.xml",
)
p = Pool(3)
waiter = p.apply_async(start, (node4,))
node1.copy_file_to_container(
os.path.join(CONFIG_DIR, "enable_keeper_node4_1.xml"),
"/etc/clickhouse-server/config.d/enable_keeper1.xml",
)
node2.copy_file_to_container(
os.path.join(CONFIG_DIR, "enable_keeper_node4_2.xml"),
"/etc/clickhouse-server/config.d/enable_keeper2.xml",
)
node1.query("SYSTEM RELOAD CONFIG")
node2.query("SYSTEM RELOAD CONFIG")
waiter.wait()
zk_conn4 = get_fake_zk(node4)
zk_conn4.sync("/test_four_0")
for i in range(100):
assert zk_conn4.exists("/test_four_" + str(i)) is not None
with pytest.raises(Exception):
# Adding and removing nodes is async operation
for i in range(10):
zk_conn3 = get_fake_zk(node3)
zk_conn3.sync("/test_four_0")
time.sleep(i)
finally:
for zk in [zk_conn, zk_conn2, zk_conn3, zk_conn4]:
if zk:
zk.stop()
zk.close()