From 28c24713a6fbedf65631cb5514858722f37e11a5 Mon Sep 17 00:00:00 2001 From: David Ansari Date: Thu, 5 Jun 2025 16:09:14 +0000 Subject: [PATCH] Run Quorum Queue property test on different OTP versions ## What? PR #13971 added a property test that applies the same quorum queue Raft command on different quorum queue members on different Erlang nodes ensuring that the state machine ends up in exaclty the same state. The different Erlang nodes run the **same** Erlang/OTP version however. This commit adds another property test where the different Erlang nodes run **different** Erlang/OTP versions. ## Why? This test allows spotting any non-determinism that could occur when running quorum queue members in a mixed version cluster, where mixed version means in our context different Erlang/OTP versions. ## How? CI runs currently tests with Erlang 27. This commit starts an Erlang 26 node in docker, specifically for the `rabbit_fifo_prop_SUITE`. Test case `two_nodes_different_otp_version` running Erlang 27 then transfers a few Erlang modules (e.g. module `rabbit_fifo`) to the Erlang 26 node. The test case then runs the Ra commands on its own node in Erlang 27 and on the Erlang 26 node in Docker. By default, this test case is skipped locally. However, to run this test case locally, simply start an Erlang node as follows: ``` erl -sname rabbit_fifo_prop@localhost ``` (cherry picked from commit eccf9fee1e54c21251fbbbfc33205f1cbe8d5991) --- .github/workflows/test-make-target.yaml | 12 +++ .github/workflows/test-make-tests.yaml | 1 + deps/rabbit/Makefile | 4 +- deps/rabbit/test/rabbit_fifo_prop_SUITE.erl | 88 ++++++++++++++----- .../src/rabbit_ct_helpers.erl | 10 ++- 5 files changed, 92 insertions(+), 23 deletions(-) diff --git a/.github/workflows/test-make-target.yaml b/.github/workflows/test-make-target.yaml index 15843138c946..9724962ae366 100644 --- a/.github/workflows/test-make-target.yaml +++ b/.github/workflows/test-make-target.yaml @@ -87,6 +87,18 @@ jobs: sudo systemctl is-active --quiet apparmor.service && sudo systemctl stop apparmor.service sudo systemctl disable apparmor.service + - name: RUN LOW VERSION ERLANG NODE IN DOCKER + if: inputs.make_target == 'ct-rabbit_fifo_prop' + run: | + # This version must be at least 1 major version lower than inputs.erlang_version + LOW_ERLANG_VERSION="26.2" + + # Create ~/.erlang.cookie by starting a distributed node + erl -sname temp_node -eval 'halt().' -noshell + + docker run -d --network host --name erlang_low_version erlang:${LOW_ERLANG_VERSION} \ + erl -sname rabbit_fifo_prop@localhost -setcookie $(cat ~/.erlang.cookie) -noinput + - name: RUN TESTS if: inputs.plugin != 'rabbitmq_cli' run: | diff --git a/.github/workflows/test-make-tests.yaml b/.github/workflows/test-make-tests.yaml index a4ffd93c453c..2536aae453d8 100644 --- a/.github/workflows/test-make-tests.yaml +++ b/.github/workflows/test-make-tests.yaml @@ -32,6 +32,7 @@ jobs: - ct-metadata_store_clustering - ct-quorum_queue - ct-rabbit_stream_queue + - ct-rabbit_fifo_prop uses: ./.github/workflows/test-make-target.yaml with: erlang_version: ${{ inputs.erlang_version }} diff --git a/deps/rabbit/Makefile b/deps/rabbit/Makefile index 8930f7088fdd..c5ff2c344a9d 100644 --- a/deps/rabbit/Makefile +++ b/deps/rabbit/Makefile @@ -268,7 +268,7 @@ PARALLEL_CT_SET_2_B = clustering_recovery crashing_queues deprecated_features di PARALLEL_CT_SET_2_C = disk_monitor dynamic_qq unit_disk_monitor unit_file_handle_cache unit_log_management unit_operator_policy PARALLEL_CT_SET_2_D = queue_length_limits queue_parallel quorum_queue_member_reconciliation rabbit_fifo rabbit_fifo_dlx rabbit_stream_coordinator -PARALLEL_CT_SET_3_A = definition_import per_user_connection_channel_limit_partitions per_vhost_connection_limit_partitions policy priority_queue_recovery rabbit_fifo_prop rabbit_fifo_v0 rabbit_stream_sac_coordinator unit_credit_flow unit_queue_consumers unit_queue_location unit_quorum_queue +PARALLEL_CT_SET_3_A = definition_import per_user_connection_channel_limit_partitions per_vhost_connection_limit_partitions policy priority_queue_recovery rabbit_fifo_v0 rabbit_stream_sac_coordinator unit_credit_flow unit_queue_consumers unit_queue_location unit_quorum_queue PARALLEL_CT_SET_3_B = cluster_upgrade list_consumers_sanity_check list_queues_online_and_offline logging lqueue maintenance_mode rabbit_fifo_q PARALLEL_CT_SET_3_C = cli_forget_cluster_node feature_flags_v2 mc_unit message_containers_deaths_v2 message_size_limit metadata_store_migration PARALLEL_CT_SET_3_D = metadata_store_phase1 metrics mirrored_supervisor peer_discovery_classic_config proxy_protocol runtime_parameters unit_stats_and_metrics unit_supervisor2 unit_vm_memory_monitor @@ -283,7 +283,7 @@ PARALLEL_CT_SET_2 = $(sort $(PARALLEL_CT_SET_2_A) $(PARALLEL_CT_SET_2_B) $(PARAL PARALLEL_CT_SET_3 = $(sort $(PARALLEL_CT_SET_3_A) $(PARALLEL_CT_SET_3_B) $(PARALLEL_CT_SET_3_C) $(PARALLEL_CT_SET_3_D)) PARALLEL_CT_SET_4 = $(sort $(PARALLEL_CT_SET_4_A) $(PARALLEL_CT_SET_4_B) $(PARALLEL_CT_SET_4_C) $(PARALLEL_CT_SET_4_D)) -SEQUENTIAL_CT_SUITES = amqp_client clustering_management dead_lettering feature_flags metadata_store_clustering quorum_queue rabbit_stream_queue +SEQUENTIAL_CT_SUITES = amqp_client clustering_management dead_lettering feature_flags metadata_store_clustering quorum_queue rabbit_stream_queue rabbit_fifo_prop PARALLEL_CT_SUITES = $(PARALLEL_CT_SET_1) $(PARALLEL_CT_SET_2) $(PARALLEL_CT_SET_3) $(PARALLEL_CT_SET_4) ifeq ($(filter-out $(SEQUENTIAL_CT_SUITES) $(PARALLEL_CT_SUITES),$(CT_SUITES)),) diff --git a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl index 37a2c8048c6b..fcc35397f2b2 100644 --- a/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_prop_SUITE.erl @@ -85,7 +85,8 @@ all_tests() -> dlx_08, dlx_09, single_active_ordering_02, - different_nodes + two_nodes_same_otp_version, + two_nodes_different_otp_version ]. groups() -> @@ -1093,14 +1094,65 @@ single_active_ordering_03(_Config) -> false end. -%% Test that running the state machine commands on different Erlang nodes -%% end up in exactly the same state. -different_nodes(Config) -> - Config1 = rabbit_ct_helpers:run_setup_steps( - Config, - rabbit_ct_broker_helpers:setup_steps()), +%% Run the log on two Erlang nodes with the same OTP version. +two_nodes_same_otp_version(Config0) -> + Config = rabbit_ct_helpers:run_setup_steps(Config0, + rabbit_ct_broker_helpers:setup_steps()), + Node = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + case is_same_otp_version(Config) of + true -> + ok = rabbit_ct_broker_helpers:add_code_path_to_node(Node, ?MODULE), + two_nodes(Node); + false -> + ct:fail("expected CT node and RabbitMQ node to have the same OTP version") + end, + rabbit_ct_helpers:run_teardown_steps(Config, + rabbit_ct_broker_helpers:teardown_steps()). + +%% Run the log on two Erlang nodes with different OTP versions. +two_nodes_different_otp_version(_Config) -> + Node = 'rabbit_fifo_prop@localhost', + case net_adm:ping(Node) of + pong -> + case is_same_otp_version(Node) of + true -> + ct:fail("expected CT node and 'rabbit_fifo_prop@localhost' " + "to have different OTP versions"); + false -> + Prefixes = ["rabbit_fifo", "rabbit_misc", "mc", + "lqueue", "priority_queue", "ra_"], + [begin + Mod = list_to_atom(ModStr), + {Mod, Bin, _File} = code:get_object_code(Mod), + {module, Mod} = erpc:call(Node, code, load_binary, [Mod, ModStr, Bin]) + end + || {ModStr, _FileName, _Loaded} <- code:all_available(), + lists:any(fun(Prefix) -> lists:prefix(Prefix, ModStr) end, Prefixes)], + two_nodes(Node) + end; + pang -> + Reason = {node_down, Node}, + case rabbit_ct_helpers:is_ci() of + true -> + ct:fail(Reason); + false -> + {skip, Reason} + end + end. - Size = 400, +is_same_otp_version(ConfigOrNode) -> + OurOTP = erlang:system_info(otp_release), + OtherOTP = case ConfigOrNode of + Cfg when is_list(Cfg) -> + rabbit_ct_broker_helpers:rpc(Cfg, erlang, system_info, [otp_release]); + Node when is_atom(Node) -> + erpc:call(Node, erlang, system_info, [otp_release]) + end, + ct:pal("Our CT node runs OTP ~s, other node runs OTP ~s", [OurOTP, OtherOTP]), + OurOTP =:= OtherOTP. + +two_nodes(Node) -> + Size = 500, run_proper( fun () -> ?FORALL({Length, Bytes, DeliveryLimit, SingleActive}, @@ -1118,13 +1170,9 @@ different_nodes(Config) -> DeliveryLimit), ?FORALL(O, ?LET(Ops, log_gen_different_nodes(Size), expand(Ops, Conf)), collect({log_size, length(O)}, - different_nodes_prop(Config1, Conf, O))) + different_nodes_prop(Node, Conf, O))) end) - end, [], Size), - - rabbit_ct_helpers:run_teardown_steps( - Config1, - rabbit_ct_broker_helpers:teardown_steps()). + end, [], Size). max_length(_Config) -> %% tests that max length is never transgressed @@ -1485,18 +1533,18 @@ single_active_prop(Conf0, Commands, ValidateOrder) -> false end. -different_nodes_prop(Config, Conf0, Commands) -> +different_nodes_prop(Node, Conf0, Commands) -> Conf = Conf0#{release_cursor_interval => 100}, Indexes = lists:seq(1, length(Commands)), Entries = lists:zip(Indexes, Commands), InitState = test_init(Conf), Fun = fun(_) -> true end, - Vsn = 6, + MachineVersion = 6, - {State0, _Effs0} = run_log(InitState, Entries, Fun, Vsn), - {State1, _Effs1} = rabbit_ct_broker_helpers:rpc(Config, ?MODULE, run_log, - [InitState, Entries, Fun, Vsn]), - State0 =:= State1. + {State1, _Effs1} = run_log(InitState, Entries, Fun, MachineVersion), + {State2, _Effs2} = erpc:call(Node, ?MODULE, run_log, + [InitState, Entries, Fun, MachineVersion]), + State1 =:= State2. messages_total_prop(Conf0, Commands) -> Conf = Conf0#{release_cursor_interval => 100}, diff --git a/deps/rabbitmq_ct_helpers/src/rabbit_ct_helpers.erl b/deps/rabbitmq_ct_helpers/src/rabbit_ct_helpers.erl index df65f808e66a..88d1f3ce8540 100644 --- a/deps/rabbitmq_ct_helpers/src/rabbit_ct_helpers.erl +++ b/deps/rabbitmq_ct_helpers/src/rabbit_ct_helpers.erl @@ -56,7 +56,9 @@ await_condition_with_retries/2, eventually/1, eventually/3, - consistently/1, consistently/3 + consistently/1, consistently/3, + + is_ci/0 ]). -define(SSL_CERT_PASSWORD, "test"). @@ -1175,6 +1177,12 @@ consistently({Line, Assertion} = TestObj, PollInterval, PollCount) timer:sleep(PollInterval), consistently(TestObj, PollInterval, PollCount - 1). +is_ci() -> + case os:getenv("CI") of + "true" -> true; + _ -> false + end. + %% ------------------------------------------------------------------- %% Cover-related functions. %% -------------------------------------------------------------------