Skip to content

Commit f020027

Browse files
authored
Merge pull request ClickHouse#80682 from azat/tests/rabbitmq-pause-fix
tests: attempt to fix RabbitMQ tests (pause/unpause breaks RabbitMQ)
2 parents 984db6b + 18487c1 commit f020027

File tree

3 files changed

+25
-29
lines changed

3 files changed

+25
-29
lines changed

tests/integration/helpers/cluster.py

Lines changed: 14 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2480,7 +2480,20 @@ def start_rabbitmq_app(self, timeout=120):
24802480
run_rabbitmqctl(
24812481
self.rabbitmq_docker_id, self.rabbitmq_cookie, "start_app", timeout
24822482
)
2483-
self.wait_rabbitmq_to_start()
2483+
self.wait_rabbitmq_to_start(timeout)
2484+
2485+
@contextmanager
2486+
def pause_rabbitmq(self, monitor=None, timeout=120):
2487+
if monitor is not None:
2488+
monitor.stop()
2489+
self.stop_rabbitmq_app(timeout)
2490+
2491+
try:
2492+
yield
2493+
finally:
2494+
self.start_rabbitmq_app(timeout)
2495+
if monitor is not None:
2496+
monitor.start(self)
24842497

24852498
def reset_rabbitmq(self, timeout=120):
24862499
self.stop_rabbitmq_app()

tests/integration/test_storage_rabbitmq/test.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2210,7 +2210,7 @@ def test_rabbitmq_no_connection_at_startup_2(rabbitmq_cluster):
22102210
)
22112211
instance.query("DETACH TABLE test.cs")
22122212

2213-
with rabbitmq_cluster.pause_container("rabbitmq1"):
2213+
with rabbitmq_cluster.pause_rabbitmq():
22142214
instance.query("ATTACH TABLE test.cs")
22152215

22162216
messages_num = 1000

tests/integration/test_storage_rabbitmq/test_failed_connection.py

Lines changed: 10 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -126,17 +126,6 @@ def stop(self):
126126
self.connection = None
127127

128128

129-
def suspend_rabbitmq(rabbitmq_cluster, rabbitmq_monitor):
130-
rabbitmq_monitor.stop()
131-
rabbitmq_cluster.stop_rabbitmq_app()
132-
133-
134-
def resume_rabbitmq(rabbitmq_cluster, rabbitmq_monitor):
135-
rabbitmq_cluster.start_rabbitmq_app()
136-
rabbitmq_cluster.wait_rabbitmq_to_start()
137-
rabbitmq_monitor.start(rabbitmq_cluster)
138-
139-
140129
# Fixtures
141130

142131
@pytest.fixture(scope="module")
@@ -234,14 +223,11 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster, r
234223
else:
235224
pytest.fail(f"Time limit of {DEFAULT_TIMEOUT_SEC} seconds reached. The count is still 0.")
236225

237-
suspend_rabbitmq(rabbitmq_cluster, rabbitmq_monitor)
238-
239-
number = int(instance.query("SELECT count() FROM test.view"))
240-
logging.debug(f"{number}/{messages_num} after suspending RabbitMQ")
241-
if number == messages_num:
242-
pytest.fail("All RabbitMQ messages have been consumed before resuming the RabbitMQ server")
243-
244-
resume_rabbitmq(rabbitmq_cluster, rabbitmq_monitor)
226+
with rabbitmq_cluster.pause_rabbitmq(rabbitmq_monitor):
227+
number = int(instance.query("SELECT count() FROM test.view"))
228+
logging.debug(f"{number}/{messages_num} after suspending RabbitMQ")
229+
if number == messages_num:
230+
pytest.fail("All RabbitMQ messages have been consumed before resuming the RabbitMQ server")
245231

246232
deadline = time.monotonic() + CLICKHOUSE_VIEW_TIMEOUT_SEC
247233
while time.monotonic() < deadline:
@@ -325,14 +311,11 @@ def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster, r
325311
else:
326312
pytest.fail(f"Time limit of {DEFAULT_TIMEOUT_SEC} seconds reached. The count is still 0.")
327313

328-
suspend_rabbitmq(rabbitmq_cluster, rabbitmq_monitor)
329-
330-
number = int(instance.query("SELECT count() FROM test.view"))
331-
logging.debug(f"{number}/{messages_num} after suspending RabbitMQ")
332-
if number == messages_num:
333-
pytest.fail("All RabbitMQ messages have been consumed before resuming the RabbitMQ server")
334-
335-
resume_rabbitmq(rabbitmq_cluster, rabbitmq_monitor)
314+
with rabbitmq_cluster.pause_rabbitmq(rabbitmq_monitor):
315+
number = int(instance.query("SELECT count() FROM test.view"))
316+
logging.debug(f"{number}/{messages_num} after suspending RabbitMQ")
317+
if number == messages_num:
318+
pytest.fail("All RabbitMQ messages have been consumed before resuming the RabbitMQ server")
336319

337320
# while int(instance.query('SELECT count() FROM test.view')) == 0:
338321
# time.sleep(0.1)

0 commit comments

Comments
 (0)