From df7b0df819f266e56aff629bab626dd473245b82 Mon Sep 17 00:00:00 2001 From: Diana Parra Corbacho Date: Thu, 5 Jan 2023 13:57:50 +0100 Subject: [PATCH] Allow to use Khepri database to store metadata instead of Mnesia MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit [Why] Mnesia is a very powerful and convenient tool for Erlang applications: it is a persistent disc-based database, it handles replication accross multiple Erlang nodes and it is available out-of-the-box from the Erlang/OTP distribution. RabbitMQ relies on Mnesia to manage all its metadata: * virtual hosts' properties * intenal users * queue, exchange and binding declarations (not queues data) * runtime parameters and policies * ... Unfortunately Mnesia makes it difficult to handle network partition and, as a consequence, the merge conflicts between Erlang nodes once the network partition is resolved. RabbitMQ provides several partition handling strategies but they are not bullet-proof. Users still hit situations where it is a pain to repair a cluster following a network partition. [How] @kjnilsson created Ra [1], a Raft consensus library that RabbitMQ already uses successfully to implement quorum queues and streams for instance. Those queues do not suffer from network partitions. We created Khepri [2], a new persistent and replicated database engine based on Ra and we want to use it in place of Mnesia in RabbitMQ to solve the problems with network partitions. This patch integrates Khepri as an experimental feature. When enabled, RabbitMQ will store all its metadata in Khepri instead of Mnesia. This change comes with behavior changes. While Khepri remains disabled, you should see no changes to the behavior of RabbitMQ. If there are changes, it is a bug. After Khepri is enabled, there are significant changes of behavior that you should be aware of. Because it is based on the Raft consensus algorithm, when there is a network partition, only the cluster members that are in the partition with at least `(Number of nodes in the cluster ÷ 2) + 1` number of nodes can "make progress". In other words, only those nodes may write to the Khepri database and read from the database and expect a consistent result. For instance in a cluster of 5 RabbitMQ nodes: * If there are two partitions, one with 3 nodes, one with 2 nodes, only the group of 3 nodes will be able to write to the database. * If there are three partitions, two with 2 nodes, one with 1 node, none of the group can write to the database. Because the Khepri database will be used for all kind of metadata, it means that RabbitMQ nodes that can't write to the database will be unable to perform some operations. A list of operations and what to expect is documented in the associated pull request and the RabbitMQ website. This requirement from Raft also affects the startup of RabbitMQ nodes in a cluster. Indeed, at least a quorum number of nodes must be started at once to allow nodes to become ready. To enable Khepri, you need to enable the `khepri_db` feature flag: rabbitmqctl enable_feature_flag khepri_db When the `khepri_db` feature flag is enabled, the migration code performs the following two tasks: 1. It synchronizes the Khepri cluster membership from the Mnesia cluster. It uses `mnesia_to_khepri:sync_cluster_membership/1` from the `khepri_mnesia_migration` application [3]. 2. It copies data from relevant Mnesia tables to Khepri, doing some conversion if necessary on the way. Again, it uses `mnesia_to_khepri:copy_tables/4` from `khepri_mnesia_migration` to do it. This can be performed on a running standalone RabbitMQ node or cluster. Data will be migrated from Mnesia to Khepri without any service interruption. Note that during the migration, the performance may decrease and the memory footprint may go up. Because this feature flag is considered experimental, it is not enabled by default even on a brand new RabbitMQ deployment. More about the implementation details below: In the past months, all accesses to Mnesia were isolated in a collection of `rabbit_db*` modules. This is where the integration of Khepri mostly takes place: we use a function called `rabbit_khepri:handle_fallback/1` which selects the database and perform the query or the transaction. Here is an example from `rabbit_db_vhost`: * Up until RabbitMQ 3.12.x: get(VHostName) when is_binary(VHostName) -> get_in_mnesia(VHostName). * Starting with RabbitMQ 3.13.0: get(VHostName) when is_binary(VHostName) -> rabbit_khepri:handle_fallback( #{mnesia => fun() -> get_in_mnesia(VHostName) end, khepri => fun() -> get_in_khepri(VHostName) end}). This `rabbit_khepri:handle_fallback/1` function relies on two things: 1. the fact that the `khepri_db` feature flag is enabled, in which case it always executes the Khepri-based variant. 4. the ability or not to read and write to Mnesia tables otherwise. Before the feature flag is enabled, or during the migration, the function will try to execute the Mnesia-based variant. If it succeeds, then it returns the result. If it fails because one or more Mnesia tables can't be used, it restarts from scratch: it means the feature flag is being enabled and depending on the outcome, either the Mnesia-based variant will succeed (the feature flag couldn't be enabled) or the feature flag will be marked as enabled and it will call the Khepri-based variant. The meat of this function really lives in the `khepri_mnesia_migration` application [3] and `rabbit_khepri:handle_fallback/1` is a wrapper on top of it that knows about the feature flag. However, some calls to the database do not depend on the existence of Mnesia tables, such as functions where we need to learn about the members of a cluster. For those, we can't rely on exceptions from Mnesia. Therefore, we just look at the state of the feature flag to determine which database to use. There are two situations though: * Sometimes, we need the feature flag state query to block because the function interested in it can't return a valid answer during the migration. Here is an example: case rabbit_khepri:is_enabled(RemoteNode) of true -> can_join_using_khepri(RemoteNode); false -> can_join_using_mnesia(RemoteNode) end * Sometimes, we need the feature flag state query to NOT block (for instance because it would cause a deadlock). Here is an example: case rabbit_khepri:get_feature_state() of enabled -> members_using_khepri(); _ -> members_using_mnesia() end Direct accesses to Mnesia still exists. They are limited to code that is specific to Mnesia such as classic queue mirroring or network partitions handling strategies. Now, to discover the Mnesia tables to migrate and how to migrate them, we use an Erlang module attribute called `rabbit_mnesia_tables_to_khepri_db` which indicates a list of Mnesia tables and an associated converter module. Here is an example in the `rabbitmq_recent_history_exchange` plugin: -rabbit_mnesia_tables_to_khepri_db( [{?RH_TABLE, rabbit_db_rh_exchange_m2k_converter}]). The converter module — `rabbit_db_rh_exchange_m2k_converter` in this example — is is fact a "sub" converter module called but `rabbit_db_m2k_converter`. See the documentation of a `mnesia_to_khepri` converter module to learn more about these modules. [1] https://github.com/rabbitmq/ra [2] https://github.com/rabbitmq/khepri [3] https://github.com/rabbitmq/khepri_mnesia_migration See #7206. Co-authored-by: Jean-Sébastien Pédron Co-authored-by: Diana Parra Corbacho Co-authored-by: Michael Davis --- .github/workflows/test-mixed-versions.yaml | 4 + .github/workflows/test-windows.yaml | 4 + .github/workflows/test.yaml | 4 + MODULE.bazel | 24 + bazel/BUILD.horus | 114 + bazel/BUILD.khepri | 185 ++ bazel/BUILD.khepri_mnesia_migration | 146 + deps/rabbit/BUILD.bazel | 47 +- deps/rabbit/Makefile | 5 +- deps/rabbit/app.bzl | 84 +- deps/rabbit/priv/schema/rabbit.schema | 4 + deps/rabbit/src/mirrored_supervisor.hrl | 1 + deps/rabbit/src/rabbit.erl | 46 +- deps/rabbit/src/rabbit_amqqueue.erl | 17 +- deps/rabbit/src/rabbit_amqqueue_process.erl | 26 +- .../src/rabbit_auth_backend_internal.erl | 13 +- deps/rabbit/src/rabbit_binding.erl | 6 +- deps/rabbit/src/rabbit_classic_queue.erl | 41 +- deps/rabbit/src/rabbit_core_ff.erl | 22 + deps/rabbit/src/rabbit_db.erl | 102 +- deps/rabbit/src/rabbit_db_binding.erl | 417 ++- .../src/rabbit_db_binding_m2k_converter.erl | 125 + deps/rabbit/src/rabbit_db_cluster.erl | 116 +- deps/rabbit/src/rabbit_db_exchange.erl | 387 ++- .../src/rabbit_db_exchange_m2k_converter.erl | 139 + deps/rabbit/src/rabbit_db_m2k_converter.erl | 127 + deps/rabbit/src/rabbit_db_maintenance.erl | 136 +- .../rabbit_db_maintenance_m2k_converter.erl | 103 + deps/rabbit/src/rabbit_db_msup.erl | 138 +- .../src/rabbit_db_msup_m2k_converter.erl | 103 + deps/rabbit/src/rabbit_db_policy.erl | 18 +- deps/rabbit/src/rabbit_db_queue.erl | 443 ++- .../src/rabbit_db_queue_m2k_converter.erl | 105 + deps/rabbit/src/rabbit_db_rtparams.erl | 141 +- .../src/rabbit_db_rtparams_m2k_converter.erl | 107 + deps/rabbit/src/rabbit_db_topic_exchange.erl | 274 +- deps/rabbit/src/rabbit_db_user.erl | 441 ++- .../src/rabbit_db_user_m2k_converter.erl | 203 ++ deps/rabbit/src/rabbit_db_vhost.erl | 175 +- .../src/rabbit_db_vhost_m2k_converter.erl | 103 + deps/rabbit/src/rabbit_definitions.erl | 8 +- deps/rabbit/src/rabbit_exchange.erl | 3 - deps/rabbit/src/rabbit_exchange_decorator.erl | 8 +- .../src/rabbit_exchange_type_direct.erl | 46 +- .../src/rabbit_exchange_type_fanout.erl | 8 +- .../src/rabbit_exchange_type_headers.erl | 8 +- .../src/rabbit_exchange_type_invalid.erl | 8 +- .../rabbit/src/rabbit_exchange_type_topic.erl | 2 +- deps/rabbit/src/rabbit_ff_controller.erl | 15 +- deps/rabbit/src/rabbit_khepri.erl | 1418 +++++++++ deps/rabbit/src/rabbit_maintenance.erl | 25 +- .../rabbit/src/rabbit_mirror_queue_master.erl | 52 +- deps/rabbit/src/rabbit_mirror_queue_misc.erl | 260 +- deps/rabbit/src/rabbit_mirror_queue_slave.erl | 51 +- deps/rabbit/src/rabbit_mnesia.erl | 33 +- deps/rabbit/src/rabbit_node_monitor.erl | 77 +- deps/rabbit/src/rabbit_policy.erl | 37 +- deps/rabbit/src/rabbit_prelaunch_cluster.erl | 37 +- deps/rabbit/src/rabbit_queue_decorator.erl | 2 +- deps/rabbit/src/rabbit_queue_location.erl | 2 + deps/rabbit/src/rabbit_quorum_queue.erl | 6 +- deps/rabbit/src/rabbit_ra_registry.erl | 2 +- deps/rabbit/src/rabbit_reader.erl | 7 +- deps/rabbit/src/rabbit_stream_coordinator.erl | 3 +- deps/rabbit/src/rabbit_stream_queue.erl | 1 + deps/rabbit/src/rabbit_table.erl | 137 +- deps/rabbit/src/rabbit_vm.erl | 27 +- deps/rabbit/src/vhost.erl | 9 +- deps/rabbit/test/bindings_SUITE.erl | 288 +- deps/rabbit/test/cluster_SUITE.erl | 10 + deps/rabbit/test/cluster_minority_SUITE.erl | 294 ++ deps/rabbit/test/cluster_rename_SUITE.erl | 46 +- .../test/clustering_management_SUITE.erl | 696 ++++- .../rabbit/test/clustering_recovery_SUITE.erl | 211 +- deps/rabbit/test/clustering_utils.erl | 108 +- deps/rabbit/test/consumer_timeout_SUITE.erl | 21 +- deps/rabbit/test/crashing_queues_SUITE.erl | 38 +- deps/rabbit/test/dead_lettering_SUITE.erl | 30 +- deps/rabbit/test/definition_import_SUITE.erl | 30 +- .../test/direct_exchange_routing_v2_SUITE.erl | 13 +- deps/rabbit/test/dynamic_ha_SUITE.erl | 29 +- deps/rabbit/test/dynamic_qq_SUITE.erl | 66 +- deps/rabbit/test/eager_sync_SUITE.erl | 9 +- deps/rabbit/test/exchanges_SUITE.erl | 83 +- deps/rabbit/test/feature_flags_SUITE.erl | 11 +- .../list_queues_online_and_offline_SUITE.erl | 6 +- deps/rabbit/test/maintenance_mode_SUITE.erl | 52 +- deps/rabbit/test/many_node_ha_SUITE.erl | 7 +- deps/rabbit/test/message_containers_SUITE.erl | 41 +- .../test/metadata_store_clustering_SUITE.erl | 340 ++ .../test/metadata_store_migration_SUITE.erl | 161 + .../test/metadata_store_phase1_SUITE.erl | 2775 +++++++++++++++++ .../peer_discovery_classic_config_SUITE.erl | 54 +- ...er_user_connection_channel_limit_SUITE.erl | 68 +- ...nection_channel_limit_partitions_SUITE.erl | 11 +- ...user_connection_channel_tracking_SUITE.erl | 39 +- .../per_user_connection_tracking_SUITE.erl | 29 +- .../test/per_vhost_connection_limit_SUITE.erl | 78 +- ...host_connection_limit_partitions_SUITE.erl | 11 +- .../rabbit/test/per_vhost_msg_store_SUITE.erl | 45 +- .../test/per_vhost_queue_limit_SUITE.erl | 87 +- deps/rabbit/test/policy_SUITE.erl | 144 +- .../publisher_confirms_parallel_SUITE.erl | 31 +- .../rabbit/test/queue_length_limits_SUITE.erl | 54 +- .../test/queue_master_location_SUITE.erl | 36 +- deps/rabbit/test/queue_parallel_SUITE.erl | 22 +- deps/rabbit/test/queue_type_SUITE.erl | 59 +- deps/rabbit/test/quorum_queue_SUITE.erl | 31 +- .../test/rabbit_core_metrics_gc_SUITE.erl | 11 +- deps/rabbit/test/rabbit_db_binding_SUITE.erl | 56 +- deps/rabbit/test/rabbit_db_exchange_SUITE.erl | 20 +- .../test/rabbit_db_maintenance_SUITE.erl | 9 - deps/rabbit/test/rabbit_db_msup_SUITE.erl | 25 +- deps/rabbit/test/rabbit_db_queue_SUITE.erl | 22 +- .../test/rabbit_db_topic_exchange_SUITE.erl | 254 +- .../rabbit_fifo_dlx_integration_SUITE.erl | 23 +- .../rabbit/test/rabbit_stream_queue_SUITE.erl | 87 +- .../test/rabbitmq_4_0_deprecations_SUITE.erl | 114 +- .../test/rabbitmqctl_integration_SUITE.erl | 11 +- deps/rabbit/test/routing_SUITE.erl | 17 +- deps/rabbit/test/simple_ha_SUITE.erl | 7 +- deps/rabbit/test/sync_detection_SUITE.erl | 7 +- deps/rabbit/test/topic_permission_SUITE.erl | 17 +- .../test/unit_policy_validators_SUITE.erl | 19 +- deps/rabbit/test/vhost_SUITE.erl | 11 + .../cli/ctl/commands/force_boot_command.ex | 4 + .../commands/force_standalone_khepri_boot.ex | 45 + .../cli/ctl/commands/join_cluster_command.ex | 15 + .../commands/rename_cluster_node_command.ex | 11 +- .../lib/rabbitmq/cli/default_output.ex | 19 +- .../commands/metadata_store_status_command.ex | 36 + .../change_cluster_node_type_command_test.exs | 19 +- .../list_global_parameters_command_test.exs | 12 +- .../ctl/set_global_parameter_command_test.exs | 11 +- .../BUILD.bazel | 8 +- .../rabbitmq_consistent_hash_exchange/app.bzl | 7 + .../src/rabbit_db_ch_exchange.erl | 124 +- .../rabbit_db_ch_exchange_m2k_converter.erl | 102 + ...it_exchange_type_consistent_hash_SUITE.erl | 187 +- .../src/rabbit_control_helper.erl | 16 + .../src/rabbit_ct_broker_helpers.erl | 97 +- ...abbit_federation_exchange_link_sup_sup.erl | 12 +- .../rabbit_federation_queue_link_sup_sup.erl | 10 +- .../rabbit_federation_upstream_exchange.erl | 8 +- .../test/exchange_SUITE.erl | 79 +- deps/rabbitmq_federation/test/queue_SUITE.erl | 55 +- .../test/rabbit_federation_test_util.erl | 16 +- deps/rabbitmq_jms_topic_exchange/BUILD.bazel | 6 +- deps/rabbitmq_jms_topic_exchange/Makefile | 1 - deps/rabbitmq_jms_topic_exchange/app.bzl | 7 + .../src/rabbit_db_jms_exchange.erl | 85 +- .../rabbit_db_jms_exchange_m2k_converter.erl | 100 + .../src/rabbit_jms_topic_exchange.erl | 3 +- .../test/rjms_topic_selector_SUITE.erl | 82 +- deps/rabbitmq_management/Makefile | 2 +- .../priv/www/js/tmpl/binary.ejs | 4 +- .../priv/www/js/tmpl/memory.ejs | 1 + .../test/clustering_SUITE.erl | 20 +- .../test/rabbit_mgmt_http_SUITE.erl | 4 +- .../rabbit_mgmt_http_health_checks_SUITE.erl | 44 +- .../test/rabbit_mgmt_only_http_SUITE.erl | 30 +- .../test/rabbit_mgmt_rabbitmqadmin_SUITE.erl | 8 +- deps/rabbitmq_management_agent/Makefile | 2 +- .../src/rabbit_mgmt_metrics_collector.erl | 2 +- deps/rabbitmq_mqtt/BUILD.bazel | 5 +- deps/rabbitmq_mqtt/test/processor_SUITE.erl | 8 + deps/rabbitmq_mqtt/test/shared_SUITE.erl | 121 +- deps/rabbitmq_mqtt/test/util.erl | 4 + deps/rabbitmq_mqtt/test/v5_SUITE.erl | 19 +- ...etheus_rabbitmq_core_metrics_collector.erl | 13 +- .../src/rabbit_exchange_type_random.erl | 4 +- .../BUILD.bazel | 6 +- deps/rabbitmq_recent_history_exchange/app.bzl | 7 + .../src/rabbit_db_rh_exchange.erl | 87 +- .../rabbit_db_rh_exchange_m2k_converter.erl | 99 + .../rabbit_exchange_type_recent_history.erl | 10 +- .../test/system_SUITE.erl | 138 +- .../src/rabbit_amqp091_shovel.erl | 70 +- .../src/rabbit_shovel_dyn_worker_sup_sup.erl | 17 +- .../src/rabbit_shovel_parameters.erl | 98 +- .../src/rabbit_shovel_worker_sup.erl | 5 +- .../rabbitmq_shovel/test/parameters_SUITE.erl | 16 +- moduleindex.yaml | 78 + 183 files changed, 14325 insertions(+), 1636 deletions(-) create mode 100644 bazel/BUILD.horus create mode 100644 bazel/BUILD.khepri create mode 100644 bazel/BUILD.khepri_mnesia_migration create mode 100644 deps/rabbit/src/mirrored_supervisor.hrl create mode 100644 deps/rabbit/src/rabbit_db_binding_m2k_converter.erl create mode 100644 deps/rabbit/src/rabbit_db_exchange_m2k_converter.erl create mode 100644 deps/rabbit/src/rabbit_db_m2k_converter.erl create mode 100644 deps/rabbit/src/rabbit_db_maintenance_m2k_converter.erl create mode 100644 deps/rabbit/src/rabbit_db_msup_m2k_converter.erl create mode 100644 deps/rabbit/src/rabbit_db_queue_m2k_converter.erl create mode 100644 deps/rabbit/src/rabbit_db_rtparams_m2k_converter.erl create mode 100644 deps/rabbit/src/rabbit_db_user_m2k_converter.erl create mode 100644 deps/rabbit/src/rabbit_db_vhost_m2k_converter.erl create mode 100644 deps/rabbit/src/rabbit_khepri.erl create mode 100644 deps/rabbit/test/cluster_minority_SUITE.erl create mode 100644 deps/rabbit/test/metadata_store_clustering_SUITE.erl create mode 100644 deps/rabbit/test/metadata_store_migration_SUITE.erl create mode 100644 deps/rabbit/test/metadata_store_phase1_SUITE.erl create mode 100644 deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/force_standalone_khepri_boot.ex create mode 100644 deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/metadata_store_status_command.ex create mode 100644 deps/rabbitmq_consistent_hash_exchange/src/rabbit_db_ch_exchange_m2k_converter.erl create mode 100644 deps/rabbitmq_jms_topic_exchange/src/rabbit_db_jms_exchange_m2k_converter.erl create mode 100644 deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange_m2k_converter.erl diff --git a/.github/workflows/test-mixed-versions.yaml b/.github/workflows/test-mixed-versions.yaml index 52f6be1c0f49..57a12783b2b2 100644 --- a/.github/workflows/test-mixed-versions.yaml +++ b/.github/workflows/test-mixed-versions.yaml @@ -131,6 +131,9 @@ jobs: matrix: otp_version_id: - "25_3" + metadata_store: + - mnesia + - khepri timeout-minutes: 120 steps: - name: CHECKOUT REPOSITORY @@ -164,6 +167,7 @@ jobs: sudo ethtool -K eth0 tso off gso off gro off tx off rx off lro off bazelisk test //... \ --config=rbe-${{ matrix.otp_version_id }} \ + --test_env RABBITMQ_METADATA_STORE=${{ matrix.metadata_store }} \ --test_tag_filters=mixed-version-cluster,-aws,-docker \ --build_tests_only \ --verbose_failures diff --git a/.github/workflows/test-windows.yaml b/.github/workflows/test-windows.yaml index 3f57fe218cb5..7f117e506143 100644 --- a/.github/workflows/test-windows.yaml +++ b/.github/workflows/test-windows.yaml @@ -13,6 +13,9 @@ jobs: include: - erlang_version: "26.1" elixir_version: "1.15.2" + metadata_store: + - mnesia + - khepri timeout-minutes: 120 steps: - name: CHECKOUT REPOSITORY @@ -57,6 +60,7 @@ jobs: run: | bazelisk test //... ^ --config=buildbuddy ^ + --test_env RABBITMQ_METADATA_STORE=${{ matrix.metadata_store }} ^ --test_tag_filters=-aws,-docker,-bats,-starts-background-broker,-dialyze ^ --build_tests_only ^ --verbose_failures diff --git a/.github/workflows/test.yaml b/.github/workflows/test.yaml index 6f571d8bb157..3c756504200d 100644 --- a/.github/workflows/test.yaml +++ b/.github/workflows/test.yaml @@ -37,6 +37,9 @@ jobs: otp_version_id: - 25_3 - 26 + metadata_store: + - mnesia + - khepri timeout-minutes: 120 steps: - name: CHECKOUT REPOSITORY @@ -72,6 +75,7 @@ jobs: sudo ethtool -K eth0 tso off gso off gro off tx off rx off lro off bazelisk test //... \ --config=rbe-${{ matrix.otp_version_id }} \ + --test_env RABBITMQ_METADATA_STORE=${{ matrix.metadata_store }} \ --test_tag_filters=-aws,-docker,-mixed-version-cluster \ --build_tests_only \ --verbose_failures diff --git a/MODULE.bazel b/MODULE.bazel index f5802aebafb9..2ed466cb126e 100644 --- a/MODULE.bazel +++ b/MODULE.bazel @@ -263,6 +263,13 @@ erlang_package.hex_package( version = "1.3.3", ) +erlang_package.git_package( + name = "horus", + build_file = "@rabbitmq-server//bazel:BUILD.horus", + commit = "253f9af23e539b7370a5105df19dcbb66762b247", + repository = "rabbitmq/horus", +) + erlang_package.git_package( name = "jose", build_file = "@rabbitmq-server//bazel:BUILD.jose", @@ -277,6 +284,20 @@ erlang_package.hex_package( version = "1.4.1", ) +erlang_package.git_package( + name = "khepri", + build_file = "@rabbitmq-server//bazel:BUILD.khepri", + commit = "952a52969eaa97a4327806b4fc062238dc3e6214", + repository = "rabbitmq/khepri", +) + +erlang_package.git_package( + name = "khepri_mnesia_migration", + build_file = "@rabbitmq-server//bazel:BUILD.khepri_mnesia_migration", + repository = "rabbitmq/khepri_mnesia_migration", + commit = "ec891800b449f6a8fc975f959447039b3df343ed", +) + erlang_package.hex_package( name = "thoas", build_file = "@rabbitmq-server//bazel:BUILD.thoas", @@ -370,8 +391,11 @@ use_repo( "gen_batch_server", "getopt", "gun", + "horus", "jose", "json", + "khepri", + "khepri_mnesia_migration", "observer_cli", "prometheus", "ranch", diff --git a/bazel/BUILD.horus b/bazel/BUILD.horus new file mode 100644 index 000000000000..e3d75efe9921 --- /dev/null +++ b/bazel/BUILD.horus @@ -0,0 +1,114 @@ +load("@rules_erlang//:erlang_bytecode2.bzl", "erlang_bytecode", "erlc_opts") +load("@rules_erlang//:erlang_app.bzl", "erlang_app") + +erlc_opts( + name = "erlc_opts", + values = select({ + "@rules_erlang//:debug_build": [ + "+debug_info", + "+warn_export_vars", + "+warnings_as_errors", + ], + "//conditions:default": [ + "+debug_info", + "+deterministic", + "+warn_export_vars", + "+warnings_as_errors", + ], + }), + visibility = [":__subpackages__"], +) + +erlang_bytecode( + name = "other_beam", + srcs = [ + "src/horus.erl", + "src/horus_cover.erl", + "src/horus_utils.erl", + ], + hdrs = [":public_and_private_hdrs"], + app_name = "horus", + dest = "ebin", + erlc_opts = "//:erlc_opts", +) + +filegroup( + name = "beam_files", + srcs = [":other_beam"], +) + +filegroup( + name = "srcs", + srcs = [ + "src/horus.app.src", + "src/horus.erl", + "src/horus_cover.erl", + "src/horus_utils.erl", + ], +) + +filegroup( + name = "private_hdrs", + srcs = [ + "src/horus_error.hrl", + "src/horus_fun.hrl", + ], +) + +filegroup( + name = "public_hdrs", + srcs = [ + "include/horus.hrl", + ], +) + +filegroup( + name = "priv", + srcs = [ + "priv/horus_cover_helper.erl", + ], +) + +filegroup( + name = "license_files", + srcs = [ + "LICENSE-Apache-2.0", + "LICENSE-MPL-2.0", + ], +) + +filegroup( + name = "public_and_private_hdrs", + srcs = [ + ":private_hdrs", + ":public_hdrs", + ], +) + +filegroup( + name = "all_srcs", + srcs = [ + ":public_and_private_hdrs", + ":srcs", + ], +) + +erlang_app( + name = "erlang_app", + srcs = [":all_srcs"], + hdrs = [":public_hdrs"], + app_name = "horus", + beam_files = [":beam_files"], + extra_apps = [ + "compiler", + "tools", + ], + license_files = [":license_files"], + priv = [":priv"], +) + +alias( + name = "horus", + actual = ":erlang_app", + visibility = ["//visibility:public"], +) diff --git a/bazel/BUILD.khepri b/bazel/BUILD.khepri new file mode 100644 index 000000000000..f30368da7c8b --- /dev/null +++ b/bazel/BUILD.khepri @@ -0,0 +1,185 @@ +load("@rules_erlang//:erlang_bytecode2.bzl", "erlang_bytecode", "erlc_opts") +load("@rules_erlang//:erlang_app.bzl", "erlang_app") + +erlc_opts( + name = "erlc_opts", + values = select({ + "@rules_erlang//:debug_build": [ + "+debug_info", + "+warn_export_vars", + "+warnings_as_errors", + ], + "//conditions:default": [ + "+debug_info", + "+deterministic", + "+warn_export_vars", + "+warnings_as_errors", + ], + }), + visibility = [":__subpackages__"], +) + +erlang_bytecode( + name = "behaviours", + srcs = [ + "src/khepri_import_export.erl", + ], + outs = [ + "ebin/khepri_import_export.beam", + ], + hdrs = [ + "src/khepri_cluster.hrl", + "src/khepri_machine.hrl", + "src/khepri_payload.hrl", + "src/khepri_tree.hrl", + ], + app_name = "khepri", + erlc_opts = "//:erlc_opts", +) + +filegroup( + name = "beam_files", + srcs = [ + ":behaviours", + ":other_beam", + ], +) + +filegroup( + name = "srcs", + srcs = [ + "src/khepri.app.src", + "src/khepri.erl", + "src/khepri_adv.erl", + "src/khepri_app.erl", + "src/khepri_cluster.erl", + "src/khepri_condition.erl", + "src/khepri_event_handler.erl", + "src/khepri_evf.erl", + "src/khepri_export_erlang.erl", + "src/khepri_import_export.erl", + "src/khepri_machine.erl", + "src/khepri_path.erl", + "src/khepri_pattern_tree.erl", + "src/khepri_payload.erl", + "src/khepri_projection.erl", + "src/khepri_sproc.erl", + "src/khepri_sup.erl", + "src/khepri_tree.erl", + "src/khepri_tx.erl", + "src/khepri_tx_adv.erl", + "src/khepri_utils.erl", + ], +) + +filegroup( + name = "private_hdrs", + srcs = [ + "src/khepri_bang.hrl", + "src/khepri_cluster.hrl", + "src/khepri_error.hrl", + "src/khepri_evf.hrl", + "src/khepri_machine.hrl", + "src/khepri_payload.hrl", + "src/khepri_projection.hrl", + "src/khepri_ret.hrl", + "src/khepri_tree.hrl", + "src/khepri_tx.hrl", + ], +) + +filegroup( + name = "public_hdrs", + srcs = [ + "include/khepri.hrl", + ], +) + +filegroup(name = "priv") + +filegroup( + name = "licenses", + srcs = [ + "LICENSE-Apache-2.0", + "LICENSE-MPL-2.0", + ], +) + +filegroup( + name = "public_and_private_hdrs", + srcs = [ + ":private_hdrs", + ":public_hdrs", + ], +) + +filegroup( + name = "all_srcs", + srcs = [ + ":public_and_private_hdrs", + ":srcs", + ], +) + +erlang_app( + name = "erlang_app", + srcs = [":all_srcs"], + hdrs = [":public_hdrs"], + app_name = "khepri", + beam_files = [":beam_files"], + extra_apps = ["compiler"], + license_files = [":license_files"], + priv = [":priv"], + deps = [ + "@horus//:erlang_app", + "@ra//:erlang_app", + ], +) + +alias( + name = "khepri", + actual = ":erlang_app", + visibility = ["//visibility:public"], +) + +erlang_bytecode( + name = "other_beam", + srcs = [ + "src/khepri.erl", + "src/khepri_adv.erl", + "src/khepri_app.erl", + "src/khepri_cluster.erl", + "src/khepri_condition.erl", + "src/khepri_event_handler.erl", + "src/khepri_evf.erl", + "src/khepri_export_erlang.erl", + "src/khepri_machine.erl", + "src/khepri_path.erl", + "src/khepri_pattern_tree.erl", + "src/khepri_payload.erl", + "src/khepri_projection.erl", + "src/khepri_sproc.erl", + "src/khepri_sup.erl", + "src/khepri_tree.erl", + "src/khepri_tx.erl", + "src/khepri_tx_adv.erl", + "src/khepri_utils.erl", + ], + hdrs = [":public_and_private_hdrs"], + app_name = "khepri", + beam = [":behaviours"], + dest = "ebin", + erlc_opts = "//:erlc_opts", + deps = [ + "@horus//:erlang_app", + "@ra//:erlang_app", + ], +) + +filegroup( + name = "license_files", + srcs = [ + "LICENSE-Apache-2.0", + "LICENSE-MPL-2.0", + ], +) diff --git a/bazel/BUILD.khepri_mnesia_migration b/bazel/BUILD.khepri_mnesia_migration new file mode 100644 index 000000000000..b01afc3951c6 --- /dev/null +++ b/bazel/BUILD.khepri_mnesia_migration @@ -0,0 +1,146 @@ +load("@rules_erlang//:erlang_bytecode2.bzl", "erlang_bytecode", "erlc_opts") +load("@rules_erlang//:erlang_app.bzl", "erlang_app") + +erlc_opts( + name = "erlc_opts", + values = select({ + "@rules_erlang//:debug_build": [ + "+debug_info", + "+warn_export_vars", + "+warnings_as_errors", + ], + "//conditions:default": [ + "+debug_info", + "+deterministic", + "+warn_export_vars", + "+warnings_as_errors", + ], + }), + visibility = [":__subpackages__"], +) + +erlang_bytecode( + name = "other_beam", + srcs = [ + "src/khepri_mnesia_migration_app.erl", + "src/khepri_mnesia_migration_sup.erl", + "src/kmm_utils.erl", + "src/m2k_cluster_sync.erl", + "src/m2k_cluster_sync_sup.erl", + "src/m2k_export.erl", + "src/m2k_subscriber.erl", + "src/m2k_table_copy.erl", + "src/m2k_table_copy_sup.erl", + "src/m2k_table_copy_sup_sup.erl", + "src/mnesia_to_khepri.erl", + "src/mnesia_to_khepri_example_converter.erl", + ], + hdrs = [":public_and_private_hdrs"], + app_name = "khepri_mnesia_migration", + beam = [":behaviours"], + dest = "ebin", + erlc_opts = "//:erlc_opts", + deps = ["@khepri//:erlang_app"], +) + +filegroup( + name = "beam_files", + srcs = [ + ":behaviours", + ":other_beam", + ], +) + +filegroup( + name = "srcs", + srcs = [ + "src/khepri_mnesia_migration.app.src", + "src/khepri_mnesia_migration_app.erl", + "src/khepri_mnesia_migration_sup.erl", + "src/kmm_utils.erl", + "src/m2k_cluster_sync.erl", + "src/m2k_cluster_sync_sup.erl", + "src/m2k_export.erl", + "src/m2k_subscriber.erl", + "src/m2k_table_copy.erl", + "src/m2k_table_copy_sup.erl", + "src/m2k_table_copy_sup_sup.erl", + "src/mnesia_to_khepri.erl", + "src/mnesia_to_khepri_example_converter.erl", + ], +) + +filegroup( + name = "private_hdrs", + srcs = [ + "src/kmm_error.hrl", + # "src/kmm_logging.hrl", # keep + ], +) + +filegroup( + name = "public_hdrs", + srcs = ["src/kmm_logging.hrl"] + glob(["include/**/*.hrl"]), # keep +) + +filegroup( + name = "priv", + srcs = glob(["priv/**/*"]), +) + +filegroup( + name = "licenses", + srcs = [ + "LICENSE-Apache-2.0", + "LICENSE-MPL-2.0", + ], +) + +filegroup( + name = "public_and_private_hdrs", + srcs = [ + ":private_hdrs", + ":public_hdrs", + ], +) + +filegroup( + name = "all_srcs", + srcs = [ + ":public_and_private_hdrs", + ":srcs", + ], +) + +erlang_app( + name = "erlang_app", + srcs = [":all_srcs"], + hdrs = [":public_hdrs"], + app_name = "khepri_mnesia_migration", + beam_files = [":beam_files"], + license_files = [":license_files"], + priv = [":priv"], + deps = ["@khepri//:erlang_app"], +) + +alias( + name = "khepri_mnesia_migration", + actual = ":erlang_app", + visibility = ["//visibility:public"], +) + +erlang_bytecode( + name = "behaviours", + srcs = [ + "src/mnesia_to_khepri_converter.erl", + ], + hdrs = [":public_and_private_hdrs"], + app_name = "khepri_mnesia_migration", + dest = "ebin", + erlc_opts = "//:erlc_opts", +) + +filegroup( + name = "license_files", + srcs = glob(["LICENSE*"]), +) diff --git a/deps/rabbit/BUILD.bazel b/deps/rabbit/BUILD.bazel index 5ff145e341cb..a17ab7398ec5 100644 --- a/deps/rabbit/BUILD.bazel +++ b/deps/rabbit/BUILD.bazel @@ -217,6 +217,8 @@ rabbitmq_app( "//deps/rabbit_common:erlang_app", "@cuttlefish//:erlang_app", "@gen_batch_server//:erlang_app", + "@khepri//:erlang_app", + "@khepri_mnesia_migration//:erlang_app", "@observer_cli//:erlang_app", "@osiris//:erlang_app", "@ra//:erlang_app", @@ -324,7 +326,7 @@ rabbitmq_integration_suite( name = "cluster_rename_SUITE", size = "large", flaky = True, - shard_count = 2, + shard_count = 3, ) rabbitmq_integration_suite( @@ -346,15 +348,18 @@ rabbitmq_integration_suite( additional_beam = [ ":test_clustering_utils_beam", ], - shard_count = 18, + shard_count = 48, sharding_method = "case", ) rabbitmq_integration_suite( name = "clustering_recovery_SUITE", size = "medium", - shard_count = 2, + shard_count = 8, sharding_method = "case", + additional_beam = [ + ":test_clustering_utils_beam", + ], ) rabbitmq_integration_suite( @@ -587,11 +592,13 @@ rabbitmq_integration_suite( rabbitmq_integration_suite( name = "per_user_connection_channel_limit_SUITE", size = "medium", + shard_count = 7, ) rabbitmq_integration_suite( name = "per_user_connection_channel_tracking_SUITE", size = "medium", + shard_count = 8, ) rabbitmq_integration_suite( @@ -607,6 +614,10 @@ rabbitmq_integration_suite( rabbitmq_integration_suite( name = "per_vhost_connection_limit_SUITE", size = "medium", + additional_beam = [ + ":cluster_rename_SUITE_beam_files", + ], + shard_count = 10, ) rabbitmq_integration_suite( @@ -660,7 +671,7 @@ rabbitmq_integration_suite( rabbitmq_integration_suite( name = "queue_master_location_SUITE", size = "large", - shard_count = 2, + shard_count = 3, ) rabbitmq_integration_suite( @@ -1166,6 +1177,26 @@ rabbitmq_integration_suite( ], ) +rabbitmq_integration_suite( + name = "metadata_store_clustering_SUITE", + size = "large", + shard_count = 18, + sharding_method = "case", +) + +rabbitmq_integration_suite( + name = "metadata_store_phase1_SUITE", + size = "small", + deps = [ + "@khepri//:erlang_app", + ], +) + +rabbitmq_integration_suite( + name = "metadata_store_migration_SUITE", + size = "small", +) + rabbitmq_integration_suite( name = "routing_SUITE", size = "large", @@ -1179,6 +1210,14 @@ rabbitmq_integration_suite( ], ) +rabbitmq_integration_suite( + name = "cluster_minority_SUITE", + additional_beam = [ + ":test_clustering_utils_beam", + ], + size = "large", +) + assert_suites() filegroup( diff --git a/deps/rabbit/Makefile b/deps/rabbit/Makefile index afc42eddc17c..4f9809a56953 100644 --- a/deps/rabbit/Makefile +++ b/deps/rabbit/Makefile @@ -127,7 +127,6 @@ define PROJECT_ENV {credentials_obfuscation_fallback_secret, <<"nocookie">>}, {dead_letter_worker_consumer_prefetch, 32}, {dead_letter_worker_publisher_confirm_timeout, 180000}, - %% EOL date for the current release series, if known/announced {release_series_eol_date, none} ] @@ -143,7 +142,7 @@ APPS_DIR := $(CURDIR)/apps LOCAL_DEPS = sasl rabbitmq_prelaunch os_mon inets compiler public_key crypto ssl syntax_tools xmerl BUILD_DEPS = rabbitmq_cli -DEPS = ranch rabbit_common ra sysmon_handler stdout_formatter recon redbug observer_cli osiris amqp10_common syslog systemd seshat +DEPS = ranch rabbit_common ra sysmon_handler stdout_formatter recon redbug observer_cli osiris amqp10_common syslog systemd seshat khepri khepri_mnesia_migration TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers amqp_client meck proper PLT_APPS += mnesia @@ -153,6 +152,8 @@ dep_osiris = git https://github.com/rabbitmq/osiris v1.6.7 dep_systemd = hex 0.6.1 dep_seshat = git https://github.com/rabbitmq/seshat v0.6.1 +dep_khepri = hex 0.8.0 +dep_khepri_mnesia_migration = hex 0.1.1 define usage_xml_to_erl $(subst __,_,$(patsubst $(DOCS_DIR)/rabbitmq%.1.xml, src/rabbit_%_usage.erl, $(subst -,_,$(1)))) diff --git a/deps/rabbit/app.bzl b/deps/rabbit/app.bzl index 7fb664c8d875..1d0c03af6538 100644 --- a/deps/rabbit/app.bzl +++ b/deps/rabbit/app.bzl @@ -85,17 +85,26 @@ def all_beam_files(name = "all_beam_files"): "src/rabbit_cuttlefish.erl", "src/rabbit_db.erl", "src/rabbit_db_binding.erl", + "src/rabbit_db_binding_m2k_converter.erl", "src/rabbit_db_cluster.erl", "src/rabbit_db_exchange.erl", + "src/rabbit_db_exchange_m2k_converter.erl", + "src/rabbit_db_m2k_converter.erl", "src/rabbit_db_maintenance.erl", + "src/rabbit_db_maintenance_m2k_converter.erl", "src/rabbit_db_msup.erl", + "src/rabbit_db_msup_m2k_converter.erl", "src/rabbit_db_policy.erl", "src/rabbit_db_queue.erl", + "src/rabbit_db_queue_m2k_converter.erl", "src/rabbit_db_rtparams.erl", + "src/rabbit_db_rtparams_m2k_converter.erl", "src/rabbit_db_topic_exchange.erl", "src/rabbit_db_user.erl", + "src/rabbit_db_user_m2k_converter.erl", "src/rabbit_db_vhost.erl", "src/rabbit_db_vhost_defaults.erl", + "src/rabbit_db_vhost_m2k_converter.erl", "src/rabbit_dead_letter.erl", "src/rabbit_definitions.erl", "src/rabbit_definitions_hashing.erl", @@ -136,6 +145,7 @@ def all_beam_files(name = "all_beam_files"): "src/rabbit_global_counters.erl", "src/rabbit_guid.erl", "src/rabbit_health_check.erl", + "src/rabbit_khepri.erl", "src/rabbit_limiter.erl", "src/rabbit_log_channel.erl", "src/rabbit_log_connection.erl", @@ -243,6 +253,8 @@ def all_beam_files(name = "all_beam_files"): deps = [ "//deps/amqp10_common:erlang_app", "//deps/rabbit_common:erlang_app", + "@khepri//:erlang_app", + "@khepri_mnesia_migration//:erlang_app", "@ra//:erlang_app", "@ranch//:erlang_app", "@stdout_formatter//:erlang_app", @@ -336,17 +348,26 @@ def all_test_beam_files(name = "all_test_beam_files"): "src/rabbit_cuttlefish.erl", "src/rabbit_db.erl", "src/rabbit_db_binding.erl", + "src/rabbit_db_binding_m2k_converter.erl", "src/rabbit_db_cluster.erl", "src/rabbit_db_exchange.erl", + "src/rabbit_db_exchange_m2k_converter.erl", + "src/rabbit_db_m2k_converter.erl", "src/rabbit_db_maintenance.erl", + "src/rabbit_db_maintenance_m2k_converter.erl", "src/rabbit_db_msup.erl", + "src/rabbit_db_msup_m2k_converter.erl", "src/rabbit_db_policy.erl", "src/rabbit_db_queue.erl", + "src/rabbit_db_queue_m2k_converter.erl", "src/rabbit_db_rtparams.erl", + "src/rabbit_db_rtparams_m2k_converter.erl", "src/rabbit_db_topic_exchange.erl", "src/rabbit_db_user.erl", + "src/rabbit_db_user_m2k_converter.erl", "src/rabbit_db_vhost.erl", "src/rabbit_db_vhost_defaults.erl", + "src/rabbit_db_vhost_m2k_converter.erl", "src/rabbit_dead_letter.erl", "src/rabbit_definitions.erl", "src/rabbit_definitions_hashing.erl", @@ -387,6 +408,7 @@ def all_test_beam_files(name = "all_test_beam_files"): "src/rabbit_global_counters.erl", "src/rabbit_guid.erl", "src/rabbit_health_check.erl", + "src/rabbit_khepri.erl", "src/rabbit_limiter.erl", "src/rabbit_log_channel.erl", "src/rabbit_log_connection.erl", @@ -494,6 +516,8 @@ def all_test_beam_files(name = "all_test_beam_files"): deps = [ "//deps/amqp10_common:erlang_app", "//deps/rabbit_common:erlang_app", + "@khepri//:erlang_app", + "@khepri_mnesia_migration//:erlang_app", "@ra//:erlang_app", "@ranch//:erlang_app", "@stdout_formatter//:erlang_app", @@ -529,6 +553,7 @@ def all_srcs(name = "all_srcs"): filegroup( name = "private_hdrs", srcs = [ + "src/mirrored_supervisor.hrl", "src/rabbit_feature_flags.hrl", "src/rabbit_fifo.hrl", "src/rabbit_fifo_dlx.hrl", @@ -601,17 +626,26 @@ def all_srcs(name = "all_srcs"): "src/rabbit_cuttlefish.erl", "src/rabbit_db.erl", "src/rabbit_db_binding.erl", + "src/rabbit_db_binding_m2k_converter.erl", "src/rabbit_db_cluster.erl", "src/rabbit_db_exchange.erl", + "src/rabbit_db_exchange_m2k_converter.erl", + "src/rabbit_db_m2k_converter.erl", "src/rabbit_db_maintenance.erl", + "src/rabbit_db_maintenance_m2k_converter.erl", "src/rabbit_db_msup.erl", + "src/rabbit_db_msup_m2k_converter.erl", "src/rabbit_db_policy.erl", "src/rabbit_db_queue.erl", + "src/rabbit_db_queue_m2k_converter.erl", "src/rabbit_db_rtparams.erl", + "src/rabbit_db_rtparams_m2k_converter.erl", "src/rabbit_db_topic_exchange.erl", "src/rabbit_db_user.erl", + "src/rabbit_db_user_m2k_converter.erl", "src/rabbit_db_vhost.erl", "src/rabbit_db_vhost_defaults.erl", + "src/rabbit_db_vhost_m2k_converter.erl", "src/rabbit_dead_letter.erl", "src/rabbit_definitions.erl", "src/rabbit_definitions_hashing.erl", @@ -653,6 +687,7 @@ def all_srcs(name = "all_srcs"): "src/rabbit_global_counters.erl", "src/rabbit_guid.erl", "src/rabbit_health_check.erl", + "src/rabbit_khepri.erl", "src/rabbit_limiter.erl", "src/rabbit_log_channel.erl", "src/rabbit_log_connection.erl", @@ -841,7 +876,7 @@ def test_suite_beam_files(name = "test_suite_beam_files"): outs = ["test/clustering_management_SUITE.beam"], app_name = "rabbit", erlc_opts = "//:test_erlc_opts", - deps = ["//deps/amqp_client:erlang_app"], + deps = ["//deps/amqp_client:erlang_app", "//deps/rabbitmq_ct_helpers:erlang_app"], ) erlang_bytecode( name = "test_clustering_utils_beam", @@ -2030,7 +2065,34 @@ def test_suite_beam_files(name = "test_suite_beam_files"): erlc_opts = "//:test_erlc_opts", deps = ["//deps/amqp_client:erlang_app"], ) - + erlang_bytecode( + name = "metadata_store_clustering_SUITE_beam_files", + testonly = True, + srcs = ["test/metadata_store_clustering_SUITE.erl"], + outs = ["test/metadata_store_clustering_SUITE.beam"], + app_name = "rabbit", + erlc_opts = "//:test_erlc_opts", + deps = ["//deps/amqp_client:erlang_app", "//deps/rabbitmq_ct_helpers:erlang_app"], + ) + erlang_bytecode( + name = "metadata_store_migration_SUITE_beam_files", + testonly = True, + srcs = ["test/metadata_store_migration_SUITE.erl"], + outs = ["test/metadata_store_migration_SUITE.beam"], + app_name = "rabbit", + erlc_opts = "//:test_erlc_opts", + deps = ["//deps/amqp_client:erlang_app"], + ) + erlang_bytecode( + name = "mc_SUITE_beam_files", + testonly = True, + srcs = ["test/mc_SUITE.erl"], + outs = ["test/mc_SUITE.beam"], + hdrs = ["include/mc.hrl"], + app_name = "rabbit", + erlc_opts = "//:test_erlc_opts", + deps = ["//deps/amqp10_common:erlang_app", "//deps/rabbit_common:erlang_app"], + ) erlang_bytecode( name = "routing_SUITE_beam_files", testonly = True, @@ -2040,6 +2102,15 @@ def test_suite_beam_files(name = "test_suite_beam_files"): erlc_opts = "//:test_erlc_opts", deps = ["//deps/amqp_client:erlang_app"], ) + erlang_bytecode( + name = "metadata_store_phase1_SUITE_beam_files", + testonly = True, + srcs = ["test/metadata_store_phase1_SUITE.erl"], + outs = ["test/metadata_store_phase1_SUITE.beam"], + app_name = "rabbit", + erlc_opts = "//:test_erlc_opts", + deps = ["//deps/rabbit_common:erlang_app", "@khepri//:erlang_app"], + ) erlang_bytecode( name = "mc_unit_SUITE_beam_files", testonly = True, @@ -2059,3 +2130,12 @@ def test_suite_beam_files(name = "test_suite_beam_files"): erlc_opts = "//:test_erlc_opts", deps = ["//deps/amqp_client:erlang_app", "//deps/rabbitmq_ct_helpers:erlang_app"], ) + erlang_bytecode( + name = "cluster_minority_SUITE_beam_files", + testonly = True, + srcs = ["test/cluster_minority_SUITE.erl"], + outs = ["test/cluster_minority_SUITE.beam"], + app_name = "rabbit", + erlc_opts = "//:test_erlc_opts", + deps = ["//deps/amqp_client:erlang_app", "//deps/rabbitmq_ct_helpers:erlang_app"], + ) diff --git a/deps/rabbit/priv/schema/rabbit.schema b/deps/rabbit/priv/schema/rabbit.schema index 977861f054f4..f2922a235561 100644 --- a/deps/rabbit/priv/schema/rabbit.schema +++ b/deps/rabbit/priv/schema/rabbit.schema @@ -427,6 +427,10 @@ end}. {mapping, "ssl_options.bypass_pem_cache", "ssl.bypass_pem_cache", [{datatype, {enum, [true, false]}}]}. + +{mapping, "metadata_store.khepri.default_timeout", "rabbit.khepri_default_timeout", + [{datatype, integer}]}. + %% =========================================================================== %% Choose the available SASL mechanism(s) to expose. diff --git a/deps/rabbit/src/mirrored_supervisor.hrl b/deps/rabbit/src/mirrored_supervisor.hrl new file mode 100644 index 000000000000..5287d1508e2a --- /dev/null +++ b/deps/rabbit/src/mirrored_supervisor.hrl @@ -0,0 +1 @@ +-record(mirrored_sup_childspec, {key, mirroring_pid, childspec}). diff --git a/deps/rabbit/src/rabbit.erl b/deps/rabbit/src/rabbit.erl index 18339b3dc137..63f50126adc5 100644 --- a/deps/rabbit/src/rabbit.erl +++ b/deps/rabbit/src/rabbit.erl @@ -356,14 +356,40 @@ run_prelaunch_second_phase() -> %% 3. Logging. ok = rabbit_prelaunch_logging:setup(Context), - %% 4. Clustering. - ok = rabbit_prelaunch_cluster:setup(Context), + %% The clustering steps requires Khepri to be started to check for + %% consistency. This is the opposite compared to Mnesia which must be + %% stopped. That's why we setup Khepri and the coordination Ra system it + %% depends on before, but only handle Mnesia after. + %% + %% We also always set it up, even when using Mnesia, to ensure it is ready + %% if/when the migration begins. + %% + %% Note that this is only the Khepri store which is started here. We + %% perform additional initialization steps in `rabbit_db:init()' which is + %% triggered from a boot step. This boot step handles both Mnesia and + %% Khepri and synchronizes the feature flags. + %% + %% To sum up: + %% 1. We start the Khepri store (always) + %% 2. We verify the cluster, including the feature flags compatibility + %% 3. We start Mnesia (if Khepri is unused) + %% 4. We synchronize feature flags in `rabbit_db:init()' + %% 4. We finish to initialize either Mnesia or Khepri in `rabbit_db:init()' + ok = rabbit_ra_systems:setup(Context), + ok = rabbit_khepri:setup(Context), - %% Start Mnesia now that everything is ready. - ?LOG_DEBUG("Starting Mnesia"), - ok = mnesia:start(), + %% 4. Clustering checks. This covers the compatibility between nodes, + %% feature-flags-wise. + ok = rabbit_prelaunch_cluster:setup(Context), - ok = rabbit_ra_systems:setup(Context), + case rabbit_khepri:is_enabled() of + true -> + ok; + false -> + %% Start Mnesia now that everything is ready. + ?LOG_DEBUG("Starting Mnesia"), + ok = mnesia:start() + end, ?LOG_DEBUG(""), ?LOG_DEBUG("== Prelaunch DONE =="), @@ -1015,7 +1041,13 @@ do_run_postlaunch_phase(Plugins) -> ok = log_broker_started(StrictlyPlugins), ?LOG_DEBUG("Marking ~ts as running", [product_name()]), - rabbit_boot_state:set(ready) + rabbit_boot_state:set(ready), + + %% Now that everything is ready, trigger the garbage collector. With + %% Khepri enabled, it seems to be more important than before; see #5515 + %% for context. + _ = rabbit_runtime:gc_all_processes(), + ok catch throw:{error, _} = Error -> rabbit_prelaunch_errors:log_error(Error), diff --git a/deps/rabbit/src/rabbit_amqqueue.erl b/deps/rabbit/src/rabbit_amqqueue.erl index f2df0d8695e9..e57d1843e871 100644 --- a/deps/rabbit/src/rabbit_amqqueue.erl +++ b/deps/rabbit/src/rabbit_amqqueue.erl @@ -34,7 +34,7 @@ -export([notify_sent/2, notify_sent_queue_down/1, resume/2]). -export([notify_down_all/2, notify_down_all/3, activate_limit_all/2, credit/5]). -export([on_node_up/1, on_node_down/1]). --export([update/2, store_queue/1, update_decorators/1, policy_changed/2]). +-export([update/2, store_queue/1, update_decorators/2, policy_changed/2]). -export([update_mirroring/1, sync_mirrors/1, cancel_sync_mirrors/1]). -export([emit_unresponsive/6, emit_unresponsive_local/5, is_unresponsive/2]). -export([has_synchronised_mirrors_online/1, is_match/2, is_in_virtual_host/2]). @@ -263,6 +263,12 @@ internal_declare(Q, Recover) -> do_internal_declare(Q, Recover). do_internal_declare(Q0, true) -> + %% TODO Why do we return the old state instead of the actual one? + %% I'm leaving it like it was before the khepri refactor, because + %% rabbit_amqqueue_process:init_it2 compares the result of this declare to decide + %% if continue or stop. If we return the actual one, it fails and the queue stops + %% silently during init. + %% Maybe we should review this bit of code at some point. Q = amqqueue:set_state(Q0, live), ok = store_queue(Q), {created, Q0}; @@ -289,10 +295,11 @@ store_queue(Q0) -> Q = rabbit_queue_decorator:set(Q0), rabbit_db_queue:set(Q). --spec update_decorators(name()) -> 'ok'. +-spec update_decorators(name(), [Decorator]) -> 'ok' when + Decorator :: atom(). -update_decorators(Name) -> - rabbit_db_queue:update_decorators(Name). +update_decorators(Name, Decorators) -> + rabbit_db_queue:update_decorators(Name, Decorators). -spec policy_changed(amqqueue:amqqueue(), amqqueue:amqqueue()) -> 'ok'. @@ -1791,6 +1798,8 @@ internal_delete(Queue, ActingUser, Reason) -> -spec forget_all_durable(node()) -> 'ok'. +%% TODO this is used by `rabbit_mnesia:remove_node_if_mnesia_running` +%% Does it make any sense once mnesia is not used/removed? forget_all_durable(Node) -> UpdateFun = fun(Q) -> forget_node_for_queue(Node, Q) diff --git a/deps/rabbit/src/rabbit_amqqueue_process.erl b/deps/rabbit/src/rabbit_amqqueue_process.erl index ac2ca6f1fb81..42ee87b40edc 100644 --- a/deps/rabbit/src/rabbit_amqqueue_process.erl +++ b/deps/rabbit/src/rabbit_amqqueue_process.erl @@ -1678,13 +1678,37 @@ handle_cast(policy_changed, State = #q{q = Q0}) -> Name = amqqueue:get_name(Q0), %% We depend on the #q.q field being up to date at least WRT %% policy (but not mirror pids) in various places, so when it - %% changes we go and read it from Mnesia again. + %% changes we go and read it from the database again. %% %% This also has the side effect of waking us up so we emit a %% stats event - so event consumers see the changed policy. {ok, Q} = rabbit_amqqueue:lookup(Name), noreply(process_args_policy(State#q{q = Q})); +handle_cast({policy_changed, Q0}, State) -> + Name = amqqueue:get_name(Q0), + PolicyVersion0 = amqqueue:get_policy_version(Q0), + %% We depend on the #q.q field being up to date at least WRT + %% policy (but not mirror pids) in various places, so when it + %% changes we go and read it from the database again. + %% + %% This also has the side effect of waking us up so we emit a + %% stats event - so event consumers see the changed policy. + {ok, Q} = rabbit_amqqueue:lookup(Name), + PolicyVersion = amqqueue:get_policy_version(Q), + case PolicyVersion >= PolicyVersion0 of + true -> + noreply(process_args_policy(State#q{q = Q})); + false -> + %% Update just the policy, as pids and mirrors could have been + %% updated simultaneously. A testcase on the `confirm_rejects_SUITE` + %% fails consistently if the internal state is updated directly to `Q0`. + Q1 = amqqueue:set_policy(Q, amqqueue:get_policy(Q0)), + Q2 = amqqueue:set_operator_policy(Q1, amqqueue:get_operator_policy(Q0)), + Q3 = amqqueue:set_policy_version(Q2, PolicyVersion0), + noreply(process_args_policy(State#q{q = Q3})) + end; + handle_cast({sync_start, _, _}, State = #q{q = Q}) -> Name = amqqueue:get_name(Q), %% Only a mirror should receive this, it means we are a duplicated master diff --git a/deps/rabbit/src/rabbit_auth_backend_internal.erl b/deps/rabbit/src/rabbit_auth_backend_internal.erl index 196fa1b7f4ef..2482bd1a768e 100644 --- a/deps/rabbit/src/rabbit_auth_backend_internal.erl +++ b/deps/rabbit/src/rabbit_auth_backend_internal.erl @@ -30,6 +30,8 @@ -export([user_info_keys/0, perms_info_keys/0, user_perms_info_keys/0, vhost_perms_info_keys/0, user_vhost_perms_info_keys/0, all_users/0, + user_topic_perms_info_keys/0, vhost_topic_perms_info_keys/0, + user_vhost_topic_perms_info_keys/0, list_users/0, list_users/2, list_permissions/0, list_user_permissions/1, list_user_permissions/3, list_topic_permissions/0, @@ -39,9 +41,13 @@ -export([state_can_expire/0]). -%% for testing -export([hashing_module_for_user/1, expand_topic_permission/2]). +-ifdef(TEST). +-export([extract_user_permission_params/2, + extract_topic_permission_params/2]). +-endif. + -import(rabbit_data_coercion, [to_atom/1, to_list/1, to_binary/1]). %%---------------------------------------------------------------------------- @@ -292,7 +298,10 @@ delete_user(Username, ActingUser) -> {user_who_performed_action, ActingUser}]), ok; false -> - ok + ok; + Error0 -> + rabbit_log:info("Failed to delete user '~ts': ~tp", [Username, Error0]), + throw(Error0) end catch Class:Error:Stacktrace -> diff --git a/deps/rabbit/src/rabbit_binding.erl b/deps/rabbit/src/rabbit_binding.erl index 81f2385f3746..cdd4cb1beba8 100644 --- a/deps/rabbit/src/rabbit_binding.erl +++ b/deps/rabbit/src/rabbit_binding.erl @@ -382,7 +382,11 @@ combine_deletions(Deletions1, Deletions2) -> merge_entry({X1, Deleted1, Bindings1}, {X2, Deleted2, Bindings2}) -> {anything_but(undefined, X1, X2), anything_but(not_deleted, Deleted1, Deleted2), - [Bindings1 | Bindings2]}. + Bindings1 ++ Bindings2}; +merge_entry({X1, Deleted1, Bindings1, none}, {X2, Deleted2, Bindings2, none}) -> + {anything_but(undefined, X1, X2), + anything_but(not_deleted, Deleted1, Deleted2), + Bindings1 ++ Bindings2, none}. notify_deletions({error, not_found}, _) -> ok; diff --git a/deps/rabbit/src/rabbit_classic_queue.erl b/deps/rabbit/src/rabbit_classic_queue.erl index e39b01dda434..58b875f7b38a 100644 --- a/deps/rabbit/src/rabbit_classic_queue.erl +++ b/deps/rabbit/src/rabbit_classic_queue.erl @@ -51,7 +51,8 @@ -export([delete_crashed/1, delete_crashed/2, - delete_crashed_internal/2]). + delete_crashed_internal/2, + delete_crashed_in_backing_queue/1]). -export([confirm_to_sender/3, send_rejection/3, @@ -169,7 +170,19 @@ find_missing_queues([Q1|Rem1], [Q2|Rem2] = Q2s, Acc) -> -spec policy_changed(amqqueue:amqqueue()) -> ok. policy_changed(Q) -> QPid = amqqueue:get_pid(Q), - gen_server2:cast(QPid, policy_changed). + case rabbit_khepri:is_enabled() of + false -> + gen_server2:cast(QPid, policy_changed); + true -> + %% When using Khepri, projections are guaranteed to be atomic on + %% the node that processes them, but there might be a slight delay + %% until they're applied on other nodes. Some test suites fail + %% intermittently, showing that rabbit_amqqueue_process is reading + %% the old policy value. We use the khepri ff to hide this API change, + %% and use the up-to-date record to update the policy on the gen_server + %% state. + gen_server2:cast(QPid, {policy_changed, Q}) + end. stat(Q) -> delegate:invoke(amqqueue:get_pid(Q), @@ -465,14 +478,30 @@ delete_crashed(Q) -> delete_crashed(Q, ?INTERNAL_USER). delete_crashed(Q, ActingUser) -> - ok = rpc:call(amqqueue:qnode(Q), ?MODULE, delete_crashed_internal, - [Q, ActingUser]). + %% Delete from `rabbit_db_queue' from the queue's node. The deletion's + %% change to the Khepri projection is immediately consistent on that node, + %% so the call will block until that node has fully deleted and forgotten + %% about the queue. + Ret = rpc:call(amqqueue:qnode(Q), ?MODULE, delete_crashed_in_backing_queue, + [Q]), + case Ret of + {badrpc, {'EXIT', {undef, _}}} -> + %% Compatibility: if the remote node doesn't yet expose this + %% function, call it directly on this node. + ok = delete_crashed_in_backing_queue(Q); + ok -> + ok + end, + ok = rabbit_amqqueue:internal_delete(Q, ActingUser). delete_crashed_internal(Q, ActingUser) -> - {ok, BQ} = application:get_env(rabbit, backing_queue_module), - BQ:delete_crashed(Q), + delete_crashed_in_backing_queue(Q), ok = rabbit_amqqueue:internal_delete(Q, ActingUser). +delete_crashed_in_backing_queue(Q) -> + {ok, BQ} = application:get_env(rabbit, backing_queue_module), + BQ:delete_crashed(Q). + recover_durable_queues(QueuesAndRecoveryTerms) -> {Results, Failures} = gen_server2:mcall( diff --git a/deps/rabbit/src/rabbit_core_ff.erl b/deps/rabbit/src/rabbit_core_ff.erl index 05c19f2c76ad..615cc1a2eb1a 100644 --- a/deps/rabbit/src/rabbit_core_ff.erl +++ b/deps/rabbit/src/rabbit_core_ff.erl @@ -126,3 +126,25 @@ stability => stable, depends_on => [feature_flags_v2] }}). + +-rabbit_feature_flag( + {khepri_db, + #{desc => "Use the new Khepri Raft-based metadata store", + doc_url => "", %% TODO + stability => experimental, + depends_on => [feature_flags_v2, + direct_exchange_routing_v2, + maintenance_mode_status, + user_limits, + virtual_host_metadata, + tracking_records_in_ets, + listener_records_in_ets, + + %% Deprecated features. + classic_queue_mirroring, + ram_node_type], + callbacks => #{enable => + {rabbit_khepri, khepri_db_migration_enable}, + post_enable => + {rabbit_khepri, khepri_db_migration_post_enable}} + }}). diff --git a/deps/rabbit/src/rabbit_db.erl b/deps/rabbit/src/rabbit_db.erl index 3dbb681b533c..4fdd3b328b13 100644 --- a/deps/rabbit/src/rabbit_db.erl +++ b/deps/rabbit/src/rabbit_db.erl @@ -7,6 +7,8 @@ -module(rabbit_db). +-include_lib("khepri/include/khepri.hrl"). + -include_lib("kernel/include/logger.hrl"). -include_lib("stdlib/include/assert.hrl"). @@ -22,7 +24,9 @@ %% Exported to be used by various rabbit_db_* modules -export([ - list_in_mnesia/2 + list_in_mnesia/2, + list_in_khepri/1, + list_in_khepri/2 ]). %% Default timeout for operations on remote nodes. @@ -48,7 +52,10 @@ init() -> pre_init(IsVirgin), - Ret = init_using_mnesia(), + Ret = case rabbit_khepri:is_enabled() of + true -> init_using_khepri(); + false -> init_using_mnesia() + end, case Ret of ok -> ?LOG_DEBUG( @@ -84,12 +91,26 @@ init_using_mnesia() -> ?assertEqual(rabbit:data_dir(), mnesia_dir()), rabbit_sup:start_child(mnesia_sync). +init_using_khepri() -> + case rabbit_khepri:members() of + [] -> + timer:sleep(1000), + init_using_khepri(); + Members -> + ?LOG_WARNING( + "Found the following metadata store members: ~p", [Members], + #{domain => ?RMQLOG_DOMAIN_DB}) + end. + -spec reset() -> Ret when Ret :: ok. %% @doc Resets the database and the node. reset() -> - ok = reset_using_mnesia(), + ok = case rabbit_khepri:is_enabled() of + true -> reset_using_khepri(); + false -> reset_using_mnesia() + end, post_reset(). reset_using_mnesia() -> @@ -98,12 +119,21 @@ reset_using_mnesia() -> #{domain => ?RMQLOG_DOMAIN_DB}), rabbit_mnesia:reset(). +reset_using_khepri() -> + ?LOG_DEBUG( + "DB: resetting node (using Khepri)", + #{domain => ?RMQLOG_DOMAIN_DB}), + rabbit_khepri:reset(). + -spec force_reset() -> Ret when Ret :: ok. %% @doc Resets the database and the node. force_reset() -> - ok = force_reset_using_mnesia(), + ok = case rabbit_khepri:is_enabled() of + true -> force_reset_using_khepri(); + false -> force_reset_using_mnesia() + end, post_reset(). force_reset_using_mnesia() -> @@ -112,6 +142,12 @@ force_reset_using_mnesia() -> #{domain => ?RMQLOG_DOMAIN_DB}), rabbit_mnesia:force_reset(). +force_reset_using_khepri() -> + ?LOG_DEBUG( + "DB: resetting node forcefully (using Khepri)", + #{domain => ?RMQLOG_DOMAIN_DB}), + rabbit_khepri:force_reset(). + -spec force_load_on_next_boot() -> Ret when Ret :: ok. %% @doc Requests that the database to be forcefully loaded during next boot. @@ -120,7 +156,14 @@ force_reset_using_mnesia() -> %% state, like if critical members are MIA. force_load_on_next_boot() -> - force_load_on_next_boot_using_mnesia(). + %% TODO force load using Khepri might need to be implemented for disaster + %% recovery scenarios where just a minority of nodes are accessible. + %% Potentially, it could also be replaced with a way to export all the + %% data. + case rabbit_khepri:is_enabled() of + true -> ok; + false -> force_load_on_next_boot_using_mnesia() + end. force_load_on_next_boot_using_mnesia() -> ?LOG_DEBUG( @@ -145,11 +188,20 @@ post_reset() -> %% @see is_virgin_node/1. is_virgin_node() -> - is_virgin_node_using_mnesia(). + case rabbit_khepri:is_enabled() of + true -> is_virgin_node_using_khepri(); + false -> is_virgin_node_using_mnesia() + end. is_virgin_node_using_mnesia() -> rabbit_mnesia:is_virgin_node(). +is_virgin_node_using_khepri() -> + case rabbit_khepri:is_empty() of + {error, _} -> true; + IsEmpty -> IsEmpty + end. + -spec is_virgin_node(Node) -> IsVirgin | undefined when Node :: node(), IsVirgin :: boolean(). @@ -171,6 +223,10 @@ is_virgin_node(Node) when is_atom(Node) -> undefined end. +%% ------------------------------------------------------------------- +%% dir(). +%% ------------------------------------------------------------------- + -spec dir() -> DBDir when DBDir :: file:filename(). %% @doc Returns the directory where the database stores its data. @@ -178,11 +234,21 @@ is_virgin_node(Node) when is_atom(Node) -> %% @returns the directory path. dir() -> - mnesia_dir(). + case rabbit_khepri:is_enabled() of + true -> khepri_dir(); + false -> mnesia_dir() + end. mnesia_dir() -> rabbit_mnesia:dir(). +khepri_dir() -> + rabbit_khepri:dir(). + +%% ------------------------------------------------------------------- +%% ensure_dir_exists(). +%% ------------------------------------------------------------------- + -spec ensure_dir_exists() -> ok | no_return(). %% @doc Ensures the database directory exists. %% @@ -210,3 +276,25 @@ list_in_mnesia(Table, Match) -> %% Not dirty_match_object since that would not be transactional when used %% in a tx context mnesia:async_dirty(fun () -> mnesia:match_object(Table, Match, read) end). + +%% ------------------------------------------------------------------- +%% list_in_khepri(). +%% ------------------------------------------------------------------- + +-spec list_in_khepri(Path) -> Objects when + Path :: khepri_path:pattern(), + Objects :: [term()]. + +list_in_khepri(Path) -> + list_in_khepri(Path, #{}). + +-spec list_in_khepri(Path, Options) -> Objects when + Path :: khepri_path:pattern(), + Options :: map(), + Objects :: [term()]. + +list_in_khepri(Path, Options) -> + case rabbit_khepri:match(Path, Options) of + {ok, Map} -> maps:values(Map); + _ -> [] + end. diff --git a/deps/rabbit/src/rabbit_db_binding.erl b/deps/rabbit/src/rabbit_db_binding.erl index a320c9d44ca1..287fa2cbf39b 100644 --- a/deps/rabbit/src/rabbit_db_binding.erl +++ b/deps/rabbit/src/rabbit_db_binding.erl @@ -7,6 +7,7 @@ -module(rabbit_db_binding). +-include_lib("khepri/include/khepri.hrl"). -include_lib("rabbit_common/include/rabbit.hrl"). -export([exists/1, @@ -25,11 +26,23 @@ %% Exported to be used by various rabbit_db_* modules -export([ delete_for_destination_in_mnesia/2, + delete_for_destination_in_khepri/2, delete_all_for_exchange_in_mnesia/3, + delete_all_for_exchange_in_khepri/3, delete_transient_for_destination_in_mnesia/1, - has_for_source_in_mnesia/1 + has_for_source_in_mnesia/1, + has_for_source_in_khepri/1, + match_source_and_destination_in_khepri_tx/2 ]). +-export([ + khepri_route_path/1, + khepri_routes_path/0, + khepri_route_exchange_path/1 + ]). + +%% Recovery is only needed for transient entities. Once mnesia is removed, these +%% functions can be deleted -export([recover/0, recover/1]). %% For testing @@ -55,7 +68,10 @@ %% @private exists(Binding) -> - exists_in_mnesia(Binding). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> exists_in_mnesia(Binding) end, + khepri => fun() -> exists_in_khepri(Binding) end + }). exists_in_mnesia(Binding) -> binding_action_in_mnesia( @@ -85,6 +101,45 @@ not_found_or_absent_errs_in_mnesia(Names) -> Errs = [not_found_or_absent_in_mnesia(Name) || Name <- Names], rabbit_misc:const({error, {resources_missing, Errs}}). +exists_in_khepri(#binding{source = SrcName, + destination = DstName} = Binding) -> + Path = khepri_route_path(Binding), + case rabbit_khepri:transaction( + fun () -> + case {lookup_resource_in_khepri_tx(SrcName), + lookup_resource_in_khepri_tx(DstName)} of + {[_Src], [_Dst]} -> + case khepri_tx:get(Path) of + {ok, Set} -> + {ok, Set}; + _ -> + {ok, not_found} + end; + Errs -> + Errs + end + end) of + {ok, not_found} -> false; + {ok, Set} -> sets:is_element(Binding, Set); + Errs -> not_found_errs_in_khepri(not_found(Errs, SrcName, DstName)) + end. + +lookup_resource_in_khepri_tx(#resource{kind = queue} = Name) -> + rabbit_db_queue:get_in_khepri_tx(Name); +lookup_resource_in_khepri_tx(#resource{kind = exchange} = Name) -> + rabbit_db_exchange:get_in_khepri_tx(Name). + +not_found_errs_in_khepri(Names) -> + Errs = [{not_found, Name} || Name <- Names], + {error, {resources_missing, Errs}}. + +not_found({[], [_]}, SrcName, _) -> + [SrcName]; +not_found({[_], []}, _, DstName) -> + [DstName]; +not_found({[], []}, SrcName, DstName) -> + [SrcName, DstName]. + %% ------------------------------------------------------------------- %% create(). %% ------------------------------------------------------------------- @@ -103,7 +158,10 @@ not_found_or_absent_errs_in_mnesia(Names) -> %% @private create(Binding, ChecksFun) -> - create_in_mnesia(Binding, ChecksFun). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> create_in_mnesia(Binding, ChecksFun) end, + khepri => fun() -> create_in_khepri(Binding, ChecksFun) end + }). create_in_mnesia(Binding, ChecksFun) -> binding_action_in_mnesia( @@ -130,6 +188,63 @@ create_in_mnesia(Binding, ChecksFun) -> end end, fun not_found_or_absent_errs_in_mnesia/1). +create_in_khepri(#binding{source = SrcName, + destination = DstName} = Binding, ChecksFun) -> + case {lookup_resource(SrcName), lookup_resource(DstName)} of + {[Src], [Dst]} -> + case ChecksFun(Src, Dst) of + ok -> + RoutePath = khepri_route_path(Binding), + MaybeSerial = rabbit_exchange:serialise_events(Src), + Serial = rabbit_khepri:transaction( + fun() -> + ExchangePath = khepri_route_exchange_path(SrcName), + ok = khepri_tx:put(ExchangePath, #{type => Src#exchange.type}), + case khepri_tx:get(RoutePath) of + {ok, Set} -> + case sets:is_element(Binding, Set) of + true -> + already_exists; + false -> + ok = khepri_tx:put(RoutePath, sets:add_element(Binding, Set)), + serial_in_khepri(MaybeSerial, Src) + end; + _ -> + ok = khepri_tx:put(RoutePath, sets:add_element(Binding, sets:new([{version, 2}]))), + serial_in_khepri(MaybeSerial, Src) + end + end, rw), + case Serial of + already_exists -> + ok; + {error, _} = Error -> + Error; + _ -> + rabbit_exchange:callback(Src, add_binding, Serial, [Src, Binding]) + end; + {error, _} = Err -> + Err + end; + Errs -> + not_found_errs_in_khepri(not_found(Errs, SrcName, DstName)) + end. + +lookup_resource(#resource{kind = queue} = Name) -> + case rabbit_db_queue:get(Name) of + {error, _} -> []; + {ok, Q} -> [Q] + end; +lookup_resource(#resource{kind = exchange} = Name) -> + case rabbit_db_exchange:get(Name) of + {ok, X} -> [X]; + _ -> [] + end. + +serial_in_khepri(false, _) -> + none; +serial_in_khepri(true, X) -> + rabbit_db_exchange:next_serial_in_khepri_tx(X). + %% ------------------------------------------------------------------- %% delete(). %% ------------------------------------------------------------------- @@ -146,7 +261,10 @@ create_in_mnesia(Binding, ChecksFun) -> %% @private delete(Binding, ChecksFun) -> - delete_in_mnesia(Binding, ChecksFun). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> delete_in_mnesia(Binding, ChecksFun) end, + khepri => fun() -> delete_in_khepri(Binding, ChecksFun) end + }). delete_in_mnesia(Binding, ChecksFun) -> binding_action_in_mnesia( @@ -173,7 +291,7 @@ delete_in_mnesia(Binding, ChecksFun) -> Src :: rabbit_types:exchange() | amqqueue:amqqueue(), Dst :: rabbit_types:exchange() | amqqueue:amqqueue(), Binding :: rabbit_types:binding(), - Ret :: fun(() -> rabbit_binding:deletions()). + Ret :: fun(() -> {ok, rabbit_binding:deletions()}). delete_in_mnesia(Src, Dst, B) -> ok = sync_route(#route{binding = B}, rabbit_binding:binding_type(Src, Dst), should_index_table(Src), fun delete/3), @@ -199,6 +317,73 @@ not_found_or_absent_in_mnesia(#resource{kind = queue} = Name) -> {ok, Q} -> {absent, Q, nodedown} end. +delete_in_khepri(#binding{source = SrcName, + destination = DstName} = Binding, ChecksFun) -> + Path = khepri_route_path(Binding), + case rabbit_khepri:transaction( + fun () -> + case {lookup_resource_in_khepri_tx(SrcName), + lookup_resource_in_khepri_tx(DstName)} of + {[Src], [Dst]} -> + case exists_in_khepri(Path, Binding) of + false -> + ok; + true -> + case ChecksFun(Src, Dst) of + ok -> + ok = delete_in_khepri(Binding), + maybe_auto_delete_exchange_in_khepri(Binding#binding.source, [Binding], rabbit_binding:new_deletions(), false); + {error, _} = Err -> + Err + end + end; + _Errs -> + %% No absent queues, always present on disk + ok + end + end) of + ok -> + ok; + {error, _} = Err -> + Err; + Deletions -> + {ok, rabbit_binding:process_deletions(Deletions)} + end. + +exists_in_khepri(Path, Binding) -> + case khepri_tx:get(Path) of + {ok, Set} -> + sets:is_element(Binding, Set); + _ -> + false + end. + +delete_in_khepri(Binding) -> + Path = khepri_route_path(Binding), + case khepri_tx:get(Path) of + {ok, Set0} -> + Set = sets:del_element(Binding, Set0), + case sets:is_empty(Set) of + true -> + ok = khepri_tx:delete(Path); + false -> + ok = khepri_tx:put(Path, Set) + end; + _ -> + ok + end. + +maybe_auto_delete_exchange_in_khepri(XName, Bindings, Deletions, OnlyDurable) -> + {Entry, Deletions1} = + case rabbit_db_exchange:maybe_auto_delete_in_khepri(XName, OnlyDurable) of + {not_deleted, X} -> + {{X, not_deleted, Bindings}, Deletions}; + {deleted, X, Deletions2} -> + {{X, deleted, Bindings}, + rabbit_binding:combine_deletions(Deletions, Deletions2)} + end, + rabbit_binding:add_deletion(XName, Entry, Deletions1). + %% ------------------------------------------------------------------- %% get_all(). %% ------------------------------------------------------------------- @@ -213,7 +398,10 @@ not_found_or_absent_in_mnesia(#resource{kind = queue} = Name) -> %% @private get_all() -> - get_all_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_in_mnesia() end, + khepri => fun() -> get_all_in_khepri() end + }). get_all_in_mnesia() -> mnesia:async_dirty( @@ -222,6 +410,9 @@ get_all_in_mnesia() -> [B || #route{binding = B} <- AllRoutes] end). +get_all_in_khepri() -> + [B || #route{binding = B} <- ets:tab2list(rabbit_khepri_bindings)]. + -spec get_all(VHostName) -> [Binding] when VHostName :: vhost:name(), Binding :: rabbit_types:binding(). @@ -232,7 +423,10 @@ get_all_in_mnesia() -> %% @private get_all(VHost) -> - get_all_in_mnesia(VHost). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_in_mnesia(VHost) end, + khepri => fun() -> get_all_in_khepri(VHost) end + }). get_all_in_mnesia(VHost) -> VHostResource = rabbit_misc:r(VHost, '_'), @@ -242,6 +436,13 @@ get_all_in_mnesia(VHost) -> _ = '_'}, [B || #route{binding = B} <- rabbit_db:list_in_mnesia(?MNESIA_TABLE, Match)]. +get_all_in_khepri(VHost) -> + VHostResource = rabbit_misc:r(VHost, '_'), + Match = #route{binding = #binding{source = VHostResource, + destination = VHostResource, + _ = '_'}}, + [B || #route{binding = B} <- ets:match_object(rabbit_khepri_bindings, Match)]. + -spec get_all(Src, Dst, Reverse) -> [Binding] when Src :: rabbit_types:binding_source(), Dst :: rabbit_types:binding_destination(), @@ -255,7 +456,10 @@ get_all_in_mnesia(VHost) -> %% @private get_all(SrcName, DstName, Reverse) -> - get_all_in_mnesia(SrcName, DstName, Reverse). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_in_mnesia(SrcName, DstName, Reverse) end, + khepri => fun() -> get_all_in_khepri(SrcName, DstName) end + }). get_all_in_mnesia(SrcName, DstName, Reverse) -> Route = #route{binding = #binding{source = SrcName, @@ -264,6 +468,12 @@ get_all_in_mnesia(SrcName, DstName, Reverse) -> Fun = list_for_route(Route, Reverse), mnesia:async_dirty(Fun). +get_all_in_khepri(SrcName, DstName) -> + MatchHead = #route{binding = #binding{source = SrcName, + destination = DstName, + _ = '_'}}, + [B || #route{binding = B} <- ets:match_object(rabbit_khepri_bindings, MatchHead)]. + %% ------------------------------------------------------------------- %% get_all_for_source(). %% ------------------------------------------------------------------- @@ -278,7 +488,10 @@ get_all_in_mnesia(SrcName, DstName, Reverse) -> %% @private get_all_for_source(Resource) -> - get_all_for_source_in_mnesia(Resource). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_for_source_in_mnesia(Resource) end, + khepri => fun() -> get_all_for_source_in_khepri(Resource) end + }). get_all_for_source_in_mnesia(Resource) -> Route = #route{binding = #binding{source = Resource, _ = '_'}}, @@ -297,6 +510,10 @@ list_for_route(Route, true) -> rabbit_binding:reverse_route(Route), read)] end. +get_all_for_source_in_khepri(Resource) -> + Route = #route{binding = #binding{source = Resource, _ = '_'}}, + [B || #route{binding = B} <- ets:match_object(rabbit_khepri_bindings, Route)]. + %% ------------------------------------------------------------------- %% get_all_for_destination(). %% ------------------------------------------------------------------- @@ -312,7 +529,10 @@ list_for_route(Route, true) -> %% @private get_all_for_destination(Dst) -> - get_all_for_destination_in_mnesia(Dst). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_for_destination_in_mnesia(Dst) end, + khepri => fun() -> get_all_for_destination_in_khepri(Dst) end + }). get_all_for_destination_in_mnesia(Dst) -> Route = #route{binding = #binding{destination = Dst, @@ -320,6 +540,11 @@ get_all_for_destination_in_mnesia(Dst) -> Fun = list_for_route(Route, true), mnesia:async_dirty(Fun). +get_all_for_destination_in_khepri(Destination) -> + Match = #route{binding = #binding{destination = Destination, + _ = '_'}}, + [B || #route{binding = B} <- ets:match_object(rabbit_khepri_bindings, Match)]. + %% ------------------------------------------------------------------- %% fold(). %% ------------------------------------------------------------------- @@ -338,13 +563,29 @@ get_all_for_destination_in_mnesia(Dst) -> %% @private fold(Fun, Acc) -> - fold_in_mnesia(Fun, Acc). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> fold_in_mnesia(Fun, Acc) end, + khepri => fun() -> fold_in_khepri(Fun, Acc) end + }). fold_in_mnesia(Fun, Acc) -> ets:foldl(fun(#route{binding = Binding}, Acc0) -> Fun(Binding, Acc0) end, Acc, ?MNESIA_TABLE). +fold_in_khepri(Fun, Acc) -> + Path = khepri_routes_path() ++ [_VHost = ?KHEPRI_WILDCARD_STAR, + _SrcName = ?KHEPRI_WILDCARD_STAR, + rabbit_khepri:if_has_data_wildcard()], + {ok, Res} = rabbit_khepri:fold( + Path, + fun(_, #{data := SetOfBindings}, Acc0) -> + lists:foldl(fun(Binding, Acc1) -> + Fun(Binding, Acc1) + end, Acc0, sets:to_list(SetOfBindings)) + end, Acc), + Res. + %% Routing - HOT CODE PATH %% ------------------------------------------------------------------- %% match(). @@ -363,7 +604,10 @@ fold_in_mnesia(Fun, Acc) -> %% @private match(SrcName, Match) -> - match_in_mnesia(SrcName, Match). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> match_in_mnesia(SrcName, Match) end, + khepri => fun() -> match_in_khepri(SrcName, Match) end + }). match_in_mnesia(SrcName, Match) -> MatchHead = #route{binding = #binding{source = SrcName, @@ -372,6 +616,12 @@ match_in_mnesia(SrcName, Match) -> [Dest || [#route{binding = Binding = #binding{destination = Dest}}] <- Routes, Match(Binding)]. +match_in_khepri(SrcName, Match) -> + MatchHead = #route{binding = #binding{source = SrcName, + _ = '_'}}, + Routes = ets:select(rabbit_khepri_bindings, [{MatchHead, [], [['$_']]}]), + [Dest || [#route{binding = Binding = #binding{destination = Dest}}] <- + Routes, Match(Binding)]. %% Routing - HOT CODE PATH %% ------------------------------------------------------------------- @@ -391,7 +641,10 @@ match_in_mnesia(SrcName, Match) -> %% @private match_routing_key(SrcName, RoutingKeys, UseIndex) -> - match_routing_key_in_mnesia(SrcName, RoutingKeys, UseIndex). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> match_routing_key_in_mnesia(SrcName, RoutingKeys, UseIndex) end, + khepri => fun() -> match_routing_key_in_khepri(SrcName, RoutingKeys) end + }). match_routing_key_in_mnesia(SrcName, RoutingKeys, UseIndex) -> case UseIndex of @@ -401,6 +654,26 @@ match_routing_key_in_mnesia(SrcName, RoutingKeys, UseIndex) -> route_in_mnesia_v1(SrcName, RoutingKeys) end. +match_routing_key_in_khepri(Src, ['_']) -> + MatchHead = #index_route{source_key = {Src, '_'}, + destination = '$1', + _ = '_'}, + ets:select(rabbit_khepri_index_route, [{MatchHead, [], ['$1']}]); + +match_routing_key_in_khepri(Src, RoutingKeys) -> + lists:foldl( + fun(RK, Acc) -> + try + Dst = ets:lookup_element( + rabbit_khepri_index_route, + {Src, RK}, + #index_route.destination), + Dst ++ Acc + catch + _:_:_ -> Acc + end + end, [], RoutingKeys). + %% ------------------------------------------------------------------- %% recover(). %% ------------------------------------------------------------------- @@ -411,7 +684,11 @@ match_routing_key_in_mnesia(SrcName, RoutingKeys, UseIndex) -> %% @private recover() -> - recover_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> recover_in_mnesia() end, + %% Nothing to do in khepri, single table storage + khepri => ok + }). recover_in_mnesia() -> rabbit_mnesia:execute_mnesia_transaction( @@ -438,7 +715,10 @@ recover_in_mnesia() -> %% @private recover(RecoverFun) -> - recover_in_mnesia(RecoverFun). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> recover_in_mnesia(RecoverFun) end, + khepri => ok + }). recover_in_mnesia(RecoverFun) -> _ = [RecoverFun(Route, Src, Dst, fun recover_semi_durable_route/2) || @@ -482,6 +762,33 @@ delete_for_source_in_mnesia(SrcName, ShouldIndexTable) -> mnesia:dirty_match_object(?MNESIA_SEMI_DURABLE_TABLE, Match)), ShouldIndexTable). +%% ------------------------------------------------------------------- +%% delete_all_for_exchange_in_khepri(). +%% ------------------------------------------------------------------- + +-spec delete_all_for_exchange_in_khepri(Exchange, OnlyDurable, RemoveBindingsForSource) + -> Ret when + Exchange :: rabbit_types:exchange(), + OnlyDurable :: boolean(), + RemoveBindingsForSource :: boolean(), + Binding :: rabbit_types:binding(), + Ret :: {deleted, Exchange, [Binding], rabbit_binding:deletions()}. + +delete_all_for_exchange_in_khepri(X = #exchange{name = XName}, OnlyDurable, RemoveBindingsForSource) -> + Bindings = case RemoveBindingsForSource of + true -> delete_for_source_in_khepri(XName); + false -> [] + end, + {deleted, X, Bindings, delete_for_destination_in_khepri(XName, OnlyDurable)}. + +delete_for_source_in_khepri(#resource{virtual_host = VHost, name = Name}) -> + Path = khepri_routes_path() ++ [VHost, Name], + {ok, Bindings} = khepri_tx:get_many(Path ++ [rabbit_khepri:if_has_data_wildcard()]), + ok = khepri_tx:delete(Path), + maps:fold(fun(_P, Set, Acc) -> + sets:to_list(Set) ++ Acc + end, [], Bindings). + %% ------------------------------------------------------------------- %% delete_for_destination_in_mnesia(). %% ------------------------------------------------------------------- @@ -513,6 +820,29 @@ delete_for_destination_in_mnesia(DstName, OnlyDurable, Fun) -> rabbit_binding:group_bindings_fold(fun maybe_auto_delete_exchange_in_mnesia/4, lists:keysort(#binding.source, Bindings), OnlyDurable). +%% ------------------------------------------------------------------- +%% delete_for_destination_in_khepri(). +%% ------------------------------------------------------------------- + +-spec delete_for_destination_in_khepri(Dst, OnlyDurable) -> Deletions when + Dst :: rabbit_types:binding_destination(), + OnlyDurable :: boolean(), + Deletions :: rabbit_binding:deletions(). + +delete_for_destination_in_khepri(DstName, OnlyDurable) -> + BindingsMap = match_destination_in_khepri(DstName), + maps:foreach(fun(K, _V) -> khepri_tx:delete(K) end, BindingsMap), + Bindings = maps:fold(fun(_, Set, Acc) -> + sets:to_list(Set) ++ Acc + end, [], BindingsMap), + rabbit_binding:group_bindings_fold(fun maybe_auto_delete_exchange_in_khepri/4, + lists:keysort(#binding.source, Bindings), OnlyDurable). + +match_destination_in_khepri(#resource{virtual_host = VHost, kind = Kind, name = Name}) -> + Path = khepri_routes_path() ++ [VHost, ?KHEPRI_WILDCARD_STAR, Kind, Name, ?KHEPRI_WILDCARD_STAR_STAR], + {ok, Map} = khepri_tx:get_many(Path), + Map. + %% ------------------------------------------------------------------- %% delete_transient_for_destination_in_mnesia(). %% ------------------------------------------------------------------- @@ -543,6 +873,38 @@ has_for_source_in_mnesia(SrcName) -> contains(?MNESIA_TABLE, Match) orelse contains(?MNESIA_SEMI_DURABLE_TABLE, Match). +%% ------------------------------------------------------------------- +%% has_for_source_in_khepri(). +%% ------------------------------------------------------------------- + +-spec has_for_source_in_khepri(rabbit_types:binding_source()) -> boolean(). + +has_for_source_in_khepri(#resource{virtual_host = VHost, name = Name}) -> + Path = khepri_routes_path() ++ [VHost, Name, rabbit_khepri:if_has_data_wildcard()], + case khepri_tx:get_many(Path) of + {ok, Map} -> + maps:size(Map) > 0; + _ -> + false + end. + +%% ------------------------------------------------------------------- +%% match_source_and_destination_in_khepri_tx(). +%% ------------------------------------------------------------------- + +-spec match_source_and_destination_in_khepri_tx(Src, Dst) -> Bindings when + Src :: rabbit_types:binding_source(), + Dst :: rabbit_types:binding_destination(), + Bindings :: [Binding :: rabbit_types:binding()]. + +match_source_and_destination_in_khepri_tx(#resource{virtual_host = VHost, name = Name}, + #resource{kind = Kind, name = DstName}) -> + Path = khepri_routes_path() ++ [VHost, Name, Kind, DstName, rabbit_khepri:if_has_data_wildcard()], + case khepri_tx:get_many(Path) of + {ok, Map} -> maps:values(Map); + _ -> [] + end. + %% ------------------------------------------------------------------- %% clear(). %% ------------------------------------------------------------------- @@ -553,7 +915,9 @@ has_for_source_in_mnesia(SrcName) -> %% @private clear() -> - clear_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> clear_in_mnesia() end, + khepri => fun() -> clear_in_khepri() end}). clear_in_mnesia() -> {atomic, ok} = mnesia:clear_table(?MNESIA_TABLE), @@ -563,6 +927,27 @@ clear_in_mnesia() -> {atomic, ok} = mnesia:clear_table(?MNESIA_INDEX_TABLE), ok. +clear_in_khepri() -> + Path = rabbit_db_binding:khepri_routes_path(), + case rabbit_khepri:delete(Path) of + ok -> ok; + Error -> throw(Error) + end. + +%% -------------------------------------------------------------- +%% Paths +%% -------------------------------------------------------------- +khepri_route_path(#binding{source = #resource{virtual_host = VHost, name = SrcName}, + destination = #resource{kind = Kind, name = DstName}, + key = RoutingKey}) -> + [?MODULE, routes, VHost, SrcName, Kind, DstName, RoutingKey]. + +khepri_routes_path() -> + [?MODULE, routes]. + +khepri_route_exchange_path(#resource{virtual_host = VHost, name = SrcName}) -> + [?MODULE, routes, VHost, SrcName]. + %% -------------------------------------------------------------- %% Internal %% -------------------------------------------------------------- diff --git a/deps/rabbit/src/rabbit_db_binding_m2k_converter.erl b/deps/rabbit/src/rabbit_db_binding_m2k_converter.erl new file mode 100644 index 000000000000..1e99f8be081f --- /dev/null +++ b/deps/rabbit/src/rabbit_db_binding_m2k_converter.erl @@ -0,0 +1,125 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright © 2022-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_binding_m2k_converter). + +-behaviour(mnesia_to_khepri_converter). + +-include_lib("kernel/include/logger.hrl"). +-include_lib("khepri/include/khepri.hrl"). +-include_lib("khepri_mnesia_migration/src/kmm_logging.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). + +-export([init_copy_to_khepri/3, + copy_to_khepri/3, + delete_from_khepri/3]). + +-record(?MODULE, {store_id :: khepri:store_id(), + exchanges :: sets:set()}). + +-spec init_copy_to_khepri(StoreId, MigrationId, Tables) -> Ret when + StoreId :: khepri:store_id(), + MigrationId :: mnesia_to_khepri:migration_id(), + Tables :: [mnesia_to_khepri:mnesia_table()], + Ret :: {ok, Priv}, + Priv :: #?MODULE{}. +%% @private + +init_copy_to_khepri(StoreId, _MigrationId, Tables) -> + %% Clean up any previous attempt to copy the Mnesia table to Khepri. + lists:foreach(fun clear_data_in_khepri/1, Tables), + + State = #?MODULE{store_id = StoreId, + exchanges = sets:new([{version, 2}])}, + {ok, State}. + +-spec copy_to_khepri(Table, Record, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Record :: tuple(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +copy_to_khepri(rabbit_route = Table, + #route{binding = #binding{source = XName} = Binding}, + #?MODULE{exchanges = Xs0} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] key: ~0p", + [Table, Binding], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_binding:khepri_route_path(Binding), + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + rabbit_khepri:transaction( + fun() -> + Xs = case sets:is_element(XName, Xs0) of + true -> + Xs0; + false -> + %% If the binding's source is a new exchange, + %% store the exchange's type in the exchange + %% name branch of the tree. + XPath = rabbit_db_binding:khepri_route_exchange_path(XName), + [#exchange{type = XType}] = + rabbit_db_exchange:get_in_khepri_tx(XName), + ok = khepri_tx:put(XPath, #{type => XType}), + sets:add_element(XName, Xs0) + end, + Set = case khepri_tx:get(Path) of + {ok, Set0} -> + Set0; + _ -> + sets:new([{version, 2}]) + end, + case khepri_tx:put(Path, sets:add_element(Binding, Set)) of + ok -> {ok, State#?MODULE{exchanges = Xs}}; + Error -> Error + end + end, rw); +copy_to_khepri(Table, Record, State) -> + ?LOG_DEBUG("Mnesia->Khepri unexpected record table ~0p record ~0p state ~0p", + [Table, Record, State]), + {error, unexpected_record}. + +-spec delete_from_khepri(Table, Key, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Key :: any(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +delete_from_khepri(rabbit_route = Table, Key, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] key: ~0p", + [Table, Key], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_binding:khepri_route_path(Key), + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:delete(StoreId, Path) of + ok -> {ok, State}; + Error -> Error + end. + +-spec clear_data_in_khepri(Table) -> ok when + Table :: atom(). + +clear_data_in_khepri(rabbit_route) -> + Path = rabbit_db_binding:khepri_routes_path(), + case rabbit_khepri:delete(Path) of + ok -> ok; + Error -> throw(Error) + end. diff --git a/deps/rabbit/src/rabbit_db_cluster.erl b/deps/rabbit/src/rabbit_db_cluster.erl index 7e377a2badc4..b42ed108efab 100644 --- a/deps/rabbit/src/rabbit_db_cluster.erl +++ b/deps/rabbit/src/rabbit_db_cluster.erl @@ -23,6 +23,12 @@ check_consistency/0, cli_cluster_status/0]). +%% These two functions are not supported by Khepri and probably +%% shouldn't be part of this API in the future, but currently +%% they're needed here so they can fail when invoked using Khepri. +-export([rename/2, + update_cluster_nodes/1]). + -type node_type() :: disc_node_type() | ram_node_type(). -type disc_node_type() :: disc. -type ram_node_type() :: ram. @@ -58,7 +64,10 @@ can_join(RemoteNode) -> #{domain => ?RMQLOG_DOMAIN_DB}), case rabbit_feature_flags:check_node_compatibility(RemoteNode) of ok -> - can_join_using_mnesia(RemoteNode); + case rabbit_khepri:is_enabled(RemoteNode) of + true -> can_join_using_khepri(RemoteNode); + false -> can_join_using_mnesia(RemoteNode) + end; Error -> Error end. @@ -66,6 +75,9 @@ can_join(RemoteNode) -> can_join_using_mnesia(RemoteNode) -> rabbit_mnesia:can_join_cluster(RemoteNode). +can_join_using_khepri(RemoteNode) -> + rabbit_khepri:can_join_cluster(RemoteNode). + -spec join(RemoteNode, NodeType) -> Ret when RemoteNode :: node(), NodeType :: rabbit_db_cluster:node_type(), @@ -83,7 +95,10 @@ join(RemoteNode, NodeType) ?LOG_INFO( "DB: joining cluster using remote nodes:~n~tp", [ClusterNodes], #{domain => ?RMQLOG_DOMAIN_DB}), - Ret = join_using_mnesia(ClusterNodes, NodeType), + Ret = case rabbit_khepri:is_enabled(RemoteNode) of + true -> join_using_khepri(ClusterNodes, NodeType); + false -> join_using_mnesia(ClusterNodes, NodeType) + end, case Ret of ok -> rabbit_feature_flags:copy_feature_states_after_reset( @@ -103,6 +118,11 @@ join_using_mnesia(ClusterNodes, NodeType) when is_list(ClusterNodes) -> ok = rabbit_mnesia:reset_gracefully(), rabbit_mnesia:join_cluster(ClusterNodes, NodeType). +join_using_khepri(ClusterNodes, disc) -> + rabbit_khepri:add_member(node(), ClusterNodes); +join_using_khepri(_ClusterNodes, ram = NodeType) -> + {error, {node_type_unsupported, khepri, NodeType}}. + -spec forget_member(Node, RemoveWhenOffline) -> ok when Node :: node(), RemoveWhenOffline :: boolean(). @@ -111,7 +131,19 @@ join_using_mnesia(ClusterNodes, NodeType) when is_list(ClusterNodes) -> forget_member(Node, RemoveWhenOffline) -> case rabbit:is_running(Node) of false -> - forget_member_using_mnesia(Node, RemoveWhenOffline); + ?LOG_DEBUG( + "DB: removing cluster member `~ts`", [Node], + #{domain => ?RMQLOG_DOMAIN_DB}), + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> + forget_member_using_mnesia( + Node, RemoveWhenOffline) + end, + khepri => fun() -> + forget_member_using_khepri( + Node, RemoveWhenOffline) + end + }); true -> {error, {failed_to_remove_node, Node, rabbit_still_running}} end. @@ -119,6 +151,15 @@ forget_member(Node, RemoveWhenOffline) -> forget_member_using_mnesia(Node, RemoveWhenOffline) -> rabbit_mnesia:forget_cluster_node(Node, RemoveWhenOffline). +forget_member_using_khepri(_Node, true) -> + ?LOG_WARNING( + "Remove node with --offline flag is not supported by Khepri. " + "Skipping...", + #{domain => ?RMQLOG_DOMAIN_DB}), + {error, not_supported}; +forget_member_using_khepri(Node, false = _RemoveWhenOffline) -> + rabbit_khepri:leave_cluster(Node). + %% ------------------------------------------------------------------- %% Cluster update. %% ------------------------------------------------------------------- @@ -130,7 +171,11 @@ forget_member_using_mnesia(Node, RemoveWhenOffline) -> %% Node types may not all be valid with all databases. change_node_type(NodeType) -> - change_node_type_using_mnesia(NodeType). + rabbit_mnesia:ensure_node_type_is_permitted(NodeType), + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> change_node_type_using_mnesia(NodeType) end, + khepri => ok + }). change_node_type_using_mnesia(NodeType) -> rabbit_mnesia:change_cluster_node_type(NodeType). @@ -144,27 +189,37 @@ change_node_type_using_mnesia(NodeType) -> %% @doc Indicates if this node is clustered with other nodes or not. is_clustered() -> - is_clustered_using_mnesia(). - -is_clustered_using_mnesia() -> - rabbit_mnesia:is_clustered(). + Members = members(), + Members =/= [] andalso Members =/= [node()]. -spec members() -> Members when Members :: [node()]. %% @doc Returns the list of cluster members. members() -> - members_using_mnesia(). + case rabbit_khepri:get_feature_state() of + enabled -> members_using_khepri(); + _ -> members_using_mnesia() + end. members_using_mnesia() -> rabbit_mnesia:members(). +members_using_khepri() -> + case rabbit_khepri:locally_known_nodes() of + [] -> [node()]; + Members -> Members + end. + -spec disc_members() -> Members when Members :: [node()]. %% @private disc_members() -> - disc_members_using_mnesia(). + case rabbit_khepri:get_feature_state() of + enabled -> members_using_khepri(); + _ -> disc_members_using_mnesia() + end. disc_members_using_mnesia() -> rabbit_mnesia:cluster_nodes(disc). @@ -176,11 +231,17 @@ disc_members_using_mnesia() -> %% Node types may not all be relevant with all databases. node_type() -> - node_type_using_mnesia(). + case rabbit_khepri:get_feature_state() of + enabled -> node_type_using_khepri(); + _ -> node_type_using_mnesia() + end. node_type_using_mnesia() -> rabbit_mnesia:node_type(). +node_type_using_khepri() -> + disc. + -spec check_compatibility(RemoteNode) -> ok | {error, Reason} when RemoteNode :: node(), Reason :: any(). @@ -190,7 +251,10 @@ node_type_using_mnesia() -> check_compatibility(RemoteNode) -> case rabbit_feature_flags:check_node_compatibility(RemoteNode) of ok -> - check_compatibility_using_mnesia(RemoteNode); + case rabbit_khepri:get_feature_state() of + enabled -> ok; + _ -> check_compatibility_using_mnesia(RemoteNode) + end; Error -> Error end. @@ -202,11 +266,17 @@ check_compatibility_using_mnesia(RemoteNode) -> %% @doc Ensures the cluster is consistent. check_consistency() -> - check_consistency_using_mnesia(). + case rabbit_khepri:get_feature_state() of + enabled -> check_consistency_using_khepri(); + _ -> check_consistency_using_mnesia() + end. check_consistency_using_mnesia() -> rabbit_mnesia:check_cluster_consistency(). +check_consistency_using_khepri() -> + rabbit_khepri:check_cluster_consistency(). + -spec cli_cluster_status() -> ClusterStatus when ClusterStatus :: [{nodes, [{rabbit_db_cluster:node_type(), [node()]}]} | {running_nodes, [node()]} | @@ -215,7 +285,25 @@ check_consistency_using_mnesia() -> %% command. cli_cluster_status() -> - cli_cluster_status_using_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun cli_cluster_status_using_mnesia/0, + khepri => fun cli_cluster_status_using_khepri/0 + }). cli_cluster_status_using_mnesia() -> rabbit_mnesia:status(). + +cli_cluster_status_using_khepri() -> + rabbit_khepri:cli_cluster_status(). + +rename(Node, NodeMapList) -> + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> rabbit_mnesia_rename:rename(Node, NodeMapList) end, + khepri => {error, not_supported} + }). + +update_cluster_nodes(DiscoveryNode) -> + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> rabbit_mnesia:update_cluster_nodes(DiscoveryNode) end, + khepri => {error, not_supported} + }). diff --git a/deps/rabbit/src/rabbit_db_exchange.erl b/deps/rabbit/src/rabbit_db_exchange.erl index 001fa7be297f..1e8531d7516d 100644 --- a/deps/rabbit/src/rabbit_db_exchange.erl +++ b/deps/rabbit/src/rabbit_db_exchange.erl @@ -7,6 +7,7 @@ -module(rabbit_db_exchange). +-include_lib("khepri/include/khepri.hrl"). -include_lib("rabbit_common/include/rabbit.hrl"). -export([ @@ -31,15 +32,28 @@ %% Used by other rabbit_db_* modules -export([ + maybe_auto_delete_in_khepri/2, maybe_auto_delete_in_mnesia/2, next_serial_in_mnesia_tx/1, + next_serial_in_khepri_tx/1, + delete_in_khepri/3, delete_in_mnesia/3, - update_in_mnesia_tx/2 + get_in_khepri_tx/1, + update_in_mnesia_tx/2, + update_in_khepri_tx/2, + path/1 ]). %% For testing -export([clear/0]). +-export([ + khepri_exchange_path/1, + khepri_exchange_serial_path/1, + khepri_exchanges_path/0, + khepri_exchange_serials_path/0 + ]). + -define(MNESIA_TABLE, rabbit_exchange). -define(MNESIA_DURABLE_TABLE, rabbit_durable_exchange). -define(MNESIA_SERIAL_TABLE, rabbit_exchange_serial). @@ -57,11 +71,17 @@ %% @private get_all() -> - get_all_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_in_mnesia() end, + khepri => fun() -> get_all_in_khepri() end + }). get_all_in_mnesia() -> rabbit_db:list_in_mnesia(?MNESIA_TABLE, #exchange{_ = '_'}). +get_all_in_khepri() -> + rabbit_db:list_in_khepri(khepri_exchanges_path() ++ [rabbit_khepri:if_has_data_wildcard()]). + -spec get_all(VHostName) -> [Exchange] when VHostName :: vhost:name(), Exchange :: rabbit_types:exchange(). @@ -72,12 +92,18 @@ get_all_in_mnesia() -> %% @private get_all(VHost) -> - get_all_in_mnesia(VHost). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_in_mnesia(VHost) end, + khepri => fun() -> get_all_in_khepri(VHost) end + }). get_all_in_mnesia(VHost) -> Match = #exchange{name = rabbit_misc:r(VHost, exchange), _ = '_'}, rabbit_db:list_in_mnesia(?MNESIA_TABLE, Match). +get_all_in_khepri(VHost) -> + rabbit_db:list_in_khepri(khepri_exchanges_path() ++ [VHost, rabbit_khepri:if_has_data_wildcard()]). + %% ------------------------------------------------------------------- %% get_all_durable(). %% ------------------------------------------------------------------- @@ -91,11 +117,17 @@ get_all_in_mnesia(VHost) -> %% @private get_all_durable() -> - get_all_durable_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_durable_in_mnesia() end, + khepri => fun() -> get_all_durable_in_khepri() end + }). get_all_durable_in_mnesia() -> rabbit_db:list_in_mnesia(rabbit_durable_exchange, #exchange{_ = '_'}). +get_all_durable_in_khepri() -> + rabbit_db:list_in_khepri(khepri_exchanges_path() ++ [rabbit_khepri:if_has_data_wildcard()]). + %% ------------------------------------------------------------------- %% list(). %% ------------------------------------------------------------------- @@ -109,11 +141,23 @@ get_all_durable_in_mnesia() -> %% @private list() -> - list_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> list_in_mnesia() end, + khepri => fun() -> list_in_khepri() end + }). list_in_mnesia() -> mnesia:dirty_all_keys(?MNESIA_TABLE). +list_in_khepri() -> + case rabbit_khepri:match(khepri_exchanges_path() ++ + [rabbit_khepri:if_has_data_wildcard()]) of + {ok, Map} -> + maps:fold(fun(_K, X, Acc) -> [X#exchange.name | Acc] end, [], Map); + _ -> + [] + end. + %% ------------------------------------------------------------------- %% get(). %% ------------------------------------------------------------------- @@ -129,11 +173,34 @@ list_in_mnesia() -> %% @private get(Name) -> - get_in_mnesia(Name). - + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_in_mnesia(Name) end, + khepri => fun() -> get_in_khepri(Name) end + }). + get_in_mnesia(Name) -> rabbit_mnesia:dirty_read({?MNESIA_TABLE, Name}). +get_in_khepri(Name) -> + case ets:lookup(rabbit_khepri_exchange, Name) of + [X] -> {ok, X}; + [] -> {error, not_found} + end. + +%% ------------------------------------------------------------------- +%% get_in_khepri_tx(). +%% ------------------------------------------------------------------- + +-spec get_in_khepri_tx(ExchangeName) -> Ret when + ExchangeName :: rabbit_exchange:name(), + Ret :: [Exchange :: rabbit_types:exchange()]. + +get_in_khepri_tx(Name) -> + case khepri_tx:get(khepri_exchange_path(Name)) of + {ok, X} -> [X]; + _ -> [] + end. + %% ------------------------------------------------------------------- %% get_many(). %% ------------------------------------------------------------------- @@ -148,7 +215,10 @@ get_in_mnesia(Name) -> %% @private get_many(Names) when is_list(Names) -> - get_many_in_mnesia(?MNESIA_TABLE, Names). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_many_in_mnesia(?MNESIA_TABLE, Names) end, + khepri => fun() -> get_many_in_khepri(Names) end + }). get_many_in_mnesia(Table, [Name]) -> ets:lookup(Table, Name); get_many_in_mnesia(Table, Names) when is_list(Names) -> @@ -156,6 +226,9 @@ get_many_in_mnesia(Table, Names) when is_list(Names) -> %% expensive for reasons explained in rabbit_mnesia:dirty_read/1. lists:append([ets:lookup(Table, Name) || Name <- Names]). +get_many_in_khepri(Names) when is_list(Names) -> + lists:append([ets:lookup(rabbit_khepri_exchange, Name) || Name <- Names]). + %% ------------------------------------------------------------------- %% count(). %% ------------------------------------------------------------------- @@ -168,11 +241,17 @@ get_many_in_mnesia(Table, Names) when is_list(Names) -> %% @private count() -> - count_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> count_in_mnesia() end, + khepri => fun() -> count_in_khepri() end + }). count_in_mnesia() -> mnesia:table_info(?MNESIA_TABLE, size). +count_in_khepri() -> + rabbit_khepri:count_children(khepri_exchanges_path() ++ [?KHEPRI_WILDCARD_STAR]). + %% ------------------------------------------------------------------- %% update(). %% ------------------------------------------------------------------- @@ -189,7 +268,10 @@ count_in_mnesia() -> %% @private update(XName, Fun) -> - update_in_mnesia(XName, Fun). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> update_in_mnesia(XName, Fun) end, + khepri => fun() -> update_in_khepri(XName, Fun) end + }). update_in_mnesia(XName, Fun) -> rabbit_mnesia:execute_mnesia_transaction( @@ -224,6 +306,52 @@ set_ram_in_mnesia_tx(X) -> ok = mnesia:write(?MNESIA_TABLE, X1, write), X1. +update_in_khepri(XName, Fun) -> + Path = khepri_exchange_path(XName), + Ret1 = rabbit_khepri:adv_get(Path), + case Ret1 of + {ok, #{data := X, payload_version := Vsn}} -> + X1 = Fun(X), + UpdatePath = + khepri_path:combine_with_conditions( + Path, [#if_payload_version{version = Vsn}]), + Ret2 = rabbit_khepri:put(UpdatePath, X1), + case Ret2 of + ok -> + ok; + {error, {khepri, mismatching_node, _}} -> + update_in_khepri(XName, Fun); + {error, {khepri, node_not_found, _}} -> + ok; + {error, _} = Error -> + Error + end; + {error, {khepri, node_not_found, _}} -> + ok; + {error, _} = Error -> + Error + end. + +%% ------------------------------------------------------------------- +%% update_in_khepri_tx(). +%% ------------------------------------------------------------------- + +-spec update_in_khepri_tx(ExchangeName, UpdateFun) -> Ret when + ExchangeName :: rabbit_exchange:name(), + Exchange :: rabbit_types:exchange(), + UpdateFun :: fun((Exchange) -> Exchange), + Ret :: not_found | Exchange. + +update_in_khepri_tx(Name, Fun) -> + Path = khepri_exchange_path(Name), + case khepri_tx:get(Path) of + {ok, X} -> + X1 = Fun(X), + ok = khepri_tx:put(Path, X1), + X1; + _ -> not_found + end. + %% ------------------------------------------------------------------- %% create_or_get(). %% ------------------------------------------------------------------- @@ -240,7 +368,10 @@ set_ram_in_mnesia_tx(X) -> %% @private create_or_get(X) -> - create_or_get_in_mnesia(X). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> create_or_get_in_mnesia(X) end, + khepri => fun() -> create_or_get_in_khepri(X) end + }). create_or_get_in_mnesia(#exchange{name = XName} = X) -> rabbit_mnesia:execute_mnesia_transaction( @@ -253,6 +384,15 @@ create_or_get_in_mnesia(#exchange{name = XName} = X) -> end end). +create_or_get_in_khepri(#exchange{name = XName} = X) -> + Path = khepri_exchange_path(XName), + case rabbit_khepri:create(Path, X) of + ok -> + {new, X}; + {error, {khepri, mismatching_node, #{node_props := #{data := ExistingX}}}} -> + {existing, ExistingX} + end. + %% ------------------------------------------------------------------- %% set(). %% ------------------------------------------------------------------- @@ -266,7 +406,10 @@ create_or_get_in_mnesia(#exchange{name = XName} = X) -> %% @private set(Xs) -> - set_in_mnesia(Xs). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> set_in_mnesia(Xs) end, + khepri => fun() -> set_in_khepri(Xs) end + }). set_in_mnesia(Xs) when is_list(Xs) -> rabbit_mnesia:execute_mnesia_transaction( @@ -275,6 +418,18 @@ set_in_mnesia(Xs) when is_list(Xs) -> end), ok. +set_in_khepri(Xs) when is_list(Xs) -> + rabbit_khepri:transaction( + fun() -> + [set_in_khepri_tx(X) || X <- Xs] + end, rw), + ok. + +set_in_khepri_tx(X) -> + Path = khepri_exchange_path(X#exchange.name), + ok = khepri_tx:put(Path, X), + X. + %% ------------------------------------------------------------------- %% peek_serial(). %% ------------------------------------------------------------------- @@ -289,7 +444,10 @@ set_in_mnesia(Xs) when is_list(Xs) -> %% @private peek_serial(XName) -> - peek_serial_in_mnesia(XName). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> peek_serial_in_mnesia(XName) end, + khepri => fun() -> peek_serial_in_khepri(XName) end + }). peek_serial_in_mnesia(XName) -> rabbit_mnesia:execute_mnesia_transaction( @@ -303,6 +461,15 @@ peek_serial_in_mnesia_tx(XName, LockType) -> _ -> 1 end. +peek_serial_in_khepri(XName) -> + Path = khepri_exchange_serial_path(XName), + case rabbit_khepri:get(Path) of + {ok, Serial} -> + Serial; + _ -> + 1 + end. + %% ------------------------------------------------------------------- %% next_serial(). %% ------------------------------------------------------------------- @@ -317,7 +484,10 @@ peek_serial_in_mnesia_tx(XName, LockType) -> %% @private next_serial(XName) -> - next_serial_in_mnesia(XName). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> next_serial_in_mnesia(XName) end, + khepri => fun() -> next_serial_in_khepri(XName) end + }). next_serial_in_mnesia(XName) -> rabbit_mnesia:execute_mnesia_transaction(fun() -> @@ -334,6 +504,43 @@ next_serial_in_mnesia_tx(XName) -> #exchange_serial{name = XName, next = Serial + 1}, write), Serial. +next_serial_in_khepri(XName) -> + %% Just storing the serial number is enough, no need to keep #exchange_serial{} + Path = khepri_exchange_serial_path(XName), + Ret1 = rabbit_khepri:adv_get(Path), + case Ret1 of + {ok, #{data := Serial, + payload_version := Vsn}} -> + UpdatePath = + khepri_path:combine_with_conditions( + Path, [#if_payload_version{version = Vsn}]), + case rabbit_khepri:put(UpdatePath, Serial + 1) of + ok -> + Serial; + {error, {khepri, mismatching_node, _}} -> + next_serial_in_khepri(XName); + Err -> + Err + end; + _ -> + Serial = 1, + ok = rabbit_khepri:put(Path, Serial + 1), + Serial + end. + +-spec next_serial_in_khepri_tx(Exchange) -> Serial when + Exchange :: rabbit_types:exchange(), + Serial :: integer(). + +next_serial_in_khepri_tx(#exchange{name = XName}) -> + Path = khepri_exchange_serial_path(XName), + Serial = case khepri_tx:get(Path) of + {ok, Serial0} -> Serial0; + _ -> 1 + end, + ok = khepri_tx:put(Path, Serial + 1), + Serial. + %% ------------------------------------------------------------------- %% delete(). %% ------------------------------------------------------------------- @@ -358,7 +565,10 @@ next_serial_in_mnesia_tx(XName) -> %% @private delete(XName, IfUnused) -> - delete_in_mnesia(XName, IfUnused). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> delete_in_mnesia(XName, IfUnused) end, + khepri => fun() -> delete_in_khepri(XName, IfUnused) end + }). delete_in_mnesia(XName, IfUnused) -> DeletionFun = case IfUnused of @@ -396,6 +606,32 @@ delete_in_mnesia(X = #exchange{name = XName}, OnlyDurable, RemoveBindingsForSour rabbit_db_binding:delete_all_for_exchange_in_mnesia( X, OnlyDurable, RemoveBindingsForSource). +delete_in_khepri(XName, IfUnused) -> + DeletionFun = case IfUnused of + true -> fun conditional_delete_in_khepri/2; + false -> fun unconditional_delete_in_khepri/2 + end, + rabbit_khepri:transaction( + fun() -> + case khepri_tx:get(khepri_exchange_path(XName)) of + {ok, X} -> DeletionFun(X, false); + _ -> {error, not_found} + end + end, rw). + +conditional_delete_in_khepri(X = #exchange{name = XName}, OnlyDurable) -> + case rabbit_db_binding:has_for_source_in_khepri(XName) of + false -> delete_in_khepri(X, OnlyDurable, false); + true -> {error, in_use} + end. + +unconditional_delete_in_khepri(X, OnlyDurable) -> + delete_in_khepri(X, OnlyDurable, true). + +delete_in_khepri(X = #exchange{name = XName}, OnlyDurable, RemoveBindingsForSource) -> + ok = khepri_tx:delete(khepri_exchange_path(XName)), + rabbit_db_binding:delete_all_for_exchange_in_khepri(X, OnlyDurable, RemoveBindingsForSource). + %% ------------------------------------------------------------------- %% delete_serial(). %% ------------------------------------------------------------------- @@ -409,7 +645,10 @@ delete_in_mnesia(X = #exchange{name = XName}, OnlyDurable, RemoveBindingsForSour %% @private delete_serial(XName) -> - delete_serial_in_mnesia(XName). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> delete_serial_in_mnesia(XName) end, + khepri => fun() -> delete_serial_in_khepri(XName) end + }). delete_serial_in_mnesia(XName) -> rabbit_mnesia:execute_mnesia_transaction( @@ -417,6 +656,10 @@ delete_serial_in_mnesia(XName) -> mnesia:delete({?MNESIA_SERIAL_TABLE, XName}) end). +delete_serial_in_khepri(XName) -> + Path = khepri_exchange_serial_path(XName), + ok = rabbit_khepri:delete(Path). + %% ------------------------------------------------------------------- %% recover(). %% ------------------------------------------------------------------- @@ -431,7 +674,10 @@ delete_serial_in_mnesia(XName) -> %% @private recover(VHost) -> - recover_in_mnesia(VHost). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> recover_in_mnesia(VHost) end, + khepri => fun() -> recover_in_khepri(VHost) end + }). recover_in_mnesia(VHost) -> rabbit_mnesia:table_filter( @@ -449,6 +695,28 @@ recover_in_mnesia(VHost) -> end, ?MNESIA_DURABLE_TABLE). +recover_in_khepri(VHost) -> + %% Transient exchanges are deprecated in Khepri, all exchanges are recovered + %% Node boot and recovery should hang until the data is ready. + %% Recovery needs to wait until progress can be done, as it + %% cannot be skipped and stopping the node is not an option - + %% the next boot most likely would behave the same way. + %% Any other request stays with the default timeout, currently 30s. + Exchanges0 = rabbit_db:list_in_khepri(khepri_exchanges_path() ++ [VHost, rabbit_khepri:if_has_data_wildcard()], + #{timeout => infinity}), + Exchanges = [rabbit_exchange_decorator:set(X) || X <- Exchanges0], + + rabbit_khepri:transaction( + fun() -> + [_ = set_in_khepri_tx(X) || X <- Exchanges] + end, rw, #{timeout => infinity}), + %% TODO once mnesia is gone, this callback should go back to `rabbit_exchange` + [begin + Serial = rabbit_exchange:serial(X), + rabbit_exchange:callback(X, create, Serial, [X]) + end || X <- Exchanges], + Exchanges. + %% ------------------------------------------------------------------- %% match(). %% ------------------------------------------------------------------- @@ -464,9 +732,12 @@ recover_in_mnesia(VHost) -> %% @private match(Pattern) -> - match_in_mnesia(Pattern). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> match_in_mnesia(Pattern) end, + khepri => fun() -> match_in_khepri(Pattern) end + }). -match_in_mnesia(Pattern) -> +match_in_mnesia(Pattern) -> case mnesia:transaction( fun() -> mnesia:match_object(?MNESIA_TABLE, Pattern, read) @@ -475,6 +746,10 @@ match_in_mnesia(Pattern) -> {aborted, Err} -> {error, Err} end. +match_in_khepri(Pattern0) -> + Pattern = #if_data_matches{pattern = Pattern0}, + rabbit_db:list_in_khepri(khepri_exchanges_path() ++ [?KHEPRI_WILDCARD_STAR, Pattern]). + %% ------------------------------------------------------------------- %% exists(). %% ------------------------------------------------------------------- @@ -489,11 +764,17 @@ match_in_mnesia(Pattern) -> %% @private exists(Name) -> - exists_in_mnesia(Name). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> exists_in_mnesia(Name) end, + khepri => fun() -> exists_in_khepri(Name) end + }). exists_in_mnesia(Name) -> ets:member(?MNESIA_TABLE, Name). +exists_in_khepri(Name) -> + rabbit_khepri:exists(khepri_exchange_path(Name)). + %% ------------------------------------------------------------------- %% clear(). %% ------------------------------------------------------------------- @@ -504,7 +785,10 @@ exists_in_mnesia(Name) -> %% @private clear() -> - clear_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> clear_in_mnesia() end, + khepri => fun() -> clear_in_khepri() end + }). clear_in_mnesia() -> {atomic, ok} = mnesia:clear_table(?MNESIA_TABLE), @@ -512,6 +796,16 @@ clear_in_mnesia() -> {atomic, ok} = mnesia:clear_table(?MNESIA_SERIAL_TABLE), ok. +clear_in_khepri() -> + khepri_delete(khepri_exchanges_path()), + khepri_delete(khepri_exchange_serials_path()). + +khepri_delete(Path) -> + case rabbit_khepri:delete(Path) of + ok -> ok; + Error -> throw(Error) + end. + %% ------------------------------------------------------------------- %% maybe_auto_delete_in_mnesia(). %% ------------------------------------------------------------------- @@ -535,3 +829,54 @@ maybe_auto_delete_in_mnesia(XName, OnlyDurable) -> {deleted, X, [], Deletions} -> {deleted, X, Deletions} end end. + +%% ------------------------------------------------------------------- +%% maybe_auto_delete_in_khepri(). +%% ------------------------------------------------------------------- + +-spec maybe_auto_delete_in_khepri(ExchangeName, boolean()) -> Ret when + ExchangeName :: rabbit_exchange:name(), + Exchange :: rabbit_types:exchange(), + Deletions :: rabbit_binding:deletions(), + Ret :: {'not_deleted', 'undefined' | Exchange} | + {'deleted', Exchange, Deletions}. + +maybe_auto_delete_in_khepri(XName, OnlyDurable) -> + case khepri_tx:get(khepri_exchange_path(XName)) of + {ok, #exchange{auto_delete = false} = X} -> + {not_deleted, X}; + {ok, #exchange{auto_delete = true} = X} -> + case conditional_delete_in_khepri(X, OnlyDurable) of + {error, in_use} -> {not_deleted, X}; + {deleted, X, [], Deletions} -> {deleted, X, Deletions} + end; + {error, _} -> + {not_deleted, undefined} + end. + +%% ------------------------------------------------------------------- +%% Khepri paths +%% ------------------------------------------------------------------- + +khepri_exchanges_path() -> + [?MODULE, exchanges]. + +khepri_exchange_path(#resource{virtual_host = VHost, name = Name}) -> + [?MODULE, exchanges, VHost, Name]. + +khepri_exchange_serials_path() -> + [?MODULE, exchange_serials]. + +khepri_exchange_serial_path(#resource{virtual_host = VHost, name = Name}) -> + [?MODULE, exchange_serials, VHost, Name]. + +%% ------------------------------------------------------------------- +%% path(). +%% ------------------------------------------------------------------- + +-spec path(ExchangeName) -> Path when + ExchangeName :: rabbit_exchange:name(), + Path :: khepri_path:path(). + +path(Name) -> + khepri_exchange_path(Name). diff --git a/deps/rabbit/src/rabbit_db_exchange_m2k_converter.erl b/deps/rabbit/src/rabbit_db_exchange_m2k_converter.erl new file mode 100644 index 000000000000..958efe5cd537 --- /dev/null +++ b/deps/rabbit/src/rabbit_db_exchange_m2k_converter.erl @@ -0,0 +1,139 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright © 2022-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_exchange_m2k_converter). + +-behaviour(mnesia_to_khepri_converter). + +-include_lib("kernel/include/logger.hrl"). +-include_lib("khepri/include/khepri.hrl"). +-include_lib("khepri_mnesia_migration/src/kmm_logging.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). + +-export([init_copy_to_khepri/3, + copy_to_khepri/3, + delete_from_khepri/3]). + +-record(?MODULE, {store_id :: khepri:store_id()}). + +-spec init_copy_to_khepri(StoreId, MigrationId, Tables) -> Ret when + StoreId :: khepri:store_id(), + MigrationId :: mnesia_to_khepri:migration_id(), + Tables :: [mnesia_to_khepri:mnesia_table()], + Ret :: {ok, Priv}, + Priv :: #?MODULE{}. +%% @private + +init_copy_to_khepri(StoreId, _MigrationId, Tables) -> + %% Clean up any previous attempt to copy the Mnesia table to Khepri. + lists:foreach(fun clear_data_in_khepri/1, Tables), + + State = #?MODULE{store_id = StoreId}, + {ok, State}. + +-spec copy_to_khepri(Table, Record, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Record :: tuple(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +copy_to_khepri( + rabbit_exchange = Table, Record, + #?MODULE{store_id = StoreId} = State) -> + Name = Record#exchange.name, + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] key: ~0p", + [Table, Name], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_exchange:khepri_exchange_path(Name), + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:put(StoreId, Path, Record) of + ok -> {ok, State}; + Error -> Error + end; +copy_to_khepri(rabbit_exchange_serial = Table, + #exchange_serial{name = Resource, next = Serial}, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] key: ~0p", + [Table, Resource], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = khepri_path:combine_with_conditions( + rabbit_db_exchange:khepri_exchange_serial_path(Resource), + [#if_node_exists{exists = false}]), + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:put(StoreId, Path, Serial) of + ok -> {ok, State}; + Error -> Error + end; +copy_to_khepri(Table, Record, State) -> + ?LOG_DEBUG("Mnesia->Khepri unexpected record table ~0p record ~0p state ~0p", + [Table, Record, State]), + {error, unexpected_record}. + +-spec delete_from_khepri(Table, Key, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Key :: any(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +delete_from_khepri(rabbit_exchange = Table, Key, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] key: ~0p", + [Table, Key], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_exchange:khepri_exchange_path(Key), + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:delete(StoreId, Path) of + ok -> {ok, State}; + Error -> Error + end; +delete_from_khepri(rabbit_exchange_serial = Table, Key, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] key: ~0p", + [Table, Key], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_exchange:khepri_exchange_serial_path(Key), + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:delete(StoreId, Path) of + ok -> {ok, State}; + Error -> Error + end. + +-spec clear_data_in_khepri(Table) -> ok when + Table :: atom(). + +clear_data_in_khepri(rabbit_exchange) -> + khepri_delete(rabbit_db_exchange:khepri_exchanges_path()); +clear_data_in_khepri(rabbit_exchange_serial) -> + khepri_delete(rabbit_db_exchange:khepri_exchange_serials_path()). + +khepri_delete(Path) -> + case rabbit_khepri:delete(Path) of + ok -> ok; + Error -> throw(Error) + end. diff --git a/deps/rabbit/src/rabbit_db_m2k_converter.erl b/deps/rabbit/src/rabbit_db_m2k_converter.erl new file mode 100644 index 000000000000..2bc3e21aadd5 --- /dev/null +++ b/deps/rabbit/src/rabbit_db_m2k_converter.erl @@ -0,0 +1,127 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright © 2022-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_m2k_converter). + +-behaviour(mnesia_to_khepri_converter). + +-include_lib("kernel/include/logger.hrl"). +-include_lib("khepri/include/khepri.hrl"). +-include_lib("khepri_mnesia_migration/src/kmm_logging.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). + +-export([init_copy_to_khepri/4, + copy_to_khepri/3, + delete_from_khepri/3]). + +-type migration() :: {mnesia_to_khepri:mnesia_table(), + mnesia_to_khepri:converter_mod()}. + +-type migrations() :: [migration()]. + +-record(?MODULE, {migrations :: migrations(), + sub_states :: #{module() => any()}}). + +-spec init_copy_to_khepri(StoreId, MigrationId, Tables, Migrations) -> + Ret when + StoreId :: khepri:store_id(), + MigrationId :: mnesia_to_khepri:migration_id(), + Tables :: [mnesia_to_khepri:mnesia_table()], + Migrations :: migrations(), + Ret :: {ok, Priv}, + Priv :: #?MODULE{}. +%% @private + +init_copy_to_khepri(StoreId, MigrationId, _Tables, Migrations) -> + TablesPerMod = lists:foldl( + fun + ({Table, Mod}, Acc) -> + Tables0 = maps:get(Mod, Acc, []), + Tables1 = Tables0 ++ [Table], + Acc#{Mod => Tables1}; + (_Table, Acc) -> + Acc + end, #{}, Migrations), + + SubStates = maps:fold( + fun(Mod, Tables, Acc) -> + {ok, SubState} = + case Mod of + {ActualMod, Args} -> + ActualMod:init_copy_to_khepri( + StoreId, MigrationId, + Tables, Args); + _ -> + Mod:init_copy_to_khepri( + StoreId, MigrationId, + Tables) + end, + Acc#{Mod => SubState} + end, #{}, TablesPerMod), + + State = #?MODULE{migrations = Migrations, + sub_states = SubStates}, + {ok, State}. + +-spec copy_to_khepri(Table, Record, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Record :: tuple(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +copy_to_khepri( + Table, Record, + #?MODULE{migrations = Migrations, sub_states = SubStates} = State) -> + case proplists:get_value(Table, Migrations) of + true -> + {ok, State}; + Mod when Mod =/= undefined -> + ActualMod = actual_mod(Mod), + SubState = maps:get(Mod, SubStates), + case ActualMod:copy_to_khepri(Table, Record, SubState) of + {ok, SubState1} -> + SubStates1 = SubStates#{Mod => SubState1}, + State1 = State#?MODULE{sub_states = SubStates1}, + {ok, State1}; + {error, _} = Error -> + Error + end + end. + +-spec delete_from_khepri(Table, Key, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Key :: any(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +delete_from_khepri( + Table, Key, + #?MODULE{migrations = Migrations, sub_states = SubStates} = State) -> + case proplists:get_value(Table, Migrations) of + true -> + {ok, State}; + Mod when Mod =/= undefined -> + ActualMod = actual_mod(Mod), + SubState = maps:get(Mod, SubStates), + case ActualMod:delete_from_khepri(Table, Key, SubState) of + {ok, SubState1} -> + SubStates1 = SubStates#{Mod => SubState1}, + State1 = State#?MODULE{sub_states = SubStates1}, + {ok, State1}; + {error, _} = Error -> + Error + end + end. + +actual_mod({Mod, _}) -> Mod; +actual_mod(Mod) -> Mod. diff --git a/deps/rabbit/src/rabbit_db_maintenance.erl b/deps/rabbit/src/rabbit_db_maintenance.erl index a70cd66b7bde..724997470992 100644 --- a/deps/rabbit/src/rabbit_db_maintenance.erl +++ b/deps/rabbit/src/rabbit_db_maintenance.erl @@ -10,85 +10,32 @@ -include_lib("rabbit_common/include/rabbit.hrl"). -export([ - setup_schema/0, + table_definitions/0, set/1, get/1, get_consistent/1 ]). --type mnesia_table() :: atom(). +-export([ + khepri_maintenance_path/1, + khepri_maintenance_path/0 + ]). -define(TABLE, rabbit_node_maintenance_states). %% ------------------------------------------------------------------- -%% setup_schema(). +%% table_definitions(). %% ------------------------------------------------------------------- --spec setup_schema() -> ok | {error, any()}. -%% @doc Creates the internal schema used by the selected metadata store -%% -%% @private - -setup_schema() -> - setup_schema_in_mnesia(). - -setup_schema_in_mnesia() -> - TableName = status_table_name(), - rabbit_log:info( - "Creating table ~ts for maintenance mode status", - [TableName]), - try - rabbit_table:create( - TableName, - status_table_definition()), - %% The `rabbit_node_maintenance_states' table used to be global but not - %% replicated. This leads to various errors during RabbitMQ boot or - %% operations on the Mnesia database. The reason is the table existed - %% on a single node and, if that node was stopped or MIA, other nodes - %% may wait forever on that node for the table to be available. - %% - %% The call below makes sure this node has a copy of the table. - case rabbit_table:ensure_table_copy(TableName, node(), ram_copies) of - ok -> - %% Next, we try to fix other nodes in the cluster if they are - %% running a version of RabbitMQ which does not replicate the - %% table. All nodes must have a replica for Mnesia operations - %% to work properly. Therefore the code below is to make older - %% compatible with newer nodes. - Replicas = mnesia:table_info(TableName, all_nodes), - Members = rabbit_nodes:list_running(), - MissingOn = Members -- Replicas, - lists:foreach( - fun(Node) -> - %% Errors from adding a replica on those older nodes - %% are ignored however. They should not be fatal. The - %% problem will solve by itself once all nodes are - %% upgraded. - _ = rpc:call( - Node, - rabbit_table, ensure_table_copy, - [TableName, Node, ram_copies]) - end, MissingOn), - ok; - Error -> - Error - end - catch throw:Reason -> - rabbit_log:error( - "Failed to create maintenance status table: ~tp", - [Reason]) - end. - --spec status_table_name() -> mnesia_table(). -status_table_name() -> - ?TABLE. +-spec table_definitions() -> [Def] when + Def :: {Name :: atom(), term()}. --spec status_table_definition() -> list(). -status_table_definition() -> - maps:to_list(#{ - record_name => node_maintenance_state, - attributes => record_info(fields, node_maintenance_state) - }). +table_definitions() -> + [{?TABLE, maps:to_list(#{ + record_name => node_maintenance_state, + attributes => record_info(fields, node_maintenance_state), + match => #node_maintenance_state{_ = '_'} + })}]. %% ------------------------------------------------------------------- %% set(). @@ -102,7 +49,10 @@ status_table_definition() -> %% @private set(Status) -> - set_in_mnesia(Status). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> set_in_mnesia(Status) end, + khepri => fun() -> set_in_khepri(Status) end + }). set_in_mnesia(Status) -> Res = mnesia:transaction( @@ -127,6 +77,18 @@ set_in_mnesia(Status) -> _ -> false end. +set_in_khepri(Status) -> + Node = node(), + Path = khepri_maintenance_path(Node), + Record = #node_maintenance_state{ + node = Node, + status = Status + }, + case rabbit_khepri:put(Path, Record) of + ok -> true; + _ -> false + end. + %% ------------------------------------------------------------------- %% get(). %% ------------------------------------------------------------------- @@ -141,7 +103,10 @@ set_in_mnesia(Status) -> %% @private get(Node) -> - get_in_mnesia(Node). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_in_mnesia(Node) end, + khepri => fun() -> get_in_khepri(Node) end + }). get_in_mnesia(Node) -> case catch mnesia:dirty_read(?TABLE, Node) of @@ -151,6 +116,15 @@ get_in_mnesia(Node) -> _ -> undefined end. +get_in_khepri(Node) -> + Path = khepri_maintenance_path(Node), + case rabbit_khepri:get(Path) of + {ok, #node_maintenance_state{status = Status}} -> + Status; + _ -> + undefined + end. + %% ------------------------------------------------------------------- %% get_consistent(). %% ------------------------------------------------------------------- @@ -165,7 +139,10 @@ get_in_mnesia(Node) -> %% @private get_consistent(Node) -> - get_consistent_in_mnesia(Node). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_consistent_in_mnesia(Node) end, + khepri => fun() -> get_consistent_in_khepri(Node) end + }). get_consistent_in_mnesia(Node) -> case mnesia:transaction(fun() -> mnesia:read(?TABLE, Node) end) of @@ -175,3 +152,22 @@ get_consistent_in_mnesia(Node) -> {atomic, _} -> undefined; {aborted, _Reason} -> undefined end. + +get_consistent_in_khepri(Node) -> + Path = khepri_maintenance_path(Node), + case rabbit_khepri:get(Path, #{favor => consistency}) of + {ok, #node_maintenance_state{status = Status}} -> + Status; + _ -> + undefined + end. + +%% ------------------------------------------------------------------- +%% Khepri paths +%% ------------------------------------------------------------------- + +khepri_maintenance_path() -> + [?MODULE, maintenance]. + +khepri_maintenance_path(Node) -> + [?MODULE, maintenance, Node]. diff --git a/deps/rabbit/src/rabbit_db_maintenance_m2k_converter.erl b/deps/rabbit/src/rabbit_db_maintenance_m2k_converter.erl new file mode 100644 index 000000000000..72b5bb70a876 --- /dev/null +++ b/deps/rabbit/src/rabbit_db_maintenance_m2k_converter.erl @@ -0,0 +1,103 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright © 2022-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_maintenance_m2k_converter). + +-behaviour(mnesia_to_khepri_converter). + +-include_lib("kernel/include/logger.hrl"). +-include_lib("khepri/include/khepri.hrl"). +-include_lib("khepri_mnesia_migration/src/kmm_logging.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). + +-export([init_copy_to_khepri/3, + copy_to_khepri/3, + delete_from_khepri/3]). + +-record(?MODULE, {store_id :: khepri:store_id()}). + +-spec init_copy_to_khepri(StoreId, MigrationId, Tables) -> Ret when + StoreId :: khepri:store_id(), + MigrationId :: mnesia_to_khepri:migration_id(), + Tables :: [mnesia_to_khepri:mnesia_table()], + Ret :: {ok, Priv}, + Priv :: #?MODULE{}. +%% @private + +init_copy_to_khepri(StoreId, _MigrationId, Tables) -> + %% Clean up any previous attempt to copy the Mnesia table to Khepri. + lists:foreach(fun clear_data_in_khepri/1, Tables), + + State = #?MODULE{store_id = StoreId}, + {ok, State}. + +-spec copy_to_khepri(Table, Record, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Record :: tuple(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +copy_to_khepri( + rabbit_node_maintenance_states = Table, Record, + #?MODULE{store_id = StoreId} = State) -> + Name = Record#node_maintenance_state.node, + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] key: ~0p", + [Table, Name], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_maintenance:khepri_maintenance_path(Name), + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:put(StoreId, Path, Record) of + ok -> {ok, State}; + Error -> Error + end; +copy_to_khepri(Table, Record, State) -> + ?LOG_DEBUG("Mnesia->Khepri unexpected record table ~0p record ~0p state ~0p", + [Table, Record, State]), + {error, unexpected_record}. + +-spec delete_from_khepri(Table, Key, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Key :: any(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +delete_from_khepri( + rabbit_node_maintenance_states = Table, Key, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] key: ~0p", + [Table, Key], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_maintenance:khepri_maintenance_path(Key), + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:delete(StoreId, Path) of + ok -> {ok, State}; + Error -> Error + end. + +-spec clear_data_in_khepri(Table) -> ok when + Table :: atom(). + +clear_data_in_khepri(rabbit_node_maintenance_states) -> + Path = rabbit_db_maintenance:khepri_maintenance_path(), + case rabbit_khepri:delete(Path) of + ok -> ok; + Error -> throw(Error) + end. diff --git a/deps/rabbit/src/rabbit_db_msup.erl b/deps/rabbit/src/rabbit_db_msup.erl index 5acb7cbfadf3..6486649fd2d2 100644 --- a/deps/rabbit/src/rabbit_db_msup.erl +++ b/deps/rabbit/src/rabbit_db_msup.erl @@ -7,6 +7,9 @@ -module(rabbit_db_msup). +-include_lib("khepri/include/khepri.hrl"). +-include("mirrored_supervisor.hrl"). + -export([ create_tables/0, table_definitions/0, @@ -19,6 +22,11 @@ -export([clear/0]). +-export([ + khepri_mirrored_supervisor_path/2, + khepri_mirrored_supervisor_path/0 + ]). + -define(TABLE, mirrored_sup_childspec). -define(TABLE_DEF, {?TABLE, @@ -27,8 +35,6 @@ {attributes, record_info(fields, mirrored_sup_childspec)}]}). -define(TABLE_MATCH, {match, #mirrored_sup_childspec{ _ = '_' }}). --record(mirrored_sup_childspec, {key, mirroring_pid, childspec}). - %% ------------------------------------------------------------------- %% create_tables(). %% ------------------------------------------------------------------- @@ -37,7 +43,10 @@ Ret :: 'ok' | {error, Reason :: term()}. create_tables() -> - create_tables_in_mnesia([?TABLE_DEF]). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> create_tables_in_mnesia([?TABLE_DEF]) end, + khepri => fun() -> ok end + }). create_tables_in_mnesia([]) -> ok; @@ -72,7 +81,16 @@ table_definitions() -> Ret :: start | undefined | pid(). create_or_update(Group, Overall, Delegate, ChildSpec, Id) -> - create_or_update_in_mnesia(Group, Overall, Delegate, ChildSpec, Id). + rabbit_khepri:handle_fallback( + #{mnesia => + fun() -> + create_or_update_in_mnesia(Group, Overall, Delegate, ChildSpec, Id) + end, + khepri => + fun() -> + create_or_update_in_khepri(Group, Overall, Delegate, ChildSpec, Id) + end + }). create_or_update_in_mnesia(Group, Overall, Delegate, ChildSpec, Id) -> rabbit_mnesia:execute_mnesia_transaction( @@ -111,6 +129,41 @@ write_in_mnesia(Group, Overall, ChildSpec, Id) -> ok = mnesia:write(?TABLE, S, write), ChildSpec. +create_or_update_in_khepri(Group, Overall, Delegate, ChildSpec, {SimpleId, _} = Id) -> + Path = khepri_mirrored_supervisor_path(Group, SimpleId), + S = #mirrored_sup_childspec{key = {Group, Id}, + mirroring_pid = Overall, + childspec = ChildSpec}, + case rabbit_khepri:adv_get(Path) of + {ok, #{data := #mirrored_sup_childspec{mirroring_pid = Pid}, + payload_version := Vsn}} -> + case Overall of + Pid -> + Delegate; + _ -> + %% The supervisor(Pid) call can't happen inside of a transaction. + %% We have to read and update the record in two different khepri calls + case mirrored_supervisor:supervisor(Pid) of + dead -> + UpdatePath = + khepri_path:combine_with_conditions( + Path, [#if_payload_version{version = Vsn}]), + Ret = rabbit_khepri:put(UpdatePath, S), + case Ret of + ok -> start; + {error, {khepri, mismatching_node, _}} -> + create_or_update_in_khepri(Group, Overall, Delegate, ChildSpec, Id); + {error, _} = Error -> Error + end; + Delegate0 -> + Delegate0 + end + end; + _ -> + ok = rabbit_khepri:put(Path, S), + start + end. + %% ------------------------------------------------------------------- %% delete(). %% ------------------------------------------------------------------- @@ -120,7 +173,10 @@ write_in_mnesia(Group, Overall, ChildSpec, Id) -> Id :: any(). delete(Group, Id) -> - delete_in_mnesia(Group, Id). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> delete_in_mnesia(Group, Id) end, + khepri => fun() -> delete_in_khepri(Group, Id) end + }). delete_in_mnesia(Group, Id) -> rabbit_mnesia:execute_mnesia_transaction( @@ -128,6 +184,9 @@ delete_in_mnesia(Group, Id) -> ok = mnesia:delete({?TABLE, {Group, Id}}) end). +delete_in_khepri(Group, {SimpleId, _}) -> + ok = rabbit_khepri:delete(khepri_mirrored_supervisor_path(Group, SimpleId)). + %% ------------------------------------------------------------------- %% find_mirror(). %% ------------------------------------------------------------------- @@ -141,7 +200,10 @@ find_mirror(Group, Id) -> %% If we did this inside a tx we could still have failover %% immediately after the tx - we can't be 100% here. So we may as %% well dirty_select. - find_mirror_in_mnesia(Group, Id). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> find_mirror_in_mnesia(Group, Id) end, + khepri => fun() -> find_mirror_in_khepri(Group, Id) end + }). find_mirror_in_mnesia(Group, Id) -> MatchHead = #mirrored_sup_childspec{mirroring_pid = '$1', @@ -152,6 +214,14 @@ find_mirror_in_mnesia(Group, Id) -> _ -> {error, not_found} end. +find_mirror_in_khepri(Group, {SimpleId, _}) -> + case rabbit_khepri:get(khepri_mirrored_supervisor_path(Group, SimpleId)) of + {ok, #mirrored_sup_childspec{mirroring_pid = Pid}} -> + {ok, Pid}; + _ -> + {error, not_found} + end. + %% ------------------------------------------------------------------- %% update_all(). %% ------------------------------------------------------------------- @@ -161,7 +231,10 @@ find_mirror_in_mnesia(Group, Id) -> ChildSpec :: supervisor2:child_spec(). update_all(Overall, OldOverall) -> - update_all_in_mnesia(Overall, OldOverall). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> update_all_in_mnesia(Overall, OldOverall) end, + khepri => fun() -> update_all_in_khepri(Overall, OldOverall) end + }). update_all_in_mnesia(Overall, OldOverall) -> rabbit_mnesia:execute_mnesia_transaction( @@ -174,6 +247,23 @@ update_all_in_mnesia(Overall, OldOverall) -> [{Group, Id}, C] <- mnesia:select(?TABLE, [{MatchHead, [], ['$$']}])] end). +update_all_in_khepri(Overall, OldOverall) -> + Pattern = #mirrored_sup_childspec{mirroring_pid = OldOverall, + _ = '_'}, + Conditions = [?KHEPRI_WILDCARD_STAR_STAR, #if_data_matches{pattern = Pattern}], + PathPattern = khepri_mirrored_supervisor_path() ++ [#if_all{conditions = Conditions}], + rabbit_khepri:transaction( + fun() -> + case khepri_tx:get_many(PathPattern) of + {ok, Map} -> + [begin + S = S0#mirrored_sup_childspec{mirroring_pid = Overall}, + ok = khepri_tx:put(Path, S), + S0#mirrored_sup_childspec.childspec + end || {Path, S0} <- maps:to_list(Map)] + end + end). + %% ------------------------------------------------------------------- %% delete_all(). %% ------------------------------------------------------------------- @@ -182,7 +272,10 @@ update_all_in_mnesia(Overall, OldOverall) -> Group :: any(). delete_all(Group) -> - delete_all_in_mnesia(Group). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> delete_all_in_mnesia(Group) end, + khepri => fun() -> delete_all_in_khepri(Group) end + }). delete_all_in_mnesia(Group) -> rabbit_mnesia:execute_mnesia_transaction( @@ -194,6 +287,13 @@ delete_all_in_mnesia(Group) -> end), ok. +delete_all_in_khepri(Group) -> + Pattern = #mirrored_sup_childspec{key = {Group, '_'}, + _ = '_'}, + Conditions = [?KHEPRI_WILDCARD_STAR_STAR, #if_data_matches{pattern = Pattern}], + rabbit_khepri:delete(khepri_mirrored_supervisor_path() ++ + [#if_all{conditions = Conditions}]). + %% ------------------------------------------------------------------- %% clear(). %% ------------------------------------------------------------------- @@ -201,8 +301,28 @@ delete_all_in_mnesia(Group) -> -spec clear() -> ok. clear() -> - clear_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> clear_in_mnesia() end, + khepri => fun() -> clear_in_khepri() end + }). clear_in_mnesia() -> {atomic, ok} = mnesia:clear_table(?TABLE), ok. + +clear_in_khepri() -> + Path = khepri_mirrored_supervisor_path(), + case rabbit_khepri:delete(Path) of + ok -> ok; + Error -> throw(Error) + end. + +%% ------------------------------------------------------------------- +%% Khepri paths +%% ------------------------------------------------------------------- + +khepri_mirrored_supervisor_path() -> + [?MODULE, mirrored_supervisor_childspec]. + +khepri_mirrored_supervisor_path(Group, Id) -> + [?MODULE, mirrored_supervisor_childspec, Group] ++ Id. diff --git a/deps/rabbit/src/rabbit_db_msup_m2k_converter.erl b/deps/rabbit/src/rabbit_db_msup_m2k_converter.erl new file mode 100644 index 000000000000..510a7c6a2f64 --- /dev/null +++ b/deps/rabbit/src/rabbit_db_msup_m2k_converter.erl @@ -0,0 +1,103 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright © 2022-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_msup_m2k_converter). + +-behaviour(mnesia_to_khepri_converter). + +-include_lib("kernel/include/logger.hrl"). +-include_lib("khepri/include/khepri.hrl"). +-include_lib("khepri_mnesia_migration/src/kmm_logging.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). +-include("mirrored_supervisor.hrl"). + +-export([init_copy_to_khepri/3, + copy_to_khepri/3, + delete_from_khepri/3]). + +-record(?MODULE, {store_id :: khepri:store_id()}). + +-spec init_copy_to_khepri(StoreId, MigrationId, Tables) -> Ret when + StoreId :: khepri:store_id(), + MigrationId :: mnesia_to_khepri:migration_id(), + Tables :: [mnesia_to_khepri:mnesia_table()], + Ret :: {ok, Priv}, + Priv :: #?MODULE{}. +%% @private + +init_copy_to_khepri(StoreId, _MigrationId, Tables) -> + %% Clean up any previous attempt to copy the Mnesia table to Khepri. + lists:foreach(fun clear_data_in_khepri/1, Tables), + + State = #?MODULE{store_id = StoreId}, + {ok, State}. + +-spec copy_to_khepri(Table, Record, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Record :: tuple(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +copy_to_khepri(mirrored_sup_childspec = Table, + #mirrored_sup_childspec{key = {Group, {SimpleId, _}} = Key} = Record, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] key: ~0p", + [Table, Key], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_msup:khepri_mirrored_supervisor_path(Group, SimpleId), + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:put(StoreId, Path, Record) of + ok -> {ok, State}; + Error -> Error + end; +copy_to_khepri(Table, Record, State) -> + ?LOG_DEBUG("Mnesia->Khepri unexpected record table ~0p record ~0p state ~0p", + [Table, Record, State]), + {error, unexpected_record}. + +-spec delete_from_khepri(Table, Key, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Key :: any(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +delete_from_khepri( + mirrored_sup_childspec = Table, {Group, Id} = Key, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] key: ~0p", + [Table, Key], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_msup:khepri_mirrored_supervisor_path(Group, Id), + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:delete(StoreId, Path) of + ok -> {ok, State}; + Error -> Error + end. + +-spec clear_data_in_khepri(Table) -> ok when + Table :: atom(). + +clear_data_in_khepri(mirrored_sup_childspec) -> + Path = rabbit_db_msup:khepri_mirrored_supervisor_path(), + case rabbit_khepri:delete(Path) of + ok -> ok; + Error -> throw(Error) + end. diff --git a/deps/rabbit/src/rabbit_db_policy.erl b/deps/rabbit/src/rabbit_db_policy.erl index e0ecf83ee09d..398c0e151e0d 100644 --- a/deps/rabbit/src/rabbit_db_policy.erl +++ b/deps/rabbit/src/rabbit_db_policy.erl @@ -27,7 +27,10 @@ Ret :: {[{Exchange, Exchange}], [{Queue, Queue}]}. update(VHost, GetUpdatedExchangeFun, GetUpdatedQueueFun) -> - update_in_mnesia(VHost, GetUpdatedExchangeFun, GetUpdatedQueueFun). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> update_in_mnesia(VHost, GetUpdatedExchangeFun, GetUpdatedQueueFun) end, + khepri => fun() -> update_in_khepri(VHost, GetUpdatedExchangeFun, GetUpdatedQueueFun) end + }). %% [1] We need to prevent this from becoming O(n^2) in a similar %% manner to rabbit_binding:remove_for_{source,destination}. So see @@ -48,6 +51,19 @@ update_in_mnesia(VHost, GetUpdatedExchangeFun, GetUpdatedQueueFun) -> || Map <- Queues, is_map(Map)]} end). +update_in_khepri(VHost, GetUpdatedExchangeFun, GetUpdatedQueueFun) -> + Exchanges0 = rabbit_db_exchange:get_all(VHost), + Queues0 = rabbit_db_queue:get_all(VHost), + Exchanges = [GetUpdatedExchangeFun(X) || X <- Exchanges0], + Queues = [GetUpdatedQueueFun(Q) || Q <- Queues0], + rabbit_khepri:transaction( + fun() -> + {[update_exchange_policies(Map, fun rabbit_db_exchange:update_in_khepri_tx/2) + || Map <- Exchanges, is_map(Map)], + [update_queue_policies(Map, fun rabbit_db_queue:update_in_khepri_tx/2) + || Map <- Queues, is_map(Map)]} + end, rw). + update_exchange_policies(#{exchange := X = #exchange{name = XName}, update_function := UpdateFun}, StoreFun) -> NewExchange = StoreFun(XName, UpdateFun), diff --git a/deps/rabbit/src/rabbit_db_queue.erl b/deps/rabbit/src/rabbit_db_queue.erl index 7993e2b4e082..ce6f096cd48c 100644 --- a/deps/rabbit/src/rabbit_db_queue.erl +++ b/deps/rabbit/src/rabbit_db_queue.erl @@ -7,6 +7,7 @@ -module(rabbit_db_queue). +-include_lib("khepri/include/khepri.hrl"). -include_lib("rabbit_common/include/rabbit.hrl"). -include_lib("stdlib/include/qlc.hrl"). -include("amqqueue.hrl"). @@ -26,10 +27,12 @@ set_many/1, delete/2, update/2, - update_decorators/1, + update_decorators/2, exists/1 ]). +%% Once mnesia is removed, all transient entities will be deleted. These can be replaced +%% with the plain get_all* functions -export([ get_all_durable/0, get_all_durable_by_type/1, @@ -40,28 +43,43 @@ consistent_exists/1 ]). -%% Used by on_node_up and on_node_down +%% Used by on_node_up and on_node_down. +%% Can be deleted once transient entities/mnesia are removed. -export([foreach_transient/1, delete_transient/1]). -%% Used only by forget all durable +%% Only used by rabbit_amqqueue:forget_node_for_queue, which is only called +%% by `rabbit_mnesia:remove_node_if_mnesia_running`. Thus, once mnesia and/or +%% HA queues are removed it can be deleted. -export([foreach_durable/2, internal_delete/3]). +%% Storing it on Khepri is not needed, this function is just used in +%% rabbit_quorum_queue to ensure the queue is present in the rabbit_queue +%% table and not just in rabbit_durable_queue. Can be deleted with mnesia removal -export([set_dirty/1]). %% Used by other rabbit_db_* modules -export([ update_in_mnesia_tx/2, - get_durable_in_mnesia_tx/1 + update_in_khepri_tx/2, + get_durable_in_mnesia_tx/1, + get_in_khepri_tx/1 ]). %% For testing -export([clear/0]). +-export([ + khepri_queue_path/1, + khepri_queues_path/0 + ]). + -define(MNESIA_TABLE, rabbit_queue). -define(MNESIA_DURABLE_TABLE, rabbit_durable_queue). +-define(KHEPRI_PROJECTION, rabbit_khepri_queue). + %% ------------------------------------------------------------------- %% get_all(). %% ------------------------------------------------------------------- @@ -76,7 +94,10 @@ %% @private get_all() -> - get_all_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_in_mnesia() end, + khepri => fun() -> get_all_in_khepri() end + }). get_all_in_mnesia() -> list_with_possible_retry_in_mnesia( @@ -84,6 +105,12 @@ get_all_in_mnesia() -> rabbit_db:list_in_mnesia(?MNESIA_TABLE, amqqueue:pattern_match_all()) end). +get_all_in_khepri() -> + list_with_possible_retry_in_khepri( + fun() -> + ets:tab2list(?KHEPRI_PROJECTION) + end). + -spec get_all(VHostName) -> [Queue] when VHostName :: vhost:name(), Queue :: amqqueue:amqqueue(). @@ -95,7 +122,10 @@ get_all_in_mnesia() -> %% @private get_all(VHostName) -> - get_all_in_mnesia(VHostName). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_in_mnesia(VHostName) end, + khepri => fun() -> get_all_in_khepri(VHostName) end + }). get_all_in_mnesia(VHostName) -> list_with_possible_retry_in_mnesia( @@ -104,6 +134,13 @@ get_all_in_mnesia(VHostName) -> rabbit_db:list_in_mnesia(?MNESIA_TABLE, Pattern) end). +get_all_in_khepri(VHostName) -> + list_with_possible_retry_in_khepri( + fun() -> + Pattern = amqqueue:pattern_match_on_name(rabbit_misc:r(VHostName, queue)), + ets:match_object(?KHEPRI_PROJECTION, Pattern) + end). + %% ------------------------------------------------------------------- %% get_all_durable(). %% ------------------------------------------------------------------- @@ -118,7 +155,10 @@ get_all_in_mnesia(VHostName) -> %% @private get_all_durable() -> - get_all_durable_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_durable_in_mnesia() end, + khepri => fun() -> get_all_durable_in_khepri() end + }). get_all_durable_in_mnesia() -> list_with_possible_retry_in_mnesia( @@ -126,6 +166,12 @@ get_all_durable_in_mnesia() -> rabbit_db:list_in_mnesia(?MNESIA_DURABLE_TABLE, amqqueue:pattern_match_all()) end). +get_all_durable_in_khepri() -> + list_with_possible_retry_in_khepri( + fun() -> + ets:tab2list(?KHEPRI_PROJECTION) + end). + -spec get_all_durable_by_type(Type) -> [Queue] when Type :: atom(), Queue :: amqqueue:amqqueue(). @@ -137,12 +183,19 @@ get_all_durable_in_mnesia() -> %% @private get_all_durable_by_type(Type) -> - get_all_durable_by_type_in_mnesia(Type). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_durable_by_type_in_mnesia(Type) end, + khepri => fun() -> get_all_durable_by_type_in_khepri(Type) end + }). get_all_durable_by_type_in_mnesia(Type) -> Pattern = amqqueue:pattern_match_on_type(Type), rabbit_db:list_in_mnesia(?MNESIA_DURABLE_TABLE, Pattern). +get_all_durable_by_type_in_khepri(Type) -> + Pattern = amqqueue:pattern_match_on_type(Type), + ets:match_object(?KHEPRI_PROJECTION, Pattern). + %% ------------------------------------------------------------------- %% filter_all_durable(). %% ------------------------------------------------------------------- @@ -158,7 +211,10 @@ get_all_durable_by_type_in_mnesia(Type) -> %% @private filter_all_durable(FilterFun) -> - filter_all_durable_in_mnesia(FilterFun). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> filter_all_durable_in_mnesia(FilterFun) end, + khepri => fun() -> filter_all_durable_in_khepri(FilterFun) end + }). filter_all_durable_in_mnesia(FilterFun) -> rabbit_mnesia:execute_mnesia_transaction( @@ -168,6 +224,16 @@ filter_all_durable_in_mnesia(FilterFun) -> ])) end). +filter_all_durable_in_khepri(FilterFun) -> + ets:foldl( + fun(Q, Acc0) -> + case FilterFun(Q) of + true -> [Q | Acc0]; + false -> Acc0 + end + end, + [], ?KHEPRI_PROJECTION). + %% ------------------------------------------------------------------- %% list(). %% ------------------------------------------------------------------- @@ -182,11 +248,18 @@ filter_all_durable_in_mnesia(FilterFun) -> %% @private list() -> - list_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> list_in_mnesia() end, + khepri => fun() -> list_in_khepri() end + }). list_in_mnesia() -> mnesia:dirty_all_keys(?MNESIA_TABLE). +list_in_khepri() -> + Pattern = amqqueue:pattern_match_on_name('$1'), + ets:select(?KHEPRI_PROJECTION, [{Pattern, [], ['$1']}]). + %% ------------------------------------------------------------------- %% count(). %% ------------------------------------------------------------------- @@ -201,11 +274,17 @@ list_in_mnesia() -> %% @private count() -> - count_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> count_in_mnesia() end, + khepri => fun() -> count_in_khepri() end + }). count_in_mnesia() -> mnesia:table_info(?MNESIA_TABLE, size). +count_in_khepri() -> + ets:info(?KHEPRI_PROJECTION, size). + -spec count(VHostName) -> Count when VHostName :: vhost:name(), Count :: integer(). @@ -226,7 +305,10 @@ count(VHostName) -> end. list_for_count(VHostName) -> - list_for_count_in_mnesia(VHostName). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> list_for_count_in_mnesia(VHostName) end, + khepri => fun() -> list_for_count_in_khepri(VHostName) end + }). list_for_count_in_mnesia(VHostName) -> %% this is certainly suboptimal but there is no way to count @@ -241,6 +323,10 @@ list_for_count_in_mnesia(VHostName) -> amqqueue:field_vhost())) end). +list_for_count_in_khepri(VHostName) -> + Pattern = amqqueue:pattern_match_on_name(rabbit_misc:r(VHostName, queue)), + ets:select_count(?KHEPRI_PROJECTION, [{Pattern, [], [true]}]). + %% ------------------------------------------------------------------- %% delete(). %% ------------------------------------------------------------------- @@ -251,7 +337,10 @@ list_for_count_in_mnesia(VHostName) -> Ret :: ok | Deletions :: rabbit_binding:deletions(). delete(QueueName, Reason) -> - delete_in_mnesia(QueueName, Reason). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> delete_in_mnesia(QueueName, Reason) end, + khepri => fun() -> delete_in_khepri(QueueName) end + }). delete_in_mnesia(QueueName, Reason) -> rabbit_mnesia:execute_mnesia_transaction( @@ -269,6 +358,23 @@ delete_in_mnesia(QueueName, Reason) -> end end). +delete_in_khepri(QueueName) -> + delete_in_khepri(QueueName, false). + +delete_in_khepri(QueueName, OnlyDurable) -> + rabbit_khepri:transaction( + fun () -> + Path = khepri_queue_path(QueueName), + case khepri_tx_adv:delete(Path) of + {ok, #{data := _}} -> + %% we want to execute some things, as decided by rabbit_exchange, + %% after the transaction. + rabbit_db_binding:delete_for_destination_in_khepri(QueueName, OnlyDurable); + {ok, _} -> + ok + end + end, rw). + %% ------------------------------------------------------------------- %% internal_delete(). %% ------------------------------------------------------------------- @@ -283,7 +389,10 @@ internal_delete(QueueName, OnlyDurable, Reason) -> %% Only used by rabbit_amqqueue:forget_node_for_queue, which is only called %% by `rabbit_mnesia:remove_node_if_mnesia_running'. Thus, once mnesia and/or %% HA queues are removed it can be removed. - internal_delete_in_mnesia(QueueName, OnlyDurable, Reason). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> internal_delete_in_mnesia(QueueName, OnlyDurable, Reason) end, + khepri => fun() -> delete_in_khepri(QueueName, OnlyDurable) end + }). internal_delete_in_mnesia(QueueName, OnlyDurable, Reason) -> ok = mnesia:delete({?MNESIA_TABLE, QueueName}), @@ -309,20 +418,20 @@ internal_delete_in_mnesia(QueueName, OnlyDurable, Reason) -> -spec get_many(rabbit_exchange:route_return()) -> [amqqueue:amqqueue() | {amqqueue:amqqueue(), rabbit_exchange:route_infos()}]. get_many(Names) when is_list(Names) -> - get_many_in_mnesia(?MNESIA_TABLE, Names). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_many_in_ets(?MNESIA_TABLE, Names) end, + khepri => fun() -> get_many_in_ets(?KHEPRI_PROJECTION, Names) end + }). -get_many_in_mnesia(Table, [{Name, RouteInfos}]) +get_many_in_ets(Table, [{Name, RouteInfos}]) when is_map(RouteInfos) -> case ets:lookup(Table, Name) of [] -> []; [Q] -> [{Q, RouteInfos}] end; -get_many_in_mnesia(Table, [Name]) -> +get_many_in_ets(Table, [Name]) -> ets:lookup(Table, Name); -get_many_in_mnesia(Table, Names) - when is_list(Names) -> - %% Normally we'd call mnesia:dirty_read/1 here, but that is quite - %% expensive for reasons explained in rabbit_mnesia:dirty_read/1. +get_many_in_ets(Table, Names) when is_list(Names) -> lists:filtermap(fun({Name, RouteInfos}) when is_map(RouteInfos) -> case ets:lookup(Table, Name) of @@ -344,11 +453,20 @@ get_many_in_mnesia(Table, Names) QName :: rabbit_amqqueue:name(), Ret :: {ok, Queue :: amqqueue:amqqueue()} | {error, not_found}. get(Name) -> - get_in_mnesia(Name). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_in_mnesia(Name) end, + khepri => fun() -> get_in_khepri(Name) end + }). get_in_mnesia(Name) -> rabbit_mnesia:dirty_read({?MNESIA_TABLE, Name}). +get_in_khepri(Name) -> + case ets:lookup(?KHEPRI_PROJECTION, Name) of + [Q] -> {ok, Q}; + [] -> {error, not_found} + end. + %% ------------------------------------------------------------------- %% get_durable(). %% ------------------------------------------------------------------- @@ -358,7 +476,10 @@ get_in_mnesia(Name) -> Ret :: {ok, Queue :: amqqueue:amqqueue()} | {error, not_found}. get_durable(Name) -> - get_durable_in_mnesia(Name). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_durable_in_mnesia(Name) end, + khepri => fun() -> get_in_khepri(Name) end + }). get_durable_in_mnesia(Name) -> rabbit_mnesia:dirty_read({?MNESIA_DURABLE_TABLE, Name}). @@ -372,7 +493,10 @@ get_durable_in_mnesia(Name) -> Ret :: [Queue :: amqqueue:amqqueue()]. get_many_durable(Names) when is_list(Names) -> - get_many_in_mnesia(?MNESIA_DURABLE_TABLE, Names). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_many_in_ets(?MNESIA_DURABLE_TABLE, Names) end, + khepri => fun() -> get_many_in_ets(?KHEPRI_PROJECTION, Names) end + }). %% ------------------------------------------------------------------- %% update(). @@ -382,13 +506,16 @@ get_many_durable(Names) when is_list(Names) -> QName :: rabbit_amqqueue:name(), Queue :: amqqueue:amqqueue(), UpdateFun :: fun((Queue) -> Queue), - Ret :: Queue | not_found. + Ret :: Queue | not_found. %% @doc Updates an existing queue record using `UpdateFun'. %% %% @private update(QName, Fun) -> - update_in_mnesia(QName, Fun). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> update_in_mnesia(QName, Fun) end, + khepri => fun() -> update_in_khepri(QName, Fun) end + }). update_in_mnesia(QName, Fun) -> rabbit_mnesia:execute_mnesia_transaction( @@ -396,29 +523,77 @@ update_in_mnesia(QName, Fun) -> update_in_mnesia_tx(QName, Fun) end). +update_in_khepri(QName, Fun) -> + Path = khepri_queue_path(QName), + Ret1 = rabbit_khepri:adv_get(Path), + case Ret1 of + {ok, #{data := Q, payload_version := Vsn}} -> + UpdatePath = khepri_path:combine_with_conditions( + Path, [#if_payload_version{version = Vsn}]), + Q1 = Fun(Q), + Ret2 = rabbit_khepri:put(UpdatePath, Q1), + case Ret2 of + ok -> Q1; + {error, {khepri, mismatching_node, _}} -> + update_in_khepri(QName, Fun); + Err -> Err + end; + _ -> + not_found + end. + %% ------------------------------------------------------------------- %% update_decorators(). %% ------------------------------------------------------------------- --spec update_decorators(QName) -> ok when - QName :: rabbit_amqqueue:name(). +-spec update_decorators(QName, [Decorator]) -> ok when + QName :: rabbit_amqqueue:name(), + Decorator :: atom(). %% @doc Updates an existing queue record adding the active queue decorators. %% %% @private -update_decorators(QName) -> - update_decorators_in_mnesia(QName). +update_decorators(QName, Decorators) -> + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> update_decorators_in_mnesia(QName, Decorators) end, + khepri => fun() -> update_decorators_in_khepri(QName, Decorators) end + }). -update_decorators_in_mnesia(Name) -> +update_decorators_in_mnesia(Name, Decorators) -> rabbit_mnesia:execute_mnesia_transaction( fun() -> case mnesia:wread({?MNESIA_TABLE, Name}) of - [Q] -> ok = mnesia:write(?MNESIA_TABLE, rabbit_queue_decorator:set(Q), + [Q] -> ok = mnesia:write(?MNESIA_TABLE, amqqueue:set_decorators(Q, Decorators), write); [] -> ok end end). +update_decorators_in_khepri(QName, Decorators) -> + %% Decorators are stored on an ETS table, so we need to query them before the transaction. + %% Also, to verify which ones are active could lead to any kind of side-effects. + %% Thus it needs to be done outside of the transaction. + %% Decorators have just been calculated on `rabbit_queue_decorator:maybe_recover/1`, thus + %% we can update them here directly. + Path = khepri_queue_path(QName), + Ret1 = rabbit_khepri:adv_get(Path), + case Ret1 of + {ok, #{data := Q0, payload_version := Vsn}} -> + Q1 = amqqueue:reset_mirroring_and_decorators(Q0), + Q2 = amqqueue:set_decorators(Q1, Decorators), + UpdatePath = khepri_path:combine_with_conditions( + Path, [#if_payload_version{version = Vsn}]), + Ret2 = rabbit_khepri:put(UpdatePath, Q2), + case Ret2 of + ok -> ok; + {error, {khepri, mismatching_node, _}} -> + update_decorators_in_khepri(QName, Decorators); + {error, _} = Error -> Error + end; + _ -> + ok + end. + %% ------------------------------------------------------------------- %% update_durable(). %% ------------------------------------------------------------------- @@ -432,7 +607,12 @@ update_decorators_in_mnesia(Name) -> %% @private update_durable(UpdateFun, FilterFun) -> - update_durable_in_mnesia(UpdateFun, FilterFun). + rabbit_khepri:handle_fallback( + #{mnesia => + fun() -> update_durable_in_mnesia(UpdateFun, FilterFun) end, + khepri => + fun() -> update_durable_in_khepri(UpdateFun, FilterFun) end + }). update_durable_in_mnesia(UpdateFun, FilterFun) -> Pattern = amqqueue:pattern_match_all(), @@ -446,6 +626,21 @@ update_durable_in_mnesia(UpdateFun, FilterFun) -> end), ok. +update_durable_in_khepri(UpdateFun, FilterFun) -> + Path = khepri_queues_path() ++ [rabbit_khepri:if_has_data_wildcard()], + rabbit_khepri:transaction( + fun() -> + khepri_tx:foreach(Path, + fun(Path0, #{data := Q}) -> + case FilterFun(Q) of + true -> + khepri_tx:put(Path0, UpdateFun(Q)); + false -> + ok + end + end) + end). + %% ------------------------------------------------------------------- %% exists(). %% ------------------------------------------------------------------- @@ -460,11 +655,17 @@ update_durable_in_mnesia(UpdateFun, FilterFun) -> %% @private exists(QName) -> - exists_in_mnesia(QName). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> exists_in_mnesia(QName) end, + khepri => fun() -> exists_in_khepri(QName) end + }). exists_in_mnesia(QName) -> ets:member(?MNESIA_TABLE, QName). +exists_in_khepri(QName) -> + ets:member(?KHEPRI_PROJECTION, QName). + %% ------------------------------------------------------------------- %% exists(). %% ------------------------------------------------------------------- @@ -481,7 +682,10 @@ exists_in_mnesia(QName) -> %% @private consistent_exists(QName) -> - consistent_exists_in_mnesia(QName). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> consistent_exists_in_mnesia(QName) end, + khepri => fun() -> exists_in_khepri(QName) end + }). consistent_exists_in_mnesia(QName) -> case mnesia:read({?MNESIA_TABLE, QName}) of @@ -505,11 +709,17 @@ consistent_exists_in_mnesia(QName) -> get_all_by_type(Type) -> Pattern = amqqueue:pattern_match_on_type(Type), - get_all_by_pattern_in_mnesia(Pattern). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_by_pattern_in_mnesia(Pattern) end, + khepri => fun() -> get_all_by_pattern_in_khepri(Pattern) end + }). get_all_by_pattern_in_mnesia(Pattern) -> rabbit_db:list_in_mnesia(?MNESIA_TABLE, Pattern). +get_all_by_pattern_in_khepri(Pattern) -> + rabbit_db:list_in_khepri(khepri_queues_path() ++ [rabbit_khepri:if_has_data([?KHEPRI_WILDCARD_STAR_STAR, #if_data_matches{pattern = Pattern}])]). + %% ------------------------------------------------------------------- %% get_all_by_type_and_node(). %% ------------------------------------------------------------------- @@ -527,7 +737,10 @@ get_all_by_pattern_in_mnesia(Pattern) -> %% @private get_all_by_type_and_node(VHostName, Type, Node) -> - get_all_by_type_and_node_in_mnesia(VHostName, Type, Node). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_by_type_and_node_in_mnesia(VHostName, Type, Node) end, + khepri => fun() -> get_all_by_type_and_node_in_khepri(VHostName, Type, Node) end + }). get_all_by_type_and_node_in_mnesia(VHostName, Type, Node) -> mnesia:async_dirty( @@ -538,6 +751,11 @@ get_all_by_type_and_node_in_mnesia(VHostName, Type, Node) -> amqqueue:qnode(Q) == Node])) end). +get_all_by_type_and_node_in_khepri(VHostName, Type, Node) -> + Pattern = amqqueue:pattern_match_on_type(Type), + Qs = rabbit_db:list_in_khepri(khepri_queues_path() ++ [VHostName, rabbit_khepri:if_has_data([?KHEPRI_WILDCARD_STAR_STAR, #if_data_matches{pattern = Pattern}])]), + [Q || Q <- Qs, amqqueue:qnode(Q) == Node]. + %% ------------------------------------------------------------------- %% create_or_get(). %% ------------------------------------------------------------------- @@ -548,12 +766,15 @@ get_all_by_type_and_node_in_mnesia(VHostName, Type, Node) -> %% @doc Writes a queue record if it doesn't exist already or returns the existing one %% %% @returns the existing record if there is one in the database already, or the newly -%% created record. +%% created record. %% %% @private create_or_get(Q) -> - create_or_get_in_mnesia(Q). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> create_or_get_in_mnesia(Q) end, + khepri => fun() -> create_or_get_in_khepri(Q) end + }). create_or_get_in_mnesia(Q) -> DurableQ = amqqueue:reset_mirroring_and_decorators(Q), @@ -574,6 +795,18 @@ create_or_get_in_mnesia(Q) -> end end). +create_or_get_in_khepri(Q) -> + QueueName = amqqueue:get_name(Q), + Path = khepri_queue_path(QueueName), + case rabbit_khepri:adv_create(Path, Q) of + {error, {khepri, mismatching_node, #{node_props := #{data := ExistingQ}}}} -> + {existing, ExistingQ}; + {ok, _} -> + {created, Q}; + Error -> + Error + end. + %% ------------------------------------------------------------------- %% set(). %% ------------------------------------------------------------------- @@ -587,7 +820,10 @@ create_or_get_in_mnesia(Q) -> %% @private set(Q) -> - set_in_mnesia(Q). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> set_in_mnesia(Q) end, + khepri => fun() -> set_in_khepri(Q) end + }). set_in_mnesia(Q) -> DurableQ = amqqueue:reset_mirroring_and_decorators(Q), @@ -605,6 +841,10 @@ set_in_mnesia_tx(DurableQ, Q) -> end, ok = mnesia:write(?MNESIA_TABLE, Q, write). +set_in_khepri(Q) -> + Path = khepri_queue_path(amqqueue:get_name(Q)), + rabbit_khepri:put(Path, Q). + %% ------------------------------------------------------------------- %% set_many(). %% ------------------------------------------------------------------- @@ -618,7 +858,10 @@ set_in_mnesia_tx(DurableQ, Q) -> %% @private set_many(Qs) -> - set_many_in_mnesia(Qs). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> set_many_in_mnesia(Qs) end, + khepri => fun() -> set_many_in_khepri(Qs) end + }). set_many_in_mnesia(Qs) -> {atomic, ok} = @@ -630,6 +873,19 @@ set_many_in_mnesia(Qs) -> end), ok. +set_many_in_khepri(Qs) -> + rabbit_khepri:transaction( + fun() -> + [begin + Path = khepri_queue_path(amqqueue:get_name(Q)), + case khepri_tx:put(Path, Q) of + ok -> ok; + Error -> khepri_tx:abort(Error) + end + end || Q <- Qs] + end), + ok. + %% ------------------------------------------------------------------- %% delete_transient(). %% ------------------------------------------------------------------- @@ -644,7 +900,10 @@ set_many_in_mnesia(Qs) -> %% @private delete_transient(FilterFun) -> - delete_transient_in_mnesia(FilterFun). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> delete_transient_in_mnesia(FilterFun) end, + khepri => fun() -> ok end + }). delete_transient_in_mnesia(FilterFun) -> Qs = rabbit_mnesia:execute_mnesia_transaction( @@ -697,7 +956,10 @@ partition_queues(T) -> %% @private foreach_transient(UpdateFun) -> - foreach_transient_in_mnesia(UpdateFun). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> foreach_transient_in_mnesia(UpdateFun) end, + khepri => fun() -> ok end + }). foreach_transient_in_mnesia(UpdateFun) -> Pattern = amqqueue:pattern_match_all(), @@ -720,7 +982,10 @@ foreach_transient_in_mnesia(UpdateFun) -> %% @private foreach_durable(UpdateFun, FilterFun) -> - foreach_durable_in_mnesia(UpdateFun, FilterFun). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> foreach_durable_in_mnesia(UpdateFun, FilterFun) end, + khepri => fun() -> foreach_durable_in_khepri(UpdateFun, FilterFun) end + }). foreach_durable_in_mnesia(UpdateFun, FilterFun) -> %% Note rabbit is not running so we avoid e.g. the worker pool. Also why @@ -733,7 +998,19 @@ foreach_durable_in_mnesia(UpdateFun, FilterFun) -> _ = [UpdateFun(Q) || Q <- Qs, FilterFun(Q)], ok end), - ok. + ok. + +foreach_durable_in_khepri(UpdateFun, FilterFun) -> + Path = khepri_queues_path() ++ [rabbit_khepri:if_has_data_wildcard()], + case rabbit_khepri:filter(Path, fun(_, #{data := Q}) -> + FilterFun(Q) + end) of + {ok, Qs} -> + _ = [UpdateFun(Q) || Q <- maps:values(Qs)], + ok; + Error -> + Error + end. %% ------------------------------------------------------------------- %% set_dirty(). @@ -746,7 +1023,10 @@ foreach_durable_in_mnesia(UpdateFun, FilterFun) -> %% @private set_dirty(Q) -> - set_dirty_in_mnesia(Q). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> set_dirty_in_mnesia(Q) end, + khepri => ok + }). set_dirty_in_mnesia(Q) -> ok = mnesia:dirty_write(?MNESIA_TABLE, rabbit_queue_decorator:set(Q)). @@ -776,6 +1056,27 @@ update_in_mnesia_tx(Name, Fun) -> not_found end. +%% ------------------------------------------------------------------- +%% update_in_khepri_tx(). +%% ------------------------------------------------------------------- + +-spec update_in_khepri_tx(QName, UpdateFun) -> Ret when + QName :: rabbit_amqqueue:name(), + Queue :: amqqueue:amqqueue(), + UpdateFun :: fun((Queue) -> Queue), + Ret :: Queue | not_found. + +update_in_khepri_tx(Name, Fun) -> + Path = khepri_queue_path(Name), + case khepri_tx:get(Path) of + {ok, Q} -> + Q1 = Fun(Q), + ok = khepri_tx:put(Path, Q1), + Q1; + _ -> + not_found + end. + %% ------------------------------------------------------------------- %% get_durable_in_mnesia_tx(). %% ------------------------------------------------------------------- @@ -790,6 +1091,13 @@ get_durable_in_mnesia_tx(Name) -> [Q] -> {ok, Q} end. +%% TODO this should be internal, it's here because of mirrored queues +get_in_khepri_tx(Name) -> + case khepri_tx:get(khepri_queue_path(Name)) of + {ok, X} -> [X]; + _ -> [] + end. + %% ------------------------------------------------------------------- %% clear(). %% ------------------------------------------------------------------- @@ -800,13 +1108,23 @@ get_durable_in_mnesia_tx(Name) -> %% @private clear() -> - clear_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> clear_in_mnesia() end, + khepri => fun() -> clear_in_khepri() end}). clear_in_mnesia() -> {atomic, ok} = mnesia:clear_table(?MNESIA_TABLE), {atomic, ok} = mnesia:clear_table(?MNESIA_DURABLE_TABLE), ok. +clear_in_khepri() -> + Path = khepri_queues_path(), + case rabbit_khepri:delete(Path) of + ok -> ok; + Error -> throw(Error) + end. + +%% -------------------------------------------------------------- %% Internal %% -------------------------------------------------------------- @@ -841,3 +1159,34 @@ list_with_possible_retry_in_mnesia(Fun) -> Ret -> Ret end. + +list_with_possible_retry_in_khepri(Fun) -> + %% See equivalent `list_with_possible_retry_in_mnesia` first. + %% Not sure how much of this is possible in Khepri, as there is no dirty read, + %% but the amqqueue record migration is still happening. + %% Let's retry just in case + AmqqueueRecordVersion = amqqueue:record_version_to_use(), + case Fun() of + [] -> + case khepri_tx:is_transaction() of + true -> + []; + false -> + case amqqueue:record_version_to_use() of + AmqqueueRecordVersion -> []; + _ -> Fun() + end + end; + Ret -> + Ret + end. + +%% -------------------------------------------------------------- +%% Khepri paths +%% -------------------------------------------------------------- + +khepri_queues_path() -> + [?MODULE, queues]. + +khepri_queue_path(#resource{virtual_host = VHost, name = Name}) -> + [?MODULE, queues, VHost, Name]. diff --git a/deps/rabbit/src/rabbit_db_queue_m2k_converter.erl b/deps/rabbit/src/rabbit_db_queue_m2k_converter.erl new file mode 100644 index 000000000000..c0a61eb010bf --- /dev/null +++ b/deps/rabbit/src/rabbit_db_queue_m2k_converter.erl @@ -0,0 +1,105 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright © 2022-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_queue_m2k_converter). + +-behaviour(mnesia_to_khepri_converter). + +-include_lib("kernel/include/logger.hrl"). +-include_lib("khepri/include/khepri.hrl"). +-include_lib("khepri_mnesia_migration/src/kmm_logging.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). + +-export([init_copy_to_khepri/3, + copy_to_khepri/3, + delete_from_khepri/3]). + +-record(?MODULE, {store_id :: khepri:store_id()}). + +-spec init_copy_to_khepri(StoreId, MigrationId, Tables) -> Ret when + StoreId :: khepri:store_id(), + MigrationId :: mnesia_to_khepri:migration_id(), + Tables :: [mnesia_to_khepri:mnesia_table()], + Ret :: {ok, Priv}, + Priv :: #?MODULE{}. +%% @private + +init_copy_to_khepri(StoreId, _MigrationId, Tables) -> + %% Clean up any previous attempt to copy the Mnesia table to Khepri. + lists:foreach(fun clear_data_in_khepri/1, Tables), + + State = #?MODULE{store_id = StoreId}, + {ok, State}. + +-spec copy_to_khepri(Table, Record, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Record :: tuple(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +copy_to_khepri(rabbit_queue = Table, Record, + #?MODULE{store_id = StoreId} = State) -> + Name = amqqueue:get_name(Record), + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] key: ~0p", + [Table, Name], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_queue:khepri_queue_path(Name), + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:put(StoreId, Path, Record) of + ok -> {ok, State}; + Error -> Error + end; +copy_to_khepri(Table, Record, State) -> + ?LOG_DEBUG("Mnesia->Khepri unexpected record table ~0p record ~0p state ~0p", + [Table, Record, State]), + {error, unexpected_record}. + +-spec delete_from_khepri(Table, Key, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Key :: any(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +delete_from_khepri(rabbit_queue = Table, Key, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] key: ~0p", + [Table, Key], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_queue:khepri_queue_path(Key), + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:delete(StoreId, Path) of + ok -> {ok, State}; + Error -> Error + end. + +-spec clear_data_in_khepri(Table) -> ok when + Table :: atom(). + +clear_data_in_khepri(rabbit_queue) -> + khepri_delete(rabbit_db_queue:khepri_queues_path()); +clear_data_in_khepri(rabbit_durable_queue) -> + khepri_delete(rabbit_db_queue:khepri_queues_path()). + +khepri_delete(Path) -> + case rabbit_khepri:delete(Path) of + ok -> ok; + Error -> throw(Error) + end. diff --git a/deps/rabbit/src/rabbit_db_rtparams.erl b/deps/rabbit/src/rabbit_db_rtparams.erl index 850dc949a6b6..bc100c177ea0 100644 --- a/deps/rabbit/src/rabbit_db_rtparams.erl +++ b/deps/rabbit/src/rabbit_db_rtparams.erl @@ -7,6 +7,7 @@ -module(rabbit_db_rtparams). +-include_lib("khepri/include/khepri.hrl"). -include_lib("rabbit_common/include/rabbit.hrl"). -export([set/2, set/4, @@ -15,7 +16,17 @@ get_all/0, get_all/2, delete/1, delete/3]). +-export([khepri_vhost_rp_path/3, + khepri_global_rp_path/1, + khepri_rp_path/0 + ]). + -define(MNESIA_TABLE, rabbit_runtime_parameters). +-define(KHEPRI_PROJECTION, rabbit_khepri_runtime_parameters). +-define(any(Value), case Value of + '_' -> ?KHEPRI_WILDCARD_STAR; + _ -> Value + end). %% ------------------------------------------------------------------- %% set(). @@ -33,12 +44,25 @@ %% @private set(Key, Term) when is_atom(Key) -> - set_in_mnesia(Key, Term). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> set_in_mnesia(Key, Term) end, + khepri => fun() -> set_in_khepri(Key, Term) end}). set_in_mnesia(Key, Term) -> rabbit_mnesia:execute_mnesia_transaction( fun() -> set_in_mnesia_tx(Key, Term) end). +set_in_khepri(Key, Term) -> + Path = khepri_rp_path(Key), + Record = #runtime_parameters{key = Key, + value = Term}, + case rabbit_khepri:adv_put(Path, Record) of + {ok, #{data := Params}} -> + {old, Params#runtime_parameters.value}; + {ok, _} -> + new + end. + -spec set(VHostName, Comp, Name, Term) -> Ret when VHostName :: vhost:name(), Comp :: binary(), @@ -58,7 +82,9 @@ set(VHostName, Comp, Name, Term) is_binary(Comp) andalso (is_binary(Name) orelse is_atom(Name)) -> Key = {VHostName, Comp, Name}, - set_in_mnesia(VHostName, Key, Term). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> set_in_mnesia(VHostName, Key, Term) end, + khepri => fun() -> set_in_khepri(VHostName, Key, Term) end}). set_in_mnesia(VHostName, Key, Term) -> rabbit_mnesia:execute_mnesia_transaction( @@ -76,6 +102,22 @@ set_in_mnesia_tx(Key, Term) -> mnesia:write(?MNESIA_TABLE, Record, write), Res. +set_in_khepri(VHostName, Key, Term) -> + rabbit_khepri:transaction( + rabbit_db_vhost:with_fun_in_khepri_tx( + VHostName, fun() -> set_in_khepri_tx(Key, Term) end), rw). + +set_in_khepri_tx(Key, Term) -> + Path = khepri_rp_path(Key), + Record = #runtime_parameters{key = Key, + value = Term}, + case khepri_tx_adv:put(Path, Record) of + {ok, #{data := Params}} -> + {old, Params#runtime_parameters.value}; + {ok, _} -> + new + end. + %% ------------------------------------------------------------------- %% get(). %% ------------------------------------------------------------------- @@ -94,9 +136,13 @@ get({VHostName, Comp, Name} = Key) when is_binary(VHostName) andalso is_binary(Comp) andalso (is_binary(Name) orelse is_atom(Name)) -> - get_in_mnesia(Key); + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_in_mnesia(Key) end, + khepri => fun() -> get_in_khepri(Key) end}); get(Key) when is_atom(Key) -> - get_in_mnesia(Key). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_in_mnesia(Key) end, + khepri => fun() -> get_in_khepri(Key) end}). get_in_mnesia(Key) -> case mnesia:dirty_read(?MNESIA_TABLE, Key) of @@ -104,6 +150,12 @@ get_in_mnesia(Key) -> [Record] -> Record end. +get_in_khepri(Key) -> + case ets:lookup(?KHEPRI_PROJECTION, Key) of + [] -> undefined; + [Record] -> Record + end. + %% ------------------------------------------------------------------- %% get_or_set(). %% ------------------------------------------------------------------- @@ -120,9 +172,14 @@ get_or_set({VHostName, Comp, Name} = Key, Default) when is_binary(VHostName) andalso is_binary(Comp) andalso (is_binary(Name) orelse is_atom(Name)) -> - get_or_set_in_mnesia(Key, Default); + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_or_set_in_mnesia(Key, Default) end, + khepri => fun() -> get_or_set_in_khepri(Key, Default) end}); get_or_set(Key, Default) -> - get_or_set_in_mnesia(Key, Default). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_or_set_in_mnesia(Key, Default) end, + khepri => fun() -> get_or_set_in_khepri(Key, Default) end + }). get_or_set_in_mnesia(Key, Default) -> rabbit_mnesia:execute_mnesia_transaction( @@ -139,6 +196,21 @@ get_or_set_in_mnesia_tx(Key, Default) -> Record end. +get_or_set_in_khepri(Key, Default) -> + Path = khepri_rp_path(Key), + rabbit_khepri:transaction( + fun () -> + case khepri_tx:get(Path) of + {ok, undefined} -> + Record = #runtime_parameters{key = Key, + value = Default}, + ok = khepri_tx:put(Path, Record), + Record; + {ok, R} -> + R + end + end). + %% ------------------------------------------------------------------- %% get_all(). %% ------------------------------------------------------------------- @@ -152,11 +224,16 @@ get_or_set_in_mnesia_tx(Key, Default) -> %% @private get_all() -> - get_all_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_in_mnesia() end, + khepri => fun() -> get_all_in_khepri() end}). get_all_in_mnesia() -> rabbit_mnesia:dirty_read_all(?MNESIA_TABLE). +get_all_in_khepri() -> + ets:tab2list(?KHEPRI_PROJECTION). + -spec get_all(VHostName, Comp) -> Ret when VHostName :: vhost:name() | '_', Comp :: binary() | '_', @@ -171,7 +248,9 @@ get_all_in_mnesia() -> get_all(VHostName, Comp) when (is_binary(VHostName) orelse VHostName =:= '_') andalso (is_binary(Comp) orelse Comp =:= '_') -> - get_all_in_mnesia(VHostName, Comp). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_in_mnesia(VHostName, Comp) end, + khepri => fun() -> get_all_in_khepri(VHostName, Comp) end}). get_all_in_mnesia(VHostName, Comp) -> mnesia:async_dirty( @@ -185,6 +264,15 @@ get_all_in_mnesia(VHostName, Comp) -> mnesia:match_object(?MNESIA_TABLE, Match, read) end). +get_all_in_khepri(VHostName, Comp) -> + case VHostName of + '_' -> ok; + _ -> rabbit_vhost:assert(VHostName) + end, + Match = #runtime_parameters{key = {VHostName, Comp, '_'}, + _ = '_'}, + ets:match_object(?KHEPRI_PROJECTION, Match). + %% ------------------------------------------------------------------- %% delete(). %% ------------------------------------------------------------------- @@ -196,7 +284,9 @@ get_all_in_mnesia(VHostName, Comp) -> %% @private delete(Key) when is_atom(Key) -> - delete_in_mnesia(Key). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> delete_in_mnesia(Key) end, + khepri => fun() -> delete_in_khepri(Key) end}). -spec delete(VHostName, Comp, Name) -> ok when VHostName :: vhost:name() | '_', @@ -212,10 +302,16 @@ delete(VHostName, Comp, Name) is_binary(Comp) andalso (is_binary(Name) orelse (is_atom(Name) andalso Name =/= '_')) -> Key = {VHostName, Comp, Name}, - delete_in_mnesia(Key); + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> delete_in_mnesia(Key) end, + khepri => fun() -> delete_in_khepri(Key) end}); delete(VHostName, Comp, Name) when VHostName =:= '_' orelse Comp =:= '_' orelse Name =:= '_' -> - delete_matching_in_mnesia(VHostName, Comp, Name). + rabbit_khepri:handle_fallback( + #{mnesia => + fun() -> delete_matching_in_mnesia(VHostName, Comp, Name) end, + khepri => + fun() -> delete_matching_in_khepri(VHostName, Comp, Name) end}). delete_in_mnesia(Key) -> rabbit_mnesia:execute_mnesia_transaction( @@ -235,3 +331,26 @@ delete_matching_in_mnesia_tx(VHostName, Comp, Name) -> || #runtime_parameters{key = Key} <- mnesia:match_object(?MNESIA_TABLE, Match, write)], ok. + +delete_in_khepri(Key) -> + Path = khepri_rp_path(Key), + ok = rabbit_khepri:delete(Path). + +delete_matching_in_khepri(VHostName, Comp, Name) -> + Key = {?any(VHostName), ?any(Comp), ?any(Name)}, + delete_in_khepri(Key). + +khepri_rp_path() -> + [?MODULE]. + +khepri_rp_path({VHost, Component, Name}) -> + khepri_vhost_rp_path(VHost, Component, Name); +khepri_rp_path(Key) -> + khepri_global_rp_path(Key). + +khepri_global_rp_path(Key) -> + [?MODULE, global, Key]. + +khepri_vhost_rp_path(VHost, Component, Name) -> + [?MODULE, per_vhost, VHost, Component, Name]. + diff --git a/deps/rabbit/src/rabbit_db_rtparams_m2k_converter.erl b/deps/rabbit/src/rabbit_db_rtparams_m2k_converter.erl new file mode 100644 index 000000000000..c13542ad72e1 --- /dev/null +++ b/deps/rabbit/src/rabbit_db_rtparams_m2k_converter.erl @@ -0,0 +1,107 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright © 2022-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_rtparams_m2k_converter). + +-behaviour(mnesia_to_khepri_converter). + +-include_lib("kernel/include/logger.hrl"). +-include_lib("khepri/include/khepri.hrl"). +-include_lib("khepri_mnesia_migration/src/kmm_logging.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). + +-export([init_copy_to_khepri/3, + copy_to_khepri/3, + delete_from_khepri/3]). + +-record(?MODULE, {store_id :: khepri:store_id()}). + +-spec init_copy_to_khepri(StoreId, MigrationId, Tables) -> Ret when + StoreId :: khepri:store_id(), + MigrationId :: mnesia_to_khepri:migration_id(), + Tables :: [mnesia_to_khepri:mnesia_table()], + Ret :: {ok, Priv}, + Priv :: #?MODULE{}. +%% @private + +init_copy_to_khepri(StoreId, _MigrationId, Tables) -> + %% Clean up any previous attempt to copy the Mnesia table to Khepri. + lists:foreach(fun clear_data_in_khepri/1, Tables), + + State = #?MODULE{store_id = StoreId}, + {ok, State}. + +-spec copy_to_khepri(Table, Record, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Record :: tuple(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +copy_to_khepri( + rabbit_runtime_parameters = Table, #runtime_parameters{key = Key} = Record, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] key: ~0p", + [Table, Key], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rtparams_path(Key), + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:put(StoreId, Path, Record) of + ok -> {ok, State}; + Error -> Error + end; +copy_to_khepri(Table, Record, State) -> + ?LOG_DEBUG("Mnesia->Khepri unexpected record table ~0p record ~0p state ~0p", + [Table, Record, State]), + {error, unexpected_record}. + +-spec delete_from_khepri(Table, Key, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Key :: any(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +delete_from_khepri( + rabbit_runtime_parameters = Table, Key, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] key: ~0p", + [Table, Key], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rtparams_path(Key), + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:delete(StoreId, Path) of + ok -> {ok, State}; + Error -> Error + end. + +rtparams_path({VHost, Comp, Name})-> + rabbit_db_rtparams:khepri_vhost_rp_path(VHost, Comp, Name); +rtparams_path(Key) -> + rabbit_db_rtparams:khepri_global_rp_path(Key). + +-spec clear_data_in_khepri(Table) -> ok when + Table :: atom(). + +clear_data_in_khepri(rabbit_runtime_parameters) -> + Path = rabbit_db_rtparams:khepri_rp_path(), + case rabbit_khepri:delete(Path) of + ok -> ok; + Error -> throw(Error) + end. diff --git a/deps/rabbit/src/rabbit_db_topic_exchange.erl b/deps/rabbit/src/rabbit_db_topic_exchange.erl index 6d77d72afbdb..c8a2dba4d175 100644 --- a/deps/rabbit/src/rabbit_db_topic_exchange.erl +++ b/deps/rabbit/src/rabbit_db_topic_exchange.erl @@ -11,6 +11,15 @@ -export([set/1, delete_all_for_exchange/1, delete/1, match/3]). +%% These functions are used to process mnesia deletion events generated during the +%% migration from mnesia to khepri +-export([ + split_topic_key/1, + split_topic_key_binary/1, + trie_binding_to_key/1, + trie_records_to_key/1 + ]). + %% For testing -export([clear/0]). @@ -21,6 +30,8 @@ -type match_result() :: [rabbit_types:binding_destination() | {rabbit_amqqueue:name(), rabbit_types:binding_key()}]. +-define(COMPILED_TOPIC_SPLIT_PATTERN, dot_binary_pattern). + %% ------------------------------------------------------------------- %% set(). %% ------------------------------------------------------------------- @@ -32,7 +43,22 @@ %% @private set(#binding{source = XName, key = BindingKey, destination = Destination, args = Args}) -> - set_in_mnesia(XName, BindingKey, Destination, Args). + rabbit_khepri:handle_fallback( + #{ + mnesia => fun() -> set_in_mnesia(XName, BindingKey, Destination, Args) end, + khepri => fun() -> set_in_khepri(XName, BindingKey, Destination, Args) end + }). + +set_in_mnesia(XName, BindingKey, Destination, Args) -> + rabbit_mnesia:execute_mnesia_transaction( + fun() -> + FinalNode = follow_down_create(XName, split_topic_key(BindingKey)), + trie_add_binding(XName, FinalNode, Destination, Args), + ok + end). + +set_in_khepri(_XName, _RoutingKey, _Destination, _Args) -> + ok. %% ------------------------------------------------------------------- %% delete_all_for_exchange(). @@ -45,7 +71,23 @@ set(#binding{source = XName, key = BindingKey, destination = Destination, args = %% @private delete_all_for_exchange(XName) -> - delete_all_for_exchange_in_mnesia(XName). + rabbit_khepri:handle_fallback( + #{ + mnesia => fun() -> delete_all_for_exchange_in_mnesia(XName) end, + khepri => fun() -> delete_all_for_exchange_in_khepri(XName) end + }). + +delete_all_for_exchange_in_mnesia(XName) -> + rabbit_mnesia:execute_mnesia_transaction( + fun() -> + trie_remove_all_nodes(XName), + trie_remove_all_edges(XName), + trie_remove_all_bindings(XName), + ok + end). + +delete_all_for_exchange_in_khepri(_XName) -> + ok. %% ------------------------------------------------------------------- %% delete(). @@ -58,7 +100,18 @@ delete_all_for_exchange(XName) -> %% @private delete(Bs) when is_list(Bs) -> - delete_in_mnesia(Bs). + rabbit_khepri:handle_fallback( + #{ + mnesia => fun() -> delete_in_mnesia(Bs) end, + khepri => fun() -> delete_in_khepri(Bs) end + }). + +delete_in_mnesia(Bs) -> + rabbit_mnesia:execute_mnesia_transaction( + fun() -> delete_in_mnesia_tx(Bs) end). + +delete_in_khepri(_Bs) -> + ok. %% ------------------------------------------------------------------- %% match(). @@ -76,7 +129,25 @@ delete(Bs) when is_list(Bs) -> match(XName, RoutingKey, Opts) -> BKeys = maps:get(return_binding_keys, Opts, false), - match_in_mnesia(XName, RoutingKey, BKeys). + rabbit_khepri:handle_fallback( + #{ + mnesia => + fun() -> + match_in_mnesia(XName, RoutingKey, BKeys) + end, + khepri => + fun() -> + match_in_khepri(XName, RoutingKey, BKeys) + end + }). + +match_in_mnesia(XName, RoutingKey, BKeys) -> + Words = split_topic_key(RoutingKey), + mnesia:async_dirty(fun trie_match/3, [XName, Words, BKeys]). + +match_in_khepri(XName, RoutingKey, BKeys) -> + Words = split_topic_key_binary(RoutingKey), + trie_match_in_khepri(XName, Words, BKeys). %% ------------------------------------------------------------------- %% clear(). @@ -88,7 +159,10 @@ match(XName, RoutingKey, Opts) -> %% @private clear() -> - clear_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> clear_in_mnesia() end, + khepri => fun() -> clear_in_khepri() end + }). clear_in_mnesia() -> {atomic, ok} = mnesia:clear_table(?MNESIA_NODE_TABLE), @@ -96,32 +170,94 @@ clear_in_mnesia() -> {atomic, ok} = mnesia:clear_table(?MNESIA_BINDING_TABLE), ok. -%% Internal +clear_in_khepri() -> + ok. + %% -------------------------------------------------------------- +%% split_topic_key(). +%% -------------------------------------------------------------- + +-spec split_topic_key(RoutingKey) -> Words when + RoutingKey :: binary(), + Words :: [[byte()]]. split_topic_key(Key) -> split_topic_key(Key, [], []). -set_in_mnesia(XName, BindingKey, Destination, Args) -> - rabbit_mnesia:execute_mnesia_transaction( - fun() -> - FinalNode = follow_down_create(XName, split_topic_key(BindingKey)), - trie_add_binding(XName, FinalNode, Destination, Args), - ok - end). +split_topic_key(<<>>, [], []) -> + []; +split_topic_key(<<>>, RevWordAcc, RevResAcc) -> + lists:reverse([lists:reverse(RevWordAcc) | RevResAcc]); +split_topic_key(<<$., Rest/binary>>, RevWordAcc, RevResAcc) -> + split_topic_key(Rest, [], [lists:reverse(RevWordAcc) | RevResAcc]); +split_topic_key(<>, RevWordAcc, RevResAcc) -> + split_topic_key(Rest, [C | RevWordAcc], RevResAcc). -delete_all_for_exchange_in_mnesia(XName) -> +%% -------------------------------------------------------------- +%% split_topic_key_binary(). +%% -------------------------------------------------------------- + +-spec split_topic_key_binary(RoutingKey) -> Words when + RoutingKey :: binary(), + Words :: [binary()]. + +split_topic_key_binary(<<>>) -> + []; +split_topic_key_binary(RoutingKey) -> + Pattern = + case persistent_term:get(?COMPILED_TOPIC_SPLIT_PATTERN, undefined) of + undefined -> + P = binary:compile_pattern(<<".">>), + persistent_term:put(?COMPILED_TOPIC_SPLIT_PATTERN, P), + P; + P -> + P + end, + binary:split(RoutingKey, Pattern, [global]). + +%% -------------------------------------------------------------- +%% trie_binding_to_key(). +%% -------------------------------------------------------------- + +-spec trie_binding_to_key(#topic_trie_binding{}) -> RoutingKey :: binary(). + +trie_binding_to_key(#topic_trie_binding{trie_binding = #trie_binding{node_id = NodeId}}) -> rabbit_mnesia:execute_mnesia_transaction( fun() -> - trie_remove_all_nodes(XName), - trie_remove_all_edges(XName), - trie_remove_all_bindings(XName), - ok + follow_up_get_path(mnesia, rabbit_topic_trie_edge, NodeId) end). -match_in_mnesia(XName, RoutingKey, BKeys) -> - Words = split_topic_key(RoutingKey), - mnesia:async_dirty(fun trie_match/3, [XName, Words, BKeys]). +%% -------------------------------------------------------------- +%% trie_records_to_key(). +%% -------------------------------------------------------------- + +-spec trie_records_to_key([#topic_trie_binding{}]) -> + [{#trie_binding{}, RoutingKey :: binary()}]. + +trie_records_to_key(Records) -> + Tab = ensure_topic_deletion_ets(), + TrieBindings = lists:foldl(fun(#topic_trie_binding{} = R, Acc) -> + [R | Acc]; + (#topic_trie_edge{} = R, Acc) -> + ets:insert(Tab, R), + Acc; + (_, Acc) -> + Acc + end, [], Records), + List = lists:foldl( + fun(#topic_trie_binding{trie_binding = #trie_binding{node_id = Node} = TB} = B, + Acc) -> + case follow_up_get_path(ets, Tab, Node) of + {error, not_found} -> [{TB, trie_binding_to_key(B)} | Acc]; + RK -> [{TB, RK} | Acc] + end + end, [], TrieBindings), + ets:delete(Tab), + List. + +%% -------------------------------------------------------------- +%% Internal +%% -------------------------------------------------------------- trie_remove_all_nodes(X) -> remove_all(?MNESIA_NODE_TABLE, @@ -166,18 +302,21 @@ delete_in_mnesia_tx(Bs) -> end || #binding{source = X, key = K, destination = D, args = Args} <- Bs], ok. -delete_in_mnesia(Bs) -> - rabbit_mnesia:execute_mnesia_transaction( - fun() -> delete_in_mnesia_tx(Bs) end). - -split_topic_key(<<>>, [], []) -> - []; -split_topic_key(<<>>, RevWordAcc, RevResAcc) -> - lists:reverse([lists:reverse(RevWordAcc) | RevResAcc]); -split_topic_key(<<$., Rest/binary>>, RevWordAcc, RevResAcc) -> - split_topic_key(Rest, [], [lists:reverse(RevWordAcc) | RevResAcc]); -split_topic_key(<>, RevWordAcc, RevResAcc) -> - split_topic_key(Rest, [C | RevWordAcc], RevResAcc). +follow_up_get_path(Mod, Tab, Node) -> + follow_up_get_path(Mod, Tab, Node, []). + +follow_up_get_path(_Mod, _Tab, root, Acc) -> + Acc; +follow_up_get_path(Mod, Tab, Node, Acc) -> + MatchHead = #topic_trie_edge{node_id = Node, + trie_edge = '$1'}, + case Mod:select(Tab, [{MatchHead, [], ['$1']}]) of + [#trie_edge{node_id = PreviousNode, + word = Word}] -> + follow_up_get_path(Mod, Tab, PreviousNode, [Word | Acc]); + [] -> + {error, not_found} + end. trie_match(X, Words, BKeys) -> trie_match(X, root, Words, BKeys, []). @@ -339,3 +478,72 @@ add_matched(DestinationsArgs, true, Acc) -> ({DestX, _BindingArgs}, L) -> [DestX | L] end, Acc, DestinationsArgs). + +ensure_topic_deletion_ets() -> + Tab = rabbit_db_topic_exchange_delete_table, + case ets:whereis(Tab) of + undefined -> + ets:new(Tab, [public, named_table, {keypos, #topic_trie_edge.trie_edge}]); + Tid -> + Tid + end. + +%% Khepri topic graph + +trie_match_in_khepri(X, Words, BKeys) -> + trie_match_in_khepri(X, root, Words, BKeys, []). + +trie_match_in_khepri(X, Node, [], BKeys, ResAcc0) -> + Destinations = trie_bindings_in_khepri(X, Node, BKeys), + ResAcc = add_matched(Destinations, BKeys, ResAcc0), + trie_match_part_in_khepri( + X, Node, <<"#">>, + fun trie_match_skip_any_in_khepri/5, [], BKeys, ResAcc); +trie_match_in_khepri(X, Node, [W | RestW] = Words, BKeys, ResAcc) -> + lists:foldl(fun ({WArg, MatchFun, RestWArg}, Acc) -> + trie_match_part_in_khepri( + X, Node, WArg, MatchFun, RestWArg, BKeys, Acc) + end, ResAcc, [{W, fun trie_match_in_khepri/5, RestW}, + {<<"*">>, fun trie_match_in_khepri/5, RestW}, + {<<"#">>, + fun trie_match_skip_any_in_khepri/5, Words}]). + +trie_match_part_in_khepri(X, Node, Search, MatchFun, RestW, BKeys, ResAcc) -> + case trie_child_in_khepri(X, Node, Search) of + {ok, NextNode} -> MatchFun(X, NextNode, RestW, BKeys, ResAcc); + error -> ResAcc + end. + +trie_match_skip_any_in_khepri(X, Node, [], BKeys, ResAcc) -> + trie_match_in_khepri(X, Node, [], BKeys, ResAcc); +trie_match_skip_any_in_khepri(X, Node, [_ | RestW] = Words, BKeys, ResAcc) -> + trie_match_skip_any_in_khepri( + X, Node, RestW, BKeys, + trie_match_in_khepri(X, Node, Words, BKeys, ResAcc)). + +trie_child_in_khepri(X, Node, Word) -> + case ets:lookup(rabbit_khepri_topic_trie, + #trie_edge{exchange_name = X, + node_id = Node, + word = Word}) of + [#topic_trie_edge{node_id = NextNode}] -> {ok, NextNode}; + [] -> error + end. + +trie_bindings_in_khepri(X, Node, BKeys) -> + case ets:lookup(rabbit_khepri_topic_trie, + #trie_edge{exchange_name = X, + node_id = Node, + word = bindings}) of + [#topic_trie_edge{node_id = {bindings, Bindings}}] -> + [case BKeys of + true -> + {Dest, Args}; + false -> + Dest + end || #binding{destination = Dest, + args = Args} <- sets:to_list(Bindings)]; + [] -> + [] + end. + diff --git a/deps/rabbit/src/rabbit_db_user.erl b/deps/rabbit/src/rabbit_db_user.erl index 84e85d64c541..492ee80b7cb9 100644 --- a/deps/rabbit/src/rabbit_db_user.erl +++ b/deps/rabbit/src/rabbit_db_user.erl @@ -9,6 +9,7 @@ -include_lib("stdlib/include/assert.hrl"). +-include_lib("khepri/include/khepri.hrl"). -include_lib("rabbit_common/include/rabbit.hrl"). -export([create/1, @@ -16,6 +17,7 @@ get/1, get_all/0, with_fun_in_mnesia_tx/2, + with_fun_in_khepri_tx/2, get_user_permissions/2, check_and_match_user_permissions/2, set_user_permissions/1, @@ -28,8 +30,44 @@ clear_matching_topic_permissions/3, delete/1]). +-export([khepri_users_path/0, + khepri_user_path/1, + khepri_user_permission_path/2, + khepri_topic_permission_path/3]). + +%% for testing -export([clear/0]). +-ifdef(TEST). +-export([get_in_mnesia/1, + get_in_khepri/1, + create_in_mnesia/2, + create_in_khepri/2, + get_all_in_mnesia/0, + get_all_in_khepri/0, + update_in_mnesia/2, + update_in_khepri/2, + delete_in_mnesia/1, + delete_in_khepri/1, + get_user_permissions_in_mnesia/2, + get_user_permissions_in_khepri/2, + set_user_permissions_in_mnesia/3, + set_user_permissions_in_khepri/3, + set_topic_permissions_in_mnesia/3, + set_topic_permissions_in_khepri/3, + match_user_permissions_in_mnesia/2, + match_user_permissions_in_khepri/2, + clear_user_permissions_in_mnesia/2, + clear_user_permissions_in_khepri/2, + get_topic_permissions_in_mnesia/3, + get_topic_permissions_in_khepri/3, + match_topic_permissions_in_mnesia/3, + match_topic_permissions_in_khepri/3, + clear_topic_permissions_in_mnesia/3, + clear_topic_permissions_in_khepri/3 + ]). +-endif. + -define(MNESIA_TABLE, rabbit_user). -define(PERM_MNESIA_TABLE, rabbit_user_permission). -define(TOPIC_PERM_MNESIA_TABLE, rabbit_topic_permission). @@ -49,7 +87,9 @@ create(User) -> Username = internal_user:get_username(User), - create_in_mnesia(Username, User). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> create_in_mnesia(Username, User) end, + khepri => fun() -> create_in_khepri(Username, User) end}). create_in_mnesia(Username, User) -> rabbit_mnesia:execute_mnesia_transaction( @@ -61,6 +101,17 @@ create_in_mnesia_tx(Username, User) -> _ -> mnesia:abort({user_already_exists, Username}) end. +create_in_khepri(Username, User) -> + Path = khepri_user_path(Username), + case rabbit_khepri:create(Path, User) of + ok -> + ok; + {error, {khepri, mismatching_node, _}} -> + throw({error, {user_already_exists, Username}}); + {error, _} = Error -> + throw(Error) + end. + %% ------------------------------------------------------------------- %% update(). %% ------------------------------------------------------------------- @@ -75,7 +126,9 @@ create_in_mnesia_tx(Username, User) -> update(Username, UpdateFun) when is_binary(Username) andalso is_function(UpdateFun, 1) -> - update_in_mnesia(Username, UpdateFun). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> update_in_mnesia(Username, UpdateFun) end, + khepri => fun() -> update_in_khepri(Username, UpdateFun) end}). update_in_mnesia(Username, UpdateFun) -> rabbit_mnesia:execute_mnesia_transaction( @@ -90,6 +143,21 @@ update_in_mnesia_tx(Username, UpdateFun) -> mnesia:abort({no_such_user, Username}) end. +update_in_khepri(Username, UpdateFun) -> + rabbit_khepri:transaction( + fun () -> + Path = khepri_user_path(Username), + case khepri_tx:get(Path) of + {ok, User} -> + case khepri_tx:put(Path, UpdateFun(User)) of + ok -> ok; + Error -> khepri_tx:abort(Error) + end; + _ -> + khepri_tx:abort({no_such_user, Username}) + end + end). + %% ------------------------------------------------------------------- %% get(). %% ------------------------------------------------------------------- @@ -97,7 +165,6 @@ update_in_mnesia_tx(Username, UpdateFun) -> -spec get(Username) -> User | undefined when Username :: internal_user:username(), User :: internal_user:internal_user(). - %% @doc Returns the record of the internal user named `Username'. %% %% @returns the internal user record or `undefined' if no internal user is named @@ -106,7 +173,9 @@ update_in_mnesia_tx(Username, UpdateFun) -> %% @private get(Username) when is_binary(Username) -> - get_in_mnesia(Username). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_in_mnesia(Username) end, + khepri => fun() -> get_in_khepri(Username) end}). get_in_mnesia(Username) -> case ets:lookup(?MNESIA_TABLE, Username) of @@ -114,6 +183,12 @@ get_in_mnesia(Username) -> [] -> undefined end. +get_in_khepri(Username) -> + case ets:lookup(rabbit_khepri_users, Username) of + [User] -> User; + _ -> undefined + end. + %% ------------------------------------------------------------------- %% get_all(). %% ------------------------------------------------------------------- @@ -127,13 +202,22 @@ get_in_mnesia(Username) -> %% @private get_all() -> - get_all_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_in_mnesia() end, + khepri => fun() -> get_all_in_khepri() end}). get_all_in_mnesia() -> mnesia:dirty_match_object( ?MNESIA_TABLE, internal_user:pattern_match_all()). +get_all_in_khepri() -> + Path = khepri_users_path(), + case rabbit_khepri:list(Path) of + {ok, Users} -> maps:values(Users); + _ -> [] + end. + %% ------------------------------------------------------------------- %% with_fun_in_*(). %% ------------------------------------------------------------------- @@ -162,6 +246,16 @@ with_fun_in_mnesia_tx(Username, TxFun) end end. +with_fun_in_khepri_tx(Username, TxFun) + when is_binary(Username) andalso is_function(TxFun, 0) -> + fun() -> + Path = khepri_user_path(Username), + case khepri_tx:exists(Path) of + true -> TxFun(); + false -> khepri_tx:abort({no_such_user, Username}) + end + end. + %% ------------------------------------------------------------------- %% get_user_permissions(). %% ------------------------------------------------------------------- @@ -180,7 +274,11 @@ with_fun_in_mnesia_tx(Username, TxFun) get_user_permissions(Username, VHostName) when is_binary(Username) andalso is_binary(VHostName) -> - get_user_permissions_in_mnesia(Username, VHostName). + rabbit_khepri:handle_fallback( + #{mnesia => + fun() -> get_user_permissions_in_mnesia(Username, VHostName) end, + khepri => + fun() -> get_user_permissions_in_khepri(Username, VHostName) end}). get_user_permissions_in_mnesia(Username, VHostName) -> Key = #user_vhost{username = Username, @@ -190,6 +288,14 @@ get_user_permissions_in_mnesia(Username, VHostName) -> [] -> undefined end. +get_user_permissions_in_khepri(Username, VHostName) -> + UserVHost = #user_vhost{username = Username, + virtual_host = VHostName}, + case ets:lookup(rabbit_khepri_user_permissions, UserVHost) of + [UserPermission] -> UserPermission; + _ -> undefined + end. + %% ------------------------------------------------------------------- %% check_and_match_user_permissions(). %% ------------------------------------------------------------------- @@ -209,7 +315,11 @@ get_user_permissions_in_mnesia(Username, VHostName) -> check_and_match_user_permissions(Username, VHostName) when (is_binary(Username) orelse Username =:= '_') andalso (is_binary(VHostName) orelse VHostName =:= '_') -> - match_user_permissions_in_mnesia(Username, VHostName). + rabbit_khepri:handle_fallback( + #{mnesia => + fun() -> match_user_permissions_in_mnesia(Username, VHostName) end, + khepri => + fun() -> match_user_permissions_in_khepri(Username, VHostName) end}). match_user_permissions_in_mnesia('_' = Username, '_' = VHostName) -> rabbit_mnesia:execute_mnesia_transaction( @@ -249,6 +359,45 @@ match_user_permissions_in_mnesia_tx(Username, VHostName) -> permission = '_'}, read). +match_user_permissions_in_khepri('_' = _Username, '_' = _VHostName) -> + Path = khepri_user_permission_path(?KHEPRI_WILDCARD_STAR, ?KHEPRI_WILDCARD_STAR), + case rabbit_khepri:match(Path) of + {ok, Map} -> + maps:values(Map); + _ -> + [] + end; +match_user_permissions_in_khepri('_' = _Username, VHostName) -> + rabbit_khepri:transaction( + rabbit_db_vhost:with_fun_in_khepri_tx( + VHostName, + fun() -> + match_user_permissions_in_khepri_tx(?KHEPRI_WILDCARD_STAR, VHostName) + end)); +match_user_permissions_in_khepri(Username, '_' = _VHostName) -> + rabbit_khepri:transaction( + with_fun_in_khepri_tx( + Username, + fun() -> + match_user_permissions_in_khepri_tx(Username, ?KHEPRI_WILDCARD_STAR) + end)); +match_user_permissions_in_khepri(Username, VHostName) -> + rabbit_khepri:transaction( + with_fun_in_khepri_tx( + Username, + rabbit_db_vhost:with_fun_in_khepri_tx( + VHostName, + fun() -> + match_user_permissions_in_khepri_tx(Username, VHostName) + end))). + +match_user_permissions_in_khepri_tx(Username, VHostName) -> + Path = khepri_user_permission_path(Username, VHostName), + case khepri_tx:get_many(Path) of + {ok, Map} -> maps:values(Map); + _ -> [] + end. + %% ------------------------------------------------------------------- %% set_user_permissions(). %% ------------------------------------------------------------------- @@ -264,7 +413,17 @@ set_user_permissions( #user_permission{user_vhost = #user_vhost{username = Username, virtual_host = VHostName}} = UserPermission) -> - set_user_permissions_in_mnesia(Username, VHostName, UserPermission). + rabbit_khepri:handle_fallback( + #{mnesia => + fun() -> + set_user_permissions_in_mnesia( + Username, VHostName, UserPermission) + end, + khepri => + fun() -> + set_user_permissions_in_khepri( + Username, VHostName, UserPermission) + end}). set_user_permissions_in_mnesia(Username, VHostName, UserPermission) -> rabbit_mnesia:execute_mnesia_transaction( @@ -277,6 +436,32 @@ set_user_permissions_in_mnesia(Username, VHostName, UserPermission) -> set_user_permissions_in_mnesia_tx(UserPermission) -> mnesia:write(?PERM_MNESIA_TABLE, UserPermission, write). +set_user_permissions_in_khepri(Username, VHostName, UserPermission) -> + rabbit_khepri:transaction( + with_fun_in_khepri_tx( + Username, + rabbit_db_vhost:with_fun_in_khepri_tx( + VHostName, + fun() -> + set_user_permissions_in_khepri_tx(Username, VHostName, UserPermission) + end)), rw). + +set_user_permissions_in_khepri_tx(Username, VHostName, UserPermission) -> + Path = khepri_user_permission_path( + #if_all{conditions = + [Username, + #if_node_exists{exists = true}]}, + VHostName), + Extra = #{keep_while => + #{rabbit_db_vhost:khepri_vhost_path(VHostName) => + #if_node_exists{exists = true}}}, + Ret = khepri_tx:put( + Path, UserPermission, Extra), + case Ret of + ok -> ok; + Error -> khepri_tx:abort(Error) + end. + %% ------------------------------------------------------------------- %% clear_user_permissions(). %% ------------------------------------------------------------------- @@ -291,7 +476,12 @@ set_user_permissions_in_mnesia_tx(UserPermission) -> clear_user_permissions(Username, VHostName) when is_binary(Username) andalso is_binary(VHostName) -> - clear_user_permissions_in_mnesia(Username, VHostName). + rabbit_khepri:handle_fallback( + #{mnesia => + fun() -> clear_user_permissions_in_mnesia(Username, VHostName) end, + khepri => + fun() -> clear_user_permissions_in_khepri(Username, VHostName) end + }). clear_user_permissions_in_mnesia(Username, VHostName) -> rabbit_mnesia:execute_mnesia_transaction( @@ -302,6 +492,13 @@ clear_user_permissions_in_mnesia_tx(Username, VHostName) -> #user_vhost{username = Username, virtual_host = VHostName}}). +clear_user_permissions_in_khepri(Username, VHostName) -> + Path = khepri_user_permission_path(Username, VHostName), + case rabbit_khepri:delete(Path) of + ok -> ok; + Error -> khepri_tx:abort(Error) + end. + %% ------------------------------------------------------------------- %% clear_matching_user_permissions(). %% ------------------------------------------------------------------- @@ -309,22 +506,31 @@ clear_user_permissions_in_mnesia_tx(Username, VHostName) -> -spec clear_matching_user_permissions(Username, VHostName) -> Ret when Username :: internal_user:username() | '_', VHostName :: vhost:name() | '_', - Ret :: [#user_permission{}]. + Ret :: ok. %% @doc Clears all user permissions matching arguments. %% -%% @returns a list of matching user permissions. -%% %% @private clear_matching_user_permissions(Username, VHostName) when (is_binary(Username) orelse Username =:= '_') andalso (is_binary(VHostName) orelse VHostName =:= '_') -> - clear_matching_user_permissions_in_mnesia(Username, VHostName). + rabbit_khepri:handle_fallback( + #{mnesia => + fun() -> + clear_matching_user_permissions_in_mnesia(Username, VHostName) + end, + khepri => + fun() -> + clear_matching_user_permissions_in_khepri(Username, VHostName) + end + }). clear_matching_user_permissions_in_mnesia(Username, VHostName) -> rabbit_mnesia:execute_mnesia_transaction( fun() -> - clear_matching_user_permissions_in_mnesia_tx( Username, VHostName) + _ = clear_matching_user_permissions_in_mnesia_tx( + Username, VHostName), + ok end). clear_matching_user_permissions_in_mnesia_tx(Username, VHostName) -> @@ -335,6 +541,13 @@ clear_matching_user_permissions_in_mnesia_tx(Username, VHostName) -> || #user_permission{user_vhost = Key} = Record <- match_user_permissions_in_mnesia_tx(Username, VHostName)]. +clear_matching_user_permissions_in_khepri(Username, VHostName) -> + Path = khepri_user_permission_path(any(Username), any(VHostName)), + ok = rabbit_khepri:delete(Path). + +any('_') -> ?KHEPRI_WILDCARD_STAR; +any(Value) -> Value. + %% ------------------------------------------------------------------- %% get_topic_permissions(). %% ------------------------------------------------------------------- @@ -356,7 +569,17 @@ get_topic_permissions(Username, VHostName, ExchangeName) when is_binary(Username) andalso is_binary(VHostName) andalso is_binary(ExchangeName) -> - get_topic_permissions_in_mnesia(Username, VHostName, ExchangeName). + rabbit_khepri:handle_fallback( + #{mnesia => + fun() -> + get_topic_permissions_in_mnesia( + Username, VHostName, ExchangeName) + end, + khepri => + fun() -> + get_topic_permissions_in_khepri( + Username, VHostName, ExchangeName) + end}). get_topic_permissions_in_mnesia(Username, VHostName, ExchangeName) -> Key = #topic_permission_key{ @@ -368,6 +591,13 @@ get_topic_permissions_in_mnesia(Username, VHostName, ExchangeName) -> [] -> undefined end. +get_topic_permissions_in_khepri(Username, VHostName, ExchangeName) -> + Path = khepri_topic_permission_path(Username, VHostName, ExchangeName), + case rabbit_khepri:get(Path) of + {ok, TopicPermission} -> TopicPermission; + _ -> undefined + end. + %% ------------------------------------------------------------------- %% check_and_match_topic_permissions(). %% ------------------------------------------------------------------- @@ -390,7 +620,18 @@ check_and_match_topic_permissions(Username, VHostName, ExchangeName) when (is_binary(Username) orelse Username =:= '_') andalso (is_binary(VHostName) orelse VHostName =:= '_') andalso (is_binary(ExchangeName) orelse ExchangeName =:= '_') -> - match_topic_permissions_in_mnesia(Username, VHostName, ExchangeName). + rabbit_khepri:handle_fallback( + #{mnesia => + fun() -> + match_topic_permissions_in_mnesia( + Username, VHostName, ExchangeName) + end, + khepri => + fun() -> + match_topic_permissions_in_khepri( + Username, VHostName, ExchangeName) + end + }). match_topic_permissions_in_mnesia( '_' = Username, '_' = VHostName, ExchangeName) -> @@ -441,6 +682,48 @@ match_topic_permissions_in_mnesia_tx(Username, VHostName, ExchangeName) -> permission = '_'}, read). +match_topic_permissions_in_khepri('_' = _Username, '_' = _VHostName, ExchangeName) -> + rabbit_khepri:transaction( + fun() -> + match_topic_permissions_in_khepri_tx( + ?KHEPRI_WILDCARD_STAR, ?KHEPRI_WILDCARD_STAR, any(ExchangeName)) + end); +match_topic_permissions_in_khepri('_' = _Username, VHostName, ExchangeName) -> + rabbit_khepri:transaction( + rabbit_db_vhost:with_fun_in_khepri_tx( + VHostName, + fun() -> + match_topic_permissions_in_khepri_tx( + ?KHEPRI_WILDCARD_STAR, VHostName, any(ExchangeName)) + end)); +match_topic_permissions_in_khepri( + Username, '_' = _VHostName, ExchangeName) -> + rabbit_khepri:transaction( + with_fun_in_khepri_tx( + Username, + fun() -> + match_topic_permissions_in_khepri_tx( + Username, ?KHEPRI_WILDCARD_STAR, any(ExchangeName)) + end)); +match_topic_permissions_in_khepri( + Username, VHostName, ExchangeName) -> + rabbit_khepri:transaction( + with_fun_in_khepri_tx( + Username, + rabbit_db_vhost:with_fun_in_khepri_tx( + VHostName, + fun() -> + match_topic_permissions_in_khepri_tx( + Username, VHostName, any(ExchangeName)) + end))). + +match_topic_permissions_in_khepri_tx(Username, VHostName, ExchangeName) -> + Path = khepri_topic_permission_path(Username, VHostName, ExchangeName), + case khepri_tx:get_many(Path) of + {ok, Map} -> maps:values(Map); + _ -> [] + end. + %% ------------------------------------------------------------------- %% set_topic_permissions(). %% ------------------------------------------------------------------- @@ -459,7 +742,18 @@ set_topic_permissions( user_vhost = #user_vhost{username = Username, virtual_host = VHostName}}} = TopicPermission) -> - set_topic_permissions_in_mnesia(Username, VHostName, TopicPermission). + rabbit_khepri:handle_fallback( + #{mnesia => + fun() -> + set_topic_permissions_in_mnesia( + Username, VHostName, TopicPermission) + end, + khepri => + fun() -> + set_topic_permissions_in_khepri( + Username, VHostName, TopicPermission) + end + }). set_topic_permissions_in_mnesia(Username, VHostName, TopicPermission) -> rabbit_mnesia:execute_mnesia_transaction( @@ -474,6 +768,34 @@ set_topic_permissions_in_mnesia(Username, VHostName, TopicPermission) -> set_topic_permissions_in_mnesia_tx(TopicPermission) -> mnesia:write(?TOPIC_PERM_MNESIA_TABLE, TopicPermission, write). +set_topic_permissions_in_khepri(Username, VHostName, TopicPermission) -> + rabbit_khepri:transaction( + with_fun_in_khepri_tx( + Username, + rabbit_db_vhost:with_fun_in_khepri_tx( + VHostName, + fun() -> + set_topic_permissions_in_khepri_tx(Username, VHostName, TopicPermission) + end)), rw). + +set_topic_permissions_in_khepri_tx(Username, VHostName, TopicPermission) -> + #topic_permission{topic_permission_key = + #topic_permission_key{exchange = ExchangeName}} = TopicPermission, + Path = khepri_topic_permission_path( + #if_all{conditions = + [Username, + #if_node_exists{exists = true}]}, + VHostName, + ExchangeName), + Extra = #{keep_while => + #{rabbit_db_vhost:khepri_vhost_path(VHostName) => + #if_node_exists{exists = true}}}, + Ret = khepri_tx:put(Path, TopicPermission, Extra), + case Ret of + ok -> ok; + Error -> khepri_tx:abort(Error) + end. + %% ------------------------------------------------------------------- %% clear_topic_permissions(). %% ------------------------------------------------------------------- @@ -490,7 +812,18 @@ set_topic_permissions_in_mnesia_tx(TopicPermission) -> clear_topic_permissions(Username, VHostName, ExchangeName) when is_binary(Username) andalso is_binary(VHostName) andalso (is_binary(ExchangeName) orelse ExchangeName =:= '_') -> - clear_topic_permissions_in_mnesia(Username, VHostName, ExchangeName). + rabbit_khepri:handle_fallback( + #{mnesia => + fun() -> + clear_topic_permissions_in_mnesia( + Username, VHostName, ExchangeName) + end, + khepri => + fun() -> + clear_topic_permissions_in_khepri( + Username, VHostName, ExchangeName) + end + }). clear_topic_permissions_in_mnesia(Username, VHostName, ExchangeName) -> rabbit_mnesia:execute_mnesia_transaction( @@ -502,6 +835,10 @@ clear_topic_permissions_in_mnesia(Username, VHostName, ExchangeName) -> clear_topic_permissions_in_mnesia_tx(Username, VHostName, ExchangeName) -> delete_topic_permission_in_mnesia_tx(Username, VHostName, ExchangeName). +clear_topic_permissions_in_khepri(Username, VHostName, ExchangeName) -> + Path = khepri_topic_permission_path(any(Username), any(VHostName), any(ExchangeName)), + rabbit_khepri:delete(Path). + %% ------------------------------------------------------------------- %% clear_matching_topic_permissions(). %% ------------------------------------------------------------------- @@ -511,26 +848,35 @@ clear_topic_permissions_in_mnesia_tx(Username, VHostName, ExchangeName) -> Username :: rabbit_types:username() | '_', VHostName :: vhost:name() | '_', ExchangeName :: binary() | '_', - Ret :: [#topic_permission{}]. + Ret :: ok. %% @doc Clears all topic permissions matching arguments. %% -%% @returns a list of matching topic permissions. -%% %% @private clear_matching_topic_permissions(Username, VHostName, ExchangeName) when (is_binary(Username) orelse Username =:= '_') andalso (is_binary(VHostName) orelse VHostName =:= '_') andalso (is_binary(ExchangeName) orelse ExchangeName =:= '_') -> - clear_matching_topic_permissions_in_mnesia( - Username, VHostName, ExchangeName). + rabbit_khepri:handle_fallback( + #{mnesia => + fun() -> + clear_matching_topic_permissions_in_mnesia( + Username, VHostName, ExchangeName) + end, + khepri => + fun() -> + clear_matching_topic_permissions_in_khepri( + Username, VHostName, ExchangeName) + end + }). clear_matching_topic_permissions_in_mnesia( Username, VHostName, ExchangeName) -> rabbit_mnesia:execute_mnesia_transaction( fun() -> - clear_matching_topic_permissions_in_mnesia_tx( - Username, VHostName, ExchangeName) + _ = clear_matching_topic_permissions_in_mnesia_tx( + Username, VHostName, ExchangeName), + ok end). clear_matching_topic_permissions_in_mnesia_tx( @@ -543,13 +889,19 @@ clear_matching_topic_permissions_in_mnesia_tx( <- match_topic_permissions_in_mnesia_tx( Username, VHostName, ExchangeName)]. +clear_matching_topic_permissions_in_khepri( + Username, VHostName, ExchangeName) -> + Path = khepri_topic_permission_path( + any(Username), any(VHostName), any(ExchangeName)), + ok = rabbit_khepri:delete(Path). + %% ------------------------------------------------------------------- %% delete(). %% ------------------------------------------------------------------- -spec delete(Username) -> Existed when Username :: internal_user:username(), - Existed :: boolean(). + Existed :: boolean() | {error, any()}. %% @doc Deletes a user and its permissions from the database. %% %% @returns a boolean indicating if the user existed. It throws an exception @@ -558,7 +910,10 @@ clear_matching_topic_permissions_in_mnesia_tx( %% @private delete(Username) when is_binary(Username) -> - delete_in_mnesia(Username). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> delete_in_mnesia(Username) end, + khepri => fun() -> delete_in_khepri(Username) end + }). delete_in_mnesia(Username) -> rabbit_mnesia:execute_mnesia_transaction( @@ -586,6 +941,14 @@ delete_topic_permission_in_mnesia_tx(Username, VHostName, ExchangeName) -> R <- mnesia:match_object(?TOPIC_PERM_MNESIA_TABLE, Pattern, write)], ok. +delete_in_khepri(Username) -> + Path = khepri_user_path(Username), + case rabbit_khepri:delete_or_fail(Path) of + ok -> true; + {error, {node_not_found, _}} -> false; + Error -> Error + end. + user_permission_pattern(Username, VHostName) -> #user_permission{user_vhost = #user_vhost{ username = Username, @@ -612,10 +975,32 @@ topic_permission_pattern(Username, VHostName, ExchangeName) -> %% @private clear() -> - clear_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> clear_in_mnesia() end, + khepri => fun() -> clear_in_khepri() end}). clear_in_mnesia() -> {atomic, ok} = mnesia:clear_table(?MNESIA_TABLE), {atomic, ok} = mnesia:clear_table(?PERM_MNESIA_TABLE), {atomic, ok} = mnesia:clear_table(?TOPIC_PERM_MNESIA_TABLE), ok. + +clear_in_khepri() -> + Path = khepri_users_path(), + case rabbit_khepri:delete(Path) of + ok -> ok; + Error -> throw(Error) + end. + +%% -------------------------------------------------------------- +%% Paths +%% -------------------------------------------------------------- + +khepri_users_path() -> [?MODULE, users]. +khepri_user_path(Username) -> [?MODULE, users, Username]. + +khepri_user_permission_path(Username, VHostName) -> + [?MODULE, users, Username, user_permissions, VHostName]. + +khepri_topic_permission_path(Username, VHostName, Exchange) -> + [?MODULE, users, Username, topic_permissions, VHostName, Exchange]. diff --git a/deps/rabbit/src/rabbit_db_user_m2k_converter.erl b/deps/rabbit/src/rabbit_db_user_m2k_converter.erl new file mode 100644 index 000000000000..5666279e824d --- /dev/null +++ b/deps/rabbit/src/rabbit_db_user_m2k_converter.erl @@ -0,0 +1,203 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright © 2022-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_user_m2k_converter). + +-behaviour(mnesia_to_khepri_converter). + +-include_lib("kernel/include/logger.hrl"). +-include_lib("khepri/include/khepri.hrl"). +-include_lib("khepri_mnesia_migration/src/kmm_logging.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). + +-export([init_copy_to_khepri/3, + copy_to_khepri/3, + delete_from_khepri/3]). + +-record(?MODULE, {store_id :: khepri:store_id()}). + +-spec init_copy_to_khepri(StoreId, MigrationId, Tables) -> Ret when + StoreId :: khepri:store_id(), + MigrationId :: mnesia_to_khepri:migration_id(), + Tables :: [mnesia_to_khepri:mnesia_table()], + Ret :: {ok, Priv}, + Priv :: #?MODULE{}. +%% @private + +init_copy_to_khepri(StoreId, _MigrationId, Tables) -> + %% Clean up any previous attempt to copy the Mnesia table to Khepri. + lists:foreach(fun clear_data_in_khepri/1, Tables), + + State = #?MODULE{store_id = StoreId}, + {ok, State}. + +-spec copy_to_khepri(Table, Record, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Record :: tuple(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +copy_to_khepri( + rabbit_user = Table, Record, + #?MODULE{store_id = StoreId} = State) -> + Username = internal_user:get_username(Record), + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] key: ~0p", + [Table, Username], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_user:khepri_user_path(Username), + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:put(StoreId, Path, Record) of + ok -> {ok, State}; + Error -> Error + end; +copy_to_khepri( + rabbit_user_permission = Table, Record, + #?MODULE{store_id = StoreId} = State) -> + #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHost}} = Record, + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] user: ~0p vhost: ~0p", + [Table, Username, VHost], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_user:khepri_user_permission_path( + #if_all{conditions = + [Username, + #if_node_exists{exists = true}]}, + VHost), + Extra = #{keep_while => + #{rabbit_db_vhost:khepri_vhost_path(VHost) => + #if_node_exists{exists = true}}}, + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:put(StoreId, Path, Record, Extra) of + ok -> {ok, State}; + Error -> Error + end; +copy_to_khepri( + rabbit_topic_permission = Table, Record, + #?MODULE{store_id = StoreId} = State) -> + #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHost}, + exchange = Exchange}} = Record, + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] user: ~0p vhost: ~0p", + [Table, Username, VHost], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_user:khepri_topic_permission_path( + #if_all{conditions = + [Username, + #if_node_exists{exists = true}]}, + VHost, + Exchange), + Extra = #{keep_while => + #{rabbit_db_vhost:khepri_vhost_path(VHost) => + #if_node_exists{exists = true}}}, + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:put(StoreId, Path, Record, Extra) of + ok -> {ok, State}; + Error -> Error + end; +copy_to_khepri(Table, Record, State) -> + ?LOG_DEBUG("Mnesia->Khepri unexpected record table ~0p record ~0p state ~0p", + [Table, Record, State]), + {error, unexpected_record}. + +-spec delete_from_khepri(Table, Key, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Key :: any(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +delete_from_khepri( + rabbit_user = Table, Key, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] key: ~0p", + [Table, Key], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_user:khepri_user_path(Key), + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:delete(StoreId, Path) of + ok -> {ok, State}; + Error -> Error + end; +delete_from_khepri( + rabbit_user_permission = Table, Key, + #?MODULE{store_id = StoreId} = State) -> + #user_vhost{ + username = Username, + virtual_host = VHost} = Key, + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] key: ~0p", + [Table, Key], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_user:khepri_user_permission_path(Username, VHost), + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:delete(StoreId, Path) of + ok -> {ok, State}; + Error -> Error + end; +delete_from_khepri( + rabbit_topic_permission = Table, Key, + #?MODULE{store_id = StoreId} = State) -> + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHost}, + exchange = Exchange} = Key, + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] key: ~0p", + [Table, Key], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_user:khepri_topic_permission_path(Username, VHost, Exchange), + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:delete(StoreId, Path) of + ok -> {ok, State}; + Error -> Error + end. + +-spec clear_data_in_khepri(Table) -> ok when + Table :: atom(). + +clear_data_in_khepri(rabbit_user) -> + Path = rabbit_db_user:khepri_users_path(), + case rabbit_khepri:delete(Path) of + ok -> ok; + Error -> throw(Error) + end; +clear_data_in_khepri(_) -> + ok. diff --git a/deps/rabbit/src/rabbit_db_vhost.erl b/deps/rabbit/src/rabbit_db_vhost.erl index be40c74fb433..f62dedbf1d6e 100644 --- a/deps/rabbit/src/rabbit_db_vhost.erl +++ b/deps/rabbit/src/rabbit_db_vhost.erl @@ -10,6 +10,7 @@ -include_lib("kernel/include/logger.hrl"). -include_lib("stdlib/include/assert.hrl"). -include_lib("rabbit_common/include/logging.hrl"). +-include_lib("khepri/include/khepri.hrl"). -include("vhost.hrl"). @@ -22,10 +23,35 @@ list/0, update/2, with_fun_in_mnesia_tx/2, + with_fun_in_khepri_tx/2, delete/1]). +-export([khepri_vhost_path/1, + khepri_vhosts_path/0]). + +%% For testing -export([clear/0]). +-ifdef(TEST). +-export([create_or_get_in_mnesia/2, + create_or_get_in_khepri/2, + get_in_mnesia/1, + get_in_khepri/1, + exists_in_mnesia/1, + exists_in_khepri/1, + list_in_mnesia/0, + list_in_khepri/0, + get_all_in_mnesia/0, + get_all_in_khepri/0, + update_in_mnesia/2, + update_in_khepri/2, + merge_metadata_in_mnesia/2, + merge_metadata_in_khepri/2, + delete_in_mnesia/1, + delete_in_khepri/1 + ]). +-endif. + -define(MNESIA_TABLE, rabbit_vhost). %% ------------------------------------------------------------------- @@ -51,7 +77,9 @@ create_or_get(VHostName, Limits, Metadata) is_list(Limits) andalso is_map(Metadata) -> VHost = vhost:new(VHostName, Limits, Metadata), - create_or_get_in_mnesia(VHostName, VHost). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> create_or_get_in_mnesia(VHostName, VHost) end, + khepri => fun() -> create_or_get_in_khepri(VHostName, VHost) end}). create_or_get_in_mnesia(VHostName, VHost) -> rabbit_mnesia:execute_mnesia_transaction( @@ -68,6 +96,20 @@ create_or_get_in_mnesia_tx(VHostName, VHost) -> {existing, ExistingVHost} end. +create_or_get_in_khepri(VHostName, VHost) -> + Path = khepri_vhost_path(VHostName), + rabbit_log:debug("Inserting a virtual host record ~tp", [VHost]), + case rabbit_khepri:create(Path, VHost) of + ok -> + {new, VHost}; + {error, {khepri, mismatching_node, + #{node_path := Path, + node_props := #{data := ExistingVHost}}}} -> + {existing, ExistingVHost}; + Error -> + throw(Error) + end. + %% ------------------------------------------------------------------- %% merge_metadata(). %% ------------------------------------------------------------------- @@ -96,7 +138,9 @@ merge_metadata(VHostName, Metadata) end. do_merge_metadata(VHostName, Metadata) -> - merge_metadata_in_mnesia(VHostName, Metadata). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> merge_metadata_in_mnesia(VHostName, Metadata) end, + khepri => fun() -> merge_metadata_in_khepri(VHostName, Metadata) end}). merge_metadata_in_mnesia(VHostName, Metadata) -> rabbit_mnesia:execute_mnesia_transaction( @@ -113,6 +157,30 @@ merge_metadata_in_mnesia_tx(VHostName, Metadata) -> {ok, VHost1} end. +merge_metadata_in_khepri(VHostName, Metadata) -> + Path = khepri_vhost_path(VHostName), + Ret1 = rabbit_khepri:adv_get(Path), + case Ret1 of + {ok, #{data := VHost0, payload_version := DVersion}} -> + VHost = vhost:merge_metadata(VHost0, Metadata), + rabbit_log:debug("Updating a virtual host record ~p", [VHost]), + Path1 = khepri_path:combine_with_conditions( + Path, [#if_payload_version{version = DVersion}]), + Ret2 = rabbit_khepri:put(Path1, VHost), + case Ret2 of + ok -> + {ok, VHost}; + {error, {khepri, mismatching_node, _}} -> + merge_metadata_in_khepri(VHostName, Metadata); + {error, _} = Error -> + Error + end; + {error, {khepri, node_not_found, _}} -> + {error, {no_such_vhost, VHostName}}; + {error, _} = Error -> + Error + end. + %% ------------------------------------------------------------------- %% set_tags(). %% ------------------------------------------------------------------- @@ -131,7 +199,9 @@ merge_metadata_in_mnesia_tx(VHostName, Metadata) -> set_tags(VHostName, Tags) when is_binary(VHostName) andalso is_list(Tags) -> ConvertedTags = lists:usort([rabbit_data_coercion:to_atom(Tag) || Tag <- Tags]), - set_tags_in_mnesia(VHostName, ConvertedTags). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> set_tags_in_mnesia(VHostName, ConvertedTags) end, + khepri => fun() -> set_tags_in_khepri(VHostName, ConvertedTags) end}). set_tags_in_mnesia(VHostName, Tags) -> rabbit_mnesia:execute_mnesia_transaction( @@ -146,6 +216,10 @@ do_set_tags(VHost, Tags) when ?is_vhost(VHost) andalso is_list(Tags) -> Metadata1 = Metadata0#{tags => Tags}, vhost:set_metadata(VHost, Metadata1). +set_tags_in_khepri(VHostName, Tags) -> + UpdateFun = fun(VHost) -> do_set_tags(VHost, Tags) end, + update_in_khepri(VHostName, UpdateFun). + %% ------------------------------------------------------------------- %% exists(). %% ------------------------------------------------------------------- @@ -160,11 +234,16 @@ do_set_tags(VHost, Tags) when ?is_vhost(VHost) andalso is_list(Tags) -> %% @private exists(VHostName) when is_binary(VHostName) -> - exists_in_mnesia(VHostName). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> exists_in_mnesia(VHostName) end, + khepri => fun() -> exists_in_khepri(VHostName) end}). exists_in_mnesia(VHostName) -> mnesia:dirty_read({?MNESIA_TABLE, VHostName}) /= []. +exists_in_khepri(VHostName) -> + ets:member(rabbit_khepri_vhost, VHostName). + %% ------------------------------------------------------------------- %% get(). %% ------------------------------------------------------------------- @@ -180,7 +259,9 @@ exists_in_mnesia(VHostName) -> %% @private get(VHostName) when is_binary(VHostName) -> - get_in_mnesia(VHostName). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_in_mnesia(VHostName) end, + khepri => fun() -> get_in_khepri(VHostName) end}). get_in_mnesia(VHostName) -> case mnesia:dirty_read({?MNESIA_TABLE, VHostName}) of @@ -188,6 +269,12 @@ get_in_mnesia(VHostName) -> [] -> undefined end. +get_in_khepri(VHostName) -> + case ets:lookup(rabbit_khepri_vhost, VHostName) of + [Record] -> Record; + _ -> undefined + end. + %% ------------------------------------------------------------------- %% get_all(). %% ------------------------------------------------------------------- @@ -201,11 +288,16 @@ get_in_mnesia(VHostName) -> %% @private get_all() -> - get_all_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_all_in_mnesia() end, + khepri => fun() -> get_all_in_khepri() end}). get_all_in_mnesia() -> mnesia:dirty_match_object(?MNESIA_TABLE, vhost:pattern_match_all()). +get_all_in_khepri() -> + ets:tab2list(rabbit_khepri_vhost). + %% ------------------------------------------------------------------- %% list(). %% ------------------------------------------------------------------- @@ -219,11 +311,16 @@ get_all_in_mnesia() -> %% @private list() -> - list_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> list_in_mnesia() end, + khepri => fun() -> list_in_khepri() end}). list_in_mnesia() -> mnesia:dirty_all_keys(?MNESIA_TABLE). +list_in_khepri() -> + ets:select(rabbit_khepri_vhost, [{vhost:pattern_match_names(), [], ['$1']}]). + %% ------------------------------------------------------------------- %% update_in_*tx(). %% ------------------------------------------------------------------- @@ -242,7 +339,9 @@ list_in_mnesia() -> update(VHostName, UpdateFun) when is_binary(VHostName) andalso is_function(UpdateFun, 1) -> - update_in_mnesia(VHostName, UpdateFun). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> update_in_mnesia(VHostName, UpdateFun) end, + khepri => fun() -> update_in_khepri(VHostName, UpdateFun) end}). update_in_mnesia(VHostName, UpdateFun) -> rabbit_mnesia:execute_mnesia_transaction( @@ -259,6 +358,27 @@ update_in_mnesia_tx(VHostName, UpdateFun) mnesia:abort({no_such_vhost, VHostName}) end. +update_in_khepri(VHostName, UpdateFun) -> + Path = khepri_vhost_path(VHostName), + case rabbit_khepri:adv_get(Path) of + {ok, #{data := V, payload_version := DVersion}} -> + V1 = UpdateFun(V), + Path1 = khepri_path:combine_with_conditions( + Path, [#if_payload_version{version = DVersion}]), + case rabbit_khepri:put(Path1, V1) of + ok -> + V1; + {error, {khepri, mismatching_node, _}} -> + update_in_khepri(VHostName, UpdateFun); + Error -> + throw(Error) + end; + {error, {khepri, node_not_found, _}} -> + throw({error, {no_such_vhost, VHostName}}); + Error -> + throw(Error) + end. + %% ------------------------------------------------------------------- %% with_fun_in_*_tx(). %% ------------------------------------------------------------------- @@ -287,6 +407,15 @@ with_fun_in_mnesia_tx(VHostName, TxFun) end end. +with_fun_in_khepri_tx(VHostName, Thunk) -> + fun() -> + Path = khepri_vhost_path(VHostName), + case khepri_tx:exists(Path) of + true -> Thunk(); + false -> khepri_tx:abort({no_such_vhost, VHostName}) + end + end. + %% ------------------------------------------------------------------- %% delete(). %% ------------------------------------------------------------------- @@ -302,7 +431,9 @@ with_fun_in_mnesia_tx(VHostName, TxFun) %% @private delete(VHostName) when is_binary(VHostName) -> - delete_in_mnesia(VHostName). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> delete_in_mnesia(VHostName) end, + khepri => fun() -> delete_in_khepri(VHostName) end}). delete_in_mnesia(VHostName) -> rabbit_mnesia:execute_mnesia_transaction( @@ -313,6 +444,14 @@ delete_in_mnesia_tx(VHostName) -> mnesia:delete({?MNESIA_TABLE, VHostName}), Existed. +delete_in_khepri(VHostName) -> + Path = khepri_vhost_path(VHostName), + case rabbit_khepri:delete_or_fail(Path) of + ok -> true; + {error, {node_not_found, _}} -> false; + _ -> false + end. + %% ------------------------------------------------------------------- %% clear(). %% ------------------------------------------------------------------- @@ -323,8 +462,24 @@ delete_in_mnesia_tx(VHostName) -> %% @private clear() -> - clear_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> clear_in_mnesia() end, + khepri => fun() -> clear_in_khepri() end}). clear_in_mnesia() -> {atomic, ok} = mnesia:clear_table(?MNESIA_TABLE), ok. + +clear_in_khepri() -> + Path = khepri_vhosts_path(), + case rabbit_khepri:delete(Path) of + ok -> ok; + Error -> throw(Error) + end. + +%% -------------------------------------------------------------- +%% Paths +%% -------------------------------------------------------------- + +khepri_vhosts_path() -> [?MODULE]. +khepri_vhost_path(VHost) -> [?MODULE, VHost]. diff --git a/deps/rabbit/src/rabbit_db_vhost_m2k_converter.erl b/deps/rabbit/src/rabbit_db_vhost_m2k_converter.erl new file mode 100644 index 000000000000..45f118e3ad0e --- /dev/null +++ b/deps/rabbit/src/rabbit_db_vhost_m2k_converter.erl @@ -0,0 +1,103 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright © 2022-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_vhost_m2k_converter). + +-behaviour(mnesia_to_khepri_converter). + +-include_lib("kernel/include/logger.hrl"). +-include_lib("khepri/include/khepri.hrl"). +-include_lib("khepri_mnesia_migration/src/kmm_logging.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). + +-export([init_copy_to_khepri/3, + copy_to_khepri/3, + delete_from_khepri/3]). + +-record(?MODULE, {store_id :: khepri:store_id()}). + +-spec init_copy_to_khepri(StoreId, MigrationId, Tables) -> Ret when + StoreId :: khepri:store_id(), + MigrationId :: mnesia_to_khepri:migration_id(), + Tables :: [mnesia_to_khepri:mnesia_table()], + Ret :: {ok, Priv}, + Priv :: #?MODULE{}. +%% @private + +init_copy_to_khepri(StoreId, _MigrationId, Tables) -> + %% Clean up any previous attempt to copy the Mnesia table to Khepri. + lists:foreach(fun clear_data_in_khepri/1, Tables), + + State = #?MODULE{store_id = StoreId}, + {ok, State}. + +-spec copy_to_khepri(Table, Record, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Record :: tuple(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +copy_to_khepri( + rabbit_vhost = Table, Record, + #?MODULE{store_id = StoreId} = State) -> + Name = vhost:get_name(Record), + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] key: ~0p", + [Table, Name], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_vhost:khepri_vhost_path(Name), + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:put(StoreId, Path, Record) of + ok -> {ok, State}; + Error -> Error + end; +copy_to_khepri(Table, Record, State) -> + ?LOG_DEBUG("Mnesia->Khepri unexpected record table ~0p record ~0p state ~0p", + [Table, Record, State]), + {error, unexpected_record}. + +-spec delete_from_khepri(Table, Key, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Key :: any(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +delete_from_khepri( + rabbit_vhost = Table, Key, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] key: ~0p", + [Table, Key], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_vhost:khepri_vhost_path(Key), + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:delete(StoreId, Path) of + ok -> {ok, State}; + Error -> Error + end. + +-spec clear_data_in_khepri(Table) -> ok when + Table :: atom(). + +clear_data_in_khepri(rabbit_vhost) -> + Path = rabbit_db_vhost:khepri_vhosts_path(), + case rabbit_khepri:delete(Path) of + ok -> ok; + Error -> throw(Error) + end. diff --git a/deps/rabbit/src/rabbit_definitions.erl b/deps/rabbit/src/rabbit_definitions.erl index ff838c398f19..dab286e54ab9 100644 --- a/deps/rabbit/src/rabbit_definitions.erl +++ b/deps/rabbit/src/rabbit_definitions.erl @@ -457,8 +457,8 @@ apply_defs(Map, ActingUser, SuccessFun) when is_function(SuccessFun) -> SuccessFun(), ok - catch {error, E} -> {error, E}; - exit:E -> {error, E} + catch {error, E} -> {error, format(E)}; + exit:E -> {error, format(E)} after rabbit_runtime:gc_all_processes() end. @@ -627,6 +627,10 @@ format({no_such_vhost, VHost}) -> [VHost])); format({vhost_limit_exceeded, ErrMsg}) -> rabbit_data_coercion:to_binary(ErrMsg); +format({shutdown, _} = Error) -> + rabbit_log:debug("Metadata store is unavailable: ~p", [Error]), + rabbit_data_coercion:to_binary( + rabbit_misc:format("Metadata store is unavailable. Please try again.", [])); format(E) -> rabbit_data_coercion:to_binary(rabbit_misc:format("~tp", [E])). diff --git a/deps/rabbit/src/rabbit_exchange.erl b/deps/rabbit/src/rabbit_exchange.erl index 60347d80afe3..3790882ebaa6 100644 --- a/deps/rabbit/src/rabbit_exchange.erl +++ b/deps/rabbit/src/rabbit_exchange.erl @@ -225,9 +225,6 @@ count() -> list_names() -> rabbit_db_exchange:list(). -%% Not dirty_match_object since that would not be transactional when used in a -%% tx context - -spec list(rabbit_types:vhost()) -> [rabbit_types:exchange()]. list(VHostPath) -> diff --git a/deps/rabbit/src/rabbit_exchange_decorator.erl b/deps/rabbit/src/rabbit_exchange_decorator.erl index 99251fa0ca50..b4144e127c76 100644 --- a/deps/rabbit/src/rabbit_exchange_decorator.erl +++ b/deps/rabbit/src/rabbit_exchange_decorator.erl @@ -26,6 +26,11 @@ -type(serial() :: pos_integer() | 'none'). +%% Callbacks on Khepri are always executed outside of a transaction, thus +%% this implementation has been updated to reflect this. The 'transaction' +%% parameter disappears, even for mnesia, callbacks run only once +%% and their implementation must ensure any transaction required. + -callback description() -> [proplists:property()]. %% Should Rabbit ensure that all binding events that are @@ -109,6 +114,7 @@ maybe_recover(X = #exchange{name = Name, case New of Old -> ok; _ -> %% TODO create a tx here for non-federation decorators - _ = [M:create(none, X) || M <- New -- Old], + Serial = rabbit_exchange:serial(X), + _ = [M:create(Serial, X) || M <- New -- Old], rabbit_exchange:update_decorators(Name, Decs1) end. diff --git a/deps/rabbit/src/rabbit_exchange_type_direct.erl b/deps/rabbit/src/rabbit_exchange_type_direct.erl index 41223721e9cd..c4250ee92626 100644 --- a/deps/rabbit/src/rabbit_exchange_type_direct.erl +++ b/deps/rabbit/src/rabbit_exchange_type_direct.erl @@ -38,49 +38,17 @@ route(#exchange{name = Name, type = Type}, Msg, _Opts) -> Routes = mc:get_annotation(routing_keys, Msg), case Type of direct -> - route_v2(Name, Routes); + rabbit_db_binding:match_routing_key(Name, Routes, true); _ -> - rabbit_router:match_routing_key(Name, Routes) - end. + rabbit_db_binding:match_routing_key(Name, Routes, false) + end. validate(_X) -> ok. validate_binding(_X, _B) -> ok. -create(_Tx, _X) -> ok. -delete(_Tx, _X) -> ok. +create(_Serial, _X) -> ok. +delete(_Serial, _X) -> ok. policy_changed(_X1, _X2) -> ok. -add_binding(_Tx, _X, _B) -> ok. -remove_bindings(_Tx, _X, _Bs) -> ok. +add_binding(_Serial, _X, _B) -> ok. +remove_bindings(_Serial, _X, _Bs) -> ok. assert_args_equivalence(X, Args) -> rabbit_exchange:assert_args_equivalence(X, Args). - -%% rabbit_router:match_routing_key/2 uses ets:select/2 to get destinations. -%% ets:select/2 is expensive because it needs to compile the match spec every -%% time and lookup does not happen by a hash key. -%% -%% In contrast, route_v2/2 increases end-to-end message sending throughput -%% (i.e. from RabbitMQ client to the queue process) by up to 35% by using ets:lookup_element/3. -%% Only the direct exchange type uses the rabbit_index_route table to store its -%% bindings by table key tuple {SourceExchange, RoutingKey}. --spec route_v2(rabbit_types:binding_source(), [rabbit_router:routing_key(), ...]) -> - rabbit_router:match_result(). -route_v2(SrcName, [RoutingKey]) -> - %% optimization - destinations(SrcName, RoutingKey); -route_v2(SrcName, [_|_] = RoutingKeys) -> - lists:flatmap(fun(Key) -> - destinations(SrcName, Key) - end, RoutingKeys). - -destinations(SrcName, RoutingKey) -> - %% Prefer try-catch block over checking Key existence with ets:member/2. - %% The latter reduces throughput by a few thousand messages per second because - %% of function db_member_hash in file erl_db_hash.c. - %% We optimise for the happy path, that is the binding / table key is present. - try - ets:lookup_element(rabbit_index_route, - {SrcName, RoutingKey}, - #index_route.destination) - catch - error:badarg -> - [] - end. diff --git a/deps/rabbit/src/rabbit_exchange_type_fanout.erl b/deps/rabbit/src/rabbit_exchange_type_fanout.erl index 992f945d2f8c..dcb15d99418b 100644 --- a/deps/rabbit/src/rabbit_exchange_type_fanout.erl +++ b/deps/rabbit/src/rabbit_exchange_type_fanout.erl @@ -39,10 +39,10 @@ route(#exchange{name = Name}, _Message, _Opts) -> validate(_X) -> ok. validate_binding(_X, _B) -> ok. -create(_Tx, _X) -> ok. -delete(_Tx, _X) -> ok. +create(_Serial, _X) -> ok. +delete(_Serial, _X) -> ok. policy_changed(_X1, _X2) -> ok. -add_binding(_Tx, _X, _B) -> ok. -remove_bindings(_Tx, _X, _Bs) -> ok. +add_binding(_Serial, _X, _B) -> ok. +remove_bindings(_Serial, _X, _Bs) -> ok. assert_args_equivalence(X, Args) -> rabbit_exchange:assert_args_equivalence(X, Args). diff --git a/deps/rabbit/src/rabbit_exchange_type_headers.erl b/deps/rabbit/src/rabbit_exchange_type_headers.erl index c9cf4b3d2ff8..c3567925dc60 100644 --- a/deps/rabbit/src/rabbit_exchange_type_headers.erl +++ b/deps/rabbit/src/rabbit_exchange_type_headers.erl @@ -104,10 +104,10 @@ validate_binding(_X, #binding{args = Args}) -> end. validate(_X) -> ok. -create(_Tx, _X) -> ok. -delete(_Tx, _X) -> ok. +create(_Serial, _X) -> ok. +delete(_Serial, _X) -> ok. policy_changed(_X1, _X2) -> ok. -add_binding(_Tx, _X, _B) -> ok. -remove_bindings(_Tx, _X, _Bs) -> ok. +add_binding(_Serial, _X, _B) -> ok. +remove_bindings(_Serial, _X, _Bs) -> ok. assert_args_equivalence(X, Args) -> rabbit_exchange:assert_args_equivalence(X, Args). diff --git a/deps/rabbit/src/rabbit_exchange_type_invalid.erl b/deps/rabbit/src/rabbit_exchange_type_invalid.erl index e7816c967b76..608b2fb5f51c 100644 --- a/deps/rabbit/src/rabbit_exchange_type_invalid.erl +++ b/deps/rabbit/src/rabbit_exchange_type_invalid.erl @@ -39,10 +39,10 @@ route(#exchange{name = Name, type = Type}, _, _Opts) -> validate(_X) -> ok. validate_binding(_X, _B) -> ok. -create(_Tx, _X) -> ok. -delete(_Tx, _X) -> ok. +create(_Serial, _X) -> ok. +delete(_Serial, _X) -> ok. policy_changed(_X1, _X2) -> ok. -add_binding(_Tx, _X, _B) -> ok. -remove_bindings(_Tx, _X, _Bs) -> ok. +add_binding(_Serial, _X, _B) -> ok. +remove_bindings(_Serial, _X, _Bs) -> ok. assert_args_equivalence(X, Args) -> rabbit_exchange:assert_args_equivalence(X, Args). diff --git a/deps/rabbit/src/rabbit_exchange_type_topic.erl b/deps/rabbit/src/rabbit_exchange_type_topic.erl index 31ee2df797c8..697b78a72671 100644 --- a/deps/rabbit/src/rabbit_exchange_type_topic.erl +++ b/deps/rabbit/src/rabbit_exchange_type_topic.erl @@ -45,7 +45,7 @@ route(#exchange{name = XName}, Msg, Opts) -> validate(_X) -> ok. validate_binding(_X, _B) -> ok. -create(_Tx, _X) -> ok. +create(_Serial, _X) -> ok. delete(_Serial, #exchange{name = X}) -> rabbit_db_topic_exchange:delete_all_for_exchange(X). diff --git a/deps/rabbit/src/rabbit_ff_controller.erl b/deps/rabbit/src/rabbit_ff_controller.erl index 53087faff97a..ee20da822129 100644 --- a/deps/rabbit/src/rabbit_ff_controller.erl +++ b/deps/rabbit/src/rabbit_ff_controller.erl @@ -1065,17 +1065,20 @@ post_enable(#{states_per_node := _}, FeatureName, Nodes, Enabled) -> -ifndef(TEST). all_nodes() -> - lists:usort([node() | rabbit_nodes:list_members()]). + lists:sort(rabbit_nodes:list_members()). running_nodes() -> lists:usort([node() | rabbit_nodes:list_running()]). -else. all_nodes() -> - RemoteNodes = case rabbit_feature_flags:get_overriden_nodes() of - undefined -> rabbit_nodes:list_members(); - Nodes -> Nodes - end, - lists:usort([node() | RemoteNodes]). + AllNodes = case rabbit_feature_flags:get_overriden_nodes() of + undefined -> + rabbit_nodes:list_members(); + Nodes -> + ?assert(lists:member(node(), Nodes)), + Nodes + end, + lists:sort(AllNodes). running_nodes() -> RemoteNodes = case rabbit_feature_flags:get_overriden_running_nodes() of diff --git a/deps/rabbit/src/rabbit_khepri.erl b/deps/rabbit/src/rabbit_khepri.erl new file mode 100644 index 000000000000..2fdac3bc7193 --- /dev/null +++ b/deps/rabbit/src/rabbit_khepri.erl @@ -0,0 +1,1418 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_khepri). + +-include_lib("kernel/include/logger.hrl"). +-include_lib("stdlib/include/assert.hrl"). + +-include_lib("khepri/include/khepri.hrl"). +-include_lib("rabbit_common/include/logging.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). + +-export([setup/0, + setup/1, + can_join_cluster/1, + add_member/2, + remove_member/1, + members/0, + locally_known_members/0, + nodes/0, + locally_known_nodes/0, + get_ra_cluster_name/0, + get_store_id/0, + transfer_leadership/1, + + is_empty/0, + create/2, + adv_create/2, + update/2, + cas/3, + fold/3, + filter/2, + + get/1, + get/2, + adv_get/1, + match/1, + match/2, + exists/1, + list/1, + list_child_nodes/1, + count_children/1, + + put/2, put/3, + adv_put/2, + clear_payload/1, + delete/1, + delete_or_fail/1, + + transaction/1, + transaction/2, + transaction/3, + + clear_store/0, + + dir/0, + info/0, + + status/0]). +%% Used during migration to join the standalone Khepri nodes and form the +%% equivalent cluster +-export([khepri_db_migration_enable/1, + khepri_db_migration_post_enable/1, + is_enabled/0, is_enabled/1, + get_feature_state/0, get_feature_state/1, + handle_fallback/1]). +-export([do_join/1]). +%% To add the current node to an existing cluster +-export([leave_cluster/1]). +-export([check_cluster_consistency/0, + check_cluster_consistency/2, + node_info/0]). +-export([reset/0, + force_reset/0]). +-export([cluster_status_from_khepri/0, + cli_cluster_status/0]). + +%% Path functions +-export([if_has_data/1, + if_has_data_wildcard/0]). + +-export([force_shrink_member_to_current_member/0]). + +-ifdef(TEST). +-export([force_metadata_store/1, + clear_forced_metadata_store/0]). +-endif. + +-compile({no_auto_import, [get/1, get/2, nodes/0]}). + +%% `sys:get_status/1''s spec only allows `sys:name()' but can work on any +%% `erlang:send_destination()' including a `ra:server_id()'. +-dialyzer({nowarn_function, get_sys_status/1}). +-dialyzer({no_match, [status/0, cluster_status_from_khepri/0]}). + +-define(RA_SYSTEM, coordination). +-define(RA_CLUSTER_NAME, rabbitmq_metadata). +-define(RA_FRIENDLY_NAME, "RabbitMQ metadata store"). +-define(STORE_ID, ?RA_CLUSTER_NAME). +-define(MIGRATION_ID, <<"rabbitmq_metadata">>). + +%% By default we should try to reply from the cluster member that makes a +%% request to change the store. Projections are immediately consistent on the +%% node that issues the reply effect and eventually consistent everywhere else. +%% There isn't a performance penalty for replying from the local node and if +%% the local node isn't a part of the cluster, the reply will come from the +%% leader instead. +-define(DEFAULT_COMMAND_OPTIONS, #{reply_from => local}). + +%% Mnesia tables to migrate and cleanup. +%% +%% This table order is important. For instance, user permissions depend on +%% both vhosts and users to exist in the metadata store. +%% +%% Channel and connection tracking are core features with difference: tables +%% cannot be predeclared as they include the node name + +-rabbit_mnesia_tables_to_khepri_db( + [ + {rabbit_vhost, rabbit_db_vhost_m2k_converter}, + {rabbit_user, rabbit_db_user_m2k_converter}, + {rabbit_user_permission, rabbit_db_user_m2k_converter}, + {rabbit_topic_permission, rabbit_db_user_m2k_converter}, + {rabbit_runtime_parameters, rabbit_db_rtparams_m2k_converter}, + {rabbit_queue, rabbit_db_queue_m2k_converter}, + {rabbit_exchange, rabbit_db_exchange_m2k_converter}, + {rabbit_exchange_serial, rabbit_db_exchange_m2k_converter}, + {rabbit_route, rabbit_db_binding_m2k_converter}, + {rabbit_node_maintenance_states, rabbit_db_maintenance_m2k_converter}, + {mirrored_sup_childspec, rabbit_db_msup_m2k_converter}, + + rabbit_durable_queue, + rabbit_durable_exchange, + rabbit_durable_route, + rabbit_semi_durable_route, + rabbit_reverse_route, + rabbit_index_route + ]). + +%% ------------------------------------------------------------------- +%% API wrapping Khepri. +%% ------------------------------------------------------------------- + +-spec setup() -> ok | no_return(). + +setup() -> + setup(rabbit_prelaunch:get_context()). + +-spec setup(map()) -> ok | no_return(). + +setup(_) -> + ?LOG_DEBUG("Starting Khepri-based " ?RA_FRIENDLY_NAME), + ok = ensure_ra_system_started(), + Timeout = application:get_env(rabbit, khepri_default_timeout, 30000), + ok = application:set_env( + [{khepri, [{default_timeout, Timeout}, + {default_store_id, ?STORE_ID}]}], + [{persistent, true}]), + RaServerConfig = #{cluster_name => ?RA_CLUSTER_NAME, + friendly_name => ?RA_FRIENDLY_NAME}, + case khepri:start(?RA_SYSTEM, RaServerConfig) of + {ok, ?STORE_ID} -> + wait_for_leader(), + register_projections(), + ?LOG_DEBUG( + "Khepri-based " ?RA_FRIENDLY_NAME " ready", + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + ok; + {error, _} = Error -> + exit(Error) + end. + +wait_for_leader() -> + wait_for_leader(retry_timeout(), retry_limit()). + +retry_timeout() -> + case application:get_env(rabbit, khepri_leader_wait_retry_timeout) of + {ok, T} -> T; + undefined -> 30000 + end. + +retry_limit() -> + case application:get_env(rabbit, khepri_leader_wait_retry_limit) of + {ok, T} -> T; + undefined -> 10 + end. + +wait_for_leader(_Timeout, 0) -> + exit(timeout_waiting_for_leader); +wait_for_leader(Timeout, Retries) -> + rabbit_log:info("Waiting for Khepri leader for ~tp ms, ~tp retries left", + [Timeout, Retries - 1]), + Options = #{timeout => Timeout, + favor => compromise}, + case khepri:exists(?STORE_ID, [], Options) of + Exists when is_boolean(Exists) -> + rabbit_log:info("Khepri leader elected"), + ok; + {error, {timeout, _ServerId}} -> + wait_for_leader(Timeout, Retries -1); + {error, Reason} -> + throw(Reason) + end. + +can_join_cluster(DiscoveryNode) when is_atom(DiscoveryNode) -> + ThisNode = node(), + try + ClusterNodes0 = erpc:call( + DiscoveryNode, + rabbit_khepri, locally_known_nodes, []), + ClusterNodes1 = ClusterNodes0 -- [ThisNode], + {ok, ClusterNodes1} + catch + _:Reason -> + {error, Reason} + end. + +add_member(JoiningNode, JoinedNode) + when JoiningNode =:= node() andalso is_atom(JoinedNode) -> + Ret = do_join(JoinedNode), + post_add_member(JoiningNode, JoinedNode, Ret); +add_member(JoiningNode, JoinedNode) when is_atom(JoinedNode) -> + Ret = rabbit_misc:rpc_call( + JoiningNode, rabbit_khepri, do_join, [JoinedNode]), + post_add_member(JoiningNode, JoinedNode, Ret); +add_member(JoiningNode, [_ | _] = Cluster) -> + JoinedNode = pick_node_in_cluster(Cluster), + ?LOG_INFO( + "Khepri clustering: Attempt to add node ~p to cluster ~0p " + "through node ~p", + [JoiningNode, Cluster, JoinedNode], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + %% Recurse with a single node taken in the `Cluster' list. + add_member(JoiningNode, JoinedNode). + +pick_node_in_cluster([_ | _] = Cluster) when is_list(Cluster) -> + ThisNode = node(), + case lists:member(ThisNode, Cluster) of + true -> ThisNode; + false -> hd(Cluster) + end. + +do_join(RemoteNode) when RemoteNode =/= node() -> + ThisNode = node(), + + ?LOG_DEBUG( + "Khepri clustering: Trying to add this node (~p) to cluster \"~s\" " + "through node ~p", + [ThisNode, ?RA_CLUSTER_NAME, RemoteNode], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + + %% Ensure the local Khepri store is running before we can reset it. It + %% could be stopped if RabbitMQ is not running for instance. + ok = setup(), + khepri:info(?RA_CLUSTER_NAME), + + %% Ensure the remote node is reachable before we add it. + pong = net_adm:ping(RemoteNode), + + %% We verify the cluster membership before adding `ThisNode' to + %% `RemoteNode''s cluster. We do it mostly to keep the same behavior as + %% what we do with Mnesia. Otherwise, the interest is limited given the + %% check and the actual join are not atomic. + + ?LOG_DEBUG( + "Adding this node (~p) to Khepri cluster \"~s\" through node ~p", + [ThisNode, ?RA_CLUSTER_NAME, RemoteNode], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + + %% If the remote node to add is running RabbitMQ, we need to put it in + %% maintenance mode at least. We remember that state to revive the node + %% only if it was fully running before this code. + IsRunning = rabbit:is_running(ThisNode), + AlreadyBeingDrained = + rabbit_maintenance:is_being_drained_consistent_read(ThisNode), + NeedToRevive = IsRunning andalso not AlreadyBeingDrained, + maybe_drain_node(IsRunning), + + %% Joining a cluster includes a reset of the local Khepri store. + Ret = khepri_cluster:join(?RA_CLUSTER_NAME, RemoteNode), + + %% Revive the remote node if it was running and not under maintenance + %% before we changed the cluster membership. + maybe_revive_node(NeedToRevive), + + Ret. + +maybe_drain_node(true) -> + ok = rabbit_maintenance:drain(); +maybe_drain_node(false) -> + ok. + +maybe_revive_node(true) -> + ok = rabbit_maintenance:revive(); +maybe_revive_node(false) -> + ok. + +post_add_member(JoiningNode, JoinedNode, ok) -> + ?LOG_INFO( + "Khepri clustering: Node ~p successfully added to cluster \"~s\" " + "through node ~p", + [JoiningNode, ?RA_CLUSTER_NAME, JoinedNode], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + ok; +post_add_member(JoiningNode, JoinedNode, Error) -> + ?LOG_INFO( + "Khepri clustering: Failed to add node ~p to cluster \"~s\" " + "through ~p: ~p", + [JoiningNode, ?RA_CLUSTER_NAME, JoinedNode, Error], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + Error. + +remove_member(NodeToRemove) when NodeToRemove =/= node() -> + ?LOG_DEBUG( + "Trying to remove node ~s from Khepri cluster \"~s\" on node ~s", + [NodeToRemove, ?RA_CLUSTER_NAME, node()], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + + %% Check if the node is part of the cluster. We query the local Ra server + %% only, in case the cluster can't elect a leader right now. + CurrentNodes = locally_known_nodes(), + case lists:member(NodeToRemove, CurrentNodes) of + true -> + %% Ensure the remote node is reachable before we remove it. + case net_adm:ping(NodeToRemove) of + pong -> + remove_reachable_member(NodeToRemove); + pang -> + remove_down_member(NodeToRemove) + end; + false -> + ?LOG_INFO( + "Asked to remove node ~s from Khepri cluster \"~s\" but not " + "member of it: ~p", + [NodeToRemove, ?RA_CLUSTER_NAME, lists:sort(CurrentNodes)], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + rabbit_mnesia:e(not_a_cluster_node) + end. + +remove_reachable_member(NodeToRemove) -> + ?LOG_DEBUG( + "Removing remote node ~s from Khepri cluster \"~s\"", + [NodeToRemove, ?RA_CLUSTER_NAME], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + + %% We need the Khepri store to run on the node to remove, to be + %% able to reset it. + ok = rabbit_misc:rpc_call( + NodeToRemove, ?MODULE, setup, []), + + Ret = rabbit_misc:rpc_call( + NodeToRemove, khepri_cluster, reset, [?RA_CLUSTER_NAME]), + case Ret of + ok -> + ?LOG_DEBUG( + "Node ~s removed from Khepri cluster \"~s\"", + [NodeToRemove, ?RA_CLUSTER_NAME], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + ok; + Error -> + ?LOG_ERROR( + "Failed to remove remote node ~s from Khepri " + "cluster \"~s\": ~p", + [NodeToRemove, ?RA_CLUSTER_NAME, Error], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + Error + end. + +remove_down_member(NodeToRemove) -> + ServerRef = khepri_cluster:node_to_member(?STORE_ID, node()), + ServerId = khepri_cluster:node_to_member(?STORE_ID, NodeToRemove), + Ret = ra:remove_member(ServerRef, ServerId), + case Ret of + {ok, _, _} -> + ?LOG_DEBUG( + "Node ~s removed from Khepri cluster \"~s\"", + [NodeToRemove, ?RA_CLUSTER_NAME], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + ok; + {error, Reason} = Error -> + ?LOG_ERROR( + "Failed to remove remote down node ~s from Khepri " + "cluster \"~s\": ~p", + [NodeToRemove, ?RA_CLUSTER_NAME, Reason], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + Error; + {timeout, _} = Reason -> + ?LOG_ERROR( + "Failed to remove remote down node ~s from Khepri " + "cluster \"~s\": ~p", + [NodeToRemove, ?RA_CLUSTER_NAME, Reason], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + {error, Reason} + end. + +reset() -> + %% Rabbit should be stopped, but Khepri needs to be running. Restart it. + ok = setup(), + ok = khepri_cluster:reset(?RA_CLUSTER_NAME), + ok = khepri:stop(?RA_CLUSTER_NAME). + +force_reset() -> + DataDir = maps:get(data_dir, ra_system:fetch(coordination)), + ok = rabbit_file:recursive_delete(filelib:wildcard(DataDir ++ "/*")). + +force_shrink_member_to_current_member() -> + ok = ra_server_proc:force_shrink_members_to_current_member( + {?RA_CLUSTER_NAME, node()}). + +ensure_ra_system_started() -> + {ok, _} = application:ensure_all_started(khepri), + ok = rabbit_ra_systems:ensure_ra_system_started(?RA_SYSTEM). + +members() -> + khepri_cluster:members(?RA_CLUSTER_NAME). + +locally_known_members() -> + khepri_cluster:locally_known_members(?RA_CLUSTER_NAME). + +nodes() -> + khepri_cluster:nodes(?RA_CLUSTER_NAME). + +locally_known_nodes() -> + khepri_cluster:locally_known_nodes(?RA_CLUSTER_NAME). + +get_ra_cluster_name() -> + ?RA_CLUSTER_NAME. + +get_store_id() -> + ?STORE_ID. + +dir() -> + filename:join(rabbit_mnesia:dir(), atom_to_list(?STORE_ID)). + +-spec transfer_leadership([node()]) -> + {ok, in_progress | undefined | node()} | {error, any()}. +transfer_leadership([]) -> + rabbit_log:warning("Skipping leadership transfer of metadata store: no candidate " + "(online, not under maintenance) nodes to transfer to!"); +transfer_leadership(TransferCandidates) -> + case get_feature_state() of + enabled -> + transfer_leadership0(TransferCandidates); + _ -> + rabbit_log:info("Skipping leadership transfer of metadata store: Khepri is not enabled") + end. + +-spec transfer_leadership0([node()]) -> + {ok, in_progress | undefined | node()} | {error, any()}. +transfer_leadership0([]) -> + rabbit_log:warning("Khepri clustering: failed to transfer leadership, no more candidates available", []), + {error, not_migrated}; +transfer_leadership0([Destination | TransferCandidates]) -> + rabbit_log:info("Khepri clustering: transferring leadership to node ~p", [Destination]), + case ra_leaderboard:lookup_leader(?STORE_ID) of + {Name, Node} = Id when Node == node() -> + case ra:transfer_leadership(Id, {Name, Destination}) of + ok -> + case ra:members(Id) of + {_, _, {_, NewNode}} -> + rabbit_log:info("Khepri clustering: successfully transferred leadership to node ~p", [Destination]), + {ok, NewNode}; + {timeout, _} -> + rabbit_log:warning("Khepri clustering: maybe failed to transfer leadership to node ~p, members query has timed out", [Destination]), + {error, not_migrated} + end; + already_leader -> + rabbit_log:info("Khepri clustering: successfully transferred leadership to node ~p, already the leader", [Destination]), + {ok, Destination}; + {error, Reason} -> + rabbit_log:warning("Khepri clustering: failed to transfer leadership to node ~p with the following error ~p", [Destination, Reason]), + transfer_leadership0(TransferCandidates); + {timeout, _} -> + rabbit_log:warning("Khepri clustering: failed to transfer leadership to node ~p with a timeout", [Destination]), + transfer_leadership0(TransferCandidates) + end; + {_, Node} -> + rabbit_log:info("Khepri clustering: skipping leadership transfer, leader is already in node ~p", [Node]), + {ok, Node}; + undefined -> + rabbit_log:info("Khepri clustering: skipping leadership transfer, leader not elected", []), + {ok, undefined} + end. + +status() -> + Nodes = rabbit_nodes:all_running(), + [begin + case get_sys_status({?RA_CLUSTER_NAME, N}) of + {ok, Sys} -> + {_, M} = lists:keyfind(ra_server_state, 1, Sys), + {_, RaftState} = lists:keyfind(raft_state, 1, Sys), + #{commit_index := Commit, + machine_version := MacVer, + current_term := Term, + log := #{last_index := Last, + snapshot_index := SnapIdx}} = M, + [{<<"Node Name">>, N}, + {<<"Raft State">>, RaftState}, + {<<"Log Index">>, Last}, + {<<"Commit Index">>, Commit}, + {<<"Snapshot Index">>, SnapIdx}, + {<<"Term">>, Term}, + {<<"Machine Version">>, MacVer} + ]; + {error, Err} -> + [{<<"Node Name">>, N}, + {<<"Raft State">>, Err}, + {<<"Log Index">>, <<>>}, + {<<"Commit Index">>, <<>>}, + {<<"Snapshot Index">>, <<>>}, + {<<"Term">>, <<>>}, + {<<"Machine Version">>, <<>>} + ] + end + end || N <- Nodes]. + + +get_sys_status(Proc) -> + try lists:nth(5, element(4, sys:get_status(Proc))) of + Sys -> {ok, Sys} + catch + _:Err when is_tuple(Err) -> + {error, element(1, Err)}; + _:_ -> + {error, other} + end. + +cli_cluster_status() -> + case rabbit:is_running() of + true -> + Nodes = locally_known_nodes(), + [{nodes, [{disc, Nodes}]}, + {running_nodes, [N || N <- Nodes, rabbit_nodes:is_running(N)]}, + {cluster_name, rabbit_nodes:cluster_name()}]; + false -> + [] + end. + +leave_cluster(Node) -> + retry_khepri_op(fun() -> remove_member(Node) end, 60). + +check_cluster_consistency() -> + %% We want to find 0 or 1 consistent nodes. + ReachableNodes = rabbit_nodes:list_reachable(), + case lists:foldl( + fun (Node, {error, _}) -> check_cluster_consistency(Node, true); + (_Node, {ok, Status}) -> {ok, Status} + end, {error, not_found}, nodes_excl_me(ReachableNodes)) + of + {ok, {RemoteAllNodes, _Running}} -> + case ordsets:is_subset(ordsets:from_list(ReachableNodes), + ordsets:from_list(RemoteAllNodes)) of + true -> + ok; + false -> + %% We delete the schema here since we think we are + %% clustered with nodes that are no longer in the + %% cluster and there is no other way to remove + %% them from our schema. On the other hand, we are + %% sure that there is another online node that we + %% can use to sync the tables with. There is a + %% race here: if between this check and the + %% `init_db' invocation the cluster gets + %% disbanded, we're left with a node with no + %% mnesia data that will try to connect to offline + %% nodes. + %% TODO delete schema in khepri ??? + ok + end; + {error, not_found} -> + ok; + {error, _} = E -> + E + end. + +nodes_excl_me(Nodes) -> Nodes -- [node()]. + +check_cluster_consistency(Node, CheckNodesConsistency) -> + case (catch remote_node_info(Node)) of + {badrpc, _Reason} -> + {error, not_found}; + {'EXIT', {badarg, _Reason}} -> + {error, not_found}; + {_OTP, _Rabbit, {error, _Reason}} -> + {error, not_found}; + {_OTP, _Rabbit, {ok, Status}} when CheckNodesConsistency -> + case rabbit_db_cluster:check_compatibility(Node) of + ok -> + case check_nodes_consistency(Node, Status) of + ok -> {ok, Status}; + Error -> Error + end; + Error -> + Error + end; + {_OTP, _Rabbit, {ok, Status}} -> + case rabbit_db_cluster:check_compatibility(Node) of + ok -> {ok, Status}; + Error -> Error + end + end. + +remote_node_info(Node) -> + rpc:call(Node, ?MODULE, node_info, []). + +check_nodes_consistency(Node, {RemoteAllNodes, _RemoteRunningNodes}) -> + case me_in_nodes(RemoteAllNodes) of + true -> + ok; + false -> + {error, {inconsistent_cluster, + format_inconsistent_cluster_message(node(), Node)}} + end. + +format_inconsistent_cluster_message(Thinker, Dissident) -> + rabbit_misc:format("Khepri: node ~tp thinks it's clustered " + "with node ~tp, but ~tp disagrees", + [Thinker, Dissident, Dissident]). + +me_in_nodes(Nodes) -> lists:member(node(), Nodes). + +node_info() -> + {rabbit_misc:otp_release(), + rabbit_misc:version(), + cluster_status_from_khepri()}. + +cluster_status_from_khepri() -> + case get_sys_status({?RA_CLUSTER_NAME, node()}) of + {ok, _} -> + All = locally_known_nodes(), + Running = lists:filter( + fun(N) -> + rabbit_nodes:is_running(N) + end, All), + {ok, {All, Running}}; + _ -> + {error, khepri_not_running} + end. + +%% ------------------------------------------------------------------- +%% "Proxy" functions to Khepri API. +%% ------------------------------------------------------------------- + +%% They just add the store ID to every calls. +%% +%% The only exceptions are get() and match() which both call khepri:get() +%% behind the scene with different options. +%% +%% They are some additional functions too, because they are useful in +%% RabbitMQ. They might be moved to Khepri in the future. + +is_empty() -> khepri:is_empty(?STORE_ID). + +create(Path, Data) -> + khepri:create(?STORE_ID, Path, Data, ?DEFAULT_COMMAND_OPTIONS). +adv_create(Path, Data) -> adv_create(Path, Data, #{}). +adv_create(Path, Data, Options0) -> + Options = maps:merge(?DEFAULT_COMMAND_OPTIONS, Options0), + khepri_adv:create(?STORE_ID, Path, Data, Options). +update(Path, Data) -> + khepri:update(?STORE_ID, Path, Data, ?DEFAULT_COMMAND_OPTIONS). +cas(Path, Pattern, Data) -> + khepri:compare_and_swap( + ?STORE_ID, Path, Pattern, Data, ?DEFAULT_COMMAND_OPTIONS). + +fold(Path, Pred, Acc) -> khepri:fold(?STORE_ID, Path, Pred, Acc). + +filter(Path, Pred) -> khepri:filter(?STORE_ID, Path, Pred). + +get(Path) -> + khepri:get(?STORE_ID, Path, #{favor => low_latency}). + +get(Path, Options) -> + khepri:get(?STORE_ID, Path, Options). + +adv_get(Path) -> + khepri_adv:get(?STORE_ID, Path, #{favor => low_latency}). + +match(Path) -> + match(Path, #{}). + +match(Path, Options) -> khepri:get_many(?STORE_ID, Path, Options). + +exists(Path) -> khepri:exists(?STORE_ID, Path, #{favor => low_latency}). + +list(Path) -> khepri:get_many(?STORE_ID, Path ++ [?KHEPRI_WILDCARD_STAR]). + +list_child_nodes(Path) -> + Options = #{props_to_return => [child_names]}, + case khepri_adv:get_many(?STORE_ID, Path, Options) of + {ok, Result} -> + case maps:values(Result) of + [#{child_names := ChildNames}] -> + {ok, ChildNames}; + [] -> + [] + end; + Error -> + Error + end. + +count_children(Path) -> + Options = #{props_to_return => [child_list_length]}, + case khepri_adv:get_many(?STORE_ID, Path, Options) of + {ok, Map} -> + lists:sum([L || #{child_list_length := L} <- maps:values(Map)]); + _ -> + 0 + end. + +clear_payload(Path) -> + khepri:clear_payload(?STORE_ID, Path, ?DEFAULT_COMMAND_OPTIONS). + +delete(Path) -> + khepri:delete_many(?STORE_ID, Path, ?DEFAULT_COMMAND_OPTIONS). + +delete_or_fail(Path) -> + case khepri_adv:delete(?STORE_ID, Path, ?DEFAULT_COMMAND_OPTIONS) of + {ok, Result} -> + case maps:size(Result) of + 0 -> {error, {node_not_found, #{}}}; + _ -> ok + end; + Error -> + Error + end. + +put(PathPattern, Data) -> + khepri:put( + ?STORE_ID, PathPattern, Data, ?DEFAULT_COMMAND_OPTIONS). + +put(PathPattern, Data, Options0) -> + Options = maps:merge(?DEFAULT_COMMAND_OPTIONS, Options0), + khepri:put( + ?STORE_ID, PathPattern, Data, Options). + +adv_put(PathPattern, Data) -> + khepri_adv:put( + ?STORE_ID, PathPattern, Data, ?DEFAULT_COMMAND_OPTIONS). + +transaction(Fun) -> + transaction(Fun, auto, #{}). + +transaction(Fun, ReadWrite) -> + transaction(Fun, ReadWrite, #{}). + +transaction(Fun, ReadWrite, Options0) -> + Options = maps:merge(?DEFAULT_COMMAND_OPTIONS, Options0), + case khepri:transaction(?STORE_ID, Fun, ReadWrite, Options) of + {ok, Result} -> Result; + {error, Reason} -> throw({error, Reason}) + end. + +clear_store() -> + khepri:delete_many(?STORE_ID, "*", ?DEFAULT_COMMAND_OPTIONS). + +info() -> + ok = setup(), + khepri:info(?STORE_ID). + +%% ------------------------------------------------------------------- +%% if_has_data_wildcard(). +%% ------------------------------------------------------------------- + +-spec if_has_data_wildcard() -> Condition when + Condition :: khepri_condition:condition(). + +if_has_data_wildcard() -> + if_has_data([?KHEPRI_WILDCARD_STAR_STAR]). + +%% ------------------------------------------------------------------- +%% if_has_data(). +%% ------------------------------------------------------------------- + +-spec if_has_data(Conditions) -> Condition when + Conditions :: [Condition], + Condition :: khepri_condition:condition(). + +if_has_data(Conditions) -> + #if_all{conditions = Conditions ++ [#if_has_data{has_data = true}]}. + +register_projections() -> + RegisterFuns = [fun register_rabbit_exchange_projection/0, + fun register_rabbit_queue_projection/0, + fun register_rabbit_vhost_projection/0, + fun register_rabbit_users_projection/0, + fun register_rabbit_runtime_parameters_projection/0, + fun register_rabbit_user_permissions_projection/0, + fun register_rabbit_bindings_projection/0, + fun register_rabbit_index_route_projection/0, + fun register_rabbit_topic_graph_projection/0], + [case RegisterFun() of + ok -> ok; + {error, exists} -> ok; + {error, Error} -> throw(Error) + end || RegisterFun <- RegisterFuns], + ok. + +register_rabbit_exchange_projection() -> + Name = rabbit_khepri_exchange, + PathPattern = [rabbit_db_exchange, + exchanges, + _VHost = ?KHEPRI_WILDCARD_STAR, + _Name = ?KHEPRI_WILDCARD_STAR], + KeyPos = #exchange.name, + register_simple_projection(Name, PathPattern, KeyPos). + +register_rabbit_queue_projection() -> + Name = rabbit_khepri_queue, + PathPattern = [rabbit_db_queue, + queues, + _VHost = ?KHEPRI_WILDCARD_STAR, + _Name = ?KHEPRI_WILDCARD_STAR], + KeyPos = 2, %% #amqqueue.name + register_simple_projection(Name, PathPattern, KeyPos). + +register_rabbit_vhost_projection() -> + Name = rabbit_khepri_vhost, + PathPattern = [rabbit_db_vhost, _VHost = ?KHEPRI_WILDCARD_STAR], + KeyPos = 2, %% #vhost.virtual_host + register_simple_projection(Name, PathPattern, KeyPos). + +register_rabbit_users_projection() -> + Name = rabbit_khepri_users, + PathPattern = [rabbit_db_user, + users, + _UserName = ?KHEPRI_WILDCARD_STAR], + KeyPos = 2, %% #internal_user.username + register_simple_projection(Name, PathPattern, KeyPos). + +register_rabbit_runtime_parameters_projection() -> + Name = rabbit_khepri_runtime_parameters, + PathPattern = [rabbit_db_rtparams, + ?KHEPRI_WILDCARD_STAR_STAR], + KeyPos = #runtime_parameters.key, + register_simple_projection(Name, PathPattern, KeyPos). + +register_rabbit_user_permissions_projection() -> + Name = rabbit_khepri_user_permissions, + PathPattern = [rabbit_db_user, + users, + _UserName = ?KHEPRI_WILDCARD_STAR, + user_permissions, + _VHost = ?KHEPRI_WILDCARD_STAR], + KeyPos = #user_permission.user_vhost, + register_simple_projection(Name, PathPattern, KeyPos). + +register_simple_projection(Name, PathPattern, KeyPos) -> + Options = #{keypos => KeyPos}, + Projection = khepri_projection:new(Name, copy, Options), + khepri:register_projection(?RA_CLUSTER_NAME, PathPattern, Projection). + +register_rabbit_bindings_projection() -> + MapFun = fun(_Path, Binding) -> + #route{binding = Binding} + end, + ProjectionFun = projection_fun_for_sets(MapFun), + Options = #{keypos => #route.binding}, + Projection = khepri_projection:new( + rabbit_khepri_bindings, ProjectionFun, Options), + PathPattern = [rabbit_db_binding, + routes, + _VHost = ?KHEPRI_WILDCARD_STAR, + _ExchangeName = ?KHEPRI_WILDCARD_STAR, + _Kind = ?KHEPRI_WILDCARD_STAR, + _DstName = ?KHEPRI_WILDCARD_STAR, + _RoutingKey = ?KHEPRI_WILDCARD_STAR], + khepri:register_projection(?RA_CLUSTER_NAME, PathPattern, Projection). + +register_rabbit_index_route_projection() -> + MapFun = fun(Path, _) -> + [rabbit_db_binding, routes, VHost, ExchangeName, Kind, + DstName, RoutingKey] = Path, + Exchange = rabbit_misc:r(VHost, exchange, ExchangeName), + Destination = rabbit_misc:r(VHost, Kind, DstName), + SourceKey = {Exchange, RoutingKey}, + #index_route{source_key = SourceKey, + destination = Destination} + end, + ProjectionFun = projection_fun_for_sets(MapFun), + Options = #{type => bag, keypos => #index_route.source_key}, + Projection = khepri_projection:new( + rabbit_khepri_index_route, ProjectionFun, Options), + DirectOrFanout = #if_data_matches{pattern = #{type => '$1'}, + conditions = [{'andalso', + {'=/=', '$1', headers}, + {'=/=', '$1', topic}}]}, + PathPattern = [rabbit_db_binding, + routes, + _VHost = ?KHEPRI_WILDCARD_STAR, + _Exchange = DirectOrFanout, + _Kind = ?KHEPRI_WILDCARD_STAR, + _DstName = ?KHEPRI_WILDCARD_STAR, + _RoutingKey = ?KHEPRI_WILDCARD_STAR], + khepri:register_projection(?RA_CLUSTER_NAME, PathPattern, Projection). + +%% Routing information is stored in the Khepri store as a `set'. +%% In order to turn these bindings into records in an ETS `bag', we use a +%% `khepri_projection:extended_projection_fun()' to determine the changes +%% `khepri_projection' should apply to the ETS table using set algebra. +projection_fun_for_sets(MapFun) -> + fun + (Table, Path, #{data := OldPayload}, #{data := NewPayload}) -> + Deletions = sets:subtract(OldPayload, NewPayload), + Creations = sets:subtract(NewPayload, OldPayload), + sets:fold( + fun(Element, _Acc) -> + ets:delete_object(Table, MapFun(Path, Element)) + end, [], Deletions), + ets:insert(Table, [MapFun(Path, Element) || + Element <- sets:to_list(Creations)]); + (Table, Path, _OldProps, #{data := NewPayload}) -> + ets:insert(Table, [MapFun(Path, Element) || + Element <- sets:to_list(NewPayload)]); + + (Table, Path, #{data := OldPayload}, _NewProps) -> + sets:fold( + fun(Element, _Acc) -> + ets:delete_object(Table, MapFun(Path, Element)) + end, [], OldPayload); + (_Table, _Path, _OldProps, _NewProps) -> + ok + end. + +register_rabbit_topic_graph_projection() -> + Name = rabbit_khepri_topic_trie, + %% This projection calls some external functions which are disallowed by + %% Horus because they interact with global or random state. We explicitly + %% allow them here for performance reasons. + ShouldProcessFun = + fun (rabbit_db_topic_exchange, split_topic_key_binary, 1, _From) -> + %% This function uses `persistent_term' to store a lazily compiled + %% binary pattern. + false; + (erlang, make_ref, 0, _From) -> + %% Randomness is discouraged in Ra effects since the effects are + %% executed separately by each cluster member. We'll use a random + %% value for trie node IDs but these IDs will live as long as the + %% projection table and do not need to be stable or reproducible + %% across restarts or across Erlang nodes. + false; + (ets, _F, _A, _From) -> + false; + (M, F, A, From) -> + khepri_tx_adv:should_process_function(M, F, A, From) + end, + Options = #{keypos => #topic_trie_edge.trie_edge, + standalone_fun_options => + #{should_process_function => ShouldProcessFun}}, + ProjectionFun = + fun(Table, Path, OldProps, NewProps) -> + [rabbit_db_binding, routes, + VHost, ExchangeName, _Kind, _DstName, RoutingKey] = Path, + Exchange = rabbit_misc:r(VHost, exchange, ExchangeName), + Words = rabbit_db_topic_exchange:split_topic_key_binary(RoutingKey), + case {OldProps, NewProps} of + {#{data := OldBindings}, #{data := NewBindings}} -> + ToInsert = sets:subtract(NewBindings, OldBindings), + ToDelete = sets:subtract(OldBindings, NewBindings), + follow_down_update( + Table, Exchange, Words, + fun(ExistingBindings) -> + sets:union( + sets:subtract(ExistingBindings, ToDelete), + ToInsert) + end); + {_, #{data := NewBindings}} -> + follow_down_update( + Table, Exchange, Words, + fun(ExistingBindings) -> + sets:union(ExistingBindings, NewBindings) + end); + {#{data := OldBindings}, _} -> + follow_down_update( + Table, Exchange, Words, + fun(ExistingBindings) -> + sets:subtract(ExistingBindings, OldBindings) + end); + {_, _} -> + ok + end + end, + Projection = khepri_projection:new(Name, ProjectionFun, Options), + PathPattern = [rabbit_db_binding, + routes, + _VHost = ?KHEPRI_WILDCARD_STAR, + _Exchange = #if_data_matches{pattern = #{type => topic}}, + _Kind = ?KHEPRI_WILDCARD_STAR, + _DstName = ?KHEPRI_WILDCARD_STAR, + _RoutingKey = ?KHEPRI_WILDCARD_STAR], + khepri:register_projection(?RA_CLUSTER_NAME, PathPattern, Projection). + +-spec follow_down_update(Table, Exchange, Words, UpdateFn) -> Ret when + Table :: ets:tid(), + Exchange :: rabbit_types:exchange_name(), + Words :: [binary()], + BindingsSet :: sets:set(rabbit_types:binding()), + UpdateFn :: fun((BindingsSet) -> BindingsSet), + Ret :: ok. + +follow_down_update(Table, Exchange, Words, UpdateFn) -> + follow_down_update(Table, Exchange, root, Words, UpdateFn), + ok. + +-spec follow_down_update(Table, Exchange, NodeId, Words, UpdateFn) -> Ret when + Table :: ets:tid(), + Exchange :: rabbit_types:exchange_name(), + NodeId :: root | rabbit_guid:guid(), + Words :: [binary()], + BindingsSet :: sets:set(rabbit_types:binding()), + UpdateFn :: fun((BindingsSet) -> BindingsSet), + Ret :: keep | delete. + +follow_down_update(Table, Exchange, FromNodeId, [To | Rest], UpdateFn) -> + TrieEdge = #trie_edge{exchange_name = Exchange, + node_id = FromNodeId, + word = To}, + ToNodeId = case ets:lookup(Table, TrieEdge) of + [#topic_trie_edge{node_id = ExistingId}] -> + ExistingId; + [] -> + %% The Khepri topic graph table uses references for node + %% IDs instead of `rabbit_guid:gen/0' used by mnesia. + %% This is possible because the topic graph table is + %% never persisted to disk. References take up slightly + %% less memory and are very cheap to produce compared to + %% `rabbit_guid' (which requires the `rabbit_guid' + %% genserver to be online). + NewNodeId = make_ref(), + NewEdge = #topic_trie_edge{trie_edge = TrieEdge, + node_id = NewNodeId}, + %% Create the intermediary node. + ets:insert(Table, NewEdge), + NewNodeId + end, + case follow_down_update(Table, Exchange, ToNodeId, Rest, UpdateFn) of + delete -> + OutEdgePattern = #topic_trie_edge{trie_edge = + TrieEdge#trie_edge{word = '_'}, + node_id = '_'}, + case ets:match(Table, OutEdgePattern, 1) of + '$end_of_table' -> + ets:delete(Table, TrieEdge), + delete; + {_Match, _Continuation} -> + keep + end; + keep -> + keep + end; +follow_down_update(Table, Exchange, LeafNodeId, [], UpdateFn) -> + TrieEdge = #trie_edge{exchange_name = Exchange, + node_id = LeafNodeId, + word = bindings}, + Bindings = case ets:lookup(Table, TrieEdge) of + [#topic_trie_edge{node_id = + {bindings, ExistingBindings}}] -> + ExistingBindings; + [] -> + sets:new([{version, 2}]) + end, + NewBindings = UpdateFn(Bindings), + case sets:is_empty(NewBindings) of + true -> + %% If the bindings have been deleted, delete the trie edge and + %% any edges that no longer lead to any bindings or other edges. + ets:delete(Table, TrieEdge), + delete; + false -> + ToNodeId = {bindings, NewBindings}, + Edge = #topic_trie_edge{trie_edge = TrieEdge, node_id = ToNodeId}, + ets:insert(Table, Edge), + keep + end. + +retry_khepri_op(Fun, 0) -> + Fun(); +retry_khepri_op(Fun, N) -> + case Fun() of + {error, {no_more_servers_to_try, Reasons}} = Err -> + case lists:member({error,cluster_change_not_permitted}, Reasons) of + true -> + timer:sleep(1000), + retry_khepri_op(Fun, N - 1); + false -> + Err + end; + {no_more_servers_to_try, Reasons} = Err -> + case lists:member({error,cluster_change_not_permitted}, Reasons) of + true -> + timer:sleep(1000), + retry_khepri_op(Fun, N - 1); + false -> + Err + end; + {error, cluster_change_not_permitted} -> + timer:sleep(1000), + retry_khepri_op(Fun, N - 1); + Any -> + Any + end. + +%% ------------------------------------------------------------------- +%% Mnesia->Khepri migration code. +%% ------------------------------------------------------------------- + +-spec is_enabled() -> IsEnabled when + IsEnabled :: boolean(). +%% @doc Returns true if Khepri is enabled, false otherwise. +%% +%% This function will block while the feature flag is being enabled and Mnesia +%% tables are migrated. + +is_enabled() -> + is_enabled__internal(blocking). + +-spec is_enabled(Node) -> IsEnabled when + Node :: node(), + IsEnabled :: boolean(). +%% @doc Returns true if Khepri is enabled on node `Node', false otherwise. +%% +%% This function will block while the feature flag is being enabled and Mnesia +%% tables are migrated. + +is_enabled(Node) -> + try + erpc:call(Node, ?MODULE, ?FUNCTION_NAME, []) + catch + error:{exception, undef, [{?MODULE, ?FUNCTION_NAME, _, _} | _]} -> + false + end. + +-spec get_feature_state() -> State when + State :: enabled | state_changing | disabled. +%% @doc Returns the current state of the Khepri use. +%% +%% This function will not block while the feature flag is being enabled and +%% Mnesia tables are migrated. + +get_feature_state() -> + Ret = is_enabled__internal(non_blocking), + case Ret of + true -> enabled; + false -> disabled; + state_changing -> Ret + end. + +-spec get_feature_state(Node) -> State when + Node :: node(), + State :: enabled | state_changing | disabled. +%% @doc Returns the current state of the Khepri use on node `Node'. +%% +%% This function will not block while the feature flag is being enabled and +%% Mnesia tables are migrated. + +get_feature_state(Node) -> + try + erpc:call(Node, ?MODULE, ?FUNCTION_NAME, []) + catch + error:{exception, undef, [{?MODULE, ?FUNCTION_NAME, _, _} | _]} -> + disabled + end. + +khepri_db_migration_enable(#{feature_name := FeatureName}) -> + case sync_cluster_membership_from_mnesia(FeatureName) of + ok -> migrate_mnesia_tables(FeatureName); + Error -> Error + end. + +khepri_db_migration_post_enable( + #{feature_name := FeatureName, enabled := true}) -> + ?LOG_ERROR( + "Feature flag `~s`: cleaning up after finished migration", + [FeatureName], + #{domain => ?RMQLOG_DOMAIN_DB}), + _ = mnesia_to_khepri:cleanup_after_table_copy(?STORE_ID, ?MIGRATION_ID), + ok; +khepri_db_migration_post_enable( + #{feature_name := FeatureName, enabled := false}) -> + ?LOG_ERROR( + "Feature flag `~s`: cleaning up after finished migration", + [FeatureName], + #{domain => ?RMQLOG_DOMAIN_DB}), + _ = mnesia_to_khepri:rollback_table_copy(?STORE_ID, ?MIGRATION_ID), + ok. + +-spec sync_cluster_membership_from_mnesia(FeatureName) -> Ret when + FeatureName :: rabbit_feature_flags:feature_name(), + Ret :: ok | {error, Reason}, + Reason :: any(). +%% @doc Initializes the Khepri cluster based on the Mnesia cluster. +%% +%% It uses the `khepri_mnesia_migration' application to synchronize membership +%% between both cluster. +%% +%% This function is called as part of the `enable' callback of the `khepri_db' +%% feature flag. + +sync_cluster_membership_from_mnesia(FeatureName) -> + %Lock = {{FeatureName, ?FUNCTION_NAME}, self()}, + %global:set_lock(Lock), + try + %% We use a global lock because `rabbit_khepri:setup()' on one node + %% can't run concurrently with the membership sync on another node: + %% the reset which is part of a join might conflict with the start in + %% `rabbit_khepri:setup()'. + sync_cluster_membership_from_mnesia_locked(FeatureName) + after + %global:del_lock(Lock) + ok + end. + +sync_cluster_membership_from_mnesia_locked(FeatureName) -> + rabbit_mnesia:ensure_mnesia_running(), + + try + ?LOG_INFO( + "Feature flag `~s`: syncing cluster membership", + [FeatureName], + #{domain => ?RMQLOG_DOMAIN_DB}), + Ret = mnesia_to_khepri:sync_cluster_membership(?STORE_ID), + ?LOG_INFO( + "Feature flag `~s`: cluster membership synchronized; " + "members are: ~1p", + [FeatureName, lists:sort(nodes())], + #{domain => ?RMQLOG_DOMAIN_DB}), + Ret + catch + error:{khepri_mnesia_migration_ex, _, _} = Error -> + ?LOG_ERROR( + "Feature flag `~s`: failed to sync membership: ~p", + [FeatureName, Error], + #{domain => ?RMQLOG_DOMAIN_DB}), + {error, Error} + end. + +migrate_mnesia_tables(FeatureName) -> + LoadedPlugins = load_disabled_plugins(), + Migrations = discover_mnesia_tables_to_migrate(), + Ret = do_migrate_mnesia_tables(FeatureName, Migrations), + unload_disabled_plugins(LoadedPlugins), + Ret. + +load_disabled_plugins() -> + #{plugins_path := PluginsPath} = rabbit_prelaunch:get_context(), + %% We need to call the application master in a short-lived process, just in + %% case it can't answer. This can happen if `rabbit` is stopped + %% concurrently. In this case, the application master is busy trying to + %% stop `rabbit`. However, `rabbit` is waiting for any feature flag + %% operations to finish before it stops. + %% + %% By using this short-lived process and killing it after some time, we + %% prevent a deadlock with the application master. + Parent = self(), + Loader = spawn_link( + fun() -> + Plugins = [P#plugin.name + || P <- rabbit_plugins:list(PluginsPath)], + Plugins1 = lists:map( + fun(Plugin) -> + case application:load(Plugin) of + ok -> {Plugin, true}; + _ -> {Plugin, false} + end + end, Plugins), + Parent ! {plugins_loading, Plugins1}, + erlang:unlink(Parent) + end), + receive + {plugins_loading, Plugins} -> + Plugins + after 60_000 -> + erlang:unlink(Loader), + throw( + {failed_to_discover_mnesia_tables_to_migrate, + plugins_loading_timeout}) + end. + +unload_disabled_plugins(Plugins) -> + %% See `load_disabled_plugins/0' for the reason why we use a short-lived + %% process here. + Parent = self(), + Unloader = spawn_link( + fun() -> + lists:foreach( + fun + ({Plugin, true}) -> _ = application:unload(Plugin); + ({_Plugin, false}) -> ok + end, Plugins), + Parent ! plugins_unloading + end), + receive + plugins_unloading -> + ok + after 30_000 -> + erlang:unlink(Unloader), + throw( + {failed_to_discover_mnesia_tables_to_migrate, + plugins_unloading_timeout}) + end. + +discover_mnesia_tables_to_migrate() -> + Apps = rabbit_misc:rabbitmq_related_apps(), + AttrsPerApp = rabbit_misc:module_attributes_from_apps( + rabbit_mnesia_tables_to_khepri_db, Apps), + discover_mnesia_tables_to_migrate1(AttrsPerApp, #{}). + +discover_mnesia_tables_to_migrate1( + [{App, _Module, Migrations} | Rest], + MigrationsPerApp) + when is_list(Migrations) -> + Migrations0 = maps:get(App, MigrationsPerApp, []), + Migrations1 = Migrations0 ++ Migrations, + MigrationsPerApp1 = MigrationsPerApp#{App => Migrations1}, + discover_mnesia_tables_to_migrate1(Rest, MigrationsPerApp1); +discover_mnesia_tables_to_migrate1([], MigrationsPerApp) -> + %% We list the applications involved and make sure `rabbit' is handled + %% first. + Apps = lists:sort( + fun + (rabbit, _) -> true; + (_, rabbit) -> false; + (A, B) -> A =< B + end, + maps:keys(MigrationsPerApp)), + lists:foldl( + fun(App, Acc) -> + Acc ++ maps:get(App, MigrationsPerApp) + end, [], Apps). + +do_migrate_mnesia_tables(FeatureName, Migrations) -> + Tables = lists:map( + fun + ({Table, _Mod}) when is_atom(Table) -> Table; + (Table) when is_atom(Table) -> Table + end, + Migrations), + ?LOG_NOTICE( + "Feature flags: `~ts`: starting migration of ~b tables from Mnesia " + "to Khepri; expect decrease in performance and increase in memory " + "footprint", + [FeatureName, length(Migrations)], + #{domain => ?RMQLOG_DOMAIN_DB}), + rabbit_table:wait(Tables, _Retry = true), + Ret = mnesia_to_khepri:copy_tables( + ?STORE_ID, ?MIGRATION_ID, Tables, + {rabbit_db_m2k_converter, Migrations}), + case Ret of + ok -> + ?LOG_NOTICE( + "Feature flags: `~ts`: migration from Mnesia to Khepri " + "finished", + [FeatureName], + #{domain => ?RMQLOG_DOMAIN_DB}), + ok; + {error, _} = Error -> + ?LOG_ERROR( + "Feature flags: `~ts`: failed to migrate Mnesia tables to " + "Khepri:~n ~p", + [FeatureName, Error], + #{domain => ?RMQLOG_DOMAIN_DB}), + {error, {migration_failure, Error}} + end. + +-spec handle_fallback(Funs) -> Ret when + Funs :: #{mnesia := Fun, khepri := Fun | Ret}, + Fun :: fun(() -> Ret), + Ret :: any(). +%% @doc Runs the function corresponding to the used database engine. +%% +%% @returns the return value of `Fun'. + +handle_fallback(#{mnesia := MnesiaFun, khepri := KhepriFunOrRet}) + when is_function(MnesiaFun, 0) -> + case get_feature_state() of + enabled when is_function(KhepriFunOrRet, 0) -> + KhepriFunOrRet(); + enabled -> + KhepriFunOrRet; + _ -> + mnesia_to_khepri:handle_fallback( + ?STORE_ID, ?MIGRATION_ID, MnesiaFun, KhepriFunOrRet) + end. + +-ifdef(TEST). +-define(FORCED_MDS_KEY, {?MODULE, forced_metadata_store}). + +force_metadata_store(Backend) -> + persistent_term:put(?FORCED_MDS_KEY, Backend). + +get_forced_metadata_store() -> + persistent_term:get(?FORCED_MDS_KEY, undefined). + +clear_forced_metadata_store() -> + _ = persistent_term:erase(?FORCED_MDS_KEY), + ok. + +is_enabled__internal(Blocking) -> + case get_forced_metadata_store() of + khepri -> + ?assert( + rabbit_feature_flags:is_enabled(khepri_db, non_blocking)), + true; + mnesia -> + ?assertNot( + rabbit_feature_flags:is_enabled(khepri_db, non_blocking)), + false; + undefined -> + rabbit_feature_flags:is_enabled(khepri_db, Blocking) + end. +-else. +is_enabled__internal(Blocking) -> + rabbit_feature_flags:is_enabled(khepri_db, Blocking). +-endif. diff --git a/deps/rabbit/src/rabbit_maintenance.erl b/deps/rabbit/src/rabbit_maintenance.erl index 342b1763c018..ceae58f66d33 100644 --- a/deps/rabbit/src/rabbit_maintenance.erl +++ b/deps/rabbit/src/rabbit_maintenance.erl @@ -28,7 +28,7 @@ random_primary_replica_transfer_candidate_node/2, transfer_leadership_of_quorum_queues/1, transfer_leadership_of_classic_mirrored_queues/1, - boot/0 + table_definitions/0 ]). -define(DEFAULT_STATUS, regular). @@ -44,13 +44,8 @@ %% Boot %% --rabbit_boot_step({rabbit_maintenance_mode_state, - [{description, "initializes maintenance mode state"}, - {mfa, {?MODULE, boot, []}}, - {requires, networking}]}). - -boot() -> - rabbit_db_maintenance:setup_schema(). +table_definitions() -> + rabbit_db_maintenance:table_definitions(). %% %% API @@ -85,6 +80,8 @@ drain() -> _Pid -> transfer_leadership_of_stream_coordinator(TransferCandidates) end, + transfer_leadership_of_metadata_store(TransferCandidates), + %% allow plugins to react rabbit_event:notify(maintenance_draining, #{ reason => <<"node is being put into maintenance">> @@ -209,6 +206,18 @@ transfer_leadership_of_quorum_queues(_TransferCandidates) -> end || Q <- Queues], rabbit_log:info("Leadership transfer for quorum queues hosted on this node has been initiated"). +transfer_leadership_of_metadata_store(TransferCandidates) -> + rabbit_log:info("Will transfer leadership of metadata store with current leader on this node", + []), + case rabbit_khepri:transfer_leadership(TransferCandidates) of + {ok, Node} when Node == node(); Node == undefined -> + rabbit_log:info("Skipping leadership transfer of metadata store: current leader is not on this node"); + {ok, Node} -> + rabbit_log:info("Leadership transfer for metadata store on this node has been done. The new leader is ~p", [Node]); + Error -> + rabbit_log:warning("Skipping leadership transfer of metadata store: ~p", [Error]) + end. + -spec transfer_leadership_of_classic_mirrored_queues([node()]) -> ok. %% This function is no longer used by maintanence mode. We retain it in case %% classic mirrored queue leadership transfer would be reconsidered. diff --git a/deps/rabbit/src/rabbit_mirror_queue_master.erl b/deps/rabbit/src/rabbit_mirror_queue_master.erl index 7cd10e15ca56..2b46a5ba9be9 100644 --- a/deps/rabbit/src/rabbit_mirror_queue_master.erl +++ b/deps/rabbit/src/rabbit_mirror_queue_master.erl @@ -93,19 +93,7 @@ init_with_existing_bq(Q0, BQ, BQS) when ?is_amqqueue(Q0) -> {ok, CPid} -> GM = rabbit_mirror_queue_coordinator:get_gm(CPid), Self = self(), - Fun = fun () -> - [Q1] = mnesia:read({rabbit_queue, QName}), - true = amqqueue:is_amqqueue(Q1), - GMPids0 = amqqueue:get_gm_pids(Q1), - GMPids1 = [{GM, Self} | GMPids0], - Q2 = amqqueue:set_gm_pids(Q1, GMPids1), - Q3 = amqqueue:set_state(Q2, live), - %% amqqueue migration: - %% The amqqueue was read from this transaction, no - %% need to handle migration. - ok = rabbit_amqqueue:store_queue(Q3) - end, - ok = rabbit_mnesia:execute_mnesia_transaction(Fun), + migrate_queue_record(QName, GM, Self), {_MNode, SNodes} = rabbit_mirror_queue_misc:suggested_queue_nodes(Q0), %% We need synchronous add here (i.e. do not return until the %% mirror is running) so that when queue declaration is finished @@ -131,6 +119,44 @@ init_with_existing_bq(Q0, BQ, BQS) when ?is_amqqueue(Q0) -> throw({coordinator_not_started, Reason}) end. +migrate_queue_record(QName, GM, Self) -> + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> migrate_queue_record_in_mnesia(QName, GM, Self) end, + khepri => fun() -> migrate_queue_record_in_khepri(QName, GM, Self) end + }). + +migrate_queue_record_in_mnesia(QName, GM, Self) -> + Fun = fun () -> + [Q1] = mnesia:read({rabbit_queue, QName}), + true = amqqueue:is_amqqueue(Q1), + GMPids0 = amqqueue:get_gm_pids(Q1), + GMPids1 = [{GM, Self} | GMPids0], + Q2 = amqqueue:set_gm_pids(Q1, GMPids1), + Q3 = amqqueue:set_state(Q2, live), + %% amqqueue migration: + %% The amqqueue was read from this transaction, no + %% need to handle migration. + ok = rabbit_amqqueue:store_queue(Q3) + end, + ok = rabbit_mnesia:execute_mnesia_transaction(Fun). + +migrate_queue_record_in_khepri(QName, GM, Self) -> + Fun = fun () -> + rabbit_db_queue:update_in_khepri_tx( + QName, + fun(Q1) -> + GMPids0 = amqqueue:get_gm_pids(Q1), + GMPids1 = [{GM, Self} | GMPids0], + Q2 = amqqueue:set_gm_pids(Q1, GMPids1), + amqqueue:set_state(Q2, live) + %% Todo it's missing the decorators, but HA is not supported + %% in khepri. This just makes things compile and maybe + %% start HA queues + end) + end, + _ = rabbit_khepri:transaction(Fun, rw), + ok. + -spec stop_mirroring(master_state()) -> {atom(), any()}. stop_mirroring(State = #state { coordinator = CPid, diff --git a/deps/rabbit/src/rabbit_mirror_queue_misc.erl b/deps/rabbit/src/rabbit_mirror_queue_misc.erl index feabab238a7d..e0fd47dbbcf0 100644 --- a/deps/rabbit/src/rabbit_mirror_queue_misc.erl +++ b/deps/rabbit/src/rabbit_mirror_queue_misc.erl @@ -114,6 +114,16 @@ {'error', {'not_synced', [pid()]}}. remove_from_queue(QueueName, Self, DeadGMPids) -> + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> + remove_from_queue_in_mnesia(QueueName, Self, DeadGMPids) + end, + khepri => fun() -> + remove_from_queue_in_khepri(QueueName, Self, DeadGMPids) + end + }). + +remove_from_queue_in_mnesia(QueueName, Self, DeadGMPids) -> rabbit_mnesia:execute_mnesia_transaction( fun () -> %% Someone else could have deleted the queue before we @@ -162,7 +172,7 @@ remove_from_queue(QueueName, Self, DeadGMPids) -> Q1 = amqqueue:set_pid(Q0, QPid1), Q2 = amqqueue:set_slave_pids(Q1, SPids1), Q3 = amqqueue:set_gm_pids(Q2, AliveGM), - _ = store_updated_slaves(Q3), + _ = store_updated_slaves_in_mnesia(Q3), %% If we add and remove nodes at the %% same time we might tell the old %% master we need to sync and then @@ -175,7 +185,7 @@ remove_from_queue(QueueName, Self, DeadGMPids) -> %% [1]. Q1 = amqqueue:set_slave_pids(Q0, Alive), Q2 = amqqueue:set_gm_pids(Q1, AliveGM), - _ = store_updated_slaves(Q2), + _ = store_updated_slaves_in_mnesia(Q2), {ok, QPid1, DeadPids, []} end end @@ -203,6 +213,76 @@ remove_from_queue(QueueName, Self, DeadGMPids) -> %% aforementioned restriction on updating the master pid, that pid may %% not be present in gm_pids, but only if said master has died. +remove_from_queue_in_khepri(QueueName, Self, DeadGMPids) -> + Decorators = rabbit_queue_decorator:list(), + rabbit_khepri:transaction( + fun () -> + %% Someone else could have deleted the queue before we + %% get here. Or, gm group could've altered. see rabbitmq-server#914 + case rabbit_db_queue:get_in_khepri_tx(QueueName) of + [] -> {error, not_found}; + [Q0] when ?is_amqqueue(Q0) -> + QPid = amqqueue:get_pid(Q0), + SPids = amqqueue:get_slave_pids(Q0), + SyncSPids = amqqueue:get_sync_slave_pids(Q0), + GMPids = amqqueue:get_gm_pids(Q0), + {DeadGM, AliveGM} = lists:partition( + fun ({GM, _}) -> + lists:member(GM, DeadGMPids) + end, GMPids), + DeadPids = [Pid || {_GM, Pid} <- DeadGM], + AlivePids = [Pid || {_GM, Pid} <- AliveGM], + Alive = [Pid || Pid <- [QPid | SPids], + lists:member(Pid, AlivePids)], + {QPid1, SPids1} = case Alive of + [] -> + %% GM altered, & if all pids are + %% perceived as dead, rather do + %% do nothing here, & trust the + %% promoted mirror to have updated + %% khepri during the alteration. + {QPid, SPids}; + _ -> promote_slave(Alive) + end, + DoNotPromote = SyncSPids =:= [] andalso + rabbit_policy:get(<<"ha-promote-on-failure">>, Q0) =:= <<"when-synced">>, + case {{QPid, SPids}, {QPid1, SPids1}} of + {Same, Same} -> + {ok, QPid1, DeadPids, []}; + _ when QPid1 =/= QPid andalso QPid1 =:= Self andalso DoNotPromote =:= true -> + %% We have been promoted to master + %% but there are no synchronised mirrors + %% hence this node is not synchronised either + %% Bailing out. + {error, {not_synced, SPids1}}; + _ when QPid =:= QPid1 orelse QPid1 =:= Self -> + %% Either master hasn't changed, so + %% we're ok to update khepri; or we have + %% become the master. If gm altered, + %% we have no choice but to proceed. + Q1 = amqqueue:set_pid(Q0, QPid1), + Q2 = amqqueue:set_slave_pids(Q1, SPids1), + Q3 = amqqueue:set_gm_pids(Q2, AliveGM), + _ = store_updated_slaves_in_khepri(Q3, Decorators), + %% If we add and remove nodes at the + %% same time we might tell the old + %% master we need to sync and then + %% shut it down. So let's check if + %% the new master needs to sync. + %% TODO I doubt this delegate call will go through khepri transactions! + _ = maybe_auto_sync(Q3), + {ok, QPid1, DeadPids, slaves_to_start_on_failure(Q3, DeadGMPids)}; + _ -> + %% Master has changed, and we're not it. + %% [1]. + Q1 = amqqueue:set_slave_pids(Q0, Alive), + Q2 = amqqueue:set_gm_pids(Q1, AliveGM), + _ = store_updated_slaves_in_khepri(Q2, Decorators), + {ok, QPid1, DeadPids, []} + end + end + end, rw). + %% Sometimes a mirror dying means we need to start more on other %% nodes - "exactly" mode can cause this to happen. slaves_to_start_on_failure(Q, DeadGMPids) -> @@ -215,39 +295,76 @@ slaves_to_start_on_failure(Q, DeadGMPids) -> NewNodes -- OldNodes. on_vhost_up(VHost) -> - QNames = - rabbit_mnesia:execute_mnesia_transaction( - fun () -> - mnesia:foldl( - fun + QNames = rabbit_khepri:handle_fallback( + #{mnesia => fun() -> + on_vhost_up_in_mnesia(VHost) + end, + khepri => fun() -> + on_vhost_up_in_khepri(VHost) + end + }), + _ = [add_mirror(QName, node(), async) || QName <- QNames], + ok. + +on_vhost_up_in_mnesia(VHost) -> + rabbit_mnesia:execute_mnesia_transaction( + fun () -> + mnesia:foldl( + fun (Q, QNames0) when not ?amqqueue_vhost_equals(Q, VHost) -> QNames0; (Q, QNames0) when ?amqqueue_is_classic(Q) -> - QName = amqqueue:get_name(Q), - Pid = amqqueue:get_pid(Q), - SPids = amqqueue:get_slave_pids(Q), - %% We don't want to pass in the whole - %% cluster - we don't want a situation - %% where starting one node causes us to - %% decide to start a mirror on another - PossibleNodes0 = [node(P) || P <- [Pid | SPids]], - PossibleNodes = - case lists:member(node(), PossibleNodes0) of - true -> PossibleNodes0; - false -> [node() | PossibleNodes0] - end, - {_MNode, SNodes} = suggested_queue_nodes( - Q, PossibleNodes), - case lists:member(node(), SNodes) of - true -> [QName | QNames0]; - false -> QNames0 - end; + QName = amqqueue:get_name(Q), + Pid = amqqueue:get_pid(Q), + SPids = amqqueue:get_slave_pids(Q), + %% We don't want to pass in the whole + %% cluster - we don't want a situation + %% where starting one node causes us to + %% decide to start a mirror on another + PossibleNodes0 = [node(P) || P <- [Pid | SPids]], + PossibleNodes = + case lists:member(node(), PossibleNodes0) of + true -> PossibleNodes0; + false -> [node() | PossibleNodes0] + end, + {_MNode, SNodes} = suggested_queue_nodes( + Q, PossibleNodes), + case lists:member(node(), SNodes) of + true -> [QName | QNames0]; + false -> QNames0 + end; (_, QNames0) -> - QNames0 - end, [], rabbit_queue) - end), - _ = [add_mirror(QName, node(), async) || QName <- QNames], - ok. + QNames0 + end, [], rabbit_queue) + end). + +on_vhost_up_in_khepri(VHost) -> + Queues = rabbit_amqqueue:list(VHost), + lists:foldl( + fun + (Q, QNames0) when ?amqqueue_is_classic(Q) -> + QName = amqqueue:get_name(Q), + Pid = amqqueue:get_pid(Q), + SPids = amqqueue:get_slave_pids(Q), + %% We don't want to pass in the whole + %% cluster - we don't want a situation + %% where starting one node causes us to + %% decide to start a mirror on another + PossibleNodes0 = [node(P) || P <- [Pid | SPids]], + PossibleNodes = + case lists:member(node(), PossibleNodes0) of + true -> PossibleNodes0; + false -> [node() | PossibleNodes0] + end, + {_MNode, SNodes} = suggested_queue_nodes( + Q, PossibleNodes), + case lists:member(node(), SNodes) of + true -> [QName | QNames0]; + false -> QNames0 + end; + (_, QNames0) -> + QNames0 + end, [], Queues). drop_mirrors(QName, Nodes) -> _ = [drop_mirror(QName, Node) || Node <- Nodes], @@ -343,6 +460,17 @@ log_warning(QName, Fmt, Args) -> amqqueue:amqqueue(). store_updated_slaves(Q0) when ?is_amqqueue(Q0) -> + Decorators = rabbit_queue_decorator:active(Q0), + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> + store_updated_slaves_in_mnesia(Q0) + end, + khepri => fun() -> + store_updated_slaves_in_khepri(Q0, Decorators) + end + }). + +store_updated_slaves_in_mnesia(Q0) -> SPids = amqqueue:get_slave_pids(Q0), SSPids = amqqueue:get_sync_slave_pids(Q0), RS0 = amqqueue:get_recoverable_slaves(Q0), @@ -361,6 +489,29 @@ store_updated_slaves(Q0) when ?is_amqqueue(Q0) -> rabbit_amqqueue:notify_policy_changed(Q3), Q3. +store_updated_slaves_in_khepri(Q0, Decorators) -> + SPids = amqqueue:get_slave_pids(Q0), + SSPids = amqqueue:get_sync_slave_pids(Q0), + RS0 = amqqueue:get_recoverable_slaves(Q0), + %% TODO now that we clear sync_slave_pids in rabbit_durable_queue, + %% do we still need this filtering? + SSPids1 = [SSPid || SSPid <- SSPids, lists:member(SSPid, SPids)], + Q1 = amqqueue:set_sync_slave_pids(Q0, SSPids1), + RS1 = update_recoverable(SPids, RS0), + Q2 = amqqueue:set_recoverable_slaves(Q1, RS1), + Q3 = amqqueue:set_state(Q2, live), + %% amqqueue migration: + %% The amqqueue was read from this transaction, no need to handle + %% migration. + Q4 = amqqueue:set_decorators(Q3, Decorators), + %% HA queues are not supported in Khepri. This update is just enough to make + %% some of the current tests work, which might start some HA queue. + %% It will be removed before Khepri is released. + _ = rabbit_db_queue:update_in_khepri_tx(amqqueue:get_name(Q0), fun(_) -> Q4 end), + %% Wake it up so that we emit a stats event + rabbit_amqqueue:notify_policy_changed(Q3), + Q3. + %% Recoverable nodes are those which we could promote if the whole %% cluster were to suddenly stop and we then lose the master; i.e. all %% nodes with running mirrors , and all stopped nodes which had running @@ -407,6 +558,17 @@ stop_all_slaves(Reason, SPids, QName, GM, WaitTimeout) -> %% Normally when we remove a mirror another mirror or master will %% notice and update Mnesia. But we just removed them all, and %% have stopped listening ourselves. So manually clean up. + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> + remove_all_slaves_in_mnesia(QName, PendingSlavePids) + end, + khepri => fun() -> + remove_all_slaves_in_khepri(QName, PendingSlavePids) + end + }), + ok = gm:forget_group(QName). + +remove_all_slaves_in_mnesia(QName, PendingSlavePids) -> rabbit_mnesia:execute_mnesia_transaction(fun () -> [Q0] = mnesia:read({rabbit_queue, QName}), Q1 = amqqueue:set_gm_pids(Q0, []), @@ -416,8 +578,21 @@ stop_all_slaves(Reason, SPids, QName, GM, WaitTimeout) -> %% the pending mirror pids. Q3 = amqqueue:set_slave_pids_pending_shutdown(Q2, PendingSlavePids), rabbit_mirror_queue_misc:store_updated_slaves(Q3) - end), - ok = gm:forget_group(QName). + end). + +remove_all_slaves_in_khepri(QName, PendingSlavePids) -> + Decorators = rabbit_queue_decorator:list(), + rabbit_khepri:transaction( + fun () -> + [Q0] = rabbit_db_queue:get_in_khepri_tx(QName), + Q1 = amqqueue:set_gm_pids(Q0, []), + Q2 = amqqueue:set_slave_pids(Q1, []), + %% Restarted mirrors on running nodes can + %% ensure old incarnations are stopped using + %% the pending mirror pids. + Q3 = amqqueue:set_slave_pids_pending_shutdown(Q2, PendingSlavePids), + store_updated_slaves_in_khepri(Q3, Decorators) + end, rw). %%---------------------------------------------------------------------------- @@ -773,6 +948,25 @@ are_cmqs_permitted() -> rabbit_deprecated_features:is_permitted(FeatureName). are_cmqs_used(_) -> + case rabbit_khepri:get_feature_state() of + enabled -> + are_cmqs_used1(); + _ -> + %% If we are using Mnesia, we want to check manually if the table + %% exists first. Otherwise it can conflict with the way + %% `rabbit_khepri:handle_fallback/1` works. Indeed, this function + %% and `rabbit_khepri:handle_fallback/1` rely on the `no_exists` + %% exception. + AllTables = mnesia:system_info(tables), + RuntimeParamsReady = lists:member( + rabbit_runtime_parameters, AllTables), + case RuntimeParamsReady of + true -> are_cmqs_used1(); + false -> false + end + end. + +are_cmqs_used1() -> try LocalPolicies = rabbit_policy:list(), LocalOpPolicies = rabbit_policy:list_op(), diff --git a/deps/rabbit/src/rabbit_mirror_queue_slave.erl b/deps/rabbit/src/rabbit_mirror_queue_slave.erl index 12c0233354c3..3acc3d30e81d 100644 --- a/deps/rabbit/src/rabbit_mirror_queue_slave.erl +++ b/deps/rabbit/src/rabbit_mirror_queue_slave.erl @@ -105,6 +105,7 @@ handle_go(Q0) when ?is_amqqueue(Q0) -> %% above. %% process_flag(trap_exit, true), %% amqqueue_process traps exits too. + %% TODO handle gm transactions!!! {ok, GM} = gm:start_link(QName, ?MODULE, [self()], fun rabbit_mnesia:execute_mnesia_transaction/1), MRef = erlang:monitor(process, GM), @@ -118,8 +119,7 @@ handle_go(Q0) when ?is_amqqueue(Q0) -> end, Self = self(), Node = node(), - case rabbit_mnesia:execute_mnesia_transaction( - fun() -> init_it(Self, GM, Node, QName) end) of + case init_it(Self, GM, Node, QName) of {new, QPid, GMPids} -> ok = file_handle_cache:register_callback( rabbit_amqqueue, set_maximum_since_use, [Self]), @@ -169,6 +169,21 @@ handle_go(Q0) when ?is_amqqueue(Q0) -> end. init_it(Self, GM, Node, QName) -> + rabbit_khepri:handle_fallback( + #{mnesia => + fun() -> + rabbit_mnesia:execute_mnesia_transaction( + fun() -> init_it_in_mnesia(Self, GM, Node, QName) end) + end, + khepri => + fun() -> + rabbit_khepri:transaction( + fun() -> init_it_in_khepri(Self, GM, Node, QName) end, + rw) + end + }). + +init_it_in_mnesia(Self, GM, Node, QName) -> case mnesia:read({rabbit_queue, QName}) of [Q] when ?is_amqqueue(Q) -> QPid = amqqueue:get_pid(Q), @@ -197,6 +212,38 @@ init_it(Self, GM, Node, QName) -> master_in_recovery end. +init_it_in_khepri(Self, GM, Node, QName) -> + case rabbit_db_queue:get_in_khepri_tx(QName) of + [Q] when ?is_amqqueue(Q) -> + QPid = amqqueue:get_pid(Q), + SPids = amqqueue:get_slave_pids(Q), + GMPids = amqqueue:get_gm_pids(Q), + PSPids = amqqueue:get_slave_pids_pending_shutdown(Q), + %% TODO we can't kill processes! + case [Pid || Pid <- [QPid | SPids], node(Pid) =:= Node] of + [] -> _ = stop_pending_slaves(QName, PSPids), + %% TODO make add_slave_in_khepri and add_slave_in_mnesia + _ = add_slave(Q, Self, GM), + {new, QPid, GMPids}; + %% TODO is_process_alive should never go on a khepri transaction! + [QPid] -> case rabbit_mnesia:is_process_alive(QPid) of + true -> duplicate_live_master; + false -> {stale, QPid} + end; + [SPid] -> case rabbit_mnesia:is_process_alive(SPid) of + true -> existing; + false -> GMPids1 = [T || T = {_, S} <- GMPids, S =/= SPid], + SPids1 = SPids -- [SPid], + Q1 = amqqueue:set_slave_pids(Q, SPids1), + Q2 = amqqueue:set_gm_pids(Q1, GMPids1), + _ = add_slave(Q2, Self, GM), + {new, QPid, GMPids1} + end + end; + [] -> + master_in_recovery + end. + %% Pending mirrors have been asked to stop by the master, but despite the node %% being up these did not answer on the expected timeout. Stop local mirrors now. stop_pending_slaves(QName, Pids) -> diff --git a/deps/rabbit/src/rabbit_mnesia.erl b/deps/rabbit/src/rabbit_mnesia.erl index 1f37c3aa6215..b3529ebe468c 100644 --- a/deps/rabbit/src/rabbit_mnesia.erl +++ b/deps/rabbit/src/rabbit_mnesia.erl @@ -23,6 +23,8 @@ force_load_next_boot/0, %% Various queries to get the status of the db + %% %% FIXME: Comment below not true anymore. + %% status/0, is_running/0, is_clustered/0, @@ -41,6 +43,8 @@ check_mnesia_consistency/1, check_cluster_consistency/0, ensure_mnesia_dir/0, + ensure_mnesia_running/0, + ensure_node_type_is_permitted/1, %% Hooks used in `rabbit_node_monitor' on_node_up/1, @@ -49,7 +53,12 @@ %% Helpers for diagnostics commands schema_info/1, - reset_gracefully/0 + start_mnesia/1, + stop_mnesia/0, + + reset_gracefully/0, + + e/1 ]). %% Mnesia queries @@ -68,6 +77,8 @@ %% Used internally in `rabbit_db_cluster'. -export([members/0]). +-export([check_reset_gracefully/0]). + -deprecated({on_running_node, 1, "Use rabbit_process:on_running_node/1 instead"}). -deprecated({is_process_alive, 1, @@ -230,7 +241,6 @@ join_cluster(DiscoveryNode, NodeType) when is_atom(DiscoveryNode) -> reset() -> ensure_mnesia_not_running(), - rabbit_log:info("Resetting Rabbit", []), reset_gracefully(). -spec force_reset() -> 'ok'. @@ -247,14 +257,17 @@ reset_gracefully() -> %% Force=true here so that reset still works when clustered with a %% node which is down. init_db_with_mnesia(AllNodes, node_type(), false, false, _Retry = false), - case is_only_clustered_disc_node() of - true -> e(resetting_only_disc_node); - false -> ok - end, + check_reset_gracefully(), leave_cluster(), rabbit_misc:ensure_ok(mnesia:delete_schema([node()]), cannot_delete_schema), wipe(). +check_reset_gracefully() -> + case is_only_clustered_disc_node() of + true -> e(resetting_only_disc_node); + false -> ok + end. + wipe() -> %% We need to make sure that we don't end up in a distributed %% Erlang system with nodes while not being in an Mnesia cluster @@ -893,6 +906,9 @@ discover_cluster0(Node) -> %% We only care about disc nodes since ram nodes are supposed to catch %% up only create_schema() -> + %% Assert we are not supposed to use Khepri. + false = rabbit_khepri:is_enabled(), + stop_mnesia(), rabbit_log:debug("Will bootstrap a schema database..."), rabbit_misc:ensure_ok(mnesia:create_schema([node()]), cannot_create_schema), @@ -1059,7 +1075,8 @@ is_virgin_node() -> rabbit_node_monitor:stream_filename(), rabbit_node_monitor:default_quorum_filename(), rabbit_node_monitor:quorum_filename(), - rabbit_feature_flags:enabled_feature_flags_list_file()], + rabbit_feature_flags:enabled_feature_flags_list_file(), + rabbit_khepri:dir()], IgnoredFiles = [filename:basename(File) || File <- IgnoredFiles0], rabbit_log:debug("Files and directories found in node's data directory: ~ts, of them to be ignored: ~ts", [string:join(lists:usort(List0), ", "), string:join(lists:usort(IgnoredFiles), ", ")]), @@ -1109,6 +1126,6 @@ error_description(no_running_cluster_nodes) -> "You cannot leave a cluster if no online nodes are present.". format_inconsistent_cluster_message(Thinker, Dissident) -> - rabbit_misc:format("Node ~tp thinks it's clustered " + rabbit_misc:format("Mnesia: node ~tp thinks it's clustered " "with node ~tp, but ~tp disagrees", [Thinker, Dissident, Dissident]). diff --git a/deps/rabbit/src/rabbit_node_monitor.erl b/deps/rabbit/src/rabbit_node_monitor.erl index fb4b94ff236a..4f855a8e3caf 100644 --- a/deps/rabbit/src/rabbit_node_monitor.erl +++ b/deps/rabbit/src/rabbit_node_monitor.erl @@ -384,7 +384,10 @@ init([]) -> %% happen. process_flag(trap_exit, true), _ = net_kernel:monitor_nodes(true, [nodedown_reason]), - {ok, _} = mnesia:subscribe(system), + _ = case rabbit_khepri:is_enabled() of + true -> ok; + false -> {ok, _} = mnesia:subscribe(system) + end, %% If the node has been restarted, Mnesia can trigger a system notification %% before the monitor subscribes to receive them. To avoid autoheal blocking due to %% the inconsistent database event never arriving, we being monitoring all running @@ -614,24 +617,13 @@ handle_info({'DOWN', _MRef, process, Pid, _Reason}, State = #state{subscribers = Subscribers}) -> {noreply, State#state{subscribers = pmon:erase(Pid, Subscribers)}}; -handle_info({nodedown, Node, Info}, State = #state{guid = MyGUID, - node_guids = GUIDs}) -> +handle_info({nodedown, Node, Info}, State) -> rabbit_log:info("node ~tp down: ~tp", [Node, proplists:get_value(nodedown_reason, Info)]), - Check = fun (N, CheckGUID, DownGUID) -> - cast(N, {check_partial_partition, - Node, node(), DownGUID, CheckGUID, MyGUID}) - end, - _ = case maps:find(Node, GUIDs) of - {ok, DownGUID} -> Alive = rabbit_mnesia:cluster_nodes(running) - -- [node(), Node], - [case maps:find(N, GUIDs) of - {ok, CheckGUID} -> Check(N, CheckGUID, DownGUID); - error -> ok - end || N <- Alive]; - error -> ok - end, - {noreply, handle_dead_node(Node, State)}; + case rabbit_khepri:is_enabled() of + true -> {noreply, State}; + false -> handle_nodedown_using_mnesia(Node, State) + end; handle_info({nodeup, Node, _Info}, State) -> rabbit_log:info("node ~tp up", [Node]), @@ -703,6 +695,23 @@ code_change(_OldVsn, State, _Extra) -> %% Functions that call the module specific hooks when nodes go up/down %%---------------------------------------------------------------------------- +handle_nodedown_using_mnesia(Node, State = #state{guid = MyGUID, + node_guids = GUIDs}) -> + Check = fun (N, CheckGUID, DownGUID) -> + cast(N, {check_partial_partition, + Node, node(), DownGUID, CheckGUID, MyGUID}) + end, + _ = case maps:find(Node, GUIDs) of + {ok, DownGUID} -> Alive = rabbit_mnesia:cluster_nodes(running) + -- [node(), Node], + [case maps:find(N, GUIDs) of + {ok, CheckGUID} -> Check(N, CheckGUID, DownGUID); + error -> ok + end || N <- Alive]; + error -> ok + end, + {noreply, handle_dead_node(Node, State)}. + handle_dead_node(Node, State = #state{autoheal = Autoheal}) -> %% In general in rabbit_node_monitor we care about whether the %% rabbit application is up rather than the node; we do this so @@ -807,13 +816,20 @@ wait_for_cluster_recovery(Condition) -> wait_for_cluster_recovery(Condition) end. -handle_dead_rabbit(Node, State = #state{partitions = Partitions, - autoheal = Autoheal}) -> +handle_dead_rabbit(Node, State) -> %% TODO: This may turn out to be a performance hog when there are %% lots of nodes. We really only need to execute some of these %% statements on *one* node, rather than all of them. ok = rabbit_amqqueue:on_node_down(Node), ok = rabbit_alarm:on_node_down(Node), + State1 = case rabbit_khepri:is_enabled() of + true -> State; + false -> on_node_down_using_mnesia(Node, State) + end, + ensure_ping_timer(State1). + +on_node_down_using_mnesia(Node, State = #state{partitions = Partitions, + autoheal = Autoheal}) -> ok = rabbit_mnesia:on_node_down(Node), ok = rabbit_quorum_queue_periodic_membership_reconciliation:on_node_down(Node), %% If we have been partitioned, and we are now in the only remaining @@ -827,9 +843,8 @@ handle_dead_rabbit(Node, State = #state{partitions = Partitions, [] -> []; _ -> Partitions end, - ensure_ping_timer( - State#state{partitions = Partitions1, - autoheal = rabbit_autoheal:rabbit_down(Node, Autoheal)}). + State#state{partitions = Partitions1, + autoheal = rabbit_autoheal:rabbit_down(Node, Autoheal)}. ensure_ping_timer(State) -> rabbit_misc:ensure_timer( @@ -844,13 +859,25 @@ ensure_keepalive_timer(State) -> handle_live_rabbit(Node) -> ok = rabbit_amqqueue:on_node_up(Node), ok = rabbit_alarm:on_node_up(Node), - ok = rabbit_mnesia:on_node_up(Node), + case rabbit_khepri:is_enabled() of + true -> ok; + false -> on_node_up_using_mnesia(Node) + end, ok = rabbit_quorum_queue_periodic_membership_reconciliation:on_node_up(Node). -maybe_autoheal(State = #state{partitions = []}) -> +on_node_up_using_mnesia(Node) -> + ok = rabbit_mnesia:on_node_up(Node). + +maybe_autoheal(State) -> + case rabbit_khepri:is_enabled() of + true -> State; + false -> maybe_autoheal1(State) + end. + +maybe_autoheal1(State = #state{partitions = []}) -> State; -maybe_autoheal(State = #state{autoheal = AState}) -> +maybe_autoheal1(State = #state{autoheal = AState}) -> case all_nodes_up() of true -> State#state{autoheal = rabbit_autoheal:maybe_start(AState)}; false -> State diff --git a/deps/rabbit/src/rabbit_policy.erl b/deps/rabbit/src/rabbit_policy.erl index c4446e72fb8d..795d6ee5d43d 100644 --- a/deps/rabbit/src/rabbit_policy.erl +++ b/deps/rabbit/src/rabbit_policy.erl @@ -185,27 +185,35 @@ get(Name, EntityName = #resource{virtual_host = VHost}) -> match(EntityName, list(VHost)), match(EntityName, list_op(VHost))). +%% It's exported, so give it a default until all khepri transformation is sorted match(NameOrQueue, Policies) -> - case match_all(NameOrQueue, Policies) of + match(NameOrQueue, Policies, is_policy_applicable). + +match(NameOrQueue, Policies, Function) -> + case match_all(NameOrQueue, Policies, Function) of [] -> undefined; [Policy | _] -> Policy end. +%% It's exported, so give it a default until all khepri transformation is sorted match_all(NameOrQueue, Policies) -> - lists:sort(fun priority_comparator/2, [P || P <- Policies, matches(NameOrQueue, P)]). + match_all(NameOrQueue, Policies, is_policy_applicable). + +match_all(NameOrQueue, Policies, Function) -> + lists:sort(fun priority_comparator/2, [P || P <- Policies, matches(NameOrQueue, P, Function)]). -matches(Q, Policy) when ?is_amqqueue(Q) -> +matches(Q, Policy, Function) when ?is_amqqueue(Q) -> #resource{name = Name, virtual_host = VHost} = amqqueue:get_name(Q), matches_queue_type(queue, amqqueue:get_type(Q), pget('apply-to', Policy)) andalso - is_applicable(Q, pget(definition, Policy)) andalso + is_applicable(Q, pget(definition, Policy), Function) andalso match =:= re:run(Name, pget(pattern, Policy), [{capture, none}]) andalso VHost =:= pget(vhost, Policy); -matches(#resource{kind = queue} = Resource, Policy) -> +matches(#resource{kind = queue} = Resource, Policy, Function) -> {ok, Q} = rabbit_amqqueue:lookup(Resource), - matches(Q, Policy); -matches(#resource{name = Name, kind = Kind, virtual_host = VHost} = Resource, Policy) -> + matches(Q, Policy, Function); +matches(#resource{name = Name, kind = Kind, virtual_host = VHost} = Resource, Policy, Function) -> matches_type(Kind, pget('apply-to', Policy)) andalso - is_applicable(Resource, pget(definition, Policy)) andalso + is_applicable(Resource, pget(definition, Policy), Function) andalso match =:= re:run(Name, pget(pattern, Policy), [{capture, none}]) andalso VHost =:= pget(vhost, Policy). @@ -389,9 +397,6 @@ notify_clear(VHost, <<"operator_policy">>, Name, ActingUser) -> %%---------------------------------------------------------------------------- -%% [1] We need to prevent this from becoming O(n^2) in a similar -%% manner to rabbit_binding:remove_for_{source,destination}. So see -%% the comment in rabbit_binding:lock_route_tables/0 for more rationale. %% [2] We could be here in a post-tx fun after the vhost has been %% deleted; in which case it's fine to do nothing. update_matched_objects(VHost, PolicyDef, ActingUser) -> @@ -492,11 +497,11 @@ matches_queue_type(queue, _, _) -> false. priority_comparator(A, B) -> pget(priority, A) >= pget(priority, B). -is_applicable(Q, Policy) when ?is_amqqueue(Q) -> - rabbit_amqqueue:is_policy_applicable(Q, rabbit_data_coercion:to_list(Policy)); -is_applicable(#resource{kind = queue} = Resource, Policy) -> - rabbit_amqqueue:is_policy_applicable(Resource, rabbit_data_coercion:to_list(Policy)); -is_applicable(_, _) -> +is_applicable(Q, Policy, Function) when ?is_amqqueue(Q) -> + rabbit_amqqueue:Function(Q, rabbit_data_coercion:to_list(Policy)); +is_applicable(#resource{kind = queue} = Resource, Policy, Function) -> + rabbit_amqqueue:Function(Resource, rabbit_data_coercion:to_list(Policy)); +is_applicable(_, _, _) -> true. %%---------------------------------------------------------------------------- diff --git a/deps/rabbit/src/rabbit_prelaunch_cluster.erl b/deps/rabbit/src/rabbit_prelaunch_cluster.erl index 7effd20cc4c1..e9fb45ff6b7e 100644 --- a/deps/rabbit/src/rabbit_prelaunch_cluster.erl +++ b/deps/rabbit/src/rabbit_prelaunch_cluster.erl @@ -10,23 +10,32 @@ setup(Context) -> ?LOG_DEBUG( "~n== Clustering ==", [], #{domain => ?RMQLOG_DOMAIN_PRELAUNCH}), - ?LOG_DEBUG( - "Preparing cluster status files", [], - #{domain => ?RMQLOG_DOMAIN_PRELAUNCH}), - rabbit_node_monitor:prepare_cluster_status_files(), - case Context of - #{initial_pass := true} -> - %% Renaming a node was partially handled by `rabbit_upgrade', the - %% old upgrade mechanism used before we introduced feature flags. - %% The following call to `rabbit_mnesia_rename' was part of - %% `rabbit_upgrade:maybe_upgrade_mnesia()'. + + case rabbit_khepri:is_enabled() of + true -> + ok; + false -> ?LOG_DEBUG( - "Finish node renaming (if any)", [], + "Preparing cluster status files", [], #{domain => ?RMQLOG_DOMAIN_PRELAUNCH}), - ok = rabbit_mnesia_rename:maybe_finish(); - _ -> - ok + rabbit_node_monitor:prepare_cluster_status_files(), + case Context of + #{initial_pass := true} -> + %% Renaming a node was partially handled by + %% `rabbit_upgrade', the old upgrade mechanism + %% used before we introduced feature flags. The + %% following call to `rabbit_mnesia_rename' was + %% part of + %% `rabbit_upgrade:maybe_upgrade_mnesia()'. + ?LOG_DEBUG( + "Finish node renaming (if any)", [], + #{domain => ?RMQLOG_DOMAIN_PRELAUNCH}), + ok = rabbit_mnesia_rename:maybe_finish(); + _ -> + ok + end end, + ?LOG_DEBUG( "Checking cluster consistency", [], #{domain => ?RMQLOG_DOMAIN_PRELAUNCH}), diff --git a/deps/rabbit/src/rabbit_queue_decorator.erl b/deps/rabbit/src/rabbit_queue_decorator.erl index 4edc93d7e70c..5d9a77317f37 100644 --- a/deps/rabbit/src/rabbit_queue_decorator.erl +++ b/deps/rabbit/src/rabbit_queue_decorator.erl @@ -72,5 +72,5 @@ maybe_recover(Q0) when ?is_amqqueue(Q0) -> _ -> %% TODO LRB JSP 160169569 should startup be passed Q1 here? _ = [M:startup(Q0) || M <- New -- Old], - rabbit_amqqueue:update_decorators(Name) + rabbit_amqqueue:update_decorators(Name, Decs1) end. diff --git a/deps/rabbit/src/rabbit_queue_location.erl b/deps/rabbit/src/rabbit_queue_location.erl index 6471a0e4ab28..85dcec30950b 100644 --- a/deps/rabbit/src/rabbit_queue_location.erl +++ b/deps/rabbit/src/rabbit_queue_location.erl @@ -32,6 +32,8 @@ select_leader_and_followers(Q, Size) true = lists:member(node(), AllNodes), QueueType = amqqueue:get_type(Q), GetQueues0 = get_queues_for_type(QueueType), + %% TODO do we always need the queue count? it can be expensive, check if it can be skipped! + %% for example, for random QueueCount = rabbit_amqqueue:count(), QueueCountStartRandom = application:get_env(rabbit, queue_count_start_random_selection, ?QUEUE_COUNT_START_RANDOM_SELECTION), diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl index 1667a62fcbac..87d1574b7e2e 100644 --- a/deps/rabbit/src/rabbit_quorum_queue.erl +++ b/deps/rabbit/src/rabbit_quorum_queue.erl @@ -253,7 +253,7 @@ start_cluster(Q) -> %% config cannot be updated ok = rabbit_fifo_client:update_machine_state(LeaderId, ra_machine_config(NewQ)), - notify_decorators(QName, startup), + notify_decorators(NewQ, startup), rabbit_quorum_queue_periodic_membership_reconciliation:queue_created(NewQ), rabbit_event:notify(queue_created, [{name, QName}, @@ -1758,6 +1758,10 @@ notify_decorators(Q) when ?is_amqqueue(Q) -> notify_decorators(QName, Event) -> notify_decorators(QName, Event, []). +notify_decorators(Q, F, A) when ?is_amqqueue(Q) -> + Ds = amqqueue:get_decorators(Q), + [ok = apply(M, F, [Q|A]) || M <- rabbit_queue_decorator:select(Ds)], + ok; notify_decorators(QName, F, A) -> %% Look up again in case policy and hence decorators have changed case rabbit_amqqueue:lookup(QName) of diff --git a/deps/rabbit/src/rabbit_ra_registry.erl b/deps/rabbit/src/rabbit_ra_registry.erl index 19e6944e5d0f..ad17a4d9df73 100644 --- a/deps/rabbit/src/rabbit_ra_registry.erl +++ b/deps/rabbit/src/rabbit_ra_registry.erl @@ -13,4 +13,4 @@ %% take them into account in operations such as memory calculation and data cleanup. %% Hardcoded atm list_not_quorum_clusters() -> - [rabbit_stream_coordinator]. + [rabbit_stream_coordinator, rabbit_khepri:get_ra_cluster_name()]. diff --git a/deps/rabbit/src/rabbit_reader.erl b/deps/rabbit/src/rabbit_reader.erl index 637b3c1a1626..9a14fc9435c8 100644 --- a/deps/rabbit/src/rabbit_reader.erl +++ b/deps/rabbit/src/rabbit_reader.erl @@ -1348,7 +1348,12 @@ is_over_vhost_connection_limit(VHostPath, User) -> [VHostPath, User#user.username, Limit]) catch throw:{error, {no_such_vhost, VHostPath}} -> - rabbit_misc:protocol_error(not_allowed, "vhost ~ts not found", [VHostPath]) + rabbit_misc:protocol_error(not_allowed, "vhost ~ts not found", [VHostPath]); + throw:{error, {cannot_get_limit, VHostPath, timeout}} -> + rabbit_misc:protocol_error(not_allowed, + "access to vhost '~ts' refused for user '~ts': " + "connection limit cannot be queried, timeout", + [VHostPath, User#user.username]) end. is_over_user_connection_limit(#user{username = Username}) -> diff --git a/deps/rabbit/src/rabbit_stream_coordinator.erl b/deps/rabbit/src/rabbit_stream_coordinator.erl index 64a9a8171707..f6c2d812b13f 100644 --- a/deps/rabbit/src/rabbit_stream_coordinator.erl +++ b/deps/rabbit/src/rabbit_stream_coordinator.erl @@ -1083,7 +1083,8 @@ phase_update_mnesia(StreamId, Args, #{reference := QName, amqqueue:set_pid(Q, LeaderPid), Conf); Ts -> S = maps:get(name, Ts, undefined), - rabbit_log:debug("~ts: refusing mnesia update for stale stream id ~ts, current ~ts", + %% TODO log as side-effect + rabbit_log:debug("~ts: refusing mnesia update for stale stream id ~s, current ~s", [?MODULE, StreamId, S]), %% if the stream id isn't a match this is a stale %% update from a previous stream incarnation for the diff --git a/deps/rabbit/src/rabbit_stream_queue.erl b/deps/rabbit/src/rabbit_stream_queue.erl index f2f09d54fcb2..fbfe2e6df116 100644 --- a/deps/rabbit/src/rabbit_stream_queue.erl +++ b/deps/rabbit/src/rabbit_stream_queue.erl @@ -1119,6 +1119,7 @@ resend_all(#stream_client{leader = LeaderPid, State. set_leader_pid(Pid, QName) -> + %% TODO this should probably be a single khepri transaction for better performance. Fun = fun (Q) -> amqqueue:set_pid(Q, Pid) end, diff --git a/deps/rabbit/src/rabbit_table.erl b/deps/rabbit/src/rabbit_table.erl index e735df876582..ac2036ca6641 100644 --- a/deps/rabbit/src/rabbit_table.erl +++ b/deps/rabbit/src/rabbit_table.erl @@ -21,6 +21,10 @@ -include_lib("rabbit_common/include/rabbit.hrl"). +-ifdef(TEST). +-export([pre_khepri_definitions/0]). +-endif. + %%---------------------------------------------------------------------------- -type retry() :: boolean(). @@ -34,7 +38,7 @@ create() -> lists:foreach( fun ({Table, Def}) -> create(Table, Def) end, - definitions()), + mandatory_definitions()), ensure_secondary_indexes(), ok. @@ -53,8 +57,13 @@ create(TableName, TableDefinition) -> %% Sets up secondary indexes in a blank node database. ensure_secondary_indexes() -> - ensure_secondary_index(rabbit_queue, vhost), - ok. + case rabbit_khepri:is_enabled() of + true -> + ok; + false -> + ensure_secondary_index(rabbit_queue, vhost), + ok + end. ensure_secondary_index(Table, Field) -> case mnesia:add_table_index(Table, Field) of @@ -101,6 +110,14 @@ wait(TableNames, Retry) -> wait(TableNames, Timeout, Retries). wait(TableNames, Timeout, Retries) -> + %% Wait for tables must only wait for tables that have already been declared. + %% Otherwise, node boot returns a timeout when the Khepri ff is enabled from the start + ExistingTables = mnesia:system_info(tables), + MissingTables = TableNames -- ExistingTables, + TablesToMigrate = TableNames -- MissingTables, + wait1(TablesToMigrate, Timeout, Retries). + +wait1(TableNames, Timeout, Retries) -> %% We might be in ctl here for offline ops, in which case we can't %% get_env() for the rabbit app. rabbit_log:info("Waiting for Mnesia tables for ~tp ms, ~tp retries left", @@ -123,7 +140,7 @@ wait(TableNames, Timeout, Retries) -> throw(Error); {_, {error, Error}} -> rabbit_log:warning("Error while waiting for Mnesia tables: ~tp", [Error]), - wait(TableNames, Timeout, Retries - 1) + wait1(TableNames, Timeout, Retries - 1) end. retry_timeout(_Retry = false) -> @@ -157,8 +174,28 @@ is_empty() -> is_empty(names()). -spec needs_default_data() -> boolean(). -needs_default_data() -> is_empty([rabbit_user, rabbit_user_permission, - rabbit_vhost]). +needs_default_data() -> + case rabbit_khepri:is_enabled() of + true -> + needs_default_data_in_khepri(); + false -> + needs_default_data_in_mnesia() + end. + +needs_default_data_in_khepri() -> + Paths = [rabbit_db_vhost:khepri_vhosts_path(), + rabbit_db_user:khepri_users_path()], + lists:all( + fun(Path) -> + case rabbit_khepri:list(Path) of + {ok, List} when is_map(List) andalso List =:= #{} -> true; + _ -> false + end + end, Paths). + +needs_default_data_in_mnesia() -> + is_empty([rabbit_user, rabbit_user_permission, + rabbit_vhost]). is_empty(Names) -> lists:all(fun (Tab) -> mnesia:dirty_first(Tab) == '$end_of_table' end, @@ -195,9 +232,18 @@ clear_ram_only_tables() -> -spec maybe_clear_ram_only_tables() -> ok. maybe_clear_ram_only_tables() -> - case rabbit_mnesia:members() of - [N] when N=:= node() -> clear_ram_only_tables(); - _ -> ok + %% We use `rabbit_khepri:get_feature_state/0' because we don't want to + %% block here. Indeed, this function is executed as part of + %% `rabbit:stop/1'. + case rabbit_khepri:get_feature_state() of + enabled -> + ok; + _ -> + _ = case rabbit_mnesia:members() of + [N] when N=:= node() -> clear_ram_only_tables(); + _ -> ok + end, + ok end. %% The sequence in which we delete the schema and then the other @@ -305,6 +351,19 @@ definitions(ram) -> {Tab, TabDef} <- definitions()]. definitions() -> + %% Checks for feature flags enabled during node boot must be non_blocking + case rabbit_khepri:get_feature_state() of + enabled -> []; + _ -> mandatory_definitions() + end. + +mandatory_definitions() -> + pre_khepri_definitions() + ++ gm:table_definitions() + ++ mirrored_supervisor:table_definitions() + ++ rabbit_maintenance:table_definitions(). + +pre_khepri_definitions() -> [{rabbit_user, [{record_name, internal_user}, {attributes, internal_user:fields()}, @@ -317,6 +376,11 @@ definitions() -> {match, #user_permission{user_vhost = #user_vhost{_='_'}, permission = #permission{_='_'}, _='_'}}]}, + {rabbit_runtime_parameters, + [{record_name, runtime_parameters}, + {attributes, record_info(fields, runtime_parameters)}, + {disc_copies, [node()]}, + {match, #runtime_parameters{_='_'}}]}, {rabbit_topic_permission, [{record_name, topic_permission}, {attributes, record_info(fields, topic_permission)}, @@ -330,6 +394,28 @@ definitions() -> {attributes, vhost:fields()}, {disc_copies, [node()]}, {match, vhost:pattern_match_all()}]}, + {rabbit_durable_queue, + [{record_name, amqqueue}, + {attributes, amqqueue:fields()}, + {disc_copies, [node()]}, + {match, amqqueue:pattern_match_on_name(queue_name_match())}]}, + {rabbit_queue, + [{record_name, amqqueue}, + {attributes, amqqueue:fields()}, + {match, amqqueue:pattern_match_on_name(queue_name_match())}]}, + {rabbit_durable_exchange, + [{record_name, exchange}, + {attributes, record_info(fields, exchange)}, + {disc_copies, [node()]}, + {match, #exchange{name = exchange_name_match(), _='_'}}]}, + {rabbit_exchange, + [{record_name, exchange}, + {attributes, record_info(fields, exchange)}, + {match, #exchange{name = exchange_name_match(), _='_'}}]}, + {rabbit_exchange_serial, + [{record_name, exchange_serial}, + {attributes, record_info(fields, exchange_serial)}, + {match, #exchange_serial{name = exchange_name_match(), _='_'}}]}, {rabbit_durable_route, [{record_name, route}, {attributes, record_info(fields, route)}, @@ -374,37 +460,8 @@ definitions() -> {attributes, record_info(fields, topic_trie_binding)}, {type, ordered_set}, {match, #topic_trie_binding{trie_binding = trie_binding_match(), - _='_'}}]}, - {rabbit_durable_exchange, - [{record_name, exchange}, - {attributes, record_info(fields, exchange)}, - {disc_copies, [node()]}, - {match, #exchange{name = exchange_name_match(), _='_'}}]}, - {rabbit_exchange, - [{record_name, exchange}, - {attributes, record_info(fields, exchange)}, - {match, #exchange{name = exchange_name_match(), _='_'}}]}, - {rabbit_exchange_serial, - [{record_name, exchange_serial}, - {attributes, record_info(fields, exchange_serial)}, - {match, #exchange_serial{name = exchange_name_match(), _='_'}}]}, - {rabbit_runtime_parameters, - [{record_name, runtime_parameters}, - {attributes, record_info(fields, runtime_parameters)}, - {disc_copies, [node()]}, - {match, #runtime_parameters{_='_'}}]}, - {rabbit_durable_queue, - [{record_name, amqqueue}, - {attributes, amqqueue:fields()}, - {disc_copies, [node()]}, - {match, amqqueue:pattern_match_on_name(queue_name_match())}]}, - {rabbit_queue, - [{record_name, amqqueue}, - {attributes, amqqueue:fields()}, - {match, amqqueue:pattern_match_on_name(queue_name_match())}]} - ] - ++ gm:table_definitions() - ++ mirrored_supervisor:table_definitions(). + _='_'}}]} + ]. binding_match() -> #binding{source = exchange_name_match(), diff --git a/deps/rabbit/src/rabbit_vm.erl b/deps/rabbit/src/rabbit_vm.erl index eb7857f2a5f5..db187cbdaf07 100644 --- a/deps/rabbit/src/rabbit_vm.erl +++ b/deps/rabbit/src/rabbit_vm.erl @@ -36,7 +36,15 @@ memory() -> error:badarg -> 0 end, + MetadataStoreProc = try + [{_, MS}] = process_info(whereis(rabbit_khepri:get_ra_cluster_name()), [memory]), + MS + catch + error:badarg -> + 0 + end, MgmtDbETS = ets_memory([rabbit_mgmt_storage]), + MetadataStoreETS = ets_memory([rabbitmq_metadata]), [{total, ErlangTotal}, {processes, Processes}, {ets, ETS}, @@ -56,8 +64,7 @@ memory() -> OtherProc = Processes - ConnsReader - ConnsWriter - ConnsChannel - ConnsOther - Qs - QsSlave - Qqs - DlxWorkers - Ssqs - Srqs - SCoor - MsgIndexProc - Plugins - - MgmtDbProc - MetricsProc, - + - MgmtDbProc - MetricsProc - MetadataStoreProc, [ %% Connections {connection_readers, ConnsReader}, @@ -76,6 +83,7 @@ memory() -> %% Processes {plugins, Plugins}, + {metadata_store, MetadataStoreProc}, {other_proc, lists:max([0, OtherProc])}, %% [1] %% Metrics @@ -85,7 +93,8 @@ memory() -> %% ETS {mnesia, MnesiaETS}, {quorum_ets, QuorumETS}, - {other_ets, ETS - MnesiaETS - MetricsETS - MgmtDbETS - MsgIndexETS - QuorumETS}, + {metadata_store_ets, MetadataStoreETS}, + {other_ets, ETS - MnesiaETS - MetricsETS - MgmtDbETS - MsgIndexETS - QuorumETS - MetadataStoreETS}, %% Messages (mostly, some binaries are not messages) {binary, Bin}, @@ -123,6 +132,15 @@ binary() -> ConnsChannel, ConnsOther, MsgIndexProc, MgmtDbProc, Plugins] = [aggregate(Names, [{other, Rest} | Sums], binary, fun sum_binary/1) || Names <- [[other] | distinguished_interesting_sups()]], + MetadataStoreProc = try + [{_, B}] = process_info(whereis(rabbit_khepri:get_ra_cluster_name()), [binary]), + lists:foldl(fun({_, Sz, _}, Acc) -> + Sz + Acc + end, 0, B) + catch + error:badarg -> + 0 + end, [{connection_readers, ConnsReader}, {connection_writers, ConnsWriter}, {connection_channels, ConnsChannel}, @@ -134,10 +152,11 @@ binary() -> {stream_queue_procs, Ssqs}, {stream_queue_replica_reader_procs, Srqs}, {stream_queue_coordinator_procs, Scoor}, + {metadata_store, MetadataStoreProc}, {plugins, Plugins}, {mgmt_db, MgmtDbProc}, {msg_index, MsgIndexProc}, - {other, Other}]. + {other, Other - MetadataStoreProc}]. %%---------------------------------------------------------------------------- diff --git a/deps/rabbit/src/vhost.erl b/deps/rabbit/src/vhost.erl index 635e0d14e4f0..d6ce7e44a06f 100644 --- a/deps/rabbit/src/vhost.erl +++ b/deps/rabbit/src/vhost.erl @@ -20,6 +20,7 @@ upgrade/1, upgrade_to/2, pattern_match_all/0, + pattern_match_names/0, get_name/1, get_limits/1, get_metadata/1, @@ -53,7 +54,7 @@ -record(vhost, { %% name as a binary - virtual_host :: name() | '_', + virtual_host :: name() | '_' | '$1', %% proplist of limits configured, if any limits :: limits() | '_', metadata :: metadata() | '_' @@ -67,7 +68,7 @@ -type vhost_pattern() :: vhost_v2_pattern(). -type vhost_v2_pattern() :: #vhost{ - virtual_host :: name() | '_', + virtual_host :: name() | '_' | '$1', limits :: '_', metadata :: '_' }. @@ -128,6 +129,10 @@ info_keys() -> pattern_match_all() -> #vhost{_ = '_'}. +-spec pattern_match_names() -> vhost_pattern(). +pattern_match_names() -> + #vhost{virtual_host = '$1', _ = '_'}. + -spec get_name(vhost()) -> name(). get_name(#vhost{virtual_host = Value}) -> Value. diff --git a/deps/rabbit/test/bindings_SUITE.erl b/deps/rabbit/test/bindings_SUITE.erl index 536a1c5f84f4..f2829494ef51 100644 --- a/deps/rabbit/test/bindings_SUITE.erl +++ b/deps/rabbit/test/bindings_SUITE.erl @@ -21,14 +21,24 @@ suite() -> all() -> [ {group, mnesia_store}, - {group, mnesia_cluster} + {group, khepri_store}, + {group, khepri_migration}, + {group, mnesia_cluster}, + {group, khepri_cluster} ]. groups() -> [ {mnesia_store, [], all_tests()}, + {khepri_store, [], all_tests()}, + {khepri_migration, [], [ + from_mnesia_to_khepri + ]}, {mnesia_cluster, [], [ transient_queue_on_node_down_mnesia + ]}, + {khepri_cluster, [], [ + transient_queue_on_node_down_khepri ]} ]. @@ -65,9 +75,20 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). -init_per_group(mnesia_store = Group, Config) -> +init_per_group(mnesia_store = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, mnesia}]), init_per_group_common(Group, Config, 1); -init_per_group(mnesia_cluster = Group, Config) -> +init_per_group(khepri_store = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, khepri}]), + init_per_group_common(Group, Config, 1); +init_per_group(khepri_migration = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, mnesia}]), + init_per_group_common(Group, Config, 1); +init_per_group(mnesia_cluster = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, mnesia}]), + init_per_group_common(Group, Config, 3); +init_per_group(khepri_cluster = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, khepri}]), init_per_group_common(Group, Config, 3). init_per_group_common(Group, Config, Size) -> @@ -107,7 +128,7 @@ end_per_testcase(Testcase, Config) -> %% ------------------------------------------------------------------- bind_and_unbind(Config) -> Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), - + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), Q = ?config(queue_name, Config), ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), @@ -115,35 +136,35 @@ bind_and_unbind(Config) -> DefaultExchange = rabbit_misc:r(<<"/">>, exchange, <<>>), QResource = rabbit_misc:r(<<"/">>, queue, Q), DefaultBinding = binding_record(DefaultExchange, QResource, Q, []), - + %% Binding to the default exchange, it's always present ?assertEqual([DefaultBinding], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), - + %% Let's bind to other exchange #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, queue = Q, routing_key = Q}), - + DirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), QResource, Q, []), Bindings = lists:sort([DefaultBinding, DirectBinding]), - + ?assertEqual(Bindings, lists:sort( rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>]))), - + #'queue.unbind_ok'{} = amqp_channel:call(Ch, #'queue.unbind'{exchange = <<"amq.direct">>, queue = Q, routing_key = Q}), - + ?assertEqual([DefaultBinding], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), ok. bind_and_delete(Config) -> Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), - + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), Q = ?config(queue_name, Config), ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), @@ -151,34 +172,34 @@ bind_and_delete(Config) -> DefaultExchange = rabbit_misc:r(<<"/">>, exchange, <<>>), QResource = rabbit_misc:r(<<"/">>, queue, Q), DefaultBinding = binding_record(DefaultExchange, QResource, Q, []), - + %% Binding to the default exchange, it's always present ?assertEqual([DefaultBinding], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), - + %% Let's bind to other exchange #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, queue = Q, routing_key = Q}), - + DirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), QResource, Q, []), Bindings = lists:sort([DefaultBinding, DirectBinding]), - + ?assertEqual(Bindings, lists:sort( rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>]))), - + ?assertMatch(#'queue.delete_ok'{}, amqp_channel:call(Ch, #'queue.delete'{queue = Q})), - + ?assertEqual([], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), ok. bind_and_delete_source_exchange(Config) -> Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), - + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), Q = ?config(queue_name, Config), X = ?config(exchange_name, Config), @@ -188,26 +209,26 @@ bind_and_delete_source_exchange(Config) -> DefaultExchange = rabbit_misc:r(<<"/">>, exchange, <<>>), QResource = rabbit_misc:r(<<"/">>, queue, Q), DefaultBinding = binding_record(DefaultExchange, QResource, Q, []), - + %% Binding to the default exchange, it's always present ?assertEqual([DefaultBinding], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), - + %% Let's bind to other exchange #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = X, queue = Q, routing_key = Q}), - + XBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, X), QResource, Q, []), Bindings = lists:sort([DefaultBinding, XBinding]), - + ?assertEqual(Bindings, lists:sort( rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>]))), - + ?assertMatch(#'exchange.delete_ok'{}, amqp_channel:call(Ch, #'exchange.delete'{exchange = X})), - + ?assertEqual([DefaultBinding], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), ok. @@ -226,7 +247,7 @@ list_bindings(Config) -> %% Binding to the default exchange, it's always present ?assertEqual([DefaultBinding], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), - + %% Let's bind to all other exchanges #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, queue = Q, @@ -246,7 +267,7 @@ list_bindings(Config) -> #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.topic">>, queue = Q, routing_key = Q}), - + DirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), QResource, Q, []), FanoutBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.fanout">>), @@ -261,7 +282,7 @@ list_bindings(Config) -> QResource, Q, []), Bindings = lists:sort([DefaultBinding, DirectBinding, FanoutBinding, HeadersBinding, MatchBinding, TraceBinding, TopicBinding]), - + ?assertEqual(Bindings, lists:sort( rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>]))), @@ -276,10 +297,10 @@ list_for_source(Config) -> QAlt = ?config(alt_queue_name, Config), ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), ?assertEqual({'queue.declare_ok', QAlt, 0, 0}, declare(Ch, QAlt, [])), - + QResource = rabbit_misc:r(<<"/">>, queue, Q), QAltResource = rabbit_misc:r(<<"/">>, queue, QAlt), - + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, queue = Q, routing_key = Q}), @@ -301,7 +322,7 @@ list_for_source(Config) -> TopicABinding = binding_record(TopicExchange, QAltResource, QAlt, []), DirectBindings = lists:sort([DirectBinding, DirectABinding]), TopicBindings = lists:sort([TopicBinding, TopicABinding]), - + ?assertEqual( DirectBindings, lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list_for_source, @@ -309,7 +330,7 @@ list_for_source(Config) -> ?assertEqual( TopicBindings, lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list_for_source, - [TopicExchange]))). + [TopicExchange]))). list_with_multiple_vhosts(Config) -> Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), @@ -334,13 +355,13 @@ list_with_multiple_vhosts(Config) -> ?assertEqual({'queue.declare_ok', QAlt, 0, 0}, declare(Ch1, QAlt, [])), ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch2, Q, [])), ?assertEqual({'queue.declare_ok', QAlt, 0, 0}, declare(Ch2, QAlt, [])), - + QResource = rabbit_misc:r(<<"/">>, queue, Q), QAltResource = rabbit_misc:r(<<"/">>, queue, QAlt), QAltResource1 = rabbit_misc:r(VHost1, queue, QAlt), QResource2 = rabbit_misc:r(VHost2, queue, Q), QAltResource2 = rabbit_misc:r(VHost2, queue, QAlt), - + %% Default vhost: %% direct - queue %% topic - altqueue @@ -405,6 +426,12 @@ list_with_multiple_vhosts(Config) -> [QAltResource2]))). list_with_multiple_arguments(Config) -> + %% Bindings are made of source, destination, routing key and arguments. + %% Arguments are difficult to use on khepri paths and also are not relevant to any + %% existing query. Thus, internally the bindings in Khepri are indexed using + %% source, destination and key. Each entry on Khepri contains a set of bindings. + %% For the `rabbit_binding` API nothing has changed, let's test here listing outputs + %% with multiple arguments for the same source, destination and routing key. Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), Ch = rabbit_ct_client_helpers:open_channel(Config, Server), @@ -414,11 +441,11 @@ list_with_multiple_arguments(Config) -> DefaultExchange = rabbit_misc:r(<<"/">>, exchange, <<>>), QResource = rabbit_misc:r(<<"/">>, queue, Q), DefaultBinding = binding_record(DefaultExchange, QResource, Q, []), - + %% Binding to the default exchange, it's always present ?assertEqual([DefaultBinding], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), - + %% Let's bind with multiple arguments #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.headers">>, queue = Q, @@ -428,7 +455,7 @@ list_with_multiple_arguments(Config) -> queue = Q, routing_key = Q, arguments = [{<<"x-match">>, longstr, <<"any">>}]}), - + AllBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.headers">>), QResource, Q, [{<<"x-match">>, longstr, <<"all">>}]), AnyBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.headers">>), @@ -449,10 +476,10 @@ list_for_destination(Config) -> QAlt = ?config(alt_queue_name, Config), ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), ?assertEqual({'queue.declare_ok', QAlt, 0, 0}, declare(Ch, QAlt, [])), - + QResource = rabbit_misc:r(<<"/">>, queue, Q), QAltResource = rabbit_misc:r(<<"/">>, queue, QAlt), - + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, queue = Q, routing_key = Q}), @@ -478,7 +505,7 @@ list_for_destination(Config) -> Bindings = lists:sort([DefaultBinding, DirectBinding, TopicBinding]), AltBindings = lists:sort([DefaultABinding, DirectABinding, TopicABinding]), - + ?assertEqual( Bindings, lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list_for_destination, @@ -496,10 +523,10 @@ list_for_source_and_destination(Config) -> QAlt = ?config(alt_queue_name, Config), ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), ?assertEqual({'queue.declare_ok', QAlt, 0, 0}, declare(Ch, QAlt, [])), - + QResource = rabbit_misc:r(<<"/">>, queue, Q), QAltResource = rabbit_misc:r(<<"/">>, queue, QAlt), - + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, queue = Q, routing_key = Q}), @@ -519,7 +546,7 @@ list_for_source_and_destination(Config) -> DirectBinding = binding_record(DirectExchange, QResource, Q, []), TopicBinding = binding_record(TopicExchange, QResource, Q, []), DefaultABinding = binding_record(DefaultExchange, QAltResource, QAlt, []), - + ?assertEqual( [DirectBinding], lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, @@ -544,10 +571,10 @@ list_for_source_and_destination(Config) -> ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), QResource = rabbit_misc:r(<<"/">>, queue, Q), - + ?assertEqual([], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list_explicit, [])), - + %% Let's bind to other exchanges #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, queue = Q, @@ -555,22 +582,22 @@ list_for_source_and_destination(Config) -> #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.fanout">>, queue = Q, routing_key = Q}), - + DirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), QResource, Q, []), FanoutBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.fanout">>), QResource, Q, []), Bindings = lists:sort([DirectBinding, FanoutBinding]), - + ?assertEqual(Bindings, lists:sort( rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list_explicit, []))), - + ok. info_all(Config) -> Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), - + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), Q = ?config(queue_name, Config), ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), @@ -593,7 +620,7 @@ info_all(Config) -> {routing_key,<<"info_all">>}, {arguments,[]}, {vhost,<<"/">>}], - + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, queue = Q, routing_key = Q}), @@ -602,9 +629,69 @@ info_all(Config) -> ?assertEqual(Infos, lists:sort( rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, info_all, [<<"/">>]))), - + ok. +from_mnesia_to_khepri(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + AltQ = ?config(alt_queue_name, Config), + ?assertEqual({'queue.declare_ok', AltQ, 0, 0}, declare(Ch, AltQ, [], false)), + + %% Combine durable and transient queues and exchanges to test the migration of durable, + %% semi-durable and transient bindings + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = AltQ, + routing_key = AltQ}), + + X = ?config(exchange_name, Config), + #'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = X, + durable = false}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = X, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = X, + queue = AltQ, + routing_key = AltQ}), + + + DefaultExchange = rabbit_misc:r(<<"/">>, exchange, <<>>), + QResource = rabbit_misc:r(<<"/">>, queue, Q), + AltQResource = rabbit_misc:r(<<"/">>, queue, AltQ), + DefaultBinding = binding_record(DefaultExchange, QResource, Q, []), + DirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + QResource, Q, []), + AltDefaultBinding = binding_record(DefaultExchange, AltQResource, AltQ, []), + AltDirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + AltQResource, AltQ, []), + XBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, X), QResource, Q, []), + AltXBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, X), + AltQResource, AltQ, []), + Bindings = lists:sort([DefaultBinding, DirectBinding, AltDefaultBinding, AltDirectBinding, + XBinding, AltXBinding]), + + ?assertEqual(Bindings, + lists:sort( + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>]))), + + case rabbit_ct_broker_helpers:enable_feature_flag(Config, khepri_db) of + ok -> + rabbit_ct_helpers:await_condition( + fun() -> + Bindings == + lists:sort( + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])) + end); + Skip -> + Skip + end. + bind_to_unknown_queue(Config) -> Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), @@ -622,32 +709,32 @@ bind_to_unknown_queue(Config) -> bind_and_unbind_exchange(Config) -> Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), - + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), X = ?config(exchange_name, Config), - + ?assertEqual([], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), - + #'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = X}), %% Let's bind to other exchange #'exchange.bind_ok'{} = amqp_channel:call(Ch, #'exchange.bind'{destination = X, source = <<"amq.direct">>, routing_key = <<"key">>}), - + DirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), rabbit_misc:r(<<"/">>, exchange, X), <<"key">>, []), - + ?assertEqual([DirectBinding], lists:sort( rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>]))), - + #'exchange.unbind_ok'{} = amqp_channel:call(Ch, #'exchange.unbind'{destination = X, source = <<"amq.direct">>, routing_key = <<"key">>}), - + ?assertEqual([], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), ok. @@ -672,58 +759,58 @@ bind_to_unknown_exchange(Config) -> bind_and_delete_exchange_destination(Config) -> Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), - + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), X = ?config(exchange_name, Config), - + ?assertEqual([], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), - + #'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = X}), %% Let's bind to other exchange #'exchange.bind_ok'{} = amqp_channel:call(Ch, #'exchange.bind'{destination = X, source = <<"amq.direct">>, routing_key = <<"key">>}), - + DirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), rabbit_misc:r(<<"/">>, exchange, X), <<"key">>, []), - + ?assertEqual([DirectBinding], lists:sort( rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>]))), - + #'exchange.delete_ok'{} = amqp_channel:call(Ch, #'exchange.delete'{exchange = X}), - + ?assertEqual([], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), ok. bind_and_delete_exchange_source(Config) -> Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), - + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), X = ?config(exchange_name, Config), - + ?assertEqual([], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), - + #'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = X}), %% Let's bind to other exchange #'exchange.bind_ok'{} = amqp_channel:call(Ch, #'exchange.bind'{destination = <<"amq.direct">>, source = X, routing_key = <<"key">>}), - + DirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, X), rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), <<"key">>, []), - + ?assertEqual([DirectBinding], lists:sort( rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>]))), - + #'exchange.delete_ok'{} = amqp_channel:call(Ch, #'exchange.delete'{exchange = X}), - + ?assertEqual([], rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), ok. @@ -783,6 +870,65 @@ transient_queue_on_node_down_mnesia(Config) -> 30000), ok. +transient_queue_on_node_down_khepri(Config) -> + %% All entities are durable in khepri + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + QAlt = ?config(alt_queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + ?assertEqual({'queue.declare_ok', QAlt, 0, 0}, declare(Ch, QAlt, [], false)), + + DefaultExchange = rabbit_misc:r(<<"/">>, exchange, <<>>), + QResource = rabbit_misc:r(<<"/">>, queue, Q), + QAltResource = rabbit_misc:r(<<"/">>, queue, QAlt), + DefaultBinding = binding_record(DefaultExchange, QResource, Q, []), + DefaultAltBinding = binding_record(DefaultExchange, QAltResource, QAlt, []), + + %% Binding to the default exchange, it's always present + ?assertEqual(lists:sort([DefaultBinding, DefaultAltBinding]), + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 1, rabbit_binding, list, [<<"/">>]))), + + %% Let's bind to other exchange + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = Q, + routing_key = Q}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = QAlt, + routing_key = QAlt}), + + DirectBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + QResource, Q, []), + DirectAltBinding = binding_record(rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + QAltResource, QAlt, []), + Bindings = lists:sort([DefaultBinding, DirectBinding, DefaultAltBinding, DirectAltBinding]), + + ?assertEqual(Bindings, + lists:sort( + rabbit_ct_broker_helpers:rpc(Config, 1, rabbit_binding, list, [<<"/">>]))), + + rabbit_ct_broker_helpers:stop_node(Config, Server), + + Bindings1 = lists:sort([DirectBinding, DirectAltBinding]), + ?awaitMatch(Bindings1, + lists:sort( + rabbit_ct_broker_helpers:rpc(Config, 1, rabbit_binding, list, [<<"/">>])), + 30000), + ?awaitMatch([], + rabbit_ct_broker_helpers:rpc(Config, 1, rabbit_amqqueue, list, [<<"/">>]), + 30000), + + rabbit_ct_broker_helpers:start_node(Config, Server), + + ?awaitMatch([_, _], rabbit_ct_broker_helpers:rpc(Config, 1, rabbit_amqqueue, list, [<<"/">>]), + 30000), + ?awaitMatch(Bindings, + lists:sort( + rabbit_ct_broker_helpers:rpc(Config, 1, rabbit_binding, list, [<<"/">>])), + 30000), + ok. + %% Internal delete_queues() -> diff --git a/deps/rabbit/test/cluster_SUITE.erl b/deps/rabbit/test/cluster_SUITE.erl index ff72656692c0..6c23ccbb5f05 100644 --- a/deps/rabbit/test/cluster_SUITE.erl +++ b/deps/rabbit/test/cluster_SUITE.erl @@ -93,6 +93,16 @@ end_per_group(Group, Config) -> Config end. +init_per_testcase(queue_cleanup = Testcase, Config) -> + case lists:any(fun(B) -> B end, + rabbit_ct_broker_helpers:rpc_all( + Config, rabbit_feature_flags, is_enabled, + [khepri_db])) of + true -> + {skip, "Invalid testcase using Khepri. All queues are durable"}; + false -> + rabbit_ct_helpers:testcase_started(Config, Testcase) + end; init_per_testcase(Testcase, Config) -> rabbit_ct_helpers:testcase_started(Config, Testcase). diff --git a/deps/rabbit/test/cluster_minority_SUITE.erl b/deps/rabbit/test/cluster_minority_SUITE.erl new file mode 100644 index 000000000000..9614e198a96e --- /dev/null +++ b/deps/rabbit/test/cluster_minority_SUITE.erl @@ -0,0 +1,294 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(cluster_minority_SUITE). + +-include_lib("common_test/include/ct.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-compile(export_all). + +all() -> + [ + {group, client_operations}, + {group, cluster_operation_add}, + {group, cluster_operation_remove} + ]. + +groups() -> + [ + {client_operations, [], [open_connection, + open_channel, + declare_exchange, + declare_binding, + declare_queue, + publish_to_exchange, + publish_and_consume_to_local_classic_queue, + consume_from_queue, + add_vhost, + update_vhost, + delete_vhost, + add_user, + update_user, + delete_user, + set_policy, + delete_policy + ]}, + {cluster_operation_add, [], [add_node]}, + {cluster_operation_remove, [], [remove_node]}, + {feature_flags, [], [enable_feature_flag]} + ]. + +suite() -> + [ + %% If a testcase hangs, no need to wait for 30 minutes. + {timetrap, {minutes, 5}} + ]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config0) -> + rabbit_ct_helpers:log_environment(), + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, khepri}]), + rabbit_ct_helpers:run_setup_steps(Config, + [ + fun rabbit_ct_broker_helpers:configure_dist_proxy/1 + ]). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(Group, Config0) when Group == client_operations; + Group == feature_flags -> + Config = rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 5}, + {rmq_nodename_suffix, Group}, + {tcp_ports_base}, + {net_ticktime, 5}]), + Config1 = rabbit_ct_helpers:run_steps(Config, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()), + case Config1 of + {skip, _} -> + Config1; + _ -> + %% Before partitioning the cluster, create a policy and queue that can be used in + %% the test cases. They're needed for delete and consume operations, which can list + %% them but fail to operate anything else. + %% + %% To be used in delete_policy + ok = rabbit_ct_broker_helpers:set_policy(Config1, 0, <<"policy-to-delete">>, <<".*">>, <<"queues">>, [{<<"max-age">>, <<"1Y">>}]), + Ch = rabbit_ct_client_helpers:open_channel(Config1, 0), + %% To be used in consume_from_queue + #'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = <<"test-queue">>, + arguments = [{<<"x-queue-type">>, longstr, <<"classic">>}]}), + + %% Create partition + partition_5_node_cluster(Config1), + Config1 + end; +init_per_group(Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 5}, + {rmq_nodename_suffix, Group}, + {rmq_nodes_clustered, false}, + {tcp_ports_base}, + {net_ticktime, 5}]), + rabbit_ct_helpers:run_steps(Config, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + +end_per_group(_, Config) -> + rabbit_ct_helpers:run_steps(Config, + rabbit_ct_broker_helpers:teardown_steps() ++ + rabbit_ct_client_helpers:teardown_steps()). + +init_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_started(Config, Testcase). + +end_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +%% ------------------------------------------------------------------- +%% Test cases +%% ------------------------------------------------------------------- +open_connection(Config) -> + [A, B | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + ConnA = rabbit_ct_client_helpers:open_unmanaged_connection(Config, A, <<"/">>), + ConnB = rabbit_ct_client_helpers:open_unmanaged_connection(Config, B, <<"/">>), + rabbit_ct_client_helpers:close_connection(ConnA), + rabbit_ct_client_helpers:close_connection(ConnB). + +open_channel(Config) -> + [A, B | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + ChA = rabbit_ct_client_helpers:open_channel(Config, A), + ChB = rabbit_ct_client_helpers:open_channel(Config, B), + rabbit_ct_client_helpers:close_channel(ChA), + rabbit_ct_client_helpers:close_channel(ChB). + +declare_exchange(Config) -> + [A | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + {_, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, A), + ?assertExit({{shutdown, {connection_closing, {server_initiated_close, 541, _}}}, _}, + amqp_channel:call(Ch, #'exchange.declare'{exchange = <<"test-exchange">>})). + +declare_binding(Config) -> + [A | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + {_, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, A), + ?assertExit({{shutdown, {connection_closing, {server_initiated_close, 541, _}}}, _}, + amqp_channel:call(Ch, #'exchange.bind'{destination = <<"amq.fanout">>, + source = <<"amq.direct">>, + routing_key = <<"key">>})). + +declare_queue(Config) -> + [A | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + {_, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, A), + ?assertExit({{shutdown, {connection_closing, {server_initiated_close, 541, _}}}, _}, + amqp_channel:call(Ch, #'queue.declare'{queue = <<"test-queue-2">>})). + +publish_to_exchange(Config) -> + [A | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + {_, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, A), + ?assertEqual(ok, amqp_channel:call(Ch, #'basic.publish'{routing_key = <<"test-queue-2">>}, + #amqp_msg{payload = <<"msg">>})). + +publish_and_consume_to_local_classic_queue(Config) -> + [A | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + {_, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, A), + ?assertEqual(ok, amqp_channel:call(Ch, #'basic.publish'{routing_key = <<"test-queue">>}, + #amqp_msg{payload = <<"msg">>})), + ?assertMatch({#'basic.get_ok'{}, _}, + amqp_channel:call(Ch, #'basic.get'{queue = <<"test-queue">>})). + +consume_from_queue(Config) -> + [A | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + {_, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, A), + ?assertMatch(#'basic.consume_ok'{}, + amqp_channel:call(Ch, #'basic.consume'{queue = <<"test-queue">>})). + +add_vhost(Config) -> + ?assertMatch({error, {timeout, _}}, + rabbit_ct_broker_helpers:add_vhost(Config, <<"vhost1">>)). + +update_vhost(Config) -> + ?assertThrow({error, {timeout, _}}, + rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_vhost, update_tags, + [<<"/">>, [carrots], <<"user">>])). + +delete_vhost(Config) -> + ?assertMatch({'EXIT', _}, rabbit_ct_broker_helpers:delete_vhost(Config, <<"vhost1">>)). + +add_user(Config) -> + ?assertMatch({error, {timeout, _}}, + rabbit_ct_broker_helpers:add_user(Config, <<"user1">>)). + +update_user(Config) -> + ?assertMatch({error, {timeout, _}}, + rabbit_ct_broker_helpers:set_user_tags(Config, 0, <<"user1">>, [<<"admin">>])). + +delete_user(Config) -> + ?assertMatch({error, {timeout, _}}, + rabbit_ct_broker_helpers:delete_user(Config, <<"user1">>)). + +set_policy(Config) -> + ?assertError(_, rabbit_ct_broker_helpers:set_policy(Config, 0, <<"max-age-policy">>, <<".*">>, <<"queues">>, [{<<"max-age">>, <<"1Y">>}])). + +delete_policy(Config) -> + ?assertError(_, rabbit_ct_broker_helpers:clear_policy(Config, 0, <<"policy-to-delete">>)). + +add_node(Config) -> + [A, B, C, D, _E] = rabbit_ct_broker_helpers:get_node_configs( + Config, nodename), + + %% Three node cluster: A, B, C + ok = rabbit_control_helper:command(stop_app, B), + ok = rabbit_control_helper:command(join_cluster, B, [atom_to_list(A)], []), + rabbit_control_helper:command(start_app, B), + + ok = rabbit_control_helper:command(stop_app, C), + ok = rabbit_control_helper:command(join_cluster, C, [atom_to_list(A)], []), + rabbit_control_helper:command(start_app, C), + + %% Minority partition: A + Cluster = [A, B, C], + partition_3_node_cluster(Config), + + ok = rabbit_control_helper:command(stop_app, D), + %% The command is appended to the log, but it will be dropped once the connectivity + %% is restored + ?assertMatch(ok, + rabbit_control_helper:command(join_cluster, D, [atom_to_list(A)], [])), + timer:sleep(10000), + join_3_node_cluster(Config), + clustering_utils:assert_cluster_status({Cluster, Cluster}, Cluster). + +remove_node(Config) -> + [A, B, C | _] = rabbit_ct_broker_helpers:get_node_configs( + Config, nodename), + + %% Three node cluster: A, B, C + ok = rabbit_control_helper:command(stop_app, B), + ok = rabbit_control_helper:command(join_cluster, B, [atom_to_list(A)], []), + rabbit_control_helper:command(start_app, B), + + ok = rabbit_control_helper:command(stop_app, C), + ok = rabbit_control_helper:command(join_cluster, C, [atom_to_list(A)], []), + rabbit_control_helper:command(start_app, C), + + %% Minority partition: A + partition_3_node_cluster(Config), + Cluster = [A, B, C], + + ok = rabbit_control_helper:command(forget_cluster_node, A, [atom_to_list(B)], []), + timer:sleep(10000), + join_3_node_cluster(Config), + clustering_utils:assert_cluster_status({Cluster, Cluster}, Cluster). + +enable_feature_flag(Config) -> + [A | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + ?assertMatch({error, missing_clustered_nodes}, rabbit_ct_broker_helpers:rpc(Config, A, rabbit_feature_flags, enable, [khepri_db])). + + +%% ------------------------------------------------------------------- +%% Internal helpers. +%% ------------------------------------------------------------------- + +partition_3_node_cluster(Config) -> + [A, B, C | _] = rabbit_ct_broker_helpers:get_node_configs( + Config, nodename), + Cluster = [A, B, C], + clustering_utils:assert_cluster_status({Cluster, Cluster}, Cluster), + NodePairs = [{A, B}, + {A, C}], + [rabbit_ct_broker_helpers:block_traffic_between(X, Y) || {X, Y} <- NodePairs], + %% Wait for the network partition to happen + clustering_utils:assert_cluster_status({Cluster, [B, C]}, [B, C]). + +partition_5_node_cluster(Config) -> + [A, B, C, D, E] = All = rabbit_ct_broker_helpers:get_node_configs( + Config, nodename), + %% Wait for the cluster to be ready + clustering_utils:assert_cluster_status({All, All}, All), + %% Minority partition A, B + NodePairs = [{A, C}, + {A, D}, + {A, E}, + {B, C}, + {B, D}, + {B, E}], + [rabbit_ct_broker_helpers:block_traffic_between(X, Y) || {X, Y} <- NodePairs], + %% Wait for the network partition to happen + clustering_utils:assert_cluster_status({All, [C, D, E]}, [C, D, E]). + +join_3_node_cluster(Config)-> + [A, B, C | _] = rabbit_ct_broker_helpers:get_node_configs( + Config, nodename), + %% Minority partition A + NodePairs = [{A, B}, + {A, C}], + [rabbit_ct_broker_helpers:allow_traffic_between(X, Y) || {X, Y} <- NodePairs]. diff --git a/deps/rabbit/test/cluster_rename_SUITE.erl b/deps/rabbit/test/cluster_rename_SUITE.erl index b9b3af7f79b4..d5c8d9786790 100644 --- a/deps/rabbit/test/cluster_rename_SUITE.erl +++ b/deps/rabbit/test/cluster_rename_SUITE.erl @@ -14,24 +14,29 @@ all() -> [ - {group, cluster_size_2}, - {group, cluster_size_3} + {group, mnesia_store}, + {group, khepri_store} ]. groups() -> [ - {cluster_size_2, [], [ - % XXX post_change_nodename, - abortive_rename, - rename_fail, - rename_twice_fail - ]}, - {cluster_size_3, [], [ - rename_cluster_one_by_one, - rename_cluster_big_bang, - partial_one_by_one, - partial_big_bang - ]} + {mnesia_store, [], [ + {cluster_size_2, [], [ + % XXX post_change_nodename, + abortive_rename, + rename_fail, + rename_twice_fail + ]}, + {cluster_size_3, [], [ + rename_cluster_one_by_one, + rename_cluster_big_bang, + partial_one_by_one, + partial_big_bang + ]} + ]}, + {khepri_store, [], [ + {cluster_size_2, [], [unsupported_command]} + ]} ]. suite() -> @@ -51,6 +56,10 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). +init_per_group(mnesia_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, mnesia}]); +init_per_group(khepri_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, khepri}]); init_per_group(cluster_size_2, Config) -> rabbit_ct_helpers:set_config(Config, [ {rmq_nodes_count, 2} %% Replaced with a list of node names later. @@ -229,6 +238,15 @@ rename_twice_fail(Config) -> ok = rename_node_fail(Config, Node1, [indecisive, jessica]), {save_config, Config1}. +unsupported_command(Config) -> + [Node1 | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Node1), + + Map = [Node1, jessica], + {error, _, Msg} = do_rename_node(Config, Node1, Map), + match = re:run(Msg, ".*not_supported.*", [{capture, none}]). + %% ---------------------------------------------------------------------------- stop_rename_start(Config, Nodename, Map) -> diff --git a/deps/rabbit/test/clustering_management_SUITE.erl b/deps/rabbit/test/clustering_management_SUITE.erl index 9f8de04c8a06..c1f47fadf390 100644 --- a/deps/rabbit/test/clustering_management_SUITE.erl +++ b/deps/rabbit/test/clustering_management_SUITE.erl @@ -10,10 +10,12 @@ -include_lib("common_test/include/ct.hrl"). -include_lib("amqp_client/include/amqp_client.hrl"). -include_lib("eunit/include/eunit.hrl"). +-include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl"). -compile(export_all). -import(clustering_utils, [ + assert_status/2, assert_cluster_status/2, assert_clustered/1, assert_not_clustered/1 @@ -21,43 +23,92 @@ all() -> [ - {group, unclustered_2_nodes}, - {group, unclustered_3_nodes}, - {group, clustered_2_nodes} + {group, mnesia_store}, + {group, khepri_store} ]. groups() -> - [ - {unclustered_2_nodes, [], [ - {cluster_size_2, [], [ - classic_config_discovery_node_list - ]} - ]}, - {unclustered_3_nodes, [], [ - {cluster_size_3, [], [ - join_and_part_cluster, - join_cluster_bad_operations, - join_to_start_interval, - forget_cluster_node, - change_cluster_node_type, - change_cluster_when_node_offline, - update_cluster_nodes, - force_reset_node - ]} - ]}, - {clustered_2_nodes, [], [ - {cluster_size_2, [], [ - forget_removes_things, - reset_removes_things, - forget_offline_removes_things, - force_boot, - status_with_alarm, - pid_file_and_await_node_startup, - await_running_count, - start_with_invalid_schema_in_path, - persistent_cluster_id - ]} - ]} + [{mnesia_store, [], [ + {unclustered_2_nodes, [], + [ + {cluster_size_2, [], [ + classic_config_discovery_node_list + ]} + ]}, + {unclustered_3_nodes, [], + [ + {cluster_size_3, [], [ + join_and_part_cluster, + join_cluster_bad_operations, + join_to_start_interval, + forget_cluster_node, + change_cluster_node_type, + change_cluster_when_node_offline, + update_cluster_nodes, + force_reset_node + ]} + ]}, + {clustered_2_nodes, [], + [ + {cluster_size_2, [], [ + forget_removes_things, + reset_removes_things, + forget_offline_removes_things, + forget_unavailable_node, + force_boot, + status_with_alarm, + pid_file_and_await_node_startup, + await_running_count, + start_with_invalid_schema_in_path, + persistent_cluster_id, + reset_last_disc_node + ]} + ]} + ]}, + {khepri_store, [], [ + {clustered_2_nodes, [], + [ + {cluster_size_2, [], [ + change_cluster_node_type_in_khepri, + forget_node_in_khepri, + forget_removes_things_in_khepri, + reset_in_khepri, + reset_removes_things_in_khepri, + reset_in_minority, + force_boot_in_khepri, + status_with_alarm, + pid_file_and_await_node_startup_in_khepri, + await_running_count_in_khepri, + start_with_invalid_schema_in_path, + persistent_cluster_id, + stop_start_cluster_node, + restart_cluster_node, + unsupported_forget_cluster_node_offline, + unsupported_update_cluster_nodes + + ]} + ]}, + {clustered_3_nodes, [], + [{cluster_size_3, [], [ + forget_unavailable_node, + forget_unavailable_node_in_minority + ]}]}, + {unclustered_3_nodes, [], + [ + {cluster_size_3, [], [ + join_and_part_cluster_in_khepri, + join_cluster_bad_operations_in_khepri, + join_cluster_in_minority, + join_cluster_with_rabbit_stopped, + force_reset_node_in_khepri, + join_to_start_interval, + forget_cluster_node_in_khepri, + start_nodes_in_reverse_order, + start_nodes_in_stop_order_in_khepri, + start_nodes_in_stop_order_with_force_boot + ]} + ]} + ]} ]. suite() -> @@ -82,12 +133,18 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). +init_per_group(mnesia_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, mnesia}]); +init_per_group(khepri_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, khepri}]); init_per_group(unclustered_2_nodes, Config) -> rabbit_ct_helpers:set_config(Config, [{rmq_nodes_clustered, false}]); init_per_group(unclustered_3_nodes, Config) -> rabbit_ct_helpers:set_config(Config, [{rmq_nodes_clustered, false}]); init_per_group(clustered_2_nodes, Config) -> rabbit_ct_helpers:set_config(Config, [{rmq_nodes_clustered, true}]); +init_per_group(clustered_3_nodes, Config) -> + rabbit_ct_helpers:set_config(Config, [{rmq_nodes_clustered, true}]); init_per_group(clustered_4_nodes, Config) -> rabbit_ct_helpers:set_config(Config, [{rmq_nodes_clustered, true}]); init_per_group(cluster_size_2, Config) -> @@ -100,7 +157,17 @@ init_per_group(cluster_size_4, Config) -> end_per_group(_, Config) -> Config. +init_per_testcase(create_bad_schema = Testcase, Config) -> + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + mnesia -> + init_per_testcase0(Testcase, Config); + _ -> + {skip, "Mnesia operations not supported by Khepri"} + end; init_per_testcase(Testcase, Config) -> + init_per_testcase0(Testcase, Config). + +init_per_testcase0(Testcase, Config) -> rabbit_ct_helpers:testcase_started(Config, Testcase), ClusterSize = ?config(rmq_nodes_count, Config), TestNumber = rabbit_ct_helpers:testcase_number(Config, ?MODULE, Testcase), @@ -130,7 +197,7 @@ start_with_invalid_schema_in_path(Config) -> create_bad_schema(Rabbit, Hare, Config), - start_app(Config, Hare), + spawn(fun() -> start_app(Config, Hare) end), case start_app(Config, Rabbit) of ok -> ok; ErrRabbit -> error({unable_to_start_with_bad_schema_in_work_dir, ErrRabbit}) @@ -204,6 +271,63 @@ join_and_part_cluster(Config) -> assert_not_clustered(Hare), assert_not_clustered(Bunny). +stop_start_cluster_node(Config) -> + [Rabbit, Hare] = cluster_members(Config), + + assert_clustered([Rabbit, Hare]), + + ok = stop_app(Config, Rabbit), + ok = start_app(Config, Rabbit), + + assert_clustered([Rabbit, Hare]), + + ok = stop_app(Config, Hare), + ok = start_app(Config, Hare), + + assert_clustered([Rabbit, Hare]). + +restart_cluster_node(Config) -> + [Rabbit, Hare] = cluster_members(Config), + + assert_clustered([Rabbit, Hare]), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Hare), + ok = rabbit_ct_broker_helpers:start_node(Config, Hare), + + assert_clustered([Rabbit, Hare]), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Rabbit), + ok = rabbit_ct_broker_helpers:start_node(Config, Rabbit), + + assert_clustered([Rabbit, Hare]). + +join_and_part_cluster_in_khepri(Config) -> + [Rabbit, Hare, Bunny] = cluster_members(Config), + assert_not_clustered(Rabbit), + assert_not_clustered(Hare), + assert_not_clustered(Bunny), + + stop_join_start(Config, Rabbit, Bunny), + assert_clustered([Rabbit, Bunny]), + + stop_join_start(Config, Hare, Bunny), + assert_clustered([Rabbit, Bunny, Hare]), + + %% Allow clustering with already clustered node + ok = stop_app(Config, Rabbit), + ?assertEqual(ok, join_cluster(Config, Rabbit, Hare)), + ok = start_app(Config, Rabbit), + + assert_clustered([Rabbit, Bunny, Hare]), + + stop_reset_start(Config, Bunny), + assert_not_clustered(Bunny), + assert_clustered([Hare, Rabbit]), + + stop_reset_start(Config, Rabbit), + assert_not_clustered(Rabbit), + assert_not_clustered(Hare). + join_cluster_bad_operations(Config) -> [Rabbit, Hare, Bunny] = cluster_members(Config), @@ -254,6 +378,27 @@ join_cluster_bad_operations(Config) -> end, ok. +join_cluster_bad_operations_in_khepri(Config) -> + [Rabbit, _Hare, Bunny] = cluster_members(Config), + + %% Nonexistent node + ok = stop_app(Config, Rabbit), + assert_failure(fun () -> join_cluster(Config, Rabbit, non@existent) end), + ok = start_app(Config, Rabbit), + assert_not_clustered(Rabbit), + + %% Trying to cluster with mnesia running + assert_failure(fun () -> join_cluster(Config, Rabbit, Bunny) end), + assert_not_clustered(Rabbit), + + %% Trying to cluster the node with itself + ok = stop_app(Config, Rabbit), + assert_failure(fun () -> join_cluster(Config, Rabbit, Rabbit) end), + ok = start_app(Config, Rabbit), + assert_not_clustered(Rabbit), + + ok. + %% This tests that the nodes in the cluster are notified immediately of a node %% join, and not just after the app is started. join_to_start_interval(Config) -> @@ -266,6 +411,44 @@ join_to_start_interval(Config) -> ok = start_app(Config, Rabbit), assert_clustered([Rabbit, Hare]). +join_cluster_in_minority(Config) -> + [Rabbit, Hare, Bunny] = cluster_members(Config), + assert_not_clustered(Rabbit), + assert_not_clustered(Hare), + assert_not_clustered(Bunny), + + stop_join_start(Config, Bunny, Rabbit), + assert_clustered([Rabbit, Bunny]), + ok = rabbit_ct_broker_helpers:stop_node(Config, Bunny), + + ok = stop_app(Config, Hare), + ?assertEqual(ok, join_cluster(Config, Hare, Rabbit, false)), + + ok = rabbit_ct_broker_helpers:start_node(Config, Bunny), + ?assertEqual(ok, join_cluster(Config, Hare, Rabbit, false)), + ?assertEqual(ok, start_app(Config, Hare)), + + assert_clustered([Rabbit, Bunny, Hare]). + +join_cluster_with_rabbit_stopped(Config) -> + [Rabbit, Hare, Bunny] = cluster_members(Config), + assert_not_clustered(Rabbit), + assert_not_clustered(Hare), + assert_not_clustered(Bunny), + + stop_join_start(Config, Bunny, Rabbit), + assert_clustered([Rabbit, Bunny]), + ok = stop_app(Config, Bunny), + + ok = stop_app(Config, Hare), + ?assertEqual(ok, join_cluster(Config, Hare, Rabbit, false)), + + ok = start_app(Config, Bunny), + ?assertEqual(ok, join_cluster(Config, Hare, Rabbit, false)), + ?assertEqual(ok, start_app(Config, Hare)), + + assert_clustered([Rabbit, Bunny, Hare]). + forget_cluster_node(Config) -> [Rabbit, Hare, Bunny] = cluster_members(Config), @@ -319,6 +502,36 @@ forget_cluster_node(Config) -> assert_not_clustered(Bunny), assert_clustered([Rabbit, Hare]). +forget_cluster_node_in_khepri(Config) -> + [Rabbit, Hare, _Bunny] = cluster_members(Config), + + %% Trying to remove a node not in the cluster should fail + assert_failure(fun () -> forget_cluster_node(Config, Hare, Rabbit) end), + + stop_join_start(Config, Rabbit, Hare), + assert_clustered([Rabbit, Hare]), + + %% Trying to remove an online node should fail + assert_failure(fun () -> forget_cluster_node(Config, Hare, Rabbit) end), + + ok = stop_app(Config, Rabbit), + %% Removing some nonexistent node will fail + assert_failure(fun () -> forget_cluster_node(Config, Hare, non@existent) end), + ok = forget_cluster_node(Config, Hare, Rabbit), + assert_not_clustered(Hare), + + ok = start_app(Config, Rabbit), + assert_not_clustered(Rabbit), + assert_not_clustered(Hare). + +unsupported_forget_cluster_node_offline(Config) -> + [Rabbit, Hare] = cluster_members(Config), + ok = rabbit_ct_broker_helpers:stop_node(Config, Hare), + ok = stop_app(Config, Rabbit), + Ret0 = rabbit_ct_broker_helpers:rabbitmqctl(Config, Hare, + ["forget_cluster_node", "--offline", Rabbit]), + is_not_supported(Ret0). + forget_removes_things(Config) -> test_removes_things(Config, fun (R, H) -> ok = forget_cluster_node(Config, H, R) end). @@ -327,7 +540,7 @@ reset_removes_things(Config) -> test_removes_things(Config, LoseRabbit) -> Unmirrored = <<"unmirrored-queue">>, - [Rabbit, Hare] = cluster_members(Config), + [Rabbit, Hare | _] = cluster_members(Config), RCh = rabbit_ct_client_helpers:open_channel(Config, Rabbit), declare(RCh, Unmirrored), ok = stop_app(Config, Rabbit), @@ -341,6 +554,186 @@ test_removes_things(Config, LoseRabbit) -> declare(HCh2, Unmirrored), ok. +forget_node_in_khepri(Config) -> + [Rabbit, Hare] = cluster_members(Config), + + assert_cluster_status({[Rabbit, Hare], [Rabbit, Hare], [Rabbit, Hare]}, + [Rabbit, Hare]), + + ok = stop_app(Config, Rabbit), + ok = forget_cluster_node(Config, Hare, Rabbit), + + assert_cluster_status({[Hare], [Hare]}, [Hare]), + + ok. + +forget_removes_things_in_khepri(Config) -> + ClassicQueue = <<"classic-queue">>, + [Rabbit, Hare | _] = cluster_members(Config), + + RCh = rabbit_ct_client_helpers:open_channel(Config, Rabbit), + ?assertMatch(#'queue.declare_ok'{}, declare(RCh, ClassicQueue)), + + ok = stop_app(Config, Rabbit), + ok = forget_cluster_node(Config, Hare, Rabbit), + + HCh = rabbit_ct_client_helpers:open_channel(Config, Hare), + ?assertExit( + {{shutdown, {server_initiated_close, 404, _}}, _}, + declare_passive(HCh, ClassicQueue)), + + ok. + +forget_unavailable_node(Config) -> + [Rabbit, Hare | _] = Nodes = cluster_members(Config), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Rabbit), + ?assertMatch(ok, forget_cluster_node(Config, Hare, Rabbit)), + + NNodes = lists:nthtail(1, Nodes), + + assert_cluster_status({NNodes, NNodes}, NNodes). + +forget_unavailable_node_in_minority(Config) -> + All = [Rabbit, Hare, Bunny] = cluster_members(Config), + + assert_cluster_status({All, All}, All), + + %% Find out the raft status of the soon to be only + %% running node + RaftStatus = get_raft_status(Config, Hare), + + %% Stop other two nodes + ok = rabbit_ct_broker_helpers:stop_node(Config, Rabbit), + ok = rabbit_ct_broker_helpers:stop_node(Config, Bunny), + %% Wait until Mnesia has detected both nodes down + ?awaitMatch( + [Hare], + rabbit_ct_broker_helpers:rpc(Config, Hare, + rabbit_mnesia, cluster_nodes, [running]), + 30000), + + %% If Hare was the leader, it is able to forget one of the nodes. Change takes place as soon as it is written on the log. The other membership change will be rejected until the last change has consensus. + case RaftStatus of + leader -> + ?assertMatch(ok, forget_cluster_node(Config, Hare, Rabbit)), + not_permitted(forget_cluster_node(Config, Hare, Bunny)); + follower -> + %% Follower might have been promoted before the second node goes down, check the status again + case get_raft_status(Config, Hare) of + leader -> + ?assertMatch(ok, forget_cluster_node(Config, Hare, Rabbit)), + not_permitted(forget_cluster_node(Config, Hare, Bunny)); + _ -> + is_in_minority(forget_cluster_node(Config, Hare, Rabbit)) + end + end. + +not_permitted(Ret) -> + ?assertMatch({error, 69, _}, Ret), + {error, _, Msg} = Ret, + ?assertMatch(match, re:run(Msg, ".*not_permitted.*", [{capture, none}])). + +get_raft_status(Config, Node) -> + AllStatus = rabbit_ct_broker_helpers:rpc(Config, Node, rabbit_khepri, status, []), + case lists:filter(fun(S) -> + proplists:get_value(<<"Node Name">>, S) == Node + end, AllStatus) of + [NodeStatus] -> + proplists:get_value(<<"Raft State">>, NodeStatus); + [] -> + unknown + end. + +reset_in_khepri(Config) -> + ClassicQueue = <<"classic-queue">>, + [Rabbit, Hare | _] = cluster_members(Config), + + RCh = rabbit_ct_client_helpers:open_channel(Config, Rabbit), + ?assertMatch(#'queue.declare_ok'{}, declare(RCh, ClassicQueue)), + + stop_app(Config, Hare), + ok = reset(Config, Hare), + + %% Rabbit is a 1-node cluster. The classic queue is still there. + assert_cluster_status({[Rabbit], [Rabbit]}, [Rabbit]), + ?assertMatch(#'queue.declare_ok'{}, declare_passive(RCh, ClassicQueue)), + + %% Can't reset a running node + ?assertMatch({error, 64, _}, reset(Config, Rabbit)), + + %% Start Hare, it should work as standalone node. + start_app(Config, Hare), + + assert_cluster_status({[Hare], [Hare]}, [Hare]), + + ok. + +reset_removes_things_in_khepri(Config) -> + ClassicQueue = <<"classic-queue">>, + [Rabbit, Hare | _] = cluster_members(Config), + + RCh = rabbit_ct_client_helpers:open_channel(Config, Rabbit), + ?assertMatch(#'queue.declare_ok'{}, declare(RCh, ClassicQueue)), + + stop_app(Config, Rabbit), + ok = reset(Config, Rabbit), + + assert_cluster_status({[Hare], [Hare]}, [Hare]), + + start_app(Config, Rabbit), + assert_cluster_status({[Rabbit], [Rabbit]}, [Rabbit]), + + %% The classic queue was declared in Rabbit, once that node is reset + %% the queue needs to be removed from the rest of the cluster + HCh = rabbit_ct_client_helpers:open_channel(Config, Hare), + ?assertExit( + {{shutdown, {server_initiated_close, 404, _}}, _}, + declare_passive(HCh, ClassicQueue)), + + ok. + +reset_in_minority(Config) -> + [Rabbit, Hare | _] = cluster_members(Config), + + rabbit_ct_broker_helpers:stop_node(Config, Hare), + + ok = rpc:call(Rabbit, application, set_env, + [rabbit, khepri_leader_wait_retry_timeout, 1000]), + ok = rpc:call(Rabbit, application, set_env, + [rabbit, khepri_leader_wait_retry_limit, 3]), + stop_app(Config, Rabbit), + + is_in_minority(reset(Config, Rabbit)), + + ok. + +is_in_minority(Ret) -> + ?assertMatch({error, 75, _}, Ret), + {error, _, Msg} = Ret, + ?assertMatch(match, re:run(Msg, ".*timed out.*minority.*", [{capture, none}])). + +reset_last_disc_node(Config) -> + Servers = [Rabbit, Hare | _] = cluster_members(Config), + + stop_app(Config, Hare), + ?assertEqual(ok, change_cluster_node_type(Config, Hare, ram)), + start_app(Config, Hare), + + case rabbit_ct_broker_helpers:enable_feature_flag(Config, Servers, khepri_db) of + ok -> + %% The reset works after the switch to Khepri because the RAM node was + %% implicitly converted to a disc one as Khepri always writes data on disc. + stop_app(Config, Rabbit), + ?assertEqual(ok, reset(Config, Rabbit)), + start_app(Config, Rabbit), + assert_not_clustered(Rabbit), + assert_not_clustered(Hare), + ok; + {skip, _} = Skip -> + Skip + end. + forget_offline_removes_things(Config) -> [Rabbit, Hare] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -430,6 +823,14 @@ force_boot(Config) -> ok = rabbit_ct_broker_helpers:start_node(Config, Rabbit), ok. +force_boot_in_khepri(Config) -> + [Rabbit, _Hare] = rabbit_ct_broker_helpers:get_node_configs(Config, + nodename), + stop_app(Config, Rabbit), + %% It executes force boot for mnesia, currently Khepri does nothing + ?assertMatch({ok, []}, rabbit_ct_broker_helpers:rabbitmqctl(Config, Rabbit, ["force_boot"])), + ok. + change_cluster_node_type(Config) -> [Rabbit, Hare, _Bunny] = cluster_members(Config), @@ -443,7 +844,7 @@ change_cluster_node_type(Config) -> assert_cluster_status({[Rabbit, Hare], [Rabbit, Hare], [Hare]}, [Rabbit, Hare]), change_cluster_node_type(Config, Rabbit, ram), - assert_cluster_status({[Rabbit, Hare], [Hare], [Hare]}, + assert_cluster_status({[Rabbit, Hare], [Hare], [Rabbit, Hare], [Hare], [Hare]}, [Rabbit, Hare]), change_cluster_node_type(Config, Rabbit, disc), @@ -459,6 +860,25 @@ change_cluster_node_type(Config) -> assert_failure(fun () -> change_cluster_node_type(Config, Hare, ram) end), ok = start_app(Config, Hare). +change_cluster_node_type_in_khepri(Config) -> + [Rabbit, Hare] = cluster_members(Config), + + assert_cluster_status({[Rabbit, Hare], [Rabbit, Hare], [Rabbit, Hare]}, + [Rabbit, Hare]), + + ok = stop_app(Config, Rabbit), + {error, 69, Msg} = change_cluster_node_type(Config, Rabbit, ram), + ?assertEqual( + match, + re:run( + Msg, "Feature `ram_node_type` is deprecated", + [{capture, none}])), + + ok = change_cluster_node_type(Config, Rabbit, disc), + ok = start_app(Config, Rabbit), + assert_cluster_status({[Rabbit, Hare], [Rabbit, Hare], [Rabbit, Hare]}, + [Rabbit, Hare]). + change_cluster_when_node_offline(Config) -> [Rabbit, Hare, Bunny] = cluster_members(Config), @@ -477,8 +897,7 @@ change_cluster_when_node_offline(Config) -> assert_cluster_status({[Bunny], [Bunny], []}, [Bunny]), assert_cluster_status({[Rabbit, Hare], [Rabbit, Hare], [Hare]}, [Hare]), assert_cluster_status( - {[Rabbit, Hare, Bunny], [Hare], [Rabbit, Hare, Bunny], - [Rabbit, Hare, Bunny], [Hare, Bunny]}, [Rabbit]), + {[Rabbit, Hare, Bunny], [Hare], [Rabbit, Hare, Bunny], [Rabbit, Hare, Bunny], [Hare, Bunny]}, [Rabbit]), %% Bring Rabbit back up ok = start_app(Config, Rabbit), @@ -531,6 +950,21 @@ update_cluster_nodes(Config) -> assert_not_clustered(Hare), assert_clustered([Rabbit, Bunny]). +unsupported_update_cluster_nodes(Config) -> + [Rabbit, Hare] = cluster_members(Config), + + %% Mnesia is running... + assert_failure(fun () -> update_cluster_nodes(Config, Rabbit, Hare) end), + + ok = stop_app(Config, Rabbit), + Ret = update_cluster_nodes(Config, Rabbit, Hare), + is_not_supported(Ret). + +is_not_supported(Ret) -> + ?assertMatch({error, _, _}, Ret), + {error, _, Msg} = Ret, + ?assertMatch(match, re:run(Msg, ".*not_supported.*", [{capture, none}])). + classic_config_discovery_node_list(Config) -> [Rabbit, Hare] = cluster_members(Config), @@ -581,6 +1015,28 @@ force_reset_node(Config) -> start_app(Config, Rabbit), assert_clustered([Rabbit, Hare]). +force_reset_node_in_khepri(Config) -> + [Rabbit, Hare, _Bunny] = cluster_members(Config), + + stop_join_start(Config, Rabbit, Hare), + stop_app(Config, Rabbit), + ok = force_reset(Config, Rabbit), + assert_cluster_status({[Rabbit, Hare], [Rabbit, Hare], [Hare]}, [Hare]), + %% Khepri is stopped, so it won't report anything. + assert_status({[Rabbit], [], [Rabbit], [Rabbit], []}, [Rabbit]), + %% Hare thinks that Rabbit is still clustered + assert_cluster_status({[Rabbit, Hare], [Rabbit, Hare], [Hare]}, + [Hare]), + ok = start_app(Config, Rabbit), + assert_not_clustered(Rabbit), + %% We can rejoin Rabbit and Hare. Unlike with Mnesia, we try to solve the + %% inconsistency instead of returning an error. + ok = stop_app(Config, Rabbit), + ?assertEqual(ok, join_cluster(Config, Rabbit, Hare, false)), + ok = start_app(Config, Rabbit), + assert_cluster_status({[Rabbit, Hare], [Rabbit, Hare], [Rabbit, Hare]}, + [Rabbit, Hare]). + status_with_alarm(Config) -> [Rabbit, Hare] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -639,6 +1095,34 @@ pid_file_and_await_node_startup(Config) -> {error, _, _} = rabbit_ct_broker_helpers:rabbitmqctl(Config, Rabbit, ["wait", RabbitPidFile]). +pid_file_and_await_node_startup_in_khepri(Config) -> + [Rabbit, Hare] = rabbit_ct_broker_helpers:get_node_configs(Config, + nodename), + RabbitConfig = rabbit_ct_broker_helpers:get_node_config(Config,Rabbit), + RabbitPidFile = ?config(pid_file, RabbitConfig), + %% ensure pid file is readable + {ok, _} = file:read_file(RabbitPidFile), + %% ensure wait works on running node + {ok, _} = rabbit_ct_broker_helpers:rabbitmqctl(Config, Rabbit, + ["wait", RabbitPidFile]), + %% stop both nodes + ok = rabbit_ct_broker_helpers:stop_node(Config, Rabbit), + ok = rabbit_ct_broker_helpers:stop_node(Config, Hare), + %% start first node in the background. It will wait for Khepri + %% and then Mnesia tables (which will already be available) + spawn_link(fun() -> + rabbit_ct_broker_helpers:start_node(Config, Rabbit) + end), + PreviousPid = pid_from_file(RabbitPidFile), + Attempts = 200, + Timeout = 50, + wait_for_pid_file_to_change(RabbitPidFile, PreviousPid, Attempts, Timeout), + %% The node is blocked waiting for Khepri, so this will timeout. Mnesia + %% alone would fail here as it wasn't the last node to stop + %% Let's make it a short wait. + {error, timeout, _} = rabbit_ct_broker_helpers:rabbitmqctl(Config, Rabbit, + ["wait", RabbitPidFile], 10000). + await_running_count(Config) -> [Rabbit, Hare] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -679,6 +1163,135 @@ await_running_count(Config) -> rabbit_nodes, await_running_count, [5, 1000])). +await_running_count_in_khepri(Config) -> + [Rabbit, Hare] = rabbit_ct_broker_helpers:get_node_configs(Config, + nodename), + RabbitConfig = rabbit_ct_broker_helpers:get_node_config(Config,Rabbit), + RabbitPidFile = ?config(pid_file, RabbitConfig), + {ok, _} = rabbit_ct_broker_helpers:rabbitmqctl(Config, Rabbit, + ["wait", RabbitPidFile]), + %% stop both nodes + ok = rabbit_ct_broker_helpers:stop_node(Config, Hare), + ok = rabbit_ct_broker_helpers:stop_node(Config, Rabbit), + %% start one node in the background + %% One khepri node in minority won't finish starting up, but will wait a reasonable + %% amount of time for a new leader to be elected. Hopefully on that time + %% a second (or more) node is brought up so they can reach consensus + %% Kind of similar to the wait for tables that we had on mnesia + rabbit_ct_broker_helpers:async_start_node(Config, Rabbit), + rabbit_ct_broker_helpers:start_node(Config, Hare), + {ok, _} = rabbit_ct_broker_helpers:rabbitmqctl(Config, Rabbit, + ["wait", RabbitPidFile]), + %% this now succeeds + ?assertEqual(ok, rabbit_ct_broker_helpers:rpc(Config, Rabbit, + rabbit_nodes, + await_running_count, [2, 30000])), + %% this still succeeds + ?assertEqual(ok, rabbit_ct_broker_helpers:rpc(Config, Rabbit, + rabbit_nodes, + await_running_count, [1, 30000])), + %% this still fails + ?assertEqual({error, timeout}, + rabbit_ct_broker_helpers:rpc(Config, Rabbit, + rabbit_nodes, + await_running_count, [5, 1000])). + +start_nodes_in_reverse_order(Config) -> + [Rabbit, Hare, Bunny] = cluster_members(Config), + assert_not_clustered(Rabbit), + assert_not_clustered(Hare), + assert_not_clustered(Bunny), + + stop_join_start(Config, Rabbit, Bunny), + stop_join_start(Config, Hare, Bunny), + assert_clustered([Rabbit, Hare, Bunny]), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Rabbit), + ok = rabbit_ct_broker_helpers:stop_node(Config, Hare), + ok = rabbit_ct_broker_helpers:stop_node(Config, Bunny), + + spawn(fun() -> ok = rabbit_ct_broker_helpers:start_node(Config, Bunny) end), + ok = rabbit_ct_broker_helpers:start_node(Config, Hare), + assert_cluster_status({[Bunny, Hare, Rabbit], [Bunny, Hare, Rabbit], [Bunny, Hare]}, + [Bunny, Hare]), + + ok = rabbit_ct_broker_helpers:start_node(Config, Rabbit), + assert_clustered([Rabbit, Hare, Bunny]). + +%% Test booting nodes in the wrong order for Mnesia. Interesting... +start_nodes_in_stop_order(Config) -> + [Rabbit, Hare, Bunny] = cluster_members(Config), + assert_not_clustered(Rabbit), + assert_not_clustered(Hare), + assert_not_clustered(Bunny), + + stop_join_start(Config, Rabbit, Bunny), + stop_join_start(Config, Hare, Bunny), + assert_clustered([Rabbit, Hare, Bunny]), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Rabbit), + ok = rabbit_ct_broker_helpers:stop_node(Config, Hare), + ok = rabbit_ct_broker_helpers:stop_node(Config, Bunny), + + Self = self(), + spawn(fun() -> + Reply = rabbit_ct_broker_helpers:start_node(Config, Rabbit), + Self ! {start_node_reply, Reply} + end), + ?assertMatch({error, {skip, _}}, rabbit_ct_broker_helpers:start_node(Config, Hare)), + receive + {start_node_reply, Reply} -> + ?assertMatch({error, {skip, _}}, Reply) + end. + +start_nodes_in_stop_order_in_khepri(Config) -> + [Rabbit, Hare, Bunny] = cluster_members(Config), + assert_not_clustered(Rabbit), + assert_not_clustered(Hare), + assert_not_clustered(Bunny), + + stop_join_start(Config, Rabbit, Bunny), + stop_join_start(Config, Hare, Bunny), + assert_clustered([Rabbit, Hare, Bunny]), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Rabbit), + ok = rabbit_ct_broker_helpers:stop_node(Config, Hare), + ok = rabbit_ct_broker_helpers:stop_node(Config, Bunny), + + ok = rabbit_ct_broker_helpers:async_start_node(Config, Rabbit), + ok = rabbit_ct_broker_helpers:async_start_node(Config, Hare), + ok = rabbit_ct_broker_helpers:async_start_node(Config, Bunny), + + ?assertMatch(ok, rabbit_ct_broker_helpers:wait_for_async_start_node(Rabbit)), + ?assertMatch(ok, rabbit_ct_broker_helpers:wait_for_async_start_node(Hare)), + ?assertMatch(ok, rabbit_ct_broker_helpers:wait_for_async_start_node(Bunny)). + +%% TODO test force_boot with Khepri involved +start_nodes_in_stop_order_with_force_boot(Config) -> + [Rabbit, Hare, Bunny] = cluster_members(Config), + assert_not_clustered(Rabbit), + assert_not_clustered(Hare), + assert_not_clustered(Bunny), + + stop_join_start(Config, Rabbit, Bunny), + stop_join_start(Config, Hare, Bunny), + assert_clustered([Rabbit, Hare, Bunny]), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Rabbit), + ok = rabbit_ct_broker_helpers:stop_node(Config, Hare), + ok = rabbit_ct_broker_helpers:stop_node(Config, Bunny), + + {ok, []} = rabbit_ct_broker_helpers:rabbitmqctl(Config, Rabbit, + ["force_boot"]), + + spawn(fun() -> rabbit_ct_broker_helpers:start_node(Config, Rabbit) end), + ok = rabbit_ct_broker_helpers:start_node(Config, Hare), + assert_cluster_status({[Bunny, Hare, Rabbit], [Bunny, Hare, Rabbit], [Rabbit, Hare]}, + [Rabbit, Hare]), + + ok = rabbit_ct_broker_helpers:start_node(Config, Bunny), + assert_clustered([Rabbit, Hare, Bunny]). + %% ---------------------------------------------------------------------------- %% Internal utils %% ---------------------------------------------------------------------------- @@ -814,3 +1427,8 @@ declare(Ch, Name) -> amqp_channel:call(Ch, #'queue.bind'{queue = Name, exchange = <<"amq.fanout">>}), Res. + +declare_passive(Ch, Name) -> + amqp_channel:call(Ch, #'queue.declare'{durable = true, + passive = true, + queue = Name}). diff --git a/deps/rabbit/test/clustering_recovery_SUITE.erl b/deps/rabbit/test/clustering_recovery_SUITE.erl index ca008d92b1e5..8311927ff596 100644 --- a/deps/rabbit/test/clustering_recovery_SUITE.erl +++ b/deps/rabbit/test/clustering_recovery_SUITE.erl @@ -13,9 +13,17 @@ -compile(export_all). +-import(clustering_utils, [ + assert_status/2, + assert_cluster_status/2, + assert_clustered/1, + assert_not_clustered/1 + ]). + all() -> [ - {group, mnesia_store} + {group, mnesia_store}, + {group, khepri_store} ]. groups() -> @@ -26,13 +34,29 @@ groups() -> force_shrink_all_quorum_queues ]} ]} + ]}, + {khepri_store, [], [ + {clustered_3_nodes, [], + [{cluster_size_3, [], [ + force_standalone_boot, + force_standalone_boot_and_restart, + force_standalone_boot_and_restart_with_quorum_queues + ]} + ]}, + {clustered_5_nodes, [], + [{cluster_size_5, [], [ + rolling_restart, + rolling_kill_restart, + forget_down_node + ]}] + } ]} ]. suite() -> [ %% If a testcase hangs, no need to wait for 30 minutes. - {timetrap, {minutes, 5}} + {timetrap, {minutes, 10}} ]. %% ------------------------------------------------------------------- @@ -46,12 +70,18 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). +init_per_group(khepri_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, khepri}]); init_per_group(mnesia_store, Config) -> - Config; + rabbit_ct_helpers:set_config(Config, [{metadata_store, mnesia}]); init_per_group(clustered_3_nodes, Config) -> rabbit_ct_helpers:set_config(Config, [{rmq_nodes_clustered, true}]); +init_per_group(clustered_5_nodes, Config) -> + rabbit_ct_helpers:set_config(Config, [{rmq_nodes_clustered, true}]); init_per_group(cluster_size_3, Config) -> - rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 3}]). + rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 3}]); +init_per_group(cluster_size_5, Config) -> + rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 5}]). end_per_group(_, Config) -> Config. @@ -82,9 +112,9 @@ end_per_testcase(Testcase, Config) -> force_shrink_all_quorum_queues(Config) -> [Rabbit, Hare, Bunny] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - QName1 = quorum_test_queue(1), - QName2 = quorum_test_queue(2), - QName3 = quorum_test_queue(3), + QName1 = quorum_queue_name(1), + QName2 = quorum_queue_name(2), + QName3 = quorum_queue_name(3), Args = [{<<"x-queue-type">>, longstr, <<"quorum">>}], declare_and_publish_to_queue(Config, Rabbit, QName1, Args), declare_and_publish_to_queue(Config, Rabbit, QName2, Args), @@ -111,7 +141,7 @@ force_shrink_all_quorum_queues(Config) -> force_shrink_quorum_queue(Config) -> [Rabbit, Hare, Bunny] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - QName1 = quorum_test_queue(1), + QName1 = quorum_queue_name(1), Args = [{<<"x-queue-type">>, longstr, <<"quorum">>}], declare_and_publish_to_queue(Config, Rabbit, QName1, Args), @@ -127,7 +157,150 @@ force_shrink_quorum_queue(Config) -> ok = rabbit_ct_broker_helpers:rpc(Config, Rabbit, rabbit_quorum_queue, force_shrink_member_to_current_member, [<<"/">>, QName1]), - ok = consume_from_queue(Config, Rabbit, QName1), + ok = consume_from_queue(Config, Rabbit, QName1). + +force_standalone_boot(Config) -> + %% Test for disaster recovery procedure command + [Rabbit, Hare, Bunny] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + assert_cluster_status({[Rabbit, Hare, Bunny], [Rabbit, Hare, Bunny], [Rabbit, Hare, Bunny]}, + [Rabbit, Hare, Bunny]), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Hare), + ok = rabbit_ct_broker_helpers:stop_node(Config, Bunny), + ok = force_standalone_khepri_boot(Rabbit), + + assert_cluster_status({[Rabbit], [Rabbit], [Rabbit], [Rabbit], [Rabbit]}, + [Rabbit]), + + ok. + +force_standalone_boot_and_restart(Config) -> + %% Test for disaster recovery procedure + %% + %% 3-node cluster. Declare and publish to a classic queue on node 1. + %% Stop the two remaining nodes. Force standalone boot on the node + %% left. Restart it. Consume all the messages. + [Rabbit, Hare, Bunny] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + assert_cluster_status({[Rabbit, Hare, Bunny], [Rabbit, Hare, Bunny], [Rabbit, Hare, Bunny]}, + [Rabbit, Hare, Bunny]), + + QName = classic_queue_name(Rabbit), + Args = [{<<"x-queue-type">>, longstr, <<"classic">>}], + declare_and_publish_to_queue(Config, Rabbit, QName, Args), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Hare), + ok = rabbit_ct_broker_helpers:stop_node(Config, Bunny), + ok = force_standalone_khepri_boot(Rabbit), + + assert_cluster_status({[Rabbit], [Rabbit], [Rabbit], [Rabbit], [Rabbit]}, + [Rabbit]), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Rabbit), + ok = rabbit_ct_broker_helpers:start_node(Config, Rabbit), + + consume_from_queue(Config, Rabbit, QName), + + ok. + +force_standalone_boot_and_restart_with_quorum_queues(Config) -> + %% Test for disaster recovery procedure + %% + %% 3-node cluster. Declare and publish to a classic queue on node 1. + %% Stop the two remaining nodes. Force standalone boot on the node + %% left. Restart it. Consume all the messages. + [Rabbit, Hare, Bunny] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + assert_cluster_status({[Rabbit, Hare, Bunny], [Rabbit, Hare, Bunny], [Rabbit, Hare, Bunny]}, + [Rabbit, Hare, Bunny]), + + QName1 = quorum_queue_name(1), + QName2 = quorum_queue_name(2), + Args = [{<<"x-queue-type">>, longstr, <<"quorum">>}], + declare_and_publish_to_queue(Config, Rabbit, QName1, Args), + declare_and_publish_to_queue(Config, Rabbit, QName2, Args), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Hare), + ok = rabbit_ct_broker_helpers:stop_node(Config, Bunny), + + ok = force_standalone_khepri_boot(Rabbit), + ok = rabbit_ct_broker_helpers:rpc(Config, Rabbit, rabbit_quorum_queue, force_all_queues_shrink_member_to_current_member, []), + + assert_cluster_status({[Rabbit], [Rabbit], [Rabbit], [Rabbit], [Rabbit]}, + [Rabbit]), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Rabbit), + ok = rabbit_ct_broker_helpers:start_node(Config, Rabbit), + + consume_from_queue(Config, Rabbit, QName1), + consume_from_queue(Config, Rabbit, QName2), + + ok. + +rolling_restart(Config) -> + Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Args = [{<<"x-queue-type">>, longstr, <<"classic">>}], + [begin + QName = classic_queue_name(N), + declare_and_publish_to_queue(Config, N, QName, Args) + end || N <- Nodes], + + [begin + ok = rabbit_ct_broker_helpers:stop_node(Config, N), + ok = rabbit_ct_broker_helpers:start_node(Config, N) + end || N <- Nodes], + + assert_cluster_status({Nodes, Nodes, Nodes}, Nodes), + [begin + QName = classic_queue_name(N), + consume_from_queue(Config, N, QName) + end || N <- Nodes], + + ok. + +rolling_kill_restart(Config) -> + Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Args = [{<<"x-queue-type">>, longstr, <<"classic">>}], + [begin + QName = classic_queue_name(N), + declare_and_publish_to_queue(Config, N, QName, Args) + end || N <- Nodes], + + Ret0 = + [begin + ok = rabbit_ct_broker_helpers:kill_node(Config, N), + {N, rabbit_ct_broker_helpers:start_node(Config, N)} + end || N <- Nodes], + Failed = [Pair || {_, V} = Pair <- Ret0, V =/= ok], + + ?assert(length(Failed) =< 1), + + case Failed of + [] -> + assert_cluster_status({Nodes, Nodes, Nodes}, Nodes), + [begin + QName = classic_queue_name(N), + consume_from_queue(Config, N, QName) + end || N <- Nodes]; + [{FailedNode, {error, _}}] -> + [Node0 | _] = RemainingNodes = Nodes -- [FailedNode], + ok = forget_cluster_node(Node0, FailedNode), + assert_cluster_status({RemainingNodes, RemainingNodes, RemainingNodes}, RemainingNodes) + end, + ok. + +forget_down_node(Config) -> + [Rabbit, Hare | _] = Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + ok = rabbit_ct_broker_helpers:stop_node(Config, Rabbit), + ok = forget_cluster_node(Hare, Rabbit), + + NNodes = lists:nthtail(1, Nodes), + + assert_cluster_status({NNodes, NNodes, NNodes}, NNodes), ok. @@ -140,16 +313,16 @@ declare_and_publish_to_queue(Config, Node, QName, Args) -> publish_many(Ch, QName, 10), rabbit_ct_client_helpers:close_connection_and_channel(Conn, Ch). -quorum_test_queue(Number) -> +quorum_queue_name(Number) -> list_to_binary(io_lib:format("quorum_queue_~p", [Number])). +classic_queue_name(Node) -> + list_to_binary(io_lib:format("classic_queue_~p", [Node])). + declare(Ch, Name, Args) -> - Res = amqp_channel:call(Ch, #'queue.declare'{durable = true, - queue = Name, - arguments = Args}), - amqp_channel:call(Ch, #'queue.bind'{queue = Name, - exchange = <<"amq.fanout">>}), - Res. + amqp_channel:call(Ch, #'queue.declare'{durable = true, + queue = Name, + arguments = Args}). consume_from_queue(Config, Node, QName) -> {Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, Node), @@ -157,6 +330,12 @@ consume_from_queue(Config, Node, QName) -> consume(10), rabbit_ct_client_helpers:close_connection_and_channel(Conn, Ch). +force_standalone_khepri_boot(Node) -> + rabbit_control_helper:command(force_standalone_khepri_boot, Node, []). + +forget_cluster_node(Node, Removee) -> + rabbit_control_helper:command(forget_cluster_node, Node, [atom_to_list(Removee)], []). + publish_many(Ch, QName, N) -> amqp_channel:call(Ch, #'confirm.select'{}), [amqp_channel:cast(Ch, #'basic.publish'{routing_key = QName}, diff --git a/deps/rabbit/test/clustering_utils.erl b/deps/rabbit/test/clustering_utils.erl index 5ff479dd5b60..39c81ddc9bcf 100644 --- a/deps/rabbit/test/clustering_utils.erl +++ b/deps/rabbit/test/clustering_utils.erl @@ -7,6 +7,7 @@ -module(clustering_utils). -export([ + assert_status/2, assert_cluster_status/2, assert_clustered/1, assert_not_clustered/1 @@ -14,9 +15,17 @@ -define(LOOP_RECURSION_DELAY, 100). -assert_cluster_status({All, Disc, Running}, Nodes) -> - assert_cluster_status({All, Running, All, Disc, Running}, Nodes); -assert_cluster_status(Status0, Nodes) -> +assert_status(Tuple, Nodes) -> + assert_cluster_status(Tuple, Nodes, fun verify_status_equal/3). + +assert_cluster_status(Tuple, Nodes) -> + assert_cluster_status(Tuple, Nodes, fun verify_cluster_status_equal/3). + +assert_cluster_status({All, Running}, Nodes, VerifyFun) -> + assert_cluster_status({All, Running, All, All, Running}, Nodes, VerifyFun); +assert_cluster_status({All, Disc, Running}, Nodes, VerifyFun) -> + assert_cluster_status({All, Running, All, Disc, Running}, Nodes, VerifyFun); +assert_cluster_status(Status0, Nodes, VerifyFun) -> Status = sort_cluster_status(Status0), AllNodes = case Status of {undef, undef, All, _, _} -> @@ -25,36 +34,41 @@ assert_cluster_status(Status0, Nodes) -> {All, _, _, _, _} -> All end, - wait_for_cluster_status(Status, AllNodes, Nodes). + wait_for_cluster_status(Status, AllNodes, Nodes, VerifyFun). -wait_for_cluster_status(Status, AllNodes, Nodes) -> +wait_for_cluster_status(Status, AllNodes, Nodes, VerifyFun) -> Max = 10000 / ?LOOP_RECURSION_DELAY, - wait_for_cluster_status(0, Max, Status, AllNodes, Nodes). + wait_for_cluster_status(0, Max, Status, AllNodes, Nodes, VerifyFun). -wait_for_cluster_status(N, Max, Status, _AllNodes, Nodes) when N >= Max -> +wait_for_cluster_status(N, Max, Status, _AllNodes, Nodes, _VerifyFun) when N >= Max -> erlang:error({cluster_status_max_tries_failed, [{nodes, Nodes}, {expected_status, Status}, {max_tried, Max}, - {status, sort_cluster_status(cluster_status(hd(Nodes)))}]}); -wait_for_cluster_status(N, Max, Status, AllNodes, Nodes) -> + {status, [{Node, sort_cluster_status(cluster_status(Node))} || Node <- Nodes]}]}); +wait_for_cluster_status(N, Max, Status, AllNodes, Nodes, VerifyFun) -> case lists:all(fun (Node) -> - verify_status_equal(Node, Status, AllNodes) + VerifyFun(Node, Status, AllNodes) end, Nodes) of true -> ok; false -> timer:sleep(?LOOP_RECURSION_DELAY), - wait_for_cluster_status(N + 1, Max, Status, AllNodes, Nodes) + wait_for_cluster_status(N + 1, Max, Status, AllNodes, Nodes, VerifyFun) end. -verify_status_equal(Node, Status, AllNodes) -> +verify_status_equal(Node, Status, _AllNodes) -> NodeStatus = sort_cluster_status(cluster_status(Node)), - IsClustered = case rpc:call(Node, rabbit_db_cluster, is_clustered, []) of - {badrpc, {'EXIT', {undef, _}}} -> - rpc:call(Node, rabbit_mnesia, is_clustered, []); - Ret -> - Ret + equal(Status, NodeStatus). + +verify_cluster_status_equal(Node, Status, AllNodes) -> + NodeStatus = sort_cluster_status(cluster_status(Node)), + %% To be compatible with mixed version clusters in 3.11.x we use here + %% rabbit_mnesia:is_clustered/0 instead of rabbit_db_cluster:is_clustered/0 + IsClustered0 = rpc:call(Node, rabbit_db_cluster, is_clustered, []), + IsClustered = case maybe_undef(IsClustered0) of + undef -> rpc:call(Node, rabbit_mnesia, is_clustered, []); + _ -> IsClustered0 end, - (AllNodes =/= [Node]) =:= IsClustered andalso equal(Status, NodeStatus). + ((AllNodes =/= [Node]) =:= IsClustered andalso equal(Status, NodeStatus)). equal({_, _, A, B, C}, {undef, undef, A, B, C}) -> true; @@ -64,17 +78,36 @@ equal(Status0, Status1) -> Status0 == Status1. cluster_status(Node) -> - AllMembers = rpc:call(Node, rabbit_nodes, list_members, []), - RunningMembers = rpc:call(Node, rabbit_nodes, list_running, []), - - AllDbNodes = case rpc:call(Node, rabbit_db_cluster, members, []) of - {badrpc, {'EXIT', {undef, _}}} -> - rpc:call(Node, rabbit_mnesia, cluster_nodes, [all]); - Ret -> - Ret + %% To be compatible with mixed version clusters in 3.11.x we use here + %% rabbit_nodes:all/0 instead of rabbit_nodes:list_members/0 and + %% rabbit_nodes:all_running/0 instead of rabbit_nodes:list_running/0 + %% which are part of the new API. + AllMembers0 = rpc:call(Node, rabbit_nodes, list_members, []), + AllMembers = case maybe_undef(AllMembers0) of + undef -> rpc:call(Node, rabbit_nodes, all, []); + _ -> AllMembers0 end, - DiscDbNodes = rpc:call(Node, rabbit_mnesia, cluster_nodes, [disc]), - RunningDbNodes = rpc:call(Node, rabbit_mnesia, cluster_nodes, [running]), + RunningMembers0 = rpc:call(Node, rabbit_nodes, list_running, []), + RunningMembers = case maybe_undef(RunningMembers0) of + undef -> rpc:call(Node, rabbit_nodes, all_running, []); + _ -> RunningMembers0 + end, + + %% To be compatible with mixed version clusters in 3.11.x we use here + %% rabbit_mnesia:cluster_nodes/1 instead of rabbit_db_cluster:members/0 + AllDbNodes0 = rpc:call(Node, rabbit_db_cluster, members, []), + AllDbNodes = case maybe_undef(AllDbNodes0) of + undef -> rpc:call(Node, rabbit_mnesia, cluster_nodes, [all]); + _ -> AllDbNodes0 + end, + {DiscDbNodes, RunningDbNodes} = + case rpc:call(Node, rabbit_khepri, is_enabled, []) of + true -> + {AllMembers, RunningMembers}; + _ -> + {rpc:call(Node, rabbit_mnesia, cluster_nodes, [disc]), + rpc:call(Node, rabbit_mnesia, cluster_nodes, [running])} + end, {AllMembers, RunningMembers, @@ -82,14 +115,25 @@ cluster_status(Node) -> DiscDbNodes, RunningDbNodes}. -sort_cluster_status({{badrpc, {'EXIT', {undef, _}}}, {badrpc, {'EXIT', {undef, _}}}, AllM, DiscM, RunningM}) -> - {undef, undef, lists:sort(AllM), lists:sort(DiscM), lists:sort(RunningM)}; sort_cluster_status({All, Running, AllM, DiscM, RunningM}) -> - {lists:sort(All), lists:sort(Running), lists:sort(AllM), lists:sort(DiscM), lists:sort(RunningM)}. + {maybe_sort(All), maybe_sort(Running), maybe_sort(AllM), maybe_sort(DiscM), maybe_sort(RunningM)}. + +maybe_sort({badrpc, {'EXIT', {undef, _}}}) -> + undef; +maybe_sort({badrpc, nodedown}) -> + nodedown; +maybe_sort({badrpc, Reason}) -> + Reason; +maybe_sort(List) -> + lists:sort(List). + +maybe_undef({badrpc, {'EXIT', {undef, _}}}) -> + undef; +maybe_undef(Any) -> + Any. assert_clustered(Nodes) -> assert_cluster_status({Nodes, Nodes, Nodes, Nodes, Nodes}, Nodes). assert_not_clustered(Node) -> assert_cluster_status({[Node], [Node], [Node], [Node], [Node]}, [Node]). - diff --git a/deps/rabbit/test/consumer_timeout_SUITE.erl b/deps/rabbit/test/consumer_timeout_SUITE.erl index 201fb8f643c2..d283108d0774 100644 --- a/deps/rabbit/test/consumer_timeout_SUITE.erl +++ b/deps/rabbit/test/consumer_timeout_SUITE.erl @@ -81,14 +81,19 @@ init_per_group(quorum_queue, Config) -> {queue_args, [{<<"x-queue-type">>, longstr, <<"quorum">>}]}, {queue_durable, true}]); init_per_group(mirrored_queue, Config) -> - rabbit_ct_broker_helpers:set_ha_policy(Config, 0, <<"^max_length.*queue">>, - <<"all">>, [{<<"ha-sync-mode">>, <<"automatic">>}]), - Config1 = rabbit_ct_helpers:set_config( - Config, [{policy_type, <<"classic_queues">>}, - {is_mirrored, true}, - {queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]}, - {queue_durable, true}]), - rabbit_ct_helpers:run_steps(Config1, []); + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + {khepri, _} -> + {skip, <<"Classic queue mirroring not supported by Khepri">>}; + mnesia -> + rabbit_ct_broker_helpers:set_ha_policy(Config, 0, <<"^max_length.*queue">>, + <<"all">>, [{<<"ha-sync-mode">>, <<"automatic">>}]), + Config1 = rabbit_ct_helpers:set_config( + Config, [{policy_type, <<"classic_queues">>}, + {is_mirrored, true}, + {queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]}, + {queue_durable, true}]), + rabbit_ct_helpers:run_steps(Config1, []) + end; init_per_group(Group, Config0) -> case lists:member({group, Group}, all()) of true -> diff --git a/deps/rabbit/test/crashing_queues_SUITE.erl b/deps/rabbit/test/crashing_queues_SUITE.erl index 55fe8acd54b8..4b7a789a1571 100644 --- a/deps/rabbit/test/crashing_queues_SUITE.erl +++ b/deps/rabbit/test/crashing_queues_SUITE.erl @@ -20,9 +20,10 @@ all() -> groups() -> [ {cluster_size_2, [], [ - crashing_unmirrored, + crashing_unmirrored_durable, crashing_mirrored, - give_up_after_repeated_crashes + give_up_after_repeated_crashes, + crashing_unmirrored_transient ]} ]. @@ -45,7 +46,24 @@ init_per_group(cluster_size_2, Config) -> end_per_group(_, Config) -> Config. +init_per_testcase(crashing_mirrored = Testcase, Config) -> + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + mnesia -> + init_per_testcase0(Testcase, Config); + _ -> + {skip, "Classic queue mirroring not supported by Khepri"} + end; +init_per_testcase(crashing_unmirrored_transient = Testcase, Config) -> + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + mnesia -> + init_per_testcase0(Testcase, Config); + _ -> + {skip, "Transient queues not supported by Khepri"} + end; init_per_testcase(Testcase, Config) -> + init_per_testcase0(Testcase, Config). + +init_per_testcase0(Testcase, Config) -> rabbit_ct_helpers:testcase_started(Config, Testcase), ClusterSize = ?config(rmq_nodes_count, Config), TestNumber = rabbit_ct_helpers:testcase_number(Config, ?MODULE, Testcase), @@ -58,16 +76,14 @@ init_per_testcase(Testcase, Config) -> rabbit_ct_client_helpers:setup_steps()). end_per_testcase(Testcase, Config) -> - Config1 = rabbit_ct_helpers:run_steps(Config, - rabbit_ct_client_helpers:teardown_steps() ++ - rabbit_ct_broker_helpers:teardown_steps()), - rabbit_ct_helpers:testcase_finished(Config1, Testcase). + rabbit_ct_helpers:testcase_finished(Config, Testcase), + rabbit_ct_helpers:run_teardown_steps(Config, rabbit_ct_broker_helpers:teardown_steps()). %% ------------------------------------------------------------------- %% Testcases. %% ------------------------------------------------------------------- -crashing_unmirrored(Config) -> +crashing_unmirrored_durable(Config) -> [A, B] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), ChA = rabbit_ct_client_helpers:open_channel(Config, A), ConnB = rabbit_ct_client_helpers:open_connection(Config, B), @@ -75,6 +91,14 @@ crashing_unmirrored(Config) -> amqp_channel:call(ChA, #'confirm.select'{}), test_queue_failure(A, ChA, ConnB, 1, 0, #'queue.declare'{queue = QName, durable = true}), + ok. + +crashing_unmirrored_transient(Config) -> + [A, B] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + ChA = rabbit_ct_client_helpers:open_channel(Config, A), + ConnB = rabbit_ct_client_helpers:open_connection(Config, B), + QName = <<"crashing_unmirrored-q">>, + amqp_channel:call(ChA, #'confirm.select'{}), test_queue_failure(A, ChA, ConnB, 0, 0, #'queue.declare'{queue = QName, durable = false}), ok. diff --git a/deps/rabbit/test/dead_lettering_SUITE.erl b/deps/rabbit/test/dead_lettering_SUITE.erl index 6513f6da566e..b3e82fe81628 100644 --- a/deps/rabbit/test/dead_lettering_SUITE.erl +++ b/deps/rabbit/test/dead_lettering_SUITE.erl @@ -106,13 +106,18 @@ init_per_group(classic_queue, Config) -> [{queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]}, {queue_durable, false}]); init_per_group(mirrored_queue, Config) -> - rabbit_ct_broker_helpers:set_ha_policy(Config, 0, <<"^max_length.*queue">>, - <<"all">>, [{<<"ha-sync-mode">>, <<"automatic">>}]), - Config1 = rabbit_ct_helpers:set_config( - Config, [{is_mirrored, true}, - {queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]}, - {queue_durable, false}]), - rabbit_ct_helpers:run_steps(Config1, []); + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + mnesia -> + rabbit_ct_broker_helpers:set_ha_policy(Config, 0, <<"^max_length.*queue">>, + <<"all">>, [{<<"ha-sync-mode">>, <<"automatic">>}]), + Config1 = rabbit_ct_helpers:set_config( + Config, [{is_mirrored, true}, + {queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]}, + {queue_durable, false}]), + rabbit_ct_helpers:run_steps(Config1, []); + _ -> + {skip, "Classic mirroring not supported by Khepri"} + end; init_per_group(quorum_queue, Config) -> rabbit_ct_helpers:set_config( Config, @@ -158,9 +163,14 @@ init_per_group(Group, Config) -> Config1, rabbit_ct_broker_helpers:setup_steps() ++ rabbit_ct_client_helpers:setup_steps()), - _ = rabbit_ct_broker_helpers:enable_feature_flag(Config2, - message_containers), - Config2; + case Config2 of + {skip, _} -> + Config2; + _ -> + _ = rabbit_ct_broker_helpers:enable_feature_flag(Config2, + message_containers), + Config2 + end; false -> rabbit_ct_helpers:run_steps(Config, []) end. diff --git a/deps/rabbit/test/definition_import_SUITE.erl b/deps/rabbit/test/definition_import_SUITE.erl index 5276c6a57136..41b674aa0e73 100644 --- a/deps/rabbit/test/definition_import_SUITE.erl +++ b/deps/rabbit/test/definition_import_SUITE.erl @@ -185,9 +185,9 @@ end_per_testcase(Testcase, Config) -> %% Tests %% -import_case1(Config) -> import_file_case(Config, "case1"). +import_case1(Config) -> import_invalid_file_case_in_khepri(Config, "case1"). import_case2(Config) -> import_file_case(Config, "case2"). -import_case3(Config) -> import_file_case(Config, "case3"). +import_case3(Config) -> import_invalid_file_case_in_khepri(Config, "case3"). import_case4(Config) -> import_file_case(Config, "case4"). import_case6(Config) -> import_file_case(Config, "case6"). import_case7(Config) -> import_file_case(Config, "case7"). @@ -389,6 +389,11 @@ import_invalid_file_case(Config, CaseName) -> rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, run_invalid_import_case, [CasePath]), ok. +import_invalid_file_case_in_khepri(Config, CaseName) -> + CasePath = filename:join(?config(data_dir, Config), CaseName ++ ".json"), + rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, run_invalid_import_case_in_khepri, [CasePath]), + ok. + import_invalid_file_case_if_unchanged(Config, CaseName) -> CasePath = filename:join(?config(data_dir, Config), CaseName ++ ".json"), rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, run_invalid_import_case_if_unchanged, [CasePath]), @@ -470,6 +475,27 @@ run_invalid_import_case_if_unchanged(Path) -> {error, _E} -> ok end. +run_invalid_import_case_in_khepri(Path) -> + case rabbit_khepri:is_enabled() of + true -> + run_invalid_import_case_in_khepri0(Path); + false -> + run_import_case(Path) + end. + +run_invalid_import_case_in_khepri0(Path) -> + {ok, Body} = file:read_file(Path), + ct:pal("Successfully loaded a definition file at ~tp~n", [Path]), + case rabbit_definitions:import_raw(Body) of + ok -> + ct:pal("Expected import case ~tp to fail~n", [Path]), + ct:fail({expected_failure, Path}); + {error, E} -> + case re:run(E, ".*mirrored queues are deprecated.*", [{capture, none}]) of + match -> ok; + _ -> ct:fail({expected_failure, Path, E}) + end + end. queue_lookup(Config, VHost, Name) -> rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_amqqueue, lookup, [rabbit_misc:r(VHost, queue, Name)]). diff --git a/deps/rabbit/test/direct_exchange_routing_v2_SUITE.erl b/deps/rabbit/test/direct_exchange_routing_v2_SUITE.erl index 0bfe75bfcde8..b811657db766 100644 --- a/deps/rabbit/test/direct_exchange_routing_v2_SUITE.erl +++ b/deps/rabbit/test/direct_exchange_routing_v2_SUITE.erl @@ -61,14 +61,21 @@ end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). init_per_group(Group = cluster_size_1, Config0) -> - Config = rabbit_ct_helpers:set_config(Config0, {rmq_nodes_count, 1}), + Config = rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 1}, + {metadata_store, mnesia}]), start_broker(Group, Config); init_per_group(Group = cluster_size_2, Config0) -> - Config = rabbit_ct_helpers:set_config(Config0, {rmq_nodes_count, 2}), + Config = rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 2}, + {metadata_store, mnesia}]), + start_broker(Group, Config); +init_per_group(Group = cluster_size_3, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 3}, + {metadata_store, mnesia}]), start_broker(Group, Config); init_per_group(Group = unclustered_cluster_size_2, Config0) -> Config = rabbit_ct_helpers:set_config(Config0, [{rmq_nodes_count, 2}, - {rmq_nodes_clustered, false}]), + {rmq_nodes_clustered, false}, + {metadata_store, mnesia}]), start_broker(Group, Config). start_broker(Group, Config0) -> diff --git a/deps/rabbit/test/dynamic_ha_SUITE.erl b/deps/rabbit/test/dynamic_ha_SUITE.erl index 1d174b9f5ece..d797e4213c8c 100644 --- a/deps/rabbit/test/dynamic_ha_SUITE.erl +++ b/deps/rabbit/test/dynamic_ha_SUITE.erl @@ -80,7 +80,12 @@ groups() -> init_per_suite(Config) -> rabbit_ct_helpers:log_environment(), - rabbit_ct_helpers:run_setup_steps(Config). + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + mnesia -> + rabbit_ct_helpers:run_setup_steps(Config); + {khepri, _} -> + {skip, "Classic queue mirroring not supported by Khepri"} + end. end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). @@ -110,15 +115,21 @@ init_per_testcase(Testcase, Config) -> Config2 = rabbit_ct_helpers:run_steps(Config1, rabbit_ct_broker_helpers:setup_steps() ++ rabbit_ct_client_helpers:setup_steps()), - case Testcase of - change_cluster -> - %% do not enable message_containers feature flag as it will stop - %% nodes in mixed versions joining later - ok; + case Config2 of + {skip, _} -> + Config2; _ -> - _ = rabbit_ct_broker_helpers:enable_feature_flag(Config2, message_containers) - end, - Config2. + case Testcase of + change_cluster -> + %% do not enable message_containers feature flag as it will + %% stop nodes in mixed versions joining later + ok; + _ -> + _ = rabbit_ct_broker_helpers:enable_feature_flag( + Config2, message_containers) + end, + Config2 + end. end_per_testcase(Testcase, Config) -> Config1 = rabbit_ct_helpers:run_steps(Config, diff --git a/deps/rabbit/test/dynamic_qq_SUITE.erl b/deps/rabbit/test/dynamic_qq_SUITE.erl index 2ddccf935cc2..fa6f9dd14927 100644 --- a/deps/rabbit/test/dynamic_qq_SUITE.erl +++ b/deps/rabbit/test/dynamic_qq_SUITE.erl @@ -26,12 +26,18 @@ groups() -> [ {clustered, [], [ {cluster_size_3, [], [ - recover_follower_after_standalone_restart, vhost_deletion, + quorum_unaffected_after_vhost_failure + ]}, + {cluster_size_5, [], [ + %% Khepri does not work on a cluster in minority. Thus, to test these + %% specific cases with quorum queues in minority we need a bigger cluster. + %% 5-nodes RMQ and 3-nodes quorum queues allows to test the same test + %% cases than a 3-nodes mnesia cluster. + recover_follower_after_standalone_restart, force_delete_if_no_consensus, takeover_on_failure, - takeover_on_shutdown, - quorum_unaffected_after_vhost_failure + takeover_on_shutdown ]} ]} ]. @@ -49,8 +55,8 @@ end_per_suite(Config) -> init_per_group(clustered, Config) -> rabbit_ct_helpers:set_config(Config, [{rmq_nodes_clustered, true}]); -init_per_group(cluster_size_2, Config) -> - rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 2}]); +init_per_group(cluster_size_5, Config) -> + rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 5}]); init_per_group(cluster_size_3, Config) -> rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 3}]). @@ -72,14 +78,20 @@ init_per_testcase(Testcase, Config) -> {rmq_nodename_suffix, Testcase}, {tcp_ports_base, {skip_n_nodes, TestNumber * ClusterSize}}, {queue_name, Q}, - {queue_args, [{<<"x-queue-type">>, longstr, <<"quorum">>}]} + {queue_args, [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-quorum-initial-group-size">>, long, 3}]} ]), Config2 = rabbit_ct_helpers:run_steps( Config1, rabbit_ct_broker_helpers:setup_steps() ++ rabbit_ct_client_helpers:setup_steps()), - _ = rabbit_ct_broker_helpers:enable_feature_flag(Config2, message_containers), - Config2 + case Config2 of + {skip, _} -> + Config2; + _ -> + _ = rabbit_ct_broker_helpers:enable_feature_flag(Config2, message_containers), + Config2 + end end. end_per_testcase(Testcase, Config) -> @@ -108,15 +120,22 @@ vhost_deletion(Config) -> ok. force_delete_if_no_consensus(Config) -> - [A, B, C] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - ACh = rabbit_ct_client_helpers:open_channel(Config, A), + [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), QName = ?config(queue_name, Config), Args = ?config(queue_args, Config), - amqp_channel:call(ACh, #'queue.declare'{queue = QName, - arguments = Args, - durable = true - }), + amqp_channel:call(Ch, #'queue.declare'{queue = QName, + arguments = Args, + durable = true + }), + rabbit_ct_client_helpers:close_channel(Ch), + + RaName = queue_utils:ra_name(QName), + {ok, [{_, A}, {_, B}, {_, C}], _} = ra:members({RaName, Server}), + + ACh = rabbit_ct_client_helpers:open_channel(Config, A), rabbit_ct_client_helpers:publish(ACh, QName, 10), + ok = rabbit_ct_broker_helpers:restart_node(Config, B), ok = rabbit_ct_broker_helpers:stop_node(Config, A), ok = rabbit_ct_broker_helpers:stop_node(Config, C), @@ -141,16 +160,19 @@ takeover_on_shutdown(Config) -> takeover_on(Config, stop_node). takeover_on(Config, Fun) -> - [A, B, C] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), - ACh = rabbit_ct_client_helpers:open_channel(Config, A), QName = ?config(queue_name, Config), Args = ?config(queue_args, Config), - amqp_channel:call(ACh, #'queue.declare'{queue = QName, + amqp_channel:call(Ch, #'queue.declare'{queue = QName, arguments = Args, durable = true }), - rabbit_ct_client_helpers:publish(ACh, QName, 10), + rabbit_ct_client_helpers:publish(Ch, QName, 10), + + RaName = queue_utils:ra_name(QName), + {ok, [{_, A}, {_, B}, {_, C}], _} = ra:members({RaName, Server}), ok = rabbit_ct_broker_helpers:restart_node(Config, B), ok = rabbit_ct_broker_helpers:Fun(Config, C), @@ -212,8 +234,8 @@ recover_follower_after_standalone_restart(Config) -> %% Tests that followers can be brought up standalone after forgetting the %% rest of the cluster. Consensus won't be reached as there is only one node in the %% new cluster. - Servers = [A, B, C] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - Ch = rabbit_ct_client_helpers:open_channel(Config, A), + [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), QName = ?config(queue_name, Config), Args = ?config(queue_args, Config), @@ -222,6 +244,10 @@ recover_follower_after_standalone_restart(Config) -> durable = true }), + RaName = queue_utils:ra_name(QName), + {ok, [{_, A}, {_, B}, {_, C}], _} = ra:members({RaName, Server}), + Servers = [A, B, C], + rabbit_ct_client_helpers:publish(Ch, QName, 15), rabbit_ct_client_helpers:close_channel(Ch), diff --git a/deps/rabbit/test/eager_sync_SUITE.erl b/deps/rabbit/test/eager_sync_SUITE.erl index 1ae07d12cbb2..74e824b4dd72 100644 --- a/deps/rabbit/test/eager_sync_SUITE.erl +++ b/deps/rabbit/test/eager_sync_SUITE.erl @@ -43,8 +43,13 @@ suite() -> %% ------------------------------------------------------------------- init_per_suite(Config) -> - rabbit_ct_helpers:log_environment(), - rabbit_ct_helpers:run_setup_steps(Config). + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + mnesia -> + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config); + _ -> + {skip, "Classic mirroring not supported by Khepri"} + end. end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). diff --git a/deps/rabbit/test/exchanges_SUITE.erl b/deps/rabbit/test/exchanges_SUITE.erl index 97924de99b9f..cec9e5948794 100644 --- a/deps/rabbit/test/exchanges_SUITE.erl +++ b/deps/rabbit/test/exchanges_SUITE.erl @@ -19,12 +19,18 @@ suite() -> all() -> [ - {group, mnesia_store} + {group, mnesia_store}, + {group, khepri_store}, + {group, khepri_migration} ]. groups() -> [ - {mnesia_store, [], all_tests()} + {mnesia_store, [], all_tests()}, + {khepri_store, [], all_tests()}, + {khepri_migration, [], [ + from_mnesia_to_khepri + ]} ]. all_tests() -> @@ -47,7 +53,14 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). -init_per_group(mnesia_store = Group, Config) -> +init_per_group(mnesia_store = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, mnesia}]), + init_per_group_common(Group, Config, 1); +init_per_group(khepri_store = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, khepri}]), + init_per_group_common(Group, Config, 1); +init_per_group(khepri_migration = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, mnesia}]), init_per_group_common(Group, Config, 1). init_per_group_common(Group, Config, Size) -> @@ -87,7 +100,7 @@ end_per_testcase(Testcase, Config) -> %% ------------------------------------------------------------------- direct_exchange(Config) -> Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), - + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), Q = ?config(queue_name, Config), ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), @@ -103,7 +116,7 @@ direct_exchange(Config) -> routing_key = AltQ}), publish(Ch, Direct, Q, <<"msg1">>), publish(Ch, Direct, <<"anyotherkey">>, <<"msg2">>), - + queue_utils:wait_for_messages(Config, [[Q, <<"1">>, <<"1">>, <<"0">>], [AltQ, <<"0">>, <<"0">>, <<"0">>]]), ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg1">>}}, @@ -116,12 +129,12 @@ direct_exchange(Config) -> topic_exchange(Config) -> Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), - + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), Q = ?config(queue_name, Config), ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), - Topic = <<"amq.topic">>, + Topic = <<"amq.topic">>, #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = Topic, queue = Q, routing_key = <<"this.*.rules">>}), @@ -162,7 +175,7 @@ topic_exchange(Config) -> amqp_channel:call(Ch, #'basic.get'{queue = Q})), ?assertMatch(#'basic.get_empty'{}, amqp_channel:call(Ch, #'basic.get'{queue = Q})), - + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = Topic, queue = Q, routing_key = <<"#">>}), @@ -176,7 +189,7 @@ topic_exchange(Config) -> fanout_exchange(Config) -> Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), - + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), Q = ?config(queue_name, Config), ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), @@ -192,7 +205,7 @@ fanout_exchange(Config) -> routing_key = AltQ}), publish(Ch, Fanout, Q, <<"msg1">>), publish(Ch, Fanout, <<"anyotherkey">>, <<"msg2">>), - + queue_utils:wait_for_messages(Config, [[Q, <<"2">>, <<"2">>, <<"0">>], [AltQ, <<"2">>, <<"2">>, <<"0">>]]), ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg1">>}}, @@ -270,6 +283,56 @@ invalid_exchange(Config) -> queue = Q, routing_key = Q})). +from_mnesia_to_khepri(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q, [])), + + %% Test transient exchanges + X = ?config(exchange_name, Config), + #'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = X, + durable = false}), + + %% Topic is the only exchange type that has its own mnesia/khepri tables. + %% Let's test that the exchange works as expected after migration + Topic = <<"amq.topic">>, + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = Topic, + queue = Q, + routing_key = <<"this.queue.rules">>}), + + Exchanges = lists:sort([rabbit_misc:r(<<"/">>, exchange, <<>>), + rabbit_misc:r(<<"/">>, exchange, <<"amq.direct">>), + rabbit_misc:r(<<"/">>, exchange, <<"amq.fanout">>), + rabbit_misc:r(<<"/">>, exchange, <<"amq.headers">>), + rabbit_misc:r(<<"/">>, exchange, <<"amq.match">>), + rabbit_misc:r(<<"/">>, exchange, <<"amq.rabbitmq.trace">>), + rabbit_misc:r(<<"/">>, exchange, <<"amq.topic">>), + rabbit_misc:r(<<"/">>, exchange, X)]), + ?assertEqual( + Exchanges, + lists:sort([X0#exchange.name || + X0 <- rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_exchange, list, [])])), + + case rabbit_ct_broker_helpers:enable_feature_flag(Config, khepri_db) of + ok -> + rabbit_ct_helpers:await_condition( + fun() -> + RecoveredExchanges = + lists:sort([X0#exchange.name || + X0 <- rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_exchange, list, [])]), + Exchanges == RecoveredExchanges + end), + publish(Ch, Topic, <<"this.queue.rules">>, <<"msg1">>), + ?assertMatch({#'basic.get_ok'{}, #amqp_msg{payload = <<"msg1">>}}, + amqp_channel:call(Ch, #'basic.get'{queue = Q})), + ?assertMatch(#'basic.get_empty'{}, + amqp_channel:call(Ch, #'basic.get'{queue = Q})); + Skip -> + Skip + end. + %% Internal delete_queues() -> diff --git a/deps/rabbit/test/feature_flags_SUITE.erl b/deps/rabbit/test/feature_flags_SUITE.erl index 360959635245..9e88e5758877 100644 --- a/deps/rabbit/test/feature_flags_SUITE.erl +++ b/deps/rabbit/test/feature_flags_SUITE.erl @@ -924,7 +924,9 @@ do_enable_feature_flag_when_ff_file_is_unwritable(Config) -> %% the `rabbit_ff_controller' process because it was pretty fragile. %% That's why the rest of the testcase is commentted out now. We should %% revisit this at some point. - [?assertEqual(ok, rabbit_ct_broker_helpers:start_node(Config, N)) + [?assertEqual(ok, rabbit_ct_broker_helpers:async_start_node(Config, N)) + || N <- lists:reverse(Nodes)], + [?assertEqual(ok, rabbit_ct_broker_helpers:wait_for_async_start_node(N)) || N <- lists:reverse(Nodes)]. % XXX ?assertEqual( @@ -960,7 +962,7 @@ enable_feature_flag_with_a_network_partition(Config) -> block(NodePairs), %% Wait for the network partition to happen - clustering_utils:assert_cluster_status({All, All, [A, C, D]}, [A, C, D]), + clustering_utils:assert_cluster_status({All, [A, C, D]}, [A, C, D]), %% Enabling the feature flag should fail in the specific case of %% `ff_from_testsuite', if the network is broken. @@ -977,7 +979,7 @@ enable_feature_flag_with_a_network_partition(Config) -> || N <- [A, C, D]], [?assertEqual(ok, rabbit_ct_broker_helpers:start_node(Config, N)) || N <- [A, C, D]], - clustering_utils:assert_cluster_status({All, All, All}, All), + clustering_utils:assert_cluster_status({All, All}, All), declare_arbitrary_feature_flag(Config), %% Enabling the feature flag works. @@ -1013,7 +1015,8 @@ mark_feature_flag_as_enabled_with_a_network_partition(Config) -> {B, D}, {B, E}], block(NodePairs), - clustering_utils:assert_cluster_status({AllNodes, AllNodes, [A, C, D, E]}, [A, C, D, E]), + clustering_utils:assert_cluster_status( + {AllNodes, [A, C, D, E]}, [A, C, D, E]), %% Mark the feature flag as enabled on all nodes from node B. This %% is expected to timeout. diff --git a/deps/rabbit/test/list_queues_online_and_offline_SUITE.erl b/deps/rabbit/test/list_queues_online_and_offline_SUITE.erl index 7db09969dec8..03065dd22e26 100644 --- a/deps/rabbit/test/list_queues_online_and_offline_SUITE.erl +++ b/deps/rabbit/test/list_queues_online_and_offline_SUITE.erl @@ -42,7 +42,7 @@ init_per_group(Group, Config) -> Config1 = rabbit_ct_helpers:set_config(Config, [ {rmq_nodename_suffix, Group}, - {rmq_nodes_count, 2} + {rmq_nodes_count, 3} ]), rabbit_ct_helpers:run_steps( Config1, @@ -65,7 +65,7 @@ end_per_testcase(Testcase, Config) -> %% --------------------------------------------------------------------------- list_queues_online_and_offline(Config) -> - [A, B] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + [A, B, C] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), ACh = rabbit_ct_client_helpers:open_channel(Config, A), %% Node B will be stopped BCh = rabbit_ct_client_helpers:open_channel(Config, B), @@ -78,7 +78,7 @@ list_queues_online_and_offline(Config) -> rabbit_ct_helpers:await_condition( fun() -> - [A] == rpc:call(A, rabbit_nodes, list_running, []) + lists:sort([A, C]) == lists:sort(rpc:call(A, rabbit_nodes, list_running, [])) end, 60000), GotUp = lists:sort(rabbit_ct_broker_helpers:rabbitmqctl_list(Config, A, diff --git a/deps/rabbit/test/maintenance_mode_SUITE.erl b/deps/rabbit/test/maintenance_mode_SUITE.erl index fd325f1244fc..2e926a699eea 100644 --- a/deps/rabbit/test/maintenance_mode_SUITE.erl +++ b/deps/rabbit/test/maintenance_mode_SUITE.erl @@ -29,7 +29,8 @@ groups() -> maintenance_mode_status, listener_suspension_status, client_connection_closure, - quorum_queue_leadership_transfer + quorum_queue_leadership_transfer, + metadata_store_leadership_transfer ]} ]. @@ -67,6 +68,20 @@ init_per_testcase(quorum_queue_leadership_transfer = Testcase, Config) -> Config1, rabbit_ct_broker_helpers:setup_steps() ++ rabbit_ct_client_helpers:setup_steps()); +init_per_testcase(metadata_store_leadership_transfer = Testcase, Config) -> + rabbit_ct_helpers:testcase_started(Config, Testcase), + ClusterSize = ?config(rmq_nodes_count, Config), + TestNumber = rabbit_ct_helpers:testcase_number(Config, ?MODULE, Testcase), + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodes_clustered, true}, + {rmq_nodename_suffix, Testcase}, + {tcp_ports_base, {skip_n_nodes, TestNumber * ClusterSize}}, + {metadata_store, khepri} + ]), + rabbit_ct_helpers:run_steps( + Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()); init_per_testcase(Testcase, Config) -> rabbit_ct_helpers:testcase_started(Config, Testcase), ClusterSize = ?config(rmq_nodes_count, Config), @@ -76,11 +91,16 @@ init_per_testcase(Testcase, Config) -> {rmq_nodename_suffix, Testcase}, {tcp_ports_base, {skip_n_nodes, TestNumber * ClusterSize}} ]), + ExtraSteps = + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + {khepri, []} -> []; + mnesia -> [fun rabbit_ct_broker_helpers:set_ha_policy_all/1] + end, rabbit_ct_helpers:run_steps( Config1, rabbit_ct_broker_helpers:setup_steps() ++ rabbit_ct_client_helpers:setup_steps() ++ - [fun rabbit_ct_broker_helpers:set_ha_policy_all/1]). + ExtraSteps). end_per_testcase(Testcase, Config) -> Config1 = rabbit_ct_helpers:run_steps(Config, @@ -262,3 +282,31 @@ quorum_queue_leadership_transfer(Config) -> end, rabbit_ct_broker_helpers:revive_node(Config, A). + +metadata_store_leadership_transfer(Config) -> + [A | _] = Nodenames = rabbit_ct_broker_helpers:get_node_configs( + Config, nodename), + + {_, LeaderNode} = rabbit_ct_broker_helpers:rpc(Config, A, ra_leaderboard, lookup_leader, + [rabbit_khepri:get_ra_cluster_name()]), + rabbit_ct_helpers:await_condition( + fun () -> not rabbit_ct_broker_helpers:is_being_drained_local_read(Config, LeaderNode) end, + 10000), + rabbit_ct_broker_helpers:drain_node(Config, LeaderNode), + rabbit_ct_helpers:await_condition( + fun () -> rabbit_ct_broker_helpers:is_being_drained_local_read(Config, LeaderNode) end, + 10000), + + %% Check it is still functional + [N | _] = Nodenames -- [LeaderNode], + Conn = rabbit_ct_client_helpers:open_connection(Config, N), + {ok, Ch} = amqp_connection:open_channel(Conn), + QName = <<"qq.1">>, + amqp_channel:call(Ch, + #'queue.declare'{queue = QName, durable = true, + arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>}]}), + + {_, NewLeaderNode} = rabbit_ct_broker_helpers:rpc(Config, N, ra_leaderboard, lookup_leader, + [rabbit_khepri:get_ra_cluster_name()]), + ?assertNot(LeaderNode == NewLeaderNode), + rabbit_ct_broker_helpers:revive_node(Config, LeaderNode). diff --git a/deps/rabbit/test/many_node_ha_SUITE.erl b/deps/rabbit/test/many_node_ha_SUITE.erl index fbaa7f90f80b..309e96a213fd 100644 --- a/deps/rabbit/test/many_node_ha_SUITE.erl +++ b/deps/rabbit/test/many_node_ha_SUITE.erl @@ -35,7 +35,12 @@ groups() -> init_per_suite(Config) -> rabbit_ct_helpers:log_environment(), - rabbit_ct_helpers:run_setup_steps(Config). + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + mnesia -> + rabbit_ct_helpers:run_setup_steps(Config); + {khepri, _} -> + {skip, "Classic queue mirroring not supported by Khepri"} + end. end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). diff --git a/deps/rabbit/test/message_containers_SUITE.erl b/deps/rabbit/test/message_containers_SUITE.erl index 5747613baaad..1ada4e9c1e10 100644 --- a/deps/rabbit/test/message_containers_SUITE.erl +++ b/deps/rabbit/test/message_containers_SUITE.erl @@ -58,17 +58,36 @@ init_per_group(Group, Config) -> Config2 = rabbit_ct_helpers:run_steps(Config1c, [fun merge_app_env/1 ] ++ rabbit_ct_broker_helpers:setup_steps()), - ok = rabbit_ct_broker_helpers:rpc( - Config2, 0, application, set_env, - [rabbit, channel_tick_interval, 100]), - - AllFFs = rabbit_ct_broker_helpers:rpc(Config2, rabbit_feature_flags, list, [all, stable]), - FFs = maps:keys(maps:remove(?FEATURE_FLAG, AllFFs)), - ct:pal("FFs ~p", [FFs]), - rabbit_ct_broker_helpers:set_policy(Config2, 0, - <<"ha-policy">>, <<".*">>, <<"queues">>, - [{<<"ha-mode">>, <<"all">>}]), - Config2. + case Config2 of + {skip, _} -> + Config2; + _ -> + ok = rabbit_ct_broker_helpers:rpc( + Config2, 0, application, set_env, + [rabbit, channel_tick_interval, 100]), + + AllFFs = rabbit_ct_broker_helpers:rpc(Config2, rabbit_feature_flags, list, [all, stable]), + FFs = maps:keys(maps:remove(?FEATURE_FLAG, AllFFs)), + ct:pal("FFs ~p", [FFs]), + case Group of + classic -> + try + rabbit_ct_broker_helpers:set_policy( + Config2, 0, + <<"ha-policy">>, <<".*">>, <<"queues">>, + [{<<"ha-mode">>, <<"all">>}]), + Config2 + catch + _:{badmatch, {error_string, Reason}} -> + rabbit_ct_helpers:run_steps( + Config2, + rabbit_ct_broker_helpers:teardown_steps()), + {skip, Reason} + end; + _ -> + Config2 + end + end. merge_app_env(Config) -> rabbit_ct_helpers:merge_app_env( diff --git a/deps/rabbit/test/metadata_store_clustering_SUITE.erl b/deps/rabbit/test/metadata_store_clustering_SUITE.erl new file mode 100644 index 000000000000..a57f093ae650 --- /dev/null +++ b/deps/rabbit/test/metadata_store_clustering_SUITE.erl @@ -0,0 +1,340 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(metadata_store_clustering_SUITE). + +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). +-include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl"). + +-compile([nowarn_export_all, export_all]). + +suite() -> + [{timetrap, 5 * 60_000}]. + +all() -> + [ + {group, unclustered} + ]. + +groups() -> + [ + {unclustered, [], [{cluster_size_2, [], cluster_size_2_tests()}, + {cluster_size_3, [], cluster_size_3_tests()}]} + ]. + +cluster_size_2_tests() -> + [ + join_khepri_khepri_cluster, + join_mnesia_khepri_cluster, + join_mnesia_khepri_cluster_reverse, + join_khepri_mnesia_cluster, + join_khepri_mnesia_cluster_reverse + ]. + +cluster_size_3_tests() -> + [ + join_khepri_khepri_khepri_cluster, + join_mnesia_khepri_khepri_cluster, + join_mnesia_khepri_khepri_cluster_reverse, + join_khepri_mnesia_khepri_cluster, + join_khepri_mnesia_khepri_cluster_reverse, + join_khepri_khepri_mnesia_cluster, + join_khepri_khepri_mnesia_cluster_reverse, + join_mnesia_mnesia_khepri_cluster, + join_mnesia_mnesia_khepri_cluster_reverse, + join_mnesia_khepri_mnesia_cluster, + join_mnesia_khepri_mnesia_cluster_reverse, + join_khepri_mnesia_mnesia_cluster, + join_khepri_mnesia_mnesia_cluster_reverse + ]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config, []). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(unclustered, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, mnesia}, + {rmq_nodes_clustered, false}, + {tcp_ports_base}, + {net_ticktime, 10}]); +init_per_group(cluster_size_2, Config) -> + rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 2}]); +init_per_group(cluster_size_3, Config) -> + rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 3}]). + +end_per_group(_, Config) -> + Config. + +init_per_testcase(Testcase, Config) -> + Q = rabbit_data_coercion:to_binary(Testcase), + Config1 = rabbit_ct_helpers:set_config(Config, + [{rmq_nodename_suffix, Testcase}, + {queue_name, Q} + ]), + Config2 = rabbit_ct_helpers:testcase_started(Config1, Testcase), + rabbit_ct_helpers:run_steps(Config2, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + +end_per_testcase(Testcase, Config) -> + Config1 = rabbit_ct_helpers:run_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()), + rabbit_ct_helpers:testcase_finished(Config1, Testcase). + +%% ------------------------------------------------------------------- +%% Testcases. +%% ------------------------------------------------------------------- + +join_khepri_khepri_cluster(Config) -> + Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag( Config, Servers, khepri_db), + case Ret of + ok -> join_size_2_cluster(Config, Servers); + {skip, _} = Skip -> Skip + end. + +join_khepri_mnesia_cluster(Config) -> + [Server0, _] = Servers = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag(Config, [Server0], khepri_db), + case Ret of + ok -> join_size_2_cluster(Config, Servers); + {skip, _} = Skip -> Skip + end. + +join_khepri_mnesia_cluster_reverse(Config) -> + [Server0, _] = Servers = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag(Config, [Server0], khepri_db), + case Ret of + ok -> join_size_2_cluster(Config, lists:reverse(Servers)); + {skip, _} = Skip -> Skip + end. + +join_mnesia_khepri_cluster(Config) -> + [_, Server1] = Servers = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag(Config, [Server1], khepri_db), + case Ret of + ok -> join_size_2_cluster(Config, Servers); + {skip, _} = Skip -> Skip + end. + +join_mnesia_khepri_cluster_reverse(Config) -> + [_, Server1] = Servers = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag(Config, [Server1], khepri_db), + case Ret of + ok -> join_size_2_cluster(Config, lists:reverse(Servers)); + {skip, _} = Skip -> Skip + end. + +join_khepri_khepri_khepri_cluster(Config) -> + Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag(Config, Servers, khepri_db), + case Ret of + ok -> join_size_3_cluster(Config, Servers); + {skip, _} = Skip -> Skip + end. + +join_mnesia_khepri_khepri_cluster(Config) -> + [_, Server1, Server2] = Servers = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag(Config, [Server1, Server2], khepri_db), + case Ret of + ok -> join_size_3_cluster(Config, Servers); + {skip, _} = Skip -> Skip + end. + +join_mnesia_khepri_khepri_cluster_reverse(Config) -> + [_, Server1, Server2] = Servers = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag(Config, [Server1, Server2], khepri_db), + case Ret of + ok -> join_size_3_cluster(Config, lists:reverse(Servers)); + {skip, _} = Skip -> Skip + end. + +join_khepri_mnesia_khepri_cluster(Config) -> + [Server0, _, Server2] = Servers = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag(Config, [Server0, Server2], khepri_db), + case Ret of + ok -> join_size_3_cluster(Config, Servers); + {skip, _} = Skip -> Skip + end. + +join_khepri_mnesia_khepri_cluster_reverse(Config) -> + [Server0, _, Server2] = Servers = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag(Config, [Server0, Server2], khepri_db), + case Ret of + ok -> join_size_3_cluster(Config, lists:reverse(Servers)); + {skip, _} = Skip -> Skip + end. + +join_khepri_khepri_mnesia_cluster(Config) -> + [Server0, Server1, _] = Servers = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag(Config, [Server0, Server1], khepri_db), + case Ret of + ok -> join_size_3_cluster(Config, Servers); + {skip, _} = Skip -> Skip + end. + +join_khepri_khepri_mnesia_cluster_reverse(Config) -> + [Server0, Server1, _] = Servers = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag(Config, [Server0, Server1], khepri_db), + case Ret of + ok -> join_size_3_cluster(Config, lists:reverse(Servers)); + {skip, _} = Skip -> Skip + end. + +join_mnesia_mnesia_khepri_cluster(Config) -> + [_, _, Server2] = Servers = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag(Config, [Server2], khepri_db), + case Ret of + ok -> join_size_3_cluster(Config, Servers); + {skip, _} = Skip -> Skip + end. + +join_mnesia_mnesia_khepri_cluster_reverse(Config) -> + [_, _, Server2] = Servers = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag(Config, [Server2], khepri_db), + case Ret of + ok -> join_size_3_cluster(Config, lists:reverse(Servers)); + {skip, _} = Skip -> Skip + end. + +join_mnesia_khepri_mnesia_cluster(Config) -> + [_, Server1, _] = Servers = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag(Config, [Server1], khepri_db), + case Ret of + ok -> join_size_3_cluster(Config, Servers); + {skip, _} = Skip -> Skip + end. + +join_mnesia_khepri_mnesia_cluster_reverse(Config) -> + [_, Server1, _] = Servers = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag(Config, [Server1], khepri_db), + case Ret of + ok -> join_size_3_cluster(Config, lists:reverse(Servers)); + {skip, _} = Skip -> Skip + end. + +join_khepri_mnesia_mnesia_cluster(Config) -> + [Server0, _, _] = Servers = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag(Config, [Server0], khepri_db), + case Ret of + ok -> join_size_3_cluster(Config, Servers); + {skip, _} = Skip -> Skip + end. + +join_khepri_mnesia_mnesia_cluster_reverse(Config) -> + [Server0, _, _] = Servers = + rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + + Ret = rabbit_ct_broker_helpers:enable_feature_flag(Config, [Server0], khepri_db), + case Ret of + ok -> join_size_3_cluster(Config, lists:reverse(Servers)); + {skip, _} = Skip -> Skip + end. + +join_size_2_cluster(Config, [Server0, Server1]) -> + Ch = rabbit_ct_client_helpers:open_channel(Config, Server0), + Q = ?config(queue_name, Config), + + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q)), + ?assertMatch([_], rpc:call(Server0, rabbit_amqqueue, list, [])), + + ok = rabbit_control_helper:command(stop_app, Server1), + ?assertMatch([_], rpc:call(Server0, rabbit_amqqueue, list, [])), + + Ret = rabbit_control_helper:command(join_cluster, Server1, [atom_to_list(Server0)], []), + case Ret of + ok -> + ?assertMatch([_], rpc:call(Server0, rabbit_amqqueue, list, [])), + + ok = rabbit_control_helper:command(start_app, Server1), + ?assertMatch([_], rpc:call(Server0, rabbit_amqqueue, list, [])); + {error, 69, <<"Error:\nincompatible_feature_flags">>} -> + {skip, "'khepri_db' feature flag is unsupported"} + end. + +join_size_3_cluster(Config, [Server0, Server1, Server2]) -> + Ch = rabbit_ct_client_helpers:open_channel(Config, Server0), + Q = ?config(queue_name, Config), + + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q)), + ?assertMatch([_], rpc:call(Server0, rabbit_amqqueue, list, [])), + + ok = rabbit_control_helper:command(stop_app, Server1), + ?assertMatch([_], rpc:call(Server0, rabbit_amqqueue, list, [])), + + Ret1 = rabbit_control_helper:command(join_cluster, Server1, [atom_to_list(Server0)], []), + case Ret1 of + ok -> + ?assertMatch([_], rpc:call(Server0, rabbit_amqqueue, list, [])), + + ok = rabbit_control_helper:command(start_app, Server1), + ?assertMatch([_], rpc:call(Server0, rabbit_amqqueue, list, [])), + + ok = rabbit_control_helper:command(stop_app, Server2), + ?assertMatch([_], rpc:call(Server0, rabbit_amqqueue, list, [])), + + Ret2 = rabbit_control_helper:command(join_cluster, Server2, [atom_to_list(Server0)], []), + case Ret2 of + ok -> + ?assertMatch([_], rpc:call(Server0, rabbit_amqqueue, list, [])), + + ok = rabbit_control_helper:command(start_app, Server2), + ?assertMatch([_], rpc:call(Server0, rabbit_amqqueue, list, [])); + {error, 69, <<"Error:\nincompatible_feature_flags">>} -> + {skip, "'khepri_db' feature flag is unsupported"} + end; + {error, 69, <<"Error:\nincompatible_feature_flags">>} -> + {skip, "'khepri_db' feature flag is unsupported"} + end. + +declare(Ch, Q) -> + amqp_channel:call(Ch, #'queue.declare'{queue = Q, + durable = true, + auto_delete = false, + arguments = []}). diff --git a/deps/rabbit/test/metadata_store_migration_SUITE.erl b/deps/rabbit/test/metadata_store_migration_SUITE.erl new file mode 100644 index 000000000000..f6fa3454ba3e --- /dev/null +++ b/deps/rabbit/test/metadata_store_migration_SUITE.erl @@ -0,0 +1,161 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2018-2022 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(metadata_store_migration_SUITE). + +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). + +-compile([nowarn_export_all, export_all]). +-compile(export_all). + +suite() -> + [{timetrap, 5 * 60000}]. + +all() -> + [ + {group, khepri_migration} + ]. + +groups() -> + [ + {khepri_migration, [], [ + from_mnesia_to_khepri + ]} + ]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config, []). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(khepri_migration = Group, Config0) -> + rabbit_ct_helpers:set_config(Config0, [{metadata_store, mnesia}, + {rmq_nodes_count, 1}, + {rmq_nodename_suffix, Group}, + {tcp_ports_base}]). + +end_per_group(_, Config) -> + Config. + +init_per_testcase(_Testcase, Config) -> + rabbit_ct_helpers:run_steps(Config, rabbit_ct_broker_helpers:setup_steps() ++ rabbit_ct_client_helpers:setup_steps()). + +end_per_testcase(Testcase, Config) -> + Config1 = rabbit_ct_helpers:run_steps( + Config, + rabbit_ct_client_helpers:teardown_steps()), + Config2 = rabbit_ct_helpers:testcase_finished(Config1, Testcase), + rabbit_ct_helpers:run_steps(Config2, + rabbit_ct_broker_helpers:teardown_steps()). + + +%% ------------------------------------------------------------------- +%% Testcases. +%% ------------------------------------------------------------------- +from_mnesia_to_khepri(Config) -> + %% 1) Ensure there is at least one entry on each Mnesia table + %% 2) Enable the Khepri feature flag + %% 3) Check that all listings return the same values than before the migration + + %% 1) + ok = rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_vhost, add, [<<"test">>, none]), + ok = rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_auth_backend_internal, set_topic_permissions, + [<<"guest">>, <<"/">>, <<"amq.topic">>, "^t", "^t", <<"acting-user">>]), + ok = rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_policy, set, + [<<"/">>, <<"policy">>, <<".*">>, [{<<"max-length">>, 100}], 0, <<"queues">>, none]), + ok = rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_runtime_parameters, set_global, + [<<"test-global-rt">>, <<"good">>, none]), + + Ch = rabbit_ct_client_helpers:open_channel(Config, 0), + amqp_channel:call(Ch, #'queue.declare'{queue = <<"test">>, + durable = true}), + amqp_channel:call(Ch, #'queue.declare'{queue = <<"test-transient">>, + durable = false}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.direct">>, + queue = <<"test">>, + routing_key = <<"test">>}), + #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{exchange = <<"amq.topic">>, + queue = <<"test">>, + routing_key = <<"test">>}), + rabbit_ct_client_helpers:close_channel(Ch), + + VHosts = lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_vhost, list, [])), + ?assertMatch(VHosts, lists:sort([<<"/">>, <<"test">>])), + Users = rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_auth_backend_internal, list_users, []), + ?assertMatch([_], Users), + UserPermissions = rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_auth_backend_internal, + list_user_permissions, [<<"guest">>]), + ?assertMatch([_], UserPermissions), + TopicPermissions = rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_auth_backend_internal, + list_user_topic_permissions, [<<"guest">>]), + ?assertMatch([_], TopicPermissions), + Policies = rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_policy, list, []), + ?assertMatch([_], Policies), + GlobalRuntimeParameters = lists:sort(rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_runtime_parameters, list_global, [])), + GRPNames = [proplists:get_value(name, RT) || RT <- GlobalRuntimeParameters], + ?assert(lists:member('test-global-rt', GRPNames)), + ?assert(lists:member('internal_cluster_id', GRPNames)), + Queues = lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_amqqueue, list, [])), + ?assertMatch([_, _], Queues), + Exchanges = lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_exchange, list, [])), + ?assertEqual(14, length(Exchanges)), + Bindings = lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + ?assertEqual(4, length(Bindings)), + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + Maintenance = rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_db_maintenance, get, [Server]), + ?assertNot(undefined == Maintenance), + + %% 2) + Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + ok = rabbit_ct_broker_helpers:enable_feature_flag(Config, Servers, khepri_db), + + %% 3) + VHostsK = lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_vhost, list, [])), + ?assertEqual(VHosts, VHostsK), + UsersK = rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_auth_backend_internal, list_users, []), + ?assertEqual(Users, UsersK), + UserPermissionsK = rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_auth_backend_internal, + list_user_permissions, [<<"guest">>]), + ?assertEqual(UserPermissions, UserPermissionsK), + TopicPermissionsK = rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_auth_backend_internal, + list_user_topic_permissions, [<<"guest">>]), + ?assertEqual(TopicPermissions, TopicPermissionsK), + PoliciesK = rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_policy, list, []), + ?assertEqual(Policies, PoliciesK), + GlobalRuntimeParametersK = lists:sort(rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_runtime_parameters, list_global, [])), + ?assertMatch(GlobalRuntimeParametersK, GlobalRuntimeParameters), + QueuesK = lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_amqqueue, list, [])), + ?assertEqual(Queues, QueuesK), + ExchangesK = lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_exchange, list, [])), + ?assertEqual(Exchanges, ExchangesK), + BindingsK = lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_binding, list, [<<"/">>])), + ?assertEqual(Bindings, BindingsK), + MaintenanceK = rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_db_maintenance, get, [Server]), + ?assertEqual(MaintenanceK, Maintenance), + + ok. + diff --git a/deps/rabbit/test/metadata_store_phase1_SUITE.erl b/deps/rabbit/test/metadata_store_phase1_SUITE.erl new file mode 100644 index 000000000000..33a358b40119 --- /dev/null +++ b/deps/rabbit/test/metadata_store_phase1_SUITE.erl @@ -0,0 +1,2775 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(metadata_store_phase1_SUITE). + +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-include_lib("khepri/include/khepri.hrl"). + +-include_lib("rabbit_common/include/rabbit.hrl"). + +-export([suite/0, + all/0, + groups/0, + init_per_suite/1, end_per_suite/1, + init_per_group/2, end_per_group/2, + init_per_testcase/2, end_per_testcase/2, + + write_non_existing_vhost/1, + write_existing_vhost/1, + check_vhost_exists/1, + list_vhost_names/1, + list_vhost_objects/1, + update_non_existing_vhost/1, + update_existing_vhost/1, + update_non_existing_vhost_desc_and_tags/1, + update_existing_vhost_desc_and_tags/1, + delete_non_existing_vhost/1, + delete_existing_vhost/1, + + write_non_existing_user/1, + write_existing_user/1, + list_users/1, + update_non_existing_user/1, + update_existing_user/1, + delete_non_existing_user/1, + delete_existing_user/1, + + write_user_permission_for_non_existing_vhost/1, + write_user_permission_for_non_existing_user/1, + write_user_permission_for_existing_user/1, + check_resource_access/1, + list_user_permissions_on_non_existing_vhost/1, + list_user_permissions_for_non_existing_user/1, + list_user_permissions/1, + clear_user_permission_for_non_existing_vhost/1, + clear_user_permission_for_non_existing_user/1, + clear_user_permission/1, + delete_user_and_check_resource_access/1, + delete_vhost_and_check_resource_access/1, + + write_topic_permission_for_non_existing_vhost/1, + write_topic_permission_for_non_existing_user/1, + write_topic_permission_for_existing_user/1, + list_topic_permissions_on_non_existing_vhost/1, + list_topic_permissions_for_non_existing_user/1, + list_topic_permissions/1, + clear_specific_topic_permission_for_non_existing_vhost/1, + clear_specific_topic_permission_for_non_existing_user/1, + clear_specific_topic_permission/1, + clear_all_topic_permission_for_non_existing_vhost/1, + clear_all_topic_permission_for_non_existing_user/1, + clear_all_topic_permissions/1, + delete_user_and_check_topic_access/1, + delete_vhost_and_check_topic_access/1 + ]). + +suite() -> + [{timetrap, {minutes, 1}}]. + +all() -> + [ + {group, vhosts}, + {group, internal_users} + ]. + +groups() -> + [ + {vhosts, [], + [ + write_non_existing_vhost, + write_existing_vhost, + check_vhost_exists, + list_vhost_names, + list_vhost_objects, + update_non_existing_vhost, + update_existing_vhost, + update_non_existing_vhost_desc_and_tags, + update_existing_vhost_desc_and_tags, + delete_non_existing_vhost, + delete_existing_vhost + ] + }, + {internal_users, [], + [ + {users, [], + [ + write_non_existing_user, + write_existing_user, + list_users, + update_non_existing_user, + update_existing_user, + delete_non_existing_user, + delete_existing_user + ] + }, + {user_permissions, [], + [ + write_user_permission_for_non_existing_vhost, + write_user_permission_for_non_existing_user, + write_user_permission_for_existing_user, + check_resource_access, + list_user_permissions_on_non_existing_vhost, + list_user_permissions_for_non_existing_user, + list_user_permissions, + clear_user_permission_for_non_existing_vhost, + clear_user_permission_for_non_existing_user, + clear_user_permission, + delete_user_and_check_resource_access, + delete_vhost_and_check_resource_access + ] + }, + {topic_permissions, [], + [ + write_topic_permission_for_non_existing_vhost, + write_topic_permission_for_non_existing_user, + write_topic_permission_for_existing_user, + list_topic_permissions_on_non_existing_vhost, + list_topic_permissions_for_non_existing_user, + list_topic_permissions, + clear_specific_topic_permission_for_non_existing_vhost, + clear_specific_topic_permission_for_non_existing_user, + clear_specific_topic_permission, + clear_all_topic_permission_for_non_existing_vhost, + clear_all_topic_permission_for_non_existing_user, + clear_all_topic_permissions, + delete_user_and_check_topic_access, + delete_vhost_and_check_topic_access + ] + } + ] + } + ]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:run_setup_steps( + Config, + [ + fun init_feature_flags/1, + fun setup_code_mocking/1, + fun setup_mnesia/1, + fun setup_khepri/1 + ]). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps( + Config, + [ + fun remove_code_mocking/1 + ]). + +setup_mnesia(Config) -> + %% Configure Mnesia directory in the common_test priv_dir and start it. + MnesiaDir = filename:join( + ?config(priv_dir, Config), + "mnesia"), + ct:pal("Mnesia directory: ~ts", [MnesiaDir]), + ok = file:make_dir(MnesiaDir), + ok = application:load(mnesia), + ok = application:set_env(mnesia, dir, MnesiaDir), + ok = application:set_env(rabbit, data_dir, MnesiaDir), + ok = mnesia:create_schema([node()]), + {ok, _} = application:ensure_all_started(mnesia), + + ct:pal("Mnesia info below:"), + mnesia:info(), + Config. + +setup_khepri(Config) -> + %% Start Khepri. + {ok, _} = application:ensure_all_started(khepri), + + %% Configure Khepri. It takes care of configuring Ra system & cluster. It + %% uses the Mnesia directory to store files. + ok = rabbit_khepri:setup(undefined), + + ct:pal("Khepri info below:"), + rabbit_khepri:info(), + Config. + +setup_code_mocking(Config) -> + %% Bypass rabbit_mnesia:execute_mnesia_transaction/1 (no worker_pool + %% configured in particular) but keep the behavior of throwing the error. + meck:new(rabbit_mnesia, [passthrough, no_link]), + meck:expect( + rabbit_mnesia, execute_mnesia_transaction, + fun(Fun) -> + case mnesia:sync_transaction(Fun) of + {atomic, Result} -> Result; + {aborted, Reason} -> throw({error, Reason}) + end + end), + ?assert(meck:validate(rabbit_mnesia)), + + %% Bypass calls inside rabbit_vhost:vhost_cluster_state/1 because these + %% are unit testcases without any sort of clustering. + meck:new(rabbit_nodes, [passthrough, no_link]), + meck:expect( + rabbit_nodes, all_running, + fun() -> [node()] end), + + meck:new(rabbit_vhost_sup_sup, [passthrough, no_link]), + meck:expect( + rabbit_vhost_sup_sup, is_vhost_alive, + fun(_) -> true end), + + %% We ensure that we use the `vhost_v2` #vhost{} record so we can play + %% with the description and tags. + meck:new(rabbit_feature_flags, [passthrough, no_link]), + meck:expect( + rabbit_feature_flags, is_enabled, + fun + (virtual_host_metadata) -> true; + (FeatureNames) -> meck:passthrough([FeatureNames]) + end), + + ct:pal("Mocked: ~p", [meck:mocked()]), + Config. + +remove_code_mocking(Config) -> + lists:foreach( + fun(Mod) -> meck:unload(Mod) end, + meck:mocked()), + Config. + +init_per_group(_, Config) -> + Config. + +end_per_group(_, Config) -> + Config. + +init_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_started(Config, Testcase), + + rabbit_khepri:clear_forced_metadata_store(), + + %% Create Mnesia tables. + TableDefs = rabbit_table:pre_khepri_definitions(), + lists:foreach( + fun ({Table, Def}) -> ok = rabbit_table:create(Table, Def) end, + TableDefs), + + Config. + +end_per_testcase(Testcase, Config) -> + rabbit_khepri:clear_forced_metadata_store(), + + %% Delete Mnesia tables to clear any data. + TableDefs = rabbit_table:pre_khepri_definitions(), + lists:foreach( + fun ({Table, _}) -> {atomic, ok} = mnesia:delete_table(Table) end, + TableDefs), + + %% Clear all data in Khepri. + ok = rabbit_khepri:clear_store(), + + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +init_feature_flags(Config) -> + FFFile = filename:join( + ?config(priv_dir, Config), + "feature_flags"), + ct:pal("Feature flags file: ~ts", [FFFile]), + ok = application:load(rabbit), + ok = application:set_env(rabbit, feature_flags_file, FFFile), + Config. + +%% ------------------------------------------------------------------- +%% Testcases. +%% ------------------------------------------------------------------- + +%% We use `_With' (with the leading underscore) on purpose: we don't know if +%% the code in `T' will use it. That code can still use `_With' of course. +%% This simply avoids compiler warnings. +-define(with(T), fun(_With) -> T end). + +-define(vhost_path(V), + [rabbit_db_vhost, V]). +-define(user_path(U), + [rabbit_db_user, users, U]). +-define(user_perm_path(U, V), + [rabbit_db_user, users, U, user_permissions, V]). +-define(topic_perm_path(U, V, E), + [rabbit_db_user, users, U, topic_permissions, V, E]). + +%% +%% Virtual hosts. +%% + +write_non_existing_vhost(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + + Tests = + [ + ?with(?assertEqual( + undefined, + lookup_vhost(_With, VHostName))), + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + VHost, + lookup_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostName) => VHost}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +write_existing_vhost(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + + Tests = + [ + ?with(?assertEqual( + undefined, + lookup_vhost(_With, VHostName))), + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + VHost, + lookup_vhost(_With, VHostName))), + ?with(?assertEqual( + {existing, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + VHost, + lookup_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostName) => VHost}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +check_vhost_exists(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assert( + vhost_exists(_With, VHostName))), + ?with(?assertNot( + vhost_exists(_With, <<"non-existing-vhost">>))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostName) => VHost}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_vhost_names(_) -> + VHostNameA = <<"vhost-a">>, + VHostDescA = <<>>, + VHostTagsA = [], + VHostA = vhost:new( + VHostNameA, + VHostTagsA, + #{description => VHostDescA, + tags => VHostTagsA}), + VHostNameB = <<"vhost-b">>, + VHostDescB = <<>>, + VHostTagsB = [], + VHostB = vhost:new( + VHostNameB, + VHostTagsB, + #{description => VHostDescB, + tags => VHostTagsB}), + + Tests = + [ + ?with(?assertEqual( + {new, VHostA}, + add_vhost(_With, VHostNameA, VHostA))), + ?with(?assertEqual( + {new, VHostB}, + add_vhost(_With, VHostNameB, VHostB))), + ?with(?assertEqual( + [VHostNameA, VHostNameB], + list_vhosts(_With))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHostA, VHostB]}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostNameA) => VHostA, + ?vhost_path(VHostNameB) => VHostB}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_vhost_objects(_) -> + VHostNameA = <<"vhost-a">>, + VHostDescA = <<>>, + VHostTagsA = [], + VHostA = vhost:new( + VHostNameA, + VHostTagsA, + #{description => VHostDescA, + tags => VHostTagsA}), + VHostNameB = <<"vhost-b">>, + VHostDescB = <<>>, + VHostTagsB = [], + VHostB = vhost:new( + VHostNameB, + VHostTagsB, + #{description => VHostDescB, + tags => VHostTagsB}), + + Tests = + [ + ?with(?assertEqual( + {new, VHostA}, + add_vhost(_With, VHostNameA, VHostA))), + ?with(?assertEqual( + {new, VHostB}, + add_vhost(_With, VHostNameB, VHostB))), + ?with(?assertEqual( + [VHostA, VHostB], + list_vhost_records(_With))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHostA, VHostB]}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostNameA) => VHostA, + ?vhost_path(VHostNameB) => VHostB}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +update_non_existing_vhost(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + UpdatedVHost = vhost:set_limits(VHost, [limits]), + Fun = fun(_) -> UpdatedVHost end, + ?assertNotEqual(VHost, UpdatedVHost), + + Tests = + [ + ?with(?assertEqual( + undefined, + lookup_vhost(_With, VHostName))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + update_vhost(_With, VHostName, Fun))), + ?with(?assertEqual( + undefined, + lookup_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {khepri, [rabbit_db_vhost], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +update_existing_vhost(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + UpdatedVHost = vhost:set_limits(VHost, [limits]), + Fun = fun(_) -> UpdatedVHost end, + ?assertNotEqual(VHost, UpdatedVHost), + + Tests = + [ + ?with(?assertEqual( + undefined, + lookup_vhost(_With, VHostName))), + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + UpdatedVHost, + update_vhost(_With, VHostName, Fun))), + ?with(?assertEqual( + UpdatedVHost, + lookup_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [UpdatedVHost]}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostName) => UpdatedVHost}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +update_non_existing_vhost_desc_and_tags(_) -> + VHostName = <<"vhost">>, + NewVHostDesc = <<"New desc">>, + NewVHostTags = [new_tag], + + Tests = + [ + ?with(?assertEqual( + undefined, + lookup_vhost(_With, VHostName))), + ?with(?assertEqual( + {error, {no_such_vhost, VHostName}}, + update_vhost(_With, VHostName, NewVHostDesc, NewVHostTags))), + ?with(?assertEqual( + undefined, + lookup_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {khepri, [rabbit_db_vhost], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +update_existing_vhost_desc_and_tags(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + NewVHostDesc = <<"New desc">>, + NewVHostTags = [new_tag], + UpdatedVHost = vhost:set_metadata( + VHost, + #{description => NewVHostDesc, + tags => NewVHostTags}), + ct:pal("VHost: ~p~nUpdatedVHost: ~p", [VHost, UpdatedVHost]), + ?assertNotEqual(VHost, UpdatedVHost), + + Tests = + [ + ?with(?assertEqual( + undefined, + lookup_vhost(_With, VHostName))), + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + {ok, UpdatedVHost}, + update_vhost(_With, VHostName, NewVHostDesc, NewVHostTags))), + ?with(?assertEqual( + UpdatedVHost, + lookup_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [UpdatedVHost]}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostName) => UpdatedVHost}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +delete_non_existing_vhost(_) -> + VHostName = <<"vhost">>, + + Tests = + [ + ?with(?assertEqual( + undefined, + lookup_vhost(_With, VHostName))), + ?with(?assertEqual( + false, + delete_vhost(_With, VHostName))), + ?with(?assertEqual( + undefined, + lookup_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {khepri, [rabbit_db_vhost], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +delete_existing_vhost(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + VHost, + lookup_vhost(_With, VHostName))), + ?with(?assertEqual( + true, + delete_vhost(_With, VHostName))), + ?with(?assertEqual( + undefined, + lookup_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {khepri, [rabbit_db_vhost], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +%% +%% Users. +%% + +write_non_existing_user(_) -> + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + + Tests = + [ + ?with(?assertEqual( + undefined, + lookup_user(_With, Username))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + User, + lookup_user(_With, Username))), + ?with(check_storage( + _With, + [{mnesia, rabbit_user, [User]}, + {khepri, [rabbit_db_user], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +write_existing_user(_) -> + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + + Tests = + [ + ?with(?assertEqual( + undefined, + lookup_user(_With, Username))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + User, + lookup_user(_With, Username))), + ?with(?assertThrow( + {error, {user_already_exists, Username}}, + add_user(_With, Username, User))), + ?with(?assertEqual( + User, + lookup_user(_With, Username))), + ?with(check_storage( + _With, + [{mnesia, rabbit_user, [User]}, + {khepri, [rabbit_db_user], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_users(_) -> + UsernameA = <<"alice">>, + UserA = internal_user:create_user(UsernameA, <<"password">>, undefined), + UsernameB = <<"bob">>, + UserB = internal_user:create_user(UsernameB, <<"password">>, undefined), + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, UsernameA, UserA))), + ?with(?assertEqual( + ok, + add_user(_With, UsernameB, UserB))), + ?with(?assertEqual( + [UserA, UserB], + list_user_records(_With))), + ?with(check_storage( + _With, + [{mnesia, rabbit_user, [UserA, UserB]}, + {khepri, [rabbit_db_user], + #{?user_path(UsernameA) => UserA, + ?user_path(UsernameB) => UserB}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +update_non_existing_user(_) -> + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + UpdatedUser = internal_user:set_password_hash( + User, <<"updated-pw">>, undefined), + Fun = fun(_) -> UpdatedUser end, + ?assertNotEqual(User, UpdatedUser), + + Tests = + [ + ?with(?assertEqual( + undefined, + lookup_user(_With, Username))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + update_user(_With, Username, Fun))), + ?with(?assertEqual( + undefined, + lookup_user(_With, Username))), + ?with(check_storage( + _With, + [{mnesia, rabbit_user, []}, + {khepri, [rabbit_db_user], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +update_existing_user(_) -> + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + UpdatedUser = internal_user:set_password_hash( + User, <<"updated-pw">>, undefined), + Fun = fun(_) -> UpdatedUser end, + ?assertNotEqual(User, UpdatedUser), + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + User, + lookup_user(_With, Username))), + ?with(?assertEqual( + ok, + update_user(_With, Username, Fun))), + ?with(?assertEqual( + UpdatedUser, + lookup_user(_With, Username))), + ?with(check_storage( + _With, + [{mnesia, rabbit_user, [UpdatedUser]}, + {khepri, [rabbit_db_user], + #{?user_path(Username) => UpdatedUser}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +delete_non_existing_user(_) -> + Username = <<"alice">>, + + Tests = + [ + ?with(?assertEqual( + undefined, + lookup_user(_With, Username))), + ?with(?assertEqual( + false, + delete_user(_With, Username))), + ?with(?assertEqual( + undefined, + lookup_user(_With, Username))), + ?with(check_storage( + _With, + [{mnesia, rabbit_user, []}, + {khepri, [rabbit_db_user], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +delete_existing_user(_) -> + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + User, + lookup_user(_With, Username))), + ?with(?assertEqual( + true, + delete_user(_With, Username))), + ?with(?assertEqual( + undefined, + lookup_user(_With, Username))), + ?with(check_storage( + _With, + [{mnesia, rabbit_user, []}, + {khepri, [rabbit_db_user], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +%% +%% User permissions. +%% + +write_user_permission_for_non_existing_vhost(_) -> + VHostName = <<"vhost">>, + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + UserPermission = #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + permission = #permission{ + configure = <<>>, + write = <<>>, + read = <<>>}}, + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + set_permissions(_With, Username, VHostName, UserPermission))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_db_vhost], + #{}}, + {khepri, [rabbit_db_user], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +write_user_permission_for_non_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + UserPermission = #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + permission = #permission{ + configure = <<>>, + write = <<>>, + read = <<>>}}, + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + set_permissions(_With, Username, VHostName, UserPermission))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_db_user], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +write_user_permission_for_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + UserPermission = #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + permission = #permission{ + configure = <<>>, + write = <<>>, + read = <<>>}}, + + Tests = + [ + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(?assertEqual( + ok, + set_permissions(_With, Username, VHostName, UserPermission))), + ?with(?assert( + check_vhost_access(_With, Username, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_user_permission, [UserPermission]}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_db_user], + #{?user_path(Username) => User, + ?user_perm_path(Username, VHostName) => UserPermission}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +check_resource_access(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + UserPermission = #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + permission = #permission{ + configure = <<"my-resource">>, + write = <<>>, + read = <<>>}}, + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + ok, + set_permissions(_With, Username, VHostName, UserPermission))), + ?with(?assert( + check_resource_access( + _With, Username, VHostName, "my-resource", configure))), + ?with(?assertNot( + check_resource_access( + _With, Username, VHostName, "my-resource", write))), + ?with(?assertNot( + check_resource_access( + _With, Username, VHostName, "other-resource", configure))) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_user_permissions_on_non_existing_vhost(_) -> + VHostName = <<"non-existing-vhost">>, + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + list_user_vhost_permissions(_With, '_', VHostName))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + list_user_vhost_permissions(_With, Username, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_db_vhost], #{}}, + {khepri, [rabbit_db_user], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_user_permissions_for_non_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"non-existing-user">>, + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + [], + list_user_vhost_permissions(_With, '_', VHostName))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + list_user_vhost_permissions(_With, Username, '_'))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + list_user_vhost_permissions(_With, Username, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_db_vhost], #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_db_user], #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_user_permissions(_) -> + VHostNameA = <<"vhost-a">>, + VHostDescA = <<>>, + VHostTagsA = [], + VHostA = vhost:new( + VHostNameA, + VHostTagsA, + #{description => VHostDescA, + tags => VHostTagsA}), + VHostNameB = <<"vhost-b">>, + VHostDescB = <<>>, + VHostTagsB = [], + VHostB = vhost:new( + VHostNameB, + VHostTagsB, + #{description => VHostDescB, + tags => VHostTagsB}), + UsernameA = <<"alice">>, + UserA = internal_user:create_user(UsernameA, <<"password">>, undefined), + UsernameB = <<"bob">>, + UserB = internal_user:create_user(UsernameB, <<"password">>, undefined), + + UserPermissionA1 = #user_permission{ + user_vhost = #user_vhost{ + username = UsernameA, + virtual_host = VHostNameA}, + permission = #permission{ + configure = <<"my-resource">>, + write = <<>>, + read = <<>>}}, + UserPermissionA2 = #user_permission{ + user_vhost = #user_vhost{ + username = UsernameA, + virtual_host = VHostNameB}, + permission = #permission{ + configure = <<"my-resource">>, + write = <<>>, + read = <<>>}}, + UserPermissionB1 = #user_permission{ + user_vhost = #user_vhost{ + username = UsernameB, + virtual_host = VHostNameA}, + permission = #permission{ + configure = <<"my-resource">>, + write = <<>>, + read = <<>>}}, + + Tests = + [ + ?with(?assertEqual( + {new, VHostA}, + add_vhost(_With, VHostNameA, VHostA))), + ?with(?assertEqual( + {new, VHostB}, + add_vhost(_With, VHostNameB, VHostB))), + ?with(?assertEqual( + ok, + add_user(_With, UsernameA, UserA))), + ?with(?assertEqual( + ok, + set_permissions( + _With, UsernameA, VHostNameA, UserPermissionA1))), + ?with(?assertEqual( + ok, + set_permissions( + _With, UsernameA, VHostNameB, UserPermissionA2))), + ?with(?assertEqual( + ok, + add_user(_With, UsernameB, UserB))), + ?with(?assertEqual( + ok, + set_permissions( + _With, UsernameB, VHostNameA, UserPermissionB1))), + ?with(?assertEqual( + [UserPermissionA1, UserPermissionA2, UserPermissionB1], + list_user_vhost_permissions(_With, '_', '_'))), + ?with(?assertEqual( + [UserPermissionA1, UserPermissionB1], + list_user_vhost_permissions(_With, '_', VHostNameA))), + ?with(?assertEqual( + [UserPermissionA1, UserPermissionA2], + list_user_vhost_permissions(_With, UsernameA, '_'))), + ?with(?assertEqual( + [UserPermissionA1], + list_user_vhost_permissions(_With, UsernameA, VHostNameA))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHostA, VHostB]}, + {mnesia, rabbit_user, [UserA, UserB]}, + {mnesia, rabbit_user_permission, [UserPermissionA1, + UserPermissionA2, + UserPermissionB1]}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostNameA) => VHostA, + ?vhost_path(VHostNameB) => VHostB}}, + {khepri, [rabbit_db_user], + #{?user_path(UsernameA) => UserA, + ?user_path(UsernameB) => UserB, + ?user_perm_path(UsernameA, VHostNameA) => + UserPermissionA1, + ?user_perm_path(UsernameA, VHostNameB) => + UserPermissionA2, + ?user_perm_path(UsernameB, VHostNameA) => + UserPermissionB1}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_user_permission_for_non_existing_vhost(_) -> + VHostName = <<"vhost">>, + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(?assertEqual( + ok, + clear_permissions(_With, Username, VHostName))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_db_vhost], + #{}}, + {khepri, [rabbit_db_user], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_user_permission_for_non_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(?assertEqual( + ok, + clear_permissions(_With, Username, VHostName))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_db_user], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_user_permission(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + UserPermission = #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + permission = #permission{ + configure = <<"my-resource">>, + write = <<>>, + read = <<>>}}, + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + ok, + set_permissions(_With, Username, VHostName, UserPermission))), + ?with(?assert( + check_resource_access( + _With, Username, VHostName, "my-resource", configure))), + ?with(?assertEqual( + ok, + clear_permissions(_With, Username, VHostName))), + ?with(?assertNot( + check_resource_access( + _With, Username, VHostName, "my-resource", configure))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_db_user], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +delete_user_and_check_resource_access(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + UserPermission = #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + permission = #permission{ + configure = <<"my-resource">>, + write = <<>>, + read = <<>>}}, + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + ok, + set_permissions(_With, Username, VHostName, UserPermission))), + ?with(?assert( + check_vhost_access(_With, Username, VHostName))), + ?with(?assert( + check_resource_access( + _With, Username, VHostName, "my-resource", configure))), + ?with(?assertEqual( + true, + delete_user(_With, Username))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(?assertNot( + check_resource_access( + _With, Username, VHostName, "my-resource", configure))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_db_user], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +delete_vhost_and_check_resource_access(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + UserPermission = #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + permission = #permission{ + configure = <<"my-resource">>, + write = <<>>, + read = <<>>}}, + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + ok, + set_permissions(_With, Username, VHostName, UserPermission))), + ?with(?assert( + check_vhost_access(_With, Username, VHostName))), + ?with(?assert( + check_resource_access( + _With, Username, VHostName, "my-resource", configure))), + ?with(?assertEqual( + true, + delete_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_user_permission, [UserPermission]}, + {khepri, [rabbit_db_vhost], + #{}}, + {khepri, [rabbit_db_user], + #{?user_path(Username) => User}}])) + ], + + %% In mnesia the permissions have to be deleted explicitly + %% Khepri permissions have a condition to automatically delete them + %% when the vhost is deleted + MnesiaTests = + [?with(?assert( + check_vhost_access(_With, Username, VHostName))), + ?with(?assert( + check_resource_access( + _With, Username, VHostName, "my-resource", configure)))], + + KhepriTests = + [?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(?assertNot( + check_resource_access( + _With, Username, VHostName, "my-resource", configure)))], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests ++ MnesiaTests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests ++ KhepriTests}]}], + [verbose])). + +%% +%% Topic permissions. +%% + +write_topic_permission_for_non_existing_vhost(_) -> + VHostName = <<"vhost">>, + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + Exchange = <<"exchange">>, + TopicPermission = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = Exchange}, + permission = #permission{ + write = <<>>, + read = <<>>} + }, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + set_topic_permissions( + _With, Username, VHostName, TopicPermission))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_topic_permission, []}, + {khepri, [rabbit_db_vhost], + #{}}, + {khepri, [rabbit_db_user], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +write_topic_permission_for_non_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + Exchange = <<"exchange">>, + TopicPermission = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = Exchange}, + permission = #permission{ + write = <<>>, + read = <<>>} + }, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + set_topic_permissions( + _With, Username, VHostName, TopicPermission))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_topic_permission, []}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_db_user], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +write_topic_permission_for_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + Exchange = <<"exchange">>, + TopicPermission = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = Exchange}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, Username, VHostName, TopicPermission))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertNot( + check_topic_access( + _With, Username, VHostName, Exchange, read, + Context#{routing_key => <<"something-else">>}))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_topic_permission, [TopicPermission]}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_db_user], + #{?user_path(Username) => User, + ?topic_perm_path(Username, VHostName, Exchange) => + TopicPermission}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_topic_permissions_on_non_existing_vhost(_) -> + VHostName = <<"non-existing-vhost">>, + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + list_topic_permissions(_With, '_', VHostName, '_'))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + list_topic_permissions( + _With, Username, VHostName, '_'))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_topic_permission, []}, + {khepri, [rabbit_db_vhost], + #{}}, + {khepri, [rabbit_db_user], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_topic_permissions_for_non_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"non-existing-user">>, + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + [], + list_topic_permissions(_With, '_', VHostName, '_'))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + list_topic_permissions(_With, Username, '_', '_'))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + list_topic_permissions(_With, Username, VHostName, '_'))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_topic_permission, []}, + {khepri, [rabbit_db_vhost], #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_db_user], #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_topic_permissions(_) -> + VHostNameA = <<"vhost-a">>, + VHostDescA = <<>>, + VHostTagsA = [], + VHostA = vhost:new( + VHostNameA, + VHostTagsA, + #{description => VHostDescA, + tags => VHostTagsA}), + VHostNameB = <<"vhost-b">>, + VHostDescB = <<>>, + VHostTagsB = [], + VHostB = vhost:new( + VHostNameB, + VHostTagsB, + #{description => VHostDescB, + tags => VHostTagsB}), + UsernameA = <<"alice">>, + UserA = internal_user:create_user(UsernameA, <<"password">>, undefined), + UsernameB = <<"bob">>, + UserB = internal_user:create_user(UsernameB, <<"password">>, undefined), + + ExchangeA = <<"exchange-a">>, + ExchangeB = <<"exchange-b">>, + TopicPermissionA1 = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = UsernameA, + virtual_host = VHostNameA}, + exchange = ExchangeA}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + TopicPermissionA2 = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = UsernameA, + virtual_host = VHostNameB}, + exchange = ExchangeB}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + TopicPermissionB1 = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = UsernameB, + virtual_host = VHostNameA}, + exchange = ExchangeA}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + + Tests = + [ + ?with(?assertEqual( + {new, VHostA}, + add_vhost(_With, VHostNameA, VHostA))), + ?with(?assertEqual( + {new, VHostB}, + add_vhost(_With, VHostNameB, VHostB))), + ?with(?assertEqual( + ok, + add_user(_With, UsernameA, UserA))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, UsernameA, VHostNameA, TopicPermissionA1))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, UsernameA, VHostNameB, TopicPermissionA2))), + ?with(?assertEqual( + ok, + add_user(_With, UsernameB, UserB))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, UsernameB, VHostNameA, TopicPermissionB1))), + ?with(?assertEqual( + [TopicPermissionA1, TopicPermissionA2, TopicPermissionB1], + list_topic_permissions(_With, '_', '_', '_'))), + ?with(?assertEqual( + [TopicPermissionA1, TopicPermissionB1], + list_topic_permissions(_With, '_', VHostNameA, '_'))), + ?with(?assertEqual( + [TopicPermissionA1, TopicPermissionA2], + list_topic_permissions(_With, UsernameA, '_', '_'))), + ?with(?assertEqual( + [TopicPermissionA1], + list_topic_permissions(_With, UsernameA, VHostNameA, '_'))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHostA, VHostB]}, + {mnesia, rabbit_user, [UserA, UserB]}, + {mnesia, rabbit_topic_permission, [TopicPermissionA1, + TopicPermissionA2, + TopicPermissionB1]}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostNameA) => VHostA, + ?vhost_path(VHostNameB) => VHostB}}, + {khepri, [rabbit_db_user], + #{?user_path(UsernameA) => UserA, + ?user_path(UsernameB) => UserB, + ?topic_perm_path(UsernameA, VHostNameA, ExchangeA) => + TopicPermissionA1, + ?topic_perm_path(UsernameA, VHostNameB, ExchangeB) => + TopicPermissionA2, + ?topic_perm_path(UsernameB, VHostNameA, ExchangeA) => + TopicPermissionB1}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_specific_topic_permission_for_non_existing_vhost(_) -> + VHostName = <<"vhost">>, + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + Exchange = <<"exchange">>, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertEqual( + ok, + clear_topic_permissions(_With, Username, VHostName, Exchange))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_db_vhost], + #{}}, + {khepri, [rabbit_db_user], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_specific_topic_permission_for_non_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + Exchange = <<"exchange">>, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertEqual( + ok, + clear_topic_permissions(_With, Username, VHostName, Exchange))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_db_user], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_specific_topic_permission(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + ExchangeA = <<"exchange-a">>, + ExchangeB = <<"exchange-b">>, + TopicPermissionA = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = ExchangeA}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + TopicPermissionB = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = ExchangeB}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, Username, VHostName, TopicPermissionA))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, Username, VHostName, TopicPermissionB))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, ExchangeA, read, Context))), + ?with(?assertNot( + check_topic_access( + _With, Username, VHostName, ExchangeA, read, + Context#{routing_key => <<"something-else">>}))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, ExchangeB, read, Context))), + ?with(?assertNot( + check_topic_access( + _With, Username, VHostName, ExchangeB, read, + Context#{routing_key => <<"something-else">>}))), + ?with(?assertEqual( + ok, + clear_topic_permissions(_With, Username, VHostName, ExchangeA))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, ExchangeA, read, Context))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, ExchangeA, read, + Context#{routing_key => <<"something-else">>}))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, ExchangeB, read, Context))), + ?with(?assertNot( + check_topic_access( + _With, Username, VHostName, ExchangeB, read, + Context#{routing_key => <<"something-else">>}))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_topic_permission, [TopicPermissionB]}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_db_user], + #{?user_path(Username) => User, + ?topic_perm_path(Username, VHostName, ExchangeB) => + TopicPermissionB}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_all_topic_permission_for_non_existing_vhost(_) -> + VHostName = <<"vhost">>, + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + Exchange = <<"exchange">>, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertEqual( + ok, + clear_topic_permissions(_With, Username, VHostName, '_'))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_db_vhost], + #{}}, + {khepri, [rabbit_db_user], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_all_topic_permission_for_non_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + Exchange = <<"exchange">>, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertEqual( + ok, + clear_topic_permissions(_With, Username, VHostName, '_'))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_db_user], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_all_topic_permissions(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + ExchangeA = <<"exchange-a">>, + ExchangeB = <<"exchange-b">>, + TopicPermissionA = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = ExchangeA}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + TopicPermissionB = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = ExchangeB}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, Username, VHostName, TopicPermissionA))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, Username, VHostName, TopicPermissionB))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, ExchangeA, read, Context))), + ?with(?assertNot( + check_topic_access( + _With, Username, VHostName, ExchangeA, read, + Context#{routing_key => <<"something-else">>}))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, ExchangeB, read, Context))), + ?with(?assertNot( + check_topic_access( + _With, Username, VHostName, ExchangeB, read, + Context#{routing_key => <<"something-else">>}))), + ?with(?assertEqual( + ok, + clear_topic_permissions(_With, Username, VHostName, '_'))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, ExchangeA, read, Context))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, ExchangeA, read, + Context#{routing_key => <<"something-else">>}))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, ExchangeB, read, Context))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, ExchangeB, read, + Context#{routing_key => <<"something-else">>}))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_topic_permission, []}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_db_user], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +delete_user_and_check_topic_access(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + Exchange = <<"exchange">>, + TopicPermission = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = Exchange}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, Username, VHostName, TopicPermission))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertNot( + check_topic_access( + _With, Username, VHostName, Exchange, read, + Context#{routing_key => <<"something-else">>}))), + ?with(?assertEqual( + true, + delete_user(_With, Username))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, Exchange, read, + Context#{routing_key => <<"something-else">>}))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_topic_permission, []}, + {khepri, [rabbit_db_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_db_user], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +delete_vhost_and_check_topic_access(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + Exchange = <<"exchange">>, + TopicPermission = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = Exchange}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + {new, VHost}, + add_vhost(_With, VHostName, VHost))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, Username, VHostName, TopicPermission))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertNot( + check_topic_access( + _With, Username, VHostName, Exchange, read, + Context#{routing_key => <<"something-else">>}))), + ?with(?assert( + delete_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_topic_permission, [TopicPermission]}, + {khepri, [rabbit_db_vhost], + #{}}, + {khepri, [rabbit_db_user], + #{?user_path(Username) => User}}])) + ], + + %% In mnesia the permissions have to be deleted explicitly + %% Khepri permissions have a condition to automatically delete them + %% when the vhost is deleted + MnesiaTests = + [?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertNot( + check_topic_access( + _With, Username, VHostName, Exchange, read, + Context#{routing_key => <<"something-else">>})))], + + KhepriTests = + [?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertEqual( + undefined, + check_topic_access( + _With, Username, VHostName, Exchange, read, + Context#{routing_key => <<"something-else">>})))], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests ++ MnesiaTests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests ++ KhepriTests}]}], + [verbose])). + +%% ------------------------------------------------------------------- +%% Helpers. +%% ------------------------------------------------------------------- + +force_mnesia_use() -> + ct:pal(?LOW_IMPORTANCE, "Using Mnesia (disabling feature flag)", []), + rabbit_khepri:force_metadata_store(mnesia). + +force_khepri_use() -> + ct:pal(?LOW_IMPORTANCE, "Using Khepri (enabling feature flag)", []), + rabbit_khepri:force_metadata_store(khepri). + +add_vhost(mnesia, VHostName, VHost) -> + rabbit_db_vhost:create_or_get_in_mnesia(VHostName, VHost); +add_vhost(khepri, VHostName, VHost) -> + rabbit_db_vhost:create_or_get_in_khepri(VHostName, VHost). + +lookup_vhost(mnesia, VHostName) -> + rabbit_db_vhost:get_in_mnesia(VHostName); +lookup_vhost(khepri, VHostName) -> + rabbit_db_vhost:get_in_khepri(VHostName). + +vhost_exists(mnesia, VHostName) -> + rabbit_db_vhost:exists_in_mnesia(VHostName); +vhost_exists(khepri, VHostName) -> + rabbit_db_vhost:exists_in_khepri(VHostName). + +list_vhosts(mnesia) -> + lists:sort(rabbit_db_vhost:list_in_mnesia()); +list_vhosts(khepri) -> + lists:sort(rabbit_db_vhost:list_in_khepri()). + +list_vhost_records(mnesia) -> + lists:sort(rabbit_db_vhost:get_all_in_mnesia()); +list_vhost_records(khepri) -> + lists:sort(rabbit_db_vhost:get_all_in_khepri()). + +update_vhost(mnesia, VHostName, Fun) -> + rabbit_db_vhost:update_in_mnesia(VHostName, Fun); +update_vhost(khepri, VHostName, Fun) -> + rabbit_db_vhost:update_in_khepri(VHostName, Fun). + +update_vhost(mnesia, VHostName, Description, Tags) -> + rabbit_db_vhost:merge_metadata_in_mnesia(VHostName, + #{description => Description, + tags => Tags}); +update_vhost(khepri, VHostName, Description, Tags) -> + rabbit_db_vhost:merge_metadata_in_khepri(VHostName, + #{description => Description, + tags => Tags}). + +delete_vhost(mnesia, VHostName) -> + rabbit_db_vhost:delete_in_mnesia(VHostName); +delete_vhost(khepri, VHostName) -> + rabbit_db_vhost:delete_in_khepri(VHostName). + +add_user(mnesia, Username, User) -> + rabbit_db_user:create_in_mnesia(Username, User); +add_user(khepri, Username, User) -> + rabbit_db_user:create_in_khepri(Username, User). + +lookup_user(mnesia, Username) -> + rabbit_db_user:get_in_mnesia(Username); +lookup_user(khepri, Username) -> + rabbit_db_user:get_in_khepri(Username). + +list_user_records(mnesia) -> + lists:sort(rabbit_db_user:get_all_in_mnesia()); +list_user_records(khepri) -> + lists:sort(rabbit_db_user:get_all_in_khepri()). + +update_user(mnesia, Username, Fun) -> + rabbit_db_user:update_in_mnesia(Username, Fun); +update_user(khepri, Username, Fun) -> + rabbit_db_user:update_in_khepri(Username, Fun). + +delete_user(mnesia, Username) -> + rabbit_db_user:delete_in_mnesia(Username); +delete_user(khepri, Username) -> + rabbit_db_user:delete_in_khepri(Username). + +set_permissions(mnesia, Username, VHostName, UserPermission) -> + rabbit_db_user:set_user_permissions_in_mnesia( + Username, VHostName, UserPermission); +set_permissions(khepri, Username, VHostName, UserPermission) -> + rabbit_db_user:set_user_permissions_in_khepri( + Username, VHostName, UserPermission). + +list_user_vhost_permissions(mnesia, Username, VHostName) -> + lists:sort( + rabbit_db_user:match_user_permissions_in_mnesia(Username, VHostName)); +list_user_vhost_permissions(khepri, Username, VHostName) -> + lists:sort( + rabbit_db_user:match_user_permissions_in_khepri(Username, VHostName)). + +list_topic_permissions(mnesia, Username, VHostName, ExchangeName) -> + lists:sort( + rabbit_db_user:match_topic_permissions_in_mnesia(Username, VHostName, ExchangeName)); +list_topic_permissions(khepri, Username, VHostName, ExchangeName) -> + lists:sort( + rabbit_db_user:match_topic_permissions_in_khepri(Username, VHostName, ExchangeName)). + +check_vhost_access(mnesia, Username, VHostName) -> + rabbit_db_user:get_user_permissions_in_mnesia( + Username, VHostName) =/= undefined; +check_vhost_access(khepri, Username, VHostName) -> + rabbit_db_user:get_user_permissions_in_khepri( + Username, VHostName) =/= undefined. + +set_topic_permissions( + mnesia, Username, VHostName, TopicPermission) -> + rabbit_db_user:set_topic_permissions_in_mnesia( + Username, VHostName, TopicPermission); +set_topic_permissions( + khepri, Username, VHostName, TopicPermission) -> + rabbit_db_user:set_topic_permissions_in_khepri( + Username, VHostName, TopicPermission). + +check_topic_access(mnesia, Username, VHostName, Exchange, Perm, Context) -> + case rabbit_db_user:get_topic_permissions_in_mnesia( + Username, VHostName, Exchange) of + undefined -> undefined; + #topic_permission{permission = P} -> + PermRegexp = case element(permission_index(Perm), P) of + <<"">> -> <<$^, $$>>; + RE -> RE + end, + case re:run(maps:get(routing_key, Context), PermRegexp, [{capture, none}]) of + match -> true; + nomatch -> false + end + end; +check_topic_access(khepri, Username, VHostName, Exchange, Perm, Context) -> + case rabbit_db_user:get_topic_permissions_in_khepri( + Username, VHostName, Exchange) of + undefined -> undefined; + #topic_permission{permission = P} -> + PermRegexp = case element(permission_index(Perm), P) of + <<"">> -> <<$^, $$>>; + RE -> RE + end, + case re:run(maps:get(routing_key, Context), PermRegexp, [{capture, none}]) of + match -> true; + nomatch -> false + end + end. + +clear_permissions(mnesia, Username, VHostName) -> + rabbit_db_user:clear_user_permissions_in_mnesia( + Username, VHostName); +clear_permissions(khepri, Username, VHostName) -> + rabbit_db_user:clear_user_permissions_in_khepri( + Username, VHostName). + +check_resource_access(mnesia, Username, VHostName, Resource, Perm) -> + case rabbit_db_user:get_user_permissions_in_mnesia(Username, VHostName) of + undefined -> false; + #user_permission{permission = P} -> + PermRegexp = case element(permission_index(Perm), P) of + <<"">> -> <<$^, $$>>; + RE -> RE + end, + case re:run(Resource, PermRegexp, [{capture, none}]) of + match -> true; + nomatch -> false + end + end; +check_resource_access(khepri, Username, VHostName, Resource, Perm) -> + case rabbit_db_user:get_user_permissions_in_khepri(Username, VHostName) of + undefined -> false; + #user_permission{permission = P} -> + PermRegexp = case element(permission_index(Perm), P) of + <<"">> -> <<$^, $$>>; + RE -> RE + end, + case re:run(Resource, PermRegexp, [{capture, none}]) of + match -> true; + nomatch -> false + end + end. + +permission_index(configure) -> #permission.configure; +permission_index(write) -> #permission.write; +permission_index(read) -> #permission.read. + +clear_topic_permissions(mnesia, Username, VHostName, Exchange) -> + rabbit_db_user:clear_topic_permissions_in_mnesia( + Username, VHostName, Exchange); +clear_topic_permissions(khepri, Username, VHostName, Exchange) -> + rabbit_db_user:clear_topic_permissions_in_khepri( + Username, VHostName, Exchange). + +check_storage(With, [{With, Source, Content} | Rest]) -> + check_storage(With, Source, Content), + check_storage(With, Rest); +check_storage(With, [_ | Rest]) -> + check_storage(With, Rest); +check_storage(_, []) -> + ok. + +check_storage(mnesia, Table, Content) -> + ?assertEqual(Content, lists:sort(ets:tab2list(Table))); +check_storage(khepri, Path, Content) -> + rabbit_khepri:info(), + Path1 = Path ++ [#if_all{conditions = [?KHEPRI_WILDCARD_STAR_STAR, + #if_has_data{has_data = true}]}], + ?assertEqual({ok, Content}, rabbit_khepri:match(Path1)). diff --git a/deps/rabbit/test/peer_discovery_classic_config_SUITE.erl b/deps/rabbit/test/peer_discovery_classic_config_SUITE.erl index df5751b2b890..beead47ef069 100644 --- a/deps/rabbit/test/peer_discovery_classic_config_SUITE.erl +++ b/deps/rabbit/test/peer_discovery_classic_config_SUITE.erl @@ -20,16 +20,29 @@ all() -> [ - {group, non_parallel} + {group, non_parallel}, + {group, cluster_size_3}, + {group, cluster_size_5}, + {group, cluster_size_7} ]. groups() -> [ {non_parallel, [], [ - successful_discovery, - successful_discovery_with_a_subset_of_nodes_coming_online, no_nodes_configured - ]} + ]}, + {cluster_size_3, [], [ + successful_discovery, + successful_discovery_with_a_subset_of_nodes_coming_online + ]}, + {cluster_size_5, [], [ + successful_discovery, + successful_discovery_with_a_subset_of_nodes_coming_online + ]}, + {cluster_size_7, [], [ + successful_discovery, + successful_discovery_with_a_subset_of_nodes_coming_online + ]} ]. suite() -> @@ -50,12 +63,23 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). +init_per_group(cluster_size_3 = Group, Config) -> + rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 3}, {group, Group}]); +init_per_group(cluster_size_5 = Group, Config) -> + rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 5}, {group, Group}]); +init_per_group(cluster_size_7 = Group, Config) -> + rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, 7}, {group, Group}]); +init_per_group(_, Config) -> + Config. + +end_per_group(_, Config) -> + Config. + init_per_testcase(successful_discovery = Testcase, Config) -> Config1 = rabbit_ct_helpers:testcase_started(Config, Testcase), - - N = 3, + N = ?config(rmq_nodes_count, Config), NodeNames = [ - list_to_atom(rabbit_misc:format("~ts-~b", [Testcase, I])) + list_to_atom(rabbit_misc:format("~ts-~ts-~b", [Testcase, ?config(group, Config), I])) || I <- lists:seq(1, N) ], Config2 = rabbit_ct_helpers:set_config(Config1, [ @@ -78,9 +102,9 @@ init_per_testcase(successful_discovery = Testcase, Config) -> init_per_testcase(successful_discovery_with_a_subset_of_nodes_coming_online = Testcase, Config) -> Config1 = rabbit_ct_helpers:testcase_started(Config, Testcase), - N = 2, + N = ?config(rmq_nodes_count, Config), NodeNames = [ - list_to_atom(rabbit_misc:format("~ts-~b", [Testcase, I])) + list_to_atom(rabbit_misc:format("~ts-~ts-~b", [Testcase, ?config(group, Config), I])) || I <- lists:seq(1, N) ], Config2 = rabbit_ct_helpers:set_config(Config1, [ @@ -89,9 +113,13 @@ init_per_testcase(successful_discovery_with_a_subset_of_nodes_coming_online = Te {rmq_nodes_count, NodeNames}, {rmq_nodes_clustered, false} ]), - NodeNamesWithHostname = [rabbit_nodes:make({Name, "localhost"}) || Name <- [nonexistent | NodeNames]], + NodeNamesWithHostname = [rabbit_nodes:make({Name, "localhost"}) || Name <- NodeNames], %% reduce retry time since we know one node on the list does %% not exist and not just unreachable + %% We no longer test non-existing nodes, it just times out + %% constantly in CI + %% To compare, this suite takes ~23min in my machine with + %% unreachable nodes vs ~6min without them Config3 = rabbit_ct_helpers:merge_app_env(Config2, {rabbit, [ {cluster_nodes, {NodeNamesWithHostname, disc}}, @@ -139,8 +167,9 @@ end_per_testcase(Testcase, Config) -> %% Test cases %% successful_discovery(Config) -> + N = length(?config(rmq_nodes_count, Config)), ?awaitMatch( - {M1, M2} when length(M1) =:= 3; length(M2) =:= 3, + {M1, M2} when length(M1) =:= N; length(M2) =:= N, {cluster_members_online(Config, 0), cluster_members_online(Config, 1)}, ?TIMEOUT). @@ -149,8 +178,9 @@ successful_discovery_with_a_subset_of_nodes_coming_online() -> [{timetrap, {minutes, 15}}]. successful_discovery_with_a_subset_of_nodes_coming_online(Config) -> + N = length(?config(rmq_nodes_count, Config)), ?awaitMatch( - {M1, M2} when length(M1) =:= 2; length(M2) =:= 2, + {M1, M2} when length(M1) =:= N; length(M2) =:= N, {cluster_members_online(Config, 0), cluster_members_online(Config, 1)}, ?TIMEOUT). diff --git a/deps/rabbit/test/per_user_connection_channel_limit_SUITE.erl b/deps/rabbit/test/per_user_connection_channel_limit_SUITE.erl index 85a8606b5ba2..23f3f201c52f 100644 --- a/deps/rabbit/test/per_user_connection_channel_limit_SUITE.erl +++ b/deps/rabbit/test/per_user_connection_channel_limit_SUITE.erl @@ -16,9 +16,9 @@ all() -> [ - {group, cluster_size_1_network}, - {group, cluster_size_2_network}, - {group, cluster_size_2_direct} + {group, mnesia_store}, + {group, khepri_store}, + {group, khepri_migration} ]. groups() -> @@ -49,9 +49,17 @@ groups() -> cluster_multiple_users_zero_limit ], [ - {cluster_size_1_network, [], ClusterSize1Tests}, - {cluster_size_2_network, [], ClusterSize2Tests}, - {cluster_size_2_direct, [], ClusterSize2Tests} + {mnesia_store, [], [ + {cluster_size_1_network, [], ClusterSize1Tests}, + {cluster_size_3_network, [], ClusterSize2Tests}, + {cluster_size_3_direct, [], ClusterSize2Tests} + ]}, + {khepri_store, [], [ + {cluster_size_1_network, [], ClusterSize1Tests}, + {cluster_size_3_network, [], ClusterSize2Tests}, + {cluster_size_3_direct, [], ClusterSize2Tests} + ]}, + {khepri_migration, [], [from_mnesia_to_khepri]} ]. suite() -> @@ -71,16 +79,23 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). +init_per_group(mnesia_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, mnesia}]); +init_per_group(khepri_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, khepri}]); +init_per_group(khepri_migration, Config) -> + Config1 = rabbit_ct_helpers:set_config(Config, [{connection_type, network}, + {metadata_store, mnesia}]), + init_per_multinode_group(cluster_size_1_network, Config1, 1); init_per_group(cluster_size_1_network, Config) -> Config1 = rabbit_ct_helpers:set_config(Config, [{connection_type, network}]), init_per_multinode_group(cluster_size_1_network, Config1, 1); -init_per_group(cluster_size_2_network, Config) -> +init_per_group(cluster_size_3_network, Config) -> Config1 = rabbit_ct_helpers:set_config(Config, [{connection_type, network}]), - init_per_multinode_group(cluster_size_2_network, Config1, 2); -init_per_group(cluster_size_2_direct, Config) -> + init_per_multinode_group(cluster_size_3_network, Config1, 3); +init_per_group(cluster_size_3_direct, Config) -> Config1 = rabbit_ct_helpers:set_config(Config, [{connection_type, direct}]), - init_per_multinode_group(cluster_size_2_direct, Config1, 2); - + init_per_multinode_group(cluster_size_3_direct, Config1, 3); init_per_group(cluster_rename, Config) -> init_per_multinode_group(cluster_rename, Config, 2). @@ -100,7 +115,7 @@ init_per_multinode_group(Group, Config, NodeCount) -> rabbit_ct_client_helpers:setup_steps()) end. -end_per_group(cluster_rename, Config) -> +end_per_group(Group, Config) when Group == mnesia_store; Group == khepri_store -> % The broker is managed by {init,end}_per_testcase(). Config; end_per_group(_Group, Config) -> @@ -670,7 +685,7 @@ cluster_node_restart_connection_and_channel_count(Config) -> end). cluster_node_list_on_node(Config) -> - [A, B] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + [A, B, _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), rabbit_ct_helpers:await_condition( fun () -> @@ -1459,6 +1474,33 @@ cluster_multiple_users_zero_limit(Config) -> set_user_connection_and_channel_limit(Config, Username1, -1, -1), set_user_connection_and_channel_limit(Config, Username2, -1, -1). +from_mnesia_to_khepri(Config) -> + Username = proplists:get_value(rmq_username, Config), + rabbit_ct_helpers:await_condition( + fun () -> + count_connections_of_user(Config, Username) =:= 0 andalso + count_channels_of_user(Config, Username) =:= 0 + end), + + [Conn] = open_connections(Config, [0]), + [_Chan] = open_channels(Conn, 1), + + rabbit_ct_helpers:await_condition( + fun () -> + count_connections_of_user(Config, Username) =:= 1 andalso + count_channels_of_user(Config, Username) =:= 1 + end), + case rabbit_ct_broker_helpers:enable_feature_flag(Config, khepri_db) of + ok -> + rabbit_ct_helpers:await_condition( + fun () -> + count_connections_of_user(Config, Username) =:= 1 andalso + count_channels_of_user(Config, Username) =:= 1 + end); + Skip -> + Skip + end. + %% ------------------------------------------------------------------- %% Helpers %% ------------------------------------------------------------------- diff --git a/deps/rabbit/test/per_user_connection_channel_limit_partitions_SUITE.erl b/deps/rabbit/test/per_user_connection_channel_limit_partitions_SUITE.erl index f542f0cc50d7..25fe73691886 100644 --- a/deps/rabbit/test/per_user_connection_channel_limit_partitions_SUITE.erl +++ b/deps/rabbit/test/per_user_connection_channel_limit_partitions_SUITE.erl @@ -124,8 +124,15 @@ cluster_full_partition_with_autoheal(Config) -> ?awaitMatch(All, list_running(Config, B), 60000, 3000), ?awaitMatch(All, list_running(Config, C), 60000, 3000), - %% during autoheal B's connections were dropped - ?awaitMatch({4, 10}, + %% During autoheal B's connections were dropped. Autoheal is not running + %% when Khepri is used. + KhepriEnabled = rabbit_ct_broker_helpers:is_feature_flag_enabled( + Config, khepri_db), + ExpectedCount = case KhepriEnabled of + true -> {6, 15}; + false -> {4, 10} + end, + ?awaitMatch(ExpectedCount, {count_connections_in(Config, Username), count_channels_in(Config, Username)}, 60000, 3000), diff --git a/deps/rabbit/test/per_user_connection_channel_tracking_SUITE.erl b/deps/rabbit/test/per_user_connection_channel_tracking_SUITE.erl index 4c4b1f7af7f8..26885009482e 100644 --- a/deps/rabbit/test/per_user_connection_channel_tracking_SUITE.erl +++ b/deps/rabbit/test/per_user_connection_channel_tracking_SUITE.erl @@ -18,10 +18,8 @@ all() -> [ - {group, cluster_size_1_network}, - {group, cluster_size_2_network}, - {group, cluster_size_1_direct}, - {group, cluster_size_2_direct} + {group, mnesia_store}, + {group, khepri_store} ]. groups() -> @@ -31,17 +29,25 @@ groups() -> single_node_vhost_down_mimic, single_node_vhost_deletion ], - ClusterSize2Tests = [ + ClusterSize3Tests = [ cluster_user_deletion, cluster_vhost_down_mimic, cluster_vhost_deletion, cluster_node_removed ], [ - {cluster_size_1_network, [], ClusterSize1Tests}, - {cluster_size_2_network, [], ClusterSize2Tests}, - {cluster_size_1_direct, [], ClusterSize1Tests}, - {cluster_size_2_direct, [], ClusterSize2Tests} + {mnesia_store, [], [ + {cluster_size_1_network, [], ClusterSize1Tests}, + {cluster_size_3_network, [], ClusterSize3Tests}, + {cluster_size_1_direct, [], ClusterSize1Tests}, + {cluster_size_3_direct, [], ClusterSize3Tests} + ]}, + {khepri_store, [], [ + {cluster_size_1_network, [], ClusterSize1Tests}, + {cluster_size_3_network, [], ClusterSize3Tests}, + {cluster_size_1_direct, [], ClusterSize1Tests}, + {cluster_size_3_direct, [], ClusterSize3Tests} + ]} ]. suite() -> @@ -61,18 +67,22 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). +init_per_group(mnesia_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, mnesia}]); +init_per_group(khepri_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, khepri}]); init_per_group(cluster_size_1_network, Config) -> Config1 = rabbit_ct_helpers:set_config(Config, [{connection_type, network}]), init_per_multinode_group(cluster_size_1_network, Config1, 1); -init_per_group(cluster_size_2_network, Config) -> +init_per_group(cluster_size_3_network, Config) -> Config1 = rabbit_ct_helpers:set_config(Config, [{connection_type, network}]), - init_per_multinode_group(cluster_size_2_network, Config1, 2); + init_per_multinode_group(cluster_size_3_network, Config1, 3); init_per_group(cluster_size_1_direct, Config) -> Config1 = rabbit_ct_helpers:set_config(Config, [{connection_type, direct}]), init_per_multinode_group(cluster_size_1_direct, Config1, 1); -init_per_group(cluster_size_2_direct, Config) -> +init_per_group(cluster_size_3_direct, Config) -> Config1 = rabbit_ct_helpers:set_config(Config, [{connection_type, direct}]), - init_per_multinode_group(cluster_size_2_direct, Config1, 2). + init_per_multinode_group(cluster_size_3_direct, Config1, 3). init_per_multinode_group(_Group, Config, NodeCount) -> Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"), @@ -84,6 +94,9 @@ init_per_multinode_group(_Group, Config, NodeCount) -> Config1, rabbit_ct_broker_helpers:setup_steps() ++ rabbit_ct_client_helpers:setup_steps()). +end_per_group(Group, Config) when Group == mnesia_store; Group == khepri_store -> + % The broker is managed by {init,end}_per_testcase(). + Config; end_per_group(_Group, Config) -> rabbit_ct_helpers:run_steps(Config, rabbit_ct_client_helpers:teardown_steps() ++ diff --git a/deps/rabbit/test/per_user_connection_tracking_SUITE.erl b/deps/rabbit/test/per_user_connection_tracking_SUITE.erl index e62ce95b837b..a2a3f1475f83 100644 --- a/deps/rabbit/test/per_user_connection_tracking_SUITE.erl +++ b/deps/rabbit/test/per_user_connection_tracking_SUITE.erl @@ -18,10 +18,8 @@ all() -> [ - {group, cluster_size_1_network}, - {group, cluster_size_2_network}, - {group, cluster_size_1_direct}, - {group, cluster_size_2_direct} + {group, mnesia_store}, + {group, khepri_store} ]. groups() -> @@ -33,10 +31,18 @@ groups() -> cluster_user_deletion_forces_connection_closure ], [ - {cluster_size_1_network, [], ClusterSize1Tests}, - {cluster_size_2_network, [], ClusterSize2Tests}, - {cluster_size_1_direct, [], ClusterSize1Tests}, - {cluster_size_2_direct, [], ClusterSize2Tests} + {mnesia_store, [], [ + {cluster_size_1_network, [], ClusterSize1Tests}, + {cluster_size_2_network, [], ClusterSize2Tests}, + {cluster_size_1_direct, [], ClusterSize1Tests}, + {cluster_size_2_direct, [], ClusterSize2Tests} + ]}, + {khepri_store, [], [ + {cluster_size_1_network, [], ClusterSize1Tests}, + {cluster_size_2_network, [], ClusterSize2Tests}, + {cluster_size_1_direct, [], ClusterSize1Tests}, + {cluster_size_2_direct, [], ClusterSize2Tests} + ]} ]. suite() -> @@ -56,6 +62,10 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). +init_per_group(mnesia_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, mnesia}]); +init_per_group(khepri_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, khepri}]); init_per_group(cluster_size_1_network, Config) -> Config1 = rabbit_ct_helpers:set_config(Config, [{connection_type, network}]), init_per_multinode_group(cluster_size_1_network, Config1, 1); @@ -79,6 +89,9 @@ init_per_multinode_group(_Group, Config, NodeCount) -> rabbit_ct_broker_helpers:setup_steps() ++ rabbit_ct_client_helpers:setup_steps()). +end_per_group(Group, Config) when Group == mnesia_store; Group == khepri_store -> + % The broker is managed by {init,end}_per_testcase(). + Config; end_per_group(_Group, Config) -> rabbit_ct_helpers:run_steps(Config, rabbit_ct_client_helpers:teardown_steps() ++ diff --git a/deps/rabbit/test/per_vhost_connection_limit_SUITE.erl b/deps/rabbit/test/per_vhost_connection_limit_SUITE.erl index 89dd01fc0cf8..c3b2b2823253 100644 --- a/deps/rabbit/test/per_vhost_connection_limit_SUITE.erl +++ b/deps/rabbit/test/per_vhost_connection_limit_SUITE.erl @@ -16,10 +16,9 @@ all() -> [ - {group, cluster_size_1_network}, - {group, cluster_size_2_network}, - {group, cluster_size_1_direct}, - {group, cluster_size_2_direct} + {group, mnesia_store}, + {group, khepri_store}, + {group, khepri_migration} ]. groups() -> @@ -33,7 +32,9 @@ groups() -> single_node_multiple_vhosts_limit, single_node_multiple_vhosts_zero_limit ], - ClusterSize2Tests = [ + %% Use a cluster size of 3 so the khepri metadata store can keep + %% making progress even if one node is down/stopped + ClusterSize3Tests = [ most_basic_cluster_connection_count, cluster_single_vhost_connection_count, cluster_multiple_vhosts_connection_count, @@ -45,13 +46,22 @@ groups() -> cluster_multiple_vhosts_zero_limit ], [ - {cluster_size_1_network, [], ClusterSize1Tests}, - {cluster_size_2_network, [], ClusterSize2Tests}, - {cluster_size_1_direct, [], ClusterSize1Tests}, - {cluster_size_2_direct, [], ClusterSize2Tests}, - {cluster_rename, [], [ - vhost_limit_after_node_renamed - ]} + {mnesia_store, [], [ + {cluster_size_1_network, [], ClusterSize1Tests}, + {cluster_size_3_network, [], ClusterSize3Tests}, + {cluster_size_1_direct, [], ClusterSize1Tests}, + {cluster_size_3_direct, [], ClusterSize3Tests}, + {cluster_rename, [], [ + vhost_limit_after_node_renamed + ]} + ]}, + {khepri_store, [], [ + {cluster_size_1_network, [], ClusterSize1Tests}, + {cluster_size_3_network, [], ClusterSize3Tests}, + {cluster_size_1_direct, [], ClusterSize1Tests}, + {cluster_size_3_direct, [], ClusterSize3Tests} + ]}, + {khepri_migration, [], [from_mnesia_to_khepri]} ]. suite() -> @@ -76,21 +86,30 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). +init_per_group(mnesia_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, mnesia}]); +init_per_group(khepri_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, khepri}]); +init_per_group(khepri_migration, Config) -> + Config1 = rabbit_ct_helpers:set_config(Config, [{connection_type, network}, + {metadata_store, mnesia}]), + init_per_multinode_group(cluster_size_1_network, Config1, 1); init_per_group(cluster_size_1_network, Config) -> Config1 = rabbit_ct_helpers:set_config(Config, [{connection_type, network}]), init_per_multinode_group(cluster_size_1_network, Config1, 1); -init_per_group(cluster_size_2_network, Config) -> +init_per_group(cluster_size_3_network, Config) -> Config1 = rabbit_ct_helpers:set_config(Config, [{connection_type, network}]), - init_per_multinode_group(cluster_size_2_network, Config1, 2); + init_per_multinode_group(cluster_size_3_network, Config1, 3); init_per_group(cluster_size_1_direct, Config) -> Config1 = rabbit_ct_helpers:set_config(Config, [{connection_type, direct}]), init_per_multinode_group(cluster_size_1_direct, Config1, 1); -init_per_group(cluster_size_2_direct, Config) -> +init_per_group(cluster_size_3_direct, Config) -> Config1 = rabbit_ct_helpers:set_config(Config, [{connection_type, direct}]), - init_per_multinode_group(cluster_size_2_direct, Config1, 2); + init_per_multinode_group(cluster_size_3_direct, Config1, 3); init_per_group(cluster_rename, Config) -> - init_per_multinode_group(cluster_rename, Config, 2). + Config1 = rabbit_ct_helpers:set_config(Config, [{connection_type, direct}]), + init_per_multinode_group(cluster_rename, Config1, 2). init_per_multinode_group(Group, Config, NodeCount) -> Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"), @@ -108,7 +127,8 @@ init_per_multinode_group(Group, Config, NodeCount) -> rabbit_ct_client_helpers:setup_steps()) end. -end_per_group(cluster_rename, Config) -> +end_per_group(Group, Config) when Group == cluster_rename; Group == mnesia_store; + Group == khepri_store; Group == khepri_migration -> % The broker is managed by {init,end}_per_testcase(). Config; end_per_group(_Group, Config) -> @@ -376,7 +396,7 @@ cluster_node_restart_connection_count(Config) -> ?awaitMatch(0, count_connections_in(Config, VHost), ?AWAIT, ?INTERVAL). cluster_node_list_on_node(Config) -> - [A, B] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + [A, B, _C] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), ?assertEqual(0, length(all_connections(Config))), ?assertEqual(0, length(connections_on_node(Config, 0))), @@ -683,16 +703,28 @@ vhost_limit_after_node_renamed(Config) -> ?awaitMatch(0, count_connections_in(Config1, VHost), ?AWAIT, ?INTERVAL), - [Conn3, Conn4, {error, not_allowed}] = open_connections(Config1, - [{0, VHost}, {1, VHost}, {0, VHost}]), + [Conn3, Conn4] = open_connections(Config1, [{0, VHost}, {1, VHost}]), + ?awaitMatch(2, count_connections_in(Config1, VHost), ?AWAIT, ?INTERVAL), + [{error, not_allowed}, {error, not_allowed}] + = open_connections(Config1, [{0, VHost}, {1, VHost}]), ?awaitMatch(2, count_connections_in(Config1, VHost), ?AWAIT, ?INTERVAL), close_connections([Conn3, Conn4]), set_vhost_connection_limit(Config1, VHost, -1), {save_config, Config1}. -%% ------------------------------------------------------------------- -%% Helpers +from_mnesia_to_khepri(Config) -> + VHost = <<"/">>, + ?assertEqual(0, count_connections_in(Config, VHost)), + [_Conn] = open_connections(Config, [{0, VHost}]), + ?awaitMatch(1, count_connections_in(Config, VHost), ?AWAIT, ?INTERVAL), + case rabbit_ct_broker_helpers:enable_feature_flag(Config, khepri_db) of + ok -> + ?awaitMatch(1, count_connections_in(Config, VHost), ?AWAIT, ?INTERVAL); + Skip -> + Skip + end. + %% ------------------------------------------------------------------- open_connections(Config, NodesAndVHosts) -> diff --git a/deps/rabbit/test/per_vhost_connection_limit_partitions_SUITE.erl b/deps/rabbit/test/per_vhost_connection_limit_partitions_SUITE.erl index 8d9e66b34792..36d8803bf388 100644 --- a/deps/rabbit/test/per_vhost_connection_limit_partitions_SUITE.erl +++ b/deps/rabbit/test/per_vhost_connection_limit_partitions_SUITE.erl @@ -113,8 +113,15 @@ cluster_full_partition_with_autoheal(Config) -> rabbit_ct_broker_helpers:allow_traffic_between(B, C), timer:sleep(?DELAY), - %% during autoheal B's connections were dropped - ?awaitMatch(Connections when length(Connections) == 4, + %% During autoheal B's connections were dropped. Autoheal is not running + %% when Khepri is used. + KhepriEnabled = rabbit_ct_broker_helpers:is_feature_flag_enabled( + Config, khepri_db), + ExpectedCount = case KhepriEnabled of + true -> 6; + false -> 4 + end, + ?awaitMatch(Connections when length(Connections) == ExpectedCount, connections_in(Config, VHost), 60000, 3000), diff --git a/deps/rabbit/test/per_vhost_msg_store_SUITE.erl b/deps/rabbit/test/per_vhost_msg_store_SUITE.erl index 44f00bf4cc39..ca71988c2ac8 100644 --- a/deps/rabbit/test/per_vhost_msg_store_SUITE.erl +++ b/deps/rabbit/test/per_vhost_msg_store_SUITE.erl @@ -17,31 +17,50 @@ all() -> [ - publish_to_different_dirs, - storage_deleted_on_vhost_delete, - single_vhost_storage_delete_is_safe + {group, mnesia_store}, + {group, khepri_store} ]. +groups() -> + [ + {mnesia_store, [], all_tests()}, + {khepri_store, [], all_tests()} + ]. +all_tests() -> + [publish_to_different_dirs, + storage_deleted_on_vhost_delete, + single_vhost_storage_delete_is_safe]. init_per_suite(Config) -> rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config, []). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(mnesia_store, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, khepri}]), + init_per_group_common(Config); +init_per_group(khepri_store, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, khepri}]), + init_per_group_common(Config). + +init_per_group_common(Config) -> Config1 = rabbit_ct_helpers:set_config( Config, [{rmq_nodename_suffix, ?MODULE}]), Config2 = rabbit_ct_helpers:merge_app_env( Config1, {rabbit, [{queue_index_embed_msgs_below, 100}]}), - rabbit_ct_helpers:run_setup_steps( - Config2, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()). - -end_per_suite(Config) -> - rabbit_ct_helpers:run_teardown_steps( - Config, - rabbit_ct_client_helpers:teardown_steps() ++ - rabbit_ct_broker_helpers:teardown_steps()). + rabbit_ct_helpers:run_steps(Config2, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + +end_per_group(_, Config) -> + rabbit_ct_helpers:run_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). init_per_testcase(_, Config) -> Vhost1 = <<"vhost1">>, diff --git a/deps/rabbit/test/per_vhost_queue_limit_SUITE.erl b/deps/rabbit/test/per_vhost_queue_limit_SUITE.erl index 13c51a220bda..4fa7a86ea86d 100644 --- a/deps/rabbit/test/per_vhost_queue_limit_SUITE.erl +++ b/deps/rabbit/test/per_vhost_queue_limit_SUITE.erl @@ -21,32 +21,44 @@ all() -> [ - {group, cluster_size_1} - , {group, cluster_size_2} + {group, mnesia_store}, + {group, khepri_store} ]. groups() -> [ - {cluster_size_1, [], [ - most_basic_single_node_queue_count, - single_node_single_vhost_queue_count, - single_node_multiple_vhosts_queue_count, - single_node_single_vhost_limit, - single_node_single_vhost_zero_limit, - single_node_single_vhost_limit_with_durable_named_queue, - single_node_single_vhost_zero_limit_with_durable_named_queue, - single_node_single_vhost_limit_with_queue_ttl, - single_node_single_vhost_limit_with_redeclaration - ]}, - {cluster_size_2, [], [ - most_basic_cluster_queue_count, - cluster_multiple_vhosts_queue_count, - cluster_multiple_vhosts_limit, - cluster_multiple_vhosts_zero_limit, - cluster_multiple_vhosts_limit_with_durable_named_queue, - cluster_multiple_vhosts_zero_limit_with_durable_named_queue, - cluster_node_restart_queue_count - ]} + {mnesia_store, [], [ + {cluster_size_1, [], cluster_size_1_tests()}, + {cluster_size_2, [], cluster_size_2_tests()} + ]}, + {khepri_store, [], [ + {cluster_size_1, [], cluster_size_1_tests()}, + {cluster_size_2, [], cluster_size_2_tests()} + ]} + ]. + +cluster_size_1_tests() -> + [ + most_basic_single_node_queue_count, + single_node_single_vhost_queue_count, + single_node_multiple_vhosts_queue_count, + single_node_single_vhost_limit, + single_node_single_vhost_zero_limit, + single_node_single_vhost_limit_with_durable_named_queue, + single_node_single_vhost_zero_limit_with_durable_named_queue, + single_node_single_vhost_limit_with_queue_ttl, + single_node_single_vhost_limit_with_redeclaration + ]. + +cluster_size_2_tests() -> + [ + most_basic_cluster_queue_count, + cluster_multiple_vhosts_queue_count, + cluster_multiple_vhosts_limit, + cluster_multiple_vhosts_zero_limit, + cluster_multiple_vhosts_limit_with_durable_named_queue, + cluster_multiple_vhosts_zero_limit_with_durable_named_queue, + cluster_node_restart_queue_count ]. suite() -> @@ -66,31 +78,26 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). -init_per_group(cluster_size_1, Config) -> - init_per_multinode_group(cluster_size_1, Config, 1); -init_per_group(cluster_size_2, Config) -> - init_per_multinode_group(cluster_size_2, Config, 2); -init_per_group(cluster_rename, Config) -> - init_per_multinode_group(cluster_rename, Config, 2). +init_per_group(mnesia_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, mnesia}]); +init_per_group(khepri_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, khepri}]); +init_per_group(cluster_size_1 = Group, Config) -> + init_per_multinode_group(Group, Config, 1); +init_per_group(cluster_size_2 = Group, Config) -> + init_per_multinode_group(Group, Config, 2). -init_per_multinode_group(Group, Config, NodeCount) -> +init_per_multinode_group(_Group, Config, NodeCount) -> Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"), Config1 = rabbit_ct_helpers:set_config(Config, [ {rmq_nodes_count, NodeCount}, {rmq_nodename_suffix, Suffix} ]), - case Group of - cluster_rename -> - % The broker is managed by {init,end}_per_testcase(). - Config1; - _ -> - rabbit_ct_helpers:run_steps(Config1, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()) - end. + rabbit_ct_helpers:run_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). -end_per_group(cluster_rename, Config) -> - % The broker is managed by {init,end}_per_testcase(). +end_per_group(Group, Config) when Group == mnesia_store; Group == khepri_store -> Config; end_per_group(_Group, Config) -> rabbit_ct_helpers:run_steps(Config, diff --git a/deps/rabbit/test/policy_SUITE.erl b/deps/rabbit/test/policy_SUITE.erl index 43bed96385f3..2952662498ae 100644 --- a/deps/rabbit/test/policy_SUITE.erl +++ b/deps/rabbit/test/policy_SUITE.erl @@ -8,6 +8,7 @@ -module(policy_SUITE). -include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). -include_lib("amqp_client/include/amqp_client.hrl"). -include_lib("stdlib/include/assert.hrl"). -include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl"). @@ -16,29 +17,37 @@ all() -> [ - {group, cluster_size_2} + {group, mnesia_store}, + {group, khepri_store}, + {group, khepri_migration} ]. groups() -> [ - {cluster_size_2, [], [ - target_count_policy, - policy_ttl, - operator_policy_ttl, - operator_retroactive_policy_ttl, - operator_retroactive_policy_publish_ttl, - queue_type_specific_policies, - classic_queue_version_policies, - is_supported_operator_policy_expires, - is_supported_operator_policy_message_ttl, - is_supported_operator_policy_max_length, - is_supported_operator_policy_max_length, - is_supported_operator_policy_max_in_memory_length, - is_supported_operator_policy_max_in_memory_bytes, - is_supported_operator_policy_delivery_limit, - is_supported_operator_policy_target_group_size, - is_supported_operator_policy_ha - ]} + {mnesia_store, [], [target_count_policy] ++ all_tests()}, + {khepri_store, [], all_tests()}, + {khepri_migration, [], [ + from_mnesia_to_khepri + ]} + ]. + +all_tests() -> + [ + policy_ttl, + operator_policy_ttl, + operator_retroactive_policy_ttl, + operator_retroactive_policy_publish_ttl, + queue_type_specific_policies, + classic_queue_version_policies, + is_supported_operator_policy_expires, + is_supported_operator_policy_message_ttl, + is_supported_operator_policy_max_length, + is_supported_operator_policy_max_length, + is_supported_operator_policy_max_in_memory_length, + is_supported_operator_policy_max_in_memory_bytes, + is_supported_operator_policy_delivery_limit, + is_supported_operator_policy_target_group_size, + is_supported_operator_policy_ha ]. %% ------------------------------------------------------------------- @@ -52,29 +61,46 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). -init_per_group(cluster_size_2, Config) -> - Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"), - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodes_count, 2}, - {rmq_nodename_suffix, Suffix} - ]), - rabbit_ct_helpers:run_steps(Config1, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()). - -end_per_group(_Group, Config) -> +init_per_group(mnesia_store = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, mnesia}]), + init_per_group_common(Group, Config, 2); +init_per_group(khepri_store = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, khepri}]), + init_per_group_common(Group, Config, 2); +init_per_group(khepri_migration = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, mnesia}]), + init_per_group_common(Group, Config, 1). + +init_per_group_common(Group, Config, Size) -> + Config1 = rabbit_ct_helpers:set_config(Config, + [{rmq_nodes_count, Size}, + {rmq_nodename_suffix, Group}, + {tcp_ports_base}]), + rabbit_ct_helpers:run_steps(Config1, rabbit_ct_broker_helpers:setup_steps()). + +end_per_group(_, Config) -> rabbit_ct_helpers:run_steps(Config, - rabbit_ct_client_helpers:teardown_steps() ++ - rabbit_ct_broker_helpers:teardown_steps()). + rabbit_ct_broker_helpers:teardown_steps()). init_per_testcase(Testcase, Config) -> - rabbit_ct_client_helpers:setup_steps(), - rabbit_ct_helpers:testcase_started(Config, Testcase). + Config1 = rabbit_ct_helpers:testcase_started(Config, Testcase), + Name = rabbit_data_coercion:to_binary(Testcase), + Group = proplists:get_value(name, ?config(tc_group_properties, Config)), + Policy = rabbit_data_coercion:to_binary(io_lib:format("~p_~p_policy", [Group, Testcase])), + OpPolicy = rabbit_data_coercion:to_binary(io_lib:format("~p_~p_op_policy", [Group, Testcase])), + Config2 = rabbit_ct_helpers:set_config(Config1, + [{queue_name, Name}, + {policy, Policy}, + {op_policy, OpPolicy} + ]), + rabbit_ct_helpers:run_steps(Config2, rabbit_ct_client_helpers:setup_steps()). end_per_testcase(Testcase, Config) -> - rabbit_ct_client_helpers:teardown_steps(), - rabbit_ct_helpers:testcase_finished(Config, Testcase). - + rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_queues, []), + _ = rabbit_ct_broker_helpers:clear_policy(Config, 0, ?config(policy, Config)), + _ = rabbit_ct_broker_helpers:clear_operator_policy(Config, 0, ?config(op_policy, Config)), + Config1 = rabbit_ct_helpers:run_steps(Config, rabbit_ct_client_helpers:teardown_steps()), + rabbit_ct_helpers:testcase_finished(Config1, Testcase). %% ------------------------------------------------------------------- %% Test cases. %% ------------------------------------------------------------------- @@ -388,7 +414,51 @@ effective_operator_policy_per_queue_type(Config, Name, Value, ClassicValue, Quor passed. %%---------------------------------------------------------------------------- +from_mnesia_to_khepri(Config) -> + Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename), + + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Q = ?config(queue_name, Config), + ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Ch, Q)), + + Policy = ?config(policy, Config), + ok = rabbit_ct_broker_helpers:set_policy(Config, 0, Policy, Q, + <<"queues">>, + [{<<"dead-letter-exchange">>, <<>>}, + {<<"dead-letter-routing-key">>, Q}]), + OpPolicy = ?config(op_policy, Config), + ok = rabbit_ct_broker_helpers:set_operator_policy(Config, 0, OpPolicy, Q, + <<"queues">>, + [{<<"max-length">>, 10000}]), + + Policies0 = lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_policy, list, [])), + Names0 = lists:sort([proplists:get_value(name, Props) || Props <- Policies0]), + + ?assertEqual([Policy], Names0), + + OpPolicies0 = lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_policy, list_op, [])), + OpNames0 = lists:sort([proplists:get_value(name, Props) || Props <- OpPolicies0]), + + ?assertEqual([OpPolicy], OpNames0), + + case rabbit_ct_broker_helpers:enable_feature_flag(Config, khepri_db) of + ok -> + rabbit_ct_helpers:await_condition( + fun() -> + (Policies0 == + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_policy, list, []))) + andalso + (OpPolicies0 == + lists:sort(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_policy, list_op, []))) + end); + Skip -> + Skip + end. +%%---------------------------------------------------------------------------- +delete_queues() -> + [{ok, _} = rabbit_amqqueue:delete(Q, false, false, <<"dummy">>) + || Q <- rabbit_amqqueue:list()]. declare(Ch, Q) -> amqp_channel:call(Ch, #'queue.declare'{queue = Q, diff --git a/deps/rabbit/test/publisher_confirms_parallel_SUITE.erl b/deps/rabbit/test/publisher_confirms_parallel_SUITE.erl index df8240811194..4072ca5b8fc9 100644 --- a/deps/rabbit/test/publisher_confirms_parallel_SUITE.erl +++ b/deps/rabbit/test/publisher_confirms_parallel_SUITE.erl @@ -19,7 +19,8 @@ all() -> [ - {group, publisher_confirm_tests} + {group, mnesia_store}, + {group, khepri_store} ]. groups() -> @@ -33,15 +34,17 @@ groups() -> confirm_mandatory_unroutable, confirm_unroutable_message], [ - {publisher_confirm_tests, [], + {mnesia_store, [], [ {classic_queue, [parallel], PublisherConfirmTests ++ [confirm_nack]}, {mirrored_queue, [parallel], PublisherConfirmTests ++ [confirm_nack]}, - {quorum_queue, [], - [ - {parllel_tests, [parallel], PublisherConfirmTests}, - confirm_minority - ]} + {quorum_queue, [parallel], PublisherConfirmTests}, + {quorum_queue, [], [confirm_minority]} + ]}, + {khepri_store, [], + [ + {classic_queue, [parallel], PublisherConfirmTests ++ [confirm_nack]}, + {quorum_queue, [parallel], PublisherConfirmTests} ]} ]. @@ -79,7 +82,14 @@ init_per_group(mirrored_queue, Config) -> {queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]}, {queue_durable, true}]), rabbit_ct_helpers:run_steps(Config1, []); -init_per_group(Group, Config) -> +init_per_group(mnesia_store = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, mnesia}]), + init_per_group0(Group, Config); +init_per_group(khepri_store = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, khepri}]), + init_per_group0(Group, Config). + +init_per_group0(Group, Config) -> case lists:member({group, Group}, all()) of true -> ClusterSize = 3, @@ -87,9 +97,10 @@ init_per_group(Group, Config) -> {rmq_nodename_suffix, Group}, {rmq_nodes_count, ClusterSize} ]), - rabbit_ct_helpers:run_steps(Config1, + Config2 = rabbit_ct_helpers:run_steps(Config1, rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()); + rabbit_ct_client_helpers:setup_steps()), + Config2; false -> Config end. diff --git a/deps/rabbit/test/queue_length_limits_SUITE.erl b/deps/rabbit/test/queue_length_limits_SUITE.erl index 881a5046be9d..7ae77fe8e419 100644 --- a/deps/rabbit/test/queue_length_limits_SUITE.erl +++ b/deps/rabbit/test/queue_length_limits_SUITE.erl @@ -22,29 +22,38 @@ all() -> [ - {group, parallel_tests} + {group, mnesia_parallel_tests}, + {group, khepri_parallel_tests} ]. groups() -> - MaxLengthTests = [max_length_default, - max_length_bytes_default, - max_length_drop_head, - max_length_bytes_drop_head, - max_length_reject_confirm, - max_length_bytes_reject_confirm, - max_length_drop_publish, - max_length_drop_publish_requeue, - max_length_bytes_drop_publish], [ - {parallel_tests, [parallel], [ - {max_length_classic, [], MaxLengthTests}, - {max_length_quorum, [], [max_length_default, - max_length_bytes_default] - }, - {max_length_mirrored, [], MaxLengthTests} + {mnesia_parallel_tests, [parallel], [ + {max_length_classic, [], max_length_tests()}, + {max_length_quorum, [], max_length_quorum_tests()}, + {max_length_mirrored, [], max_length_tests()} + ]}, + {khepri_parallel_tests, [parallel], [ + {max_length_classic, [], max_length_tests()}, + {max_length_quorum, [], max_length_quorum_tests()} ]} ]. +max_length_tests() -> + [max_length_default, + max_length_bytes_default, + max_length_drop_head, + max_length_bytes_drop_head, + max_length_reject_confirm, + max_length_bytes_reject_confirm, + max_length_drop_publish, + max_length_drop_publish_requeue, + max_length_bytes_drop_publish]. + +max_length_quorum_tests() -> + [max_length_default, + max_length_bytes_default]. + suite() -> [ {timetrap, {minutes, 3}} @@ -79,7 +88,14 @@ init_per_group(max_length_mirrored, Config) -> {queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]}, {queue_durable, false}]), rabbit_ct_helpers:run_steps(Config1, []); -init_per_group(Group, Config) -> +init_per_group(mnesia_parallel_tests = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, mnesia}]), + init_per_group0(Group, Config); +init_per_group(khepri_parallel_tests = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, khepri}]), + init_per_group0(Group, Config). + +init_per_group0(Group, Config) -> case lists:member({group, Group}, all()) of true -> ClusterSize = 3, @@ -88,8 +104,8 @@ init_per_group(Group, Config) -> {rmq_nodes_count, ClusterSize} ]), rabbit_ct_helpers:run_steps(Config1, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()); + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()); false -> rabbit_ct_helpers:run_steps(Config, []) end. diff --git a/deps/rabbit/test/queue_master_location_SUITE.erl b/deps/rabbit/test/queue_master_location_SUITE.erl index f80de76baca5..261fc453c3a5 100644 --- a/deps/rabbit/test/queue_master_location_SUITE.erl +++ b/deps/rabbit/test/queue_master_location_SUITE.erl @@ -41,19 +41,20 @@ all() -> groups() -> [ - {cluster_size_3, [], [ - declare_args, - declare_policy, - declare_invalid_policy, - declare_policy_nodes, - declare_policy_all, - declare_policy_exactly, - declare_config, - calculate_min_master, - calculate_min_master_with_bindings, - calculate_random, - calculate_client_local - ]}, + {cluster_size_3, [], [{non_mirrored, [], [ + declare_args, + declare_policy, + declare_config, + calculate_min_master, + calculate_min_master_with_bindings, + calculate_random, + calculate_client_local + ]}, + {mirrored, [], [declare_invalid_policy, + declare_policy_nodes, + declare_policy_all, + declare_policy_exactly]}] + }, {maintenance_mode, [], [ declare_with_min_masters_and_some_nodes_under_maintenance, @@ -83,6 +84,15 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). +init_per_group(mirrored, Config) -> + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + mnesia -> + Config; + {khepri, _} -> + {skip, "Classic queue mirroring not supported by Khepri"} + end; +init_per_group(non_mirrored, Config) -> + Config; init_per_group(cluster_size_3, Config) -> rabbit_ct_helpers:set_config(Config, [ %% Replaced with a list of node names later diff --git a/deps/rabbit/test/queue_parallel_SUITE.erl b/deps/rabbit/test/queue_parallel_SUITE.erl index 82582a29fc6b..6e313d0d5294 100644 --- a/deps/rabbit/test/queue_parallel_SUITE.erl +++ b/deps/rabbit/test/queue_parallel_SUITE.erl @@ -119,14 +119,20 @@ init_per_group(quorum_queue_in_memory_bytes, Config) -> {consumer_args, []}, {queue_durable, true}]); init_per_group(mirrored_queue, Config) -> - rabbit_ct_broker_helpers:set_ha_policy(Config, 0, <<"^max_length.*queue">>, - <<"all">>, [{<<"ha-sync-mode">>, <<"automatic">>}]), - Config1 = rabbit_ct_helpers:set_config( - Config, [{is_mirrored, true}, - {queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]}, - {consumer_args, []}, - {queue_durable, true}]), - rabbit_ct_helpers:run_steps(Config1, []); + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + mnesia -> + rabbit_ct_broker_helpers:set_ha_policy( + Config, 0, <<"^max_length.*queue">>, + <<"all">>, [{<<"ha-sync-mode">>, <<"automatic">>}]), + Config1 = rabbit_ct_helpers:set_config( + Config, [{is_mirrored, true}, + {queue_args, [{<<"x-queue-type">>, longstr, <<"classic">>}]}, + {consumer_args, []}, + {queue_durable, true}]), + rabbit_ct_helpers:run_steps(Config1, []); + {khepri, _} -> + {skip, "Classic queue mirroring not supported by Khepri"} + end; init_per_group(stream_queue, Config) -> rabbit_ct_helpers:set_config( Config, diff --git a/deps/rabbit/test/queue_type_SUITE.erl b/deps/rabbit/test/queue_type_SUITE.erl index d787ea4a8ab8..200ae6d67a20 100644 --- a/deps/rabbit/test/queue_type_SUITE.erl +++ b/deps/rabbit/test/queue_type_SUITE.erl @@ -44,8 +44,24 @@ end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config), ok. +init_per_group(classic = Group, Config0) -> + ct:pal("init per group ~p", [Group]), + case rabbit_ct_broker_helpers:configured_metadata_store(Config0) of + mnesia -> + Config = init_per_group0(classic, Config0), + rabbit_ct_broker_helpers:set_policy( + Config, 0, + <<"ha-policy">>, <<".*">>, <<"queues">>, + [{<<"ha-mode">>, <<"all">>}]), + Config; + {khepri, _} -> + {skip, "Classic queue mirroring not supported by Khepri"} + end; init_per_group(Group, Config) -> ct:pal("init per group ~p", [Group]), + init_per_group0(Group, Config). + +init_per_group0(Group, Config) -> ClusterSize = 3, Config1 = rabbit_ct_helpers:set_config(Config, [{rmq_nodes_count, ClusterSize}, @@ -57,27 +73,28 @@ init_per_group(Group, Config) -> Config2 = rabbit_ct_helpers:run_steps(Config1b, [fun merge_app_env/1 ] ++ rabbit_ct_broker_helpers:setup_steps()), - ok = rabbit_ct_broker_helpers:rpc( - Config2, 0, application, set_env, - [rabbit, channel_tick_interval, 100]), - %% HACK: the larger cluster sizes benefit for a bit more time - %% after clustering before running the tests. - Config3 = case Group of - cluster_size_5 -> - timer:sleep(5000), - Config2; - _ -> - Config2 - end, - EnableFF = rabbit_ct_broker_helpers:enable_feature_flag(Config3, - message_containers), - ct:pal("message_containers ff ~p", [EnableFF]), - - rabbit_ct_broker_helpers:set_policy( - Config3, 0, - <<"ha-policy">>, <<".*">>, <<"queues">>, - [{<<"ha-mode">>, <<"all">>}]), - Config3. + case Config2 of + {skip, _Reason} = Skip -> + %% To support mixed-version clusters, + %% Khepri feature flag is unsupported + Skip; + _ -> + ok = rabbit_ct_broker_helpers:rpc( + Config2, 0, application, set_env, + [rabbit, channel_tick_interval, 100]), + %% HACK: the larger cluster sizes benefit for a bit more time + %% after clustering before running the tests. + case Group of + cluster_size_5 -> + timer:sleep(5000); + _ -> + ok + end, + EnableFF = rabbit_ct_broker_helpers:enable_feature_flag(Config2, + message_containers), + ct:pal("message_containers ff ~p", [EnableFF]), + Config2 + end. merge_app_env(Config) -> rabbit_ct_helpers:merge_app_env( diff --git a/deps/rabbit/test/quorum_queue_SUITE.erl b/deps/rabbit/test/quorum_queue_SUITE.erl index 28476eb64bc8..d6528e620e7b 100644 --- a/deps/rabbit/test/quorum_queue_SUITE.erl +++ b/deps/rabbit/test/quorum_queue_SUITE.erl @@ -57,10 +57,10 @@ groups() -> delete_member_queue_not_found, delete_member, delete_member_not_a_member, - node_removal_is_quorum_critical, - cleanup_data_dir] + node_removal_is_quorum_critical] ++ memory_tests()}, {cluster_size_3, [], [ + cleanup_data_dir, channel_handles_ra_event, declare_during_node_down, simple_confirm_availability_on_leader_change, @@ -955,8 +955,11 @@ consume_in_minority(Config) -> ?assertExit({{shutdown, {connection_closing, {server_initiated_close, 541, _}}}, _}, amqp_channel:call(Ch, #'basic.get'{queue = QQ, no_ack = false})), - ok = rabbit_ct_broker_helpers:start_node(Config, Server1), - ok = rabbit_ct_broker_helpers:start_node(Config, Server2), + + ok = rabbit_ct_broker_helpers:async_start_node(Config, Server1), + ok = rabbit_ct_broker_helpers:async_start_node(Config, Server2), + ok = rabbit_ct_broker_helpers:wait_for_async_start_node(Server1), + ok = rabbit_ct_broker_helpers:wait_for_async_start_node(Server2), ok. reject_after_leader_transfer(Config) -> @@ -1169,12 +1172,12 @@ invalid_policy(Config) -> ?assertEqual({'queue.declare_ok', QQ, 0, 0}, declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), ok = rabbit_ct_broker_helpers:set_policy( - Config, 0, <<"ha">>, <<"invalid_policy.*">>, <<"queues">>, - [{<<"ha-mode">>, <<"all">>}]), + Config, 0, <<"max-age">>, <<"invalid_policy.*">>, <<"queues">>, + [{<<"max-age">>, <<"5s">>}]), Info = rpc:call(Server, rabbit_quorum_queue, infos, [rabbit_misc:r(<<"/">>, queue, QQ)]), ?assertEqual('', proplists:get_value(policy, Info)), - ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, <<"ha">>). + ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, <<"max-age">>). pre_existing_invalid_policy(Config) -> [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), @@ -1182,14 +1185,14 @@ pre_existing_invalid_policy(Config) -> Ch = rabbit_ct_client_helpers:open_channel(Config, Server), QQ = ?config(queue_name, Config), ok = rabbit_ct_broker_helpers:set_policy( - Config, 0, <<"ha">>, <<"invalid_policy.*">>, <<"queues">>, - [{<<"ha-mode">>, <<"all">>}]), + Config, 0, <<"max-age">>, <<"invalid_policy.*">>, <<"queues">>, + [{<<"max-age">>, <<"5s">>}]), ?assertEqual({'queue.declare_ok', QQ, 0, 0}, declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), Info = rpc:call(Server, rabbit_quorum_queue, infos, [rabbit_misc:r(<<"/">>, queue, QQ)]), ?assertEqual('', proplists:get_value(policy, Info)), - ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, <<"ha">>), + ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, <<"max-age">>), ok. dead_letter_to_quorum_queue(Config) -> @@ -2018,16 +2021,19 @@ file_handle_reservations_above_limit(Config) -> ok = rpc:call(S3, file_handle_cache, set_limit, [Limit]). cleanup_data_dir(Config) -> + %% With Khepri this test needs to run in a 3-node cluster, otherwise the queue can't + %% be deleted in minority + %% %% This test is slow, but also checks that we handle properly errors when %% trying to delete a queue in minority. A case clause there had gone %% previously unnoticed. - [Server1, Server2] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + [Server1, Server2, _Server3] = Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), Ch = rabbit_ct_client_helpers:open_channel(Config, Server1), QQ = ?config(queue_name, Config), ?assertEqual({'queue.declare_ok', QQ, 0, 0}, declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), - ?awaitMatch(2, count_online_nodes(Server1, <<"/">>, QQ), ?DEFAULT_AWAIT), + ?awaitMatch(3, count_online_nodes(Server1, <<"/">>, QQ), ?DEFAULT_AWAIT), UId1 = proplists:get_value(ra_name(QQ), rpc:call(Server1, ra_directory, list_registered, [quorum_queues])), UId2 = proplists:get_value(ra_name(QQ), rpc:call(Server2, ra_directory, list_registered, [quorum_queues])), @@ -2041,7 +2047,6 @@ cleanup_data_dir(Config) -> ?assertMatch(#'queue.delete_ok'{}, amqp_channel:call(Ch, #'queue.delete'{queue = QQ})), - ok = rabbit_ct_broker_helpers:stop_node(Config, Server2), %% data dir 1 should be force deleted at this point ?assert(not filelib:is_dir(DataDir1)), ?assert(filelib:is_dir(DataDir2)), diff --git a/deps/rabbit/test/rabbit_core_metrics_gc_SUITE.erl b/deps/rabbit/test/rabbit_core_metrics_gc_SUITE.erl index 382c4320b232..d59e07af76e6 100644 --- a/deps/rabbit/test/rabbit_core_metrics_gc_SUITE.erl +++ b/deps/rabbit/test/rabbit_core_metrics_gc_SUITE.erl @@ -45,9 +45,14 @@ merge_app_env(Config) -> init_per_group(cluster_tests, Config) -> rabbit_ct_helpers:log_environment(), - Conf = [{rmq_nodename_suffix, cluster_tests}, {rmq_nodes_count, 2}], - Config1 = rabbit_ct_helpers:set_config(Config, Conf), - rabbit_ct_helpers:run_setup_steps(Config1, setup_steps()); + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + mnesia -> + Conf = [{rmq_nodename_suffix, cluster_tests}, {rmq_nodes_count, 2}], + Config1 = rabbit_ct_helpers:set_config(Config, Conf), + rabbit_ct_helpers:run_setup_steps(Config1, setup_steps()); + {khepri, _} -> + {skip, "Classic queue mirroring not supported by Khepri"} + end; init_per_group(non_parallel_tests, Config) -> rabbit_ct_helpers:log_environment(), Conf = [{rmq_nodename_suffix, non_parallel_tests}], diff --git a/deps/rabbit/test/rabbit_db_binding_SUITE.erl b/deps/rabbit/test/rabbit_db_binding_SUITE.erl index ed5376e3ea09..99beaa7792bd 100644 --- a/deps/rabbit/test/rabbit_db_binding_SUITE.erl +++ b/deps/rabbit/test/rabbit_db_binding_SUITE.erl @@ -85,8 +85,8 @@ create(Config) -> create1(_Config) -> XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), - Exchange1 = #exchange{name = XName1, durable = true}, - Exchange2 = #exchange{name = XName2, durable = true}, + Exchange1 = #exchange{name = XName1, durable = true, decorators = {[], []}}, + Exchange2 = #exchange{name = XName2, durable = true, decorators = {[], []}}, Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, ?assertMatch({error, {resources_missing, [_, _]}}, rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), @@ -105,12 +105,14 @@ exists(Config) -> exists1(_Config) -> XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), - Exchange1 = #exchange{name = XName1, durable = true}, - Exchange2 = #exchange{name = XName2, durable = true}, + Exchange1 = #exchange{name = XName1, durable = true, decorators = {[], []}}, + Exchange2 = #exchange{name = XName2, durable = true, decorators = {[], []}}, Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, - ?assertEqual(false, rabbit_db_exchange:exists(Binding)), + ?assertMatch({error, {resources_missing, [{not_found, _}, {not_found, _}]}}, + rabbit_db_binding:exists(Binding)), ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange2)), + ?assertEqual(false, rabbit_db_binding:exists(Binding)), ?assertMatch(ok, rabbit_db_binding:create(Binding, fun(_, _) -> ok end)), ?assertEqual(true, rabbit_db_binding:exists(Binding)), passed. @@ -121,8 +123,8 @@ delete(Config) -> delete1(_Config) -> XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), - Exchange1 = #exchange{name = XName1, durable = true, auto_delete = false}, - Exchange2 = #exchange{name = XName2, durable = true, auto_delete = false}, + Exchange1 = #exchange{name = XName1, durable = true, auto_delete = false, decorators = {[], []}}, + Exchange2 = #exchange{name = XName2, durable = true, auto_delete = false, decorators = {[], []}}, Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, ?assertEqual(ok, rabbit_db_binding:delete(Binding, fun(_, _) -> ok end)), ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), @@ -142,8 +144,8 @@ auto_delete(Config) -> auto_delete1(_Config) -> XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), - Exchange1 = #exchange{name = XName1, durable = true, auto_delete = true}, - Exchange2 = #exchange{name = XName2, durable = true, auto_delete = false}, + Exchange1 = #exchange{name = XName1, durable = true, auto_delete = true, decorators = {[], []}}, + Exchange2 = #exchange{name = XName2, durable = true, auto_delete = false, decorators = {[], []}}, Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, ?assertEqual(ok, rabbit_db_binding:delete(Binding, fun(_, _) -> ok end)), ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), @@ -163,8 +165,8 @@ get_all(Config) -> get_all1(_Config) -> XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), - Exchange1 = #exchange{name = XName1, durable = true}, - Exchange2 = #exchange{name = XName2, durable = true}, + Exchange1 = #exchange{name = XName1, durable = true, decorators = {[], []}}, + Exchange2 = #exchange{name = XName2, durable = true, decorators = {[], []}}, Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, ?assertEqual([], rabbit_db_binding:get_all()), ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), @@ -179,8 +181,8 @@ get_all_by_vhost(Config) -> get_all_by_vhost1(_Config) -> XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), - Exchange1 = #exchange{name = XName1, durable = true}, - Exchange2 = #exchange{name = XName2, durable = true}, + Exchange1 = #exchange{name = XName1, durable = true, decorators = {[], []}}, + Exchange2 = #exchange{name = XName2, durable = true, decorators = {[], []}}, Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, ?assertEqual([], rabbit_db_binding:get_all(?VHOST)), ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), @@ -197,8 +199,8 @@ get_all_for_source(Config) -> get_all_for_source1(_Config) -> XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), - Exchange1 = #exchange{name = XName1, durable = true}, - Exchange2 = #exchange{name = XName2, durable = true}, + Exchange1 = #exchange{name = XName1, durable = true, decorators = {[], []}}, + Exchange2 = #exchange{name = XName2, durable = true, decorators = {[], []}}, Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, ?assertEqual([], rabbit_db_binding:get_all_for_source(XName1)), ?assertEqual([], rabbit_db_binding:get_all_for_source(XName2)), @@ -216,8 +218,8 @@ get_all_for_destination(Config) -> get_all_for_destination1(_Config) -> XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), - Exchange1 = #exchange{name = XName1, durable = true}, - Exchange2 = #exchange{name = XName2, durable = true}, + Exchange1 = #exchange{name = XName1, durable = true, decorators = {[], []}}, + Exchange2 = #exchange{name = XName2, durable = true, decorators = {[], []}}, Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, ?assertEqual([], rabbit_db_binding:get_all_for_destination(XName1)), ?assertEqual([], rabbit_db_binding:get_all_for_destination(XName2)), @@ -235,8 +237,8 @@ get_all_for_source_and_destination(Config) -> get_all_for_source_and_destination1(_Config) -> XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), - Exchange1 = #exchange{name = XName1, durable = true}, - Exchange2 = #exchange{name = XName2, durable = true}, + Exchange1 = #exchange{name = XName1, durable = true, decorators = {[], []}}, + Exchange2 = #exchange{name = XName2, durable = true, decorators = {[], []}}, Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, ?assertEqual([], rabbit_db_binding:get_all(XName1, XName2, false)), ?assertEqual([], rabbit_db_binding:get_all(XName2, XName1, false)), @@ -256,8 +258,8 @@ get_all_for_source_and_destination_reverse(Config) -> get_all_for_source_and_destination_reverse1(_Config) -> XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), - Exchange1 = #exchange{name = XName1, durable = true}, - Exchange2 = #exchange{name = XName2, durable = true}, + Exchange1 = #exchange{name = XName1, durable = true, decorators = {[], []}}, + Exchange2 = #exchange{name = XName2, durable = true, decorators = {[], []}}, Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, ?assertEqual([], rabbit_db_binding:get_all(XName1, XName2, true)), ?assertEqual([], rabbit_db_binding:get_all(XName2, XName1, true)), @@ -276,8 +278,8 @@ fold(Config) -> fold1(_Config) -> XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), - Exchange1 = #exchange{name = XName1, durable = true}, - Exchange2 = #exchange{name = XName2, durable = true}, + Exchange1 = #exchange{name = XName1, durable = true, decorators = {[], []}}, + Exchange2 = #exchange{name = XName2, durable = true, decorators = {[], []}}, Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, ?assertEqual([], rabbit_db_binding:fold(fun(B, Acc) -> [B | Acc] end, [])), ?assertMatch({new, #exchange{}}, rabbit_db_exchange:create_or_get(Exchange1)), @@ -292,8 +294,8 @@ match(Config) -> match1(_Config) -> XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), - Exchange1 = #exchange{name = XName1, durable = true}, - Exchange2 = #exchange{name = XName2, durable = true}, + Exchange1 = #exchange{name = XName1, durable = true, decorators = {[], []}}, + Exchange2 = #exchange{name = XName2, durable = true, decorators = {[], []}}, Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{foo => bar}}, ?assertEqual([], rabbit_db_binding:match(XName1, fun(#binding{args = Args}) -> @@ -318,8 +320,8 @@ match_routing_key(Config) -> match_routing_key1(_Config) -> XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), - Exchange1 = #exchange{name = XName1, durable = true}, - Exchange2 = #exchange{name = XName2, durable = true}, + Exchange1 = #exchange{name = XName1, durable = true, decorators = {[], []}}, + Exchange2 = #exchange{name = XName2, durable = true, decorators = {[], []}}, Binding = #binding{source = XName1, key = <<"*.*">>, destination = XName2, args = #{foo => bar}}, ?assertEqual([], rabbit_db_binding:match_routing_key(XName1, [<<"a.b.c">>], false)), diff --git a/deps/rabbit/test/rabbit_db_exchange_SUITE.erl b/deps/rabbit/test/rabbit_db_exchange_SUITE.erl index 33982e8b34e0..97b0c1f645f8 100644 --- a/deps/rabbit/test/rabbit_db_exchange_SUITE.erl +++ b/deps/rabbit/test/rabbit_db_exchange_SUITE.erl @@ -92,8 +92,8 @@ create_or_get1(_Config) -> XName = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), Exchange0 = #exchange{name = XName, durable = true}, Exchange = rabbit_exchange_decorator:set(Exchange0), - ?assertMatch({new, Exchange}, rabbit_db_exchange:create_or_get(Exchange0)), - ?assertEqual({existing, Exchange}, rabbit_db_exchange:create_or_get(Exchange0)), + ?assertMatch({new, Exchange}, rabbit_db_exchange:create_or_get(Exchange)), + ?assertEqual({existing, Exchange}, rabbit_db_exchange:create_or_get(Exchange)), passed. get(Config) -> @@ -104,7 +104,7 @@ get1(_Config) -> Exchange0 = #exchange{name = XName, durable = true}, Exchange = rabbit_exchange_decorator:set(Exchange0), ?assertEqual({error, not_found}, rabbit_db_exchange:get(XName)), - ?assertEqual({new, Exchange}, rabbit_db_exchange:create_or_get(Exchange0)), + ?assertEqual({new, Exchange}, rabbit_db_exchange:create_or_get(Exchange)), ?assertEqual({ok, Exchange}, rabbit_db_exchange:get(XName)), passed. @@ -116,7 +116,7 @@ get_many1(_Config) -> Exchange0 = #exchange{name = XName, durable = true}, Exchange = rabbit_exchange_decorator:set(Exchange0), ?assertEqual([], rabbit_db_exchange:get_many([XName])), - ?assertEqual({new, Exchange}, rabbit_db_exchange:create_or_get(Exchange0)), + ?assertEqual({new, Exchange}, rabbit_db_exchange:create_or_get(Exchange)), ?assertEqual([Exchange], rabbit_db_exchange:get_many([XName])), passed. @@ -132,7 +132,7 @@ get_all1(_Config) -> Exchange2 = rabbit_exchange_decorator:set(Exchange2_0), All = lists:sort([Exchange1, Exchange2]), ?assertEqual([], rabbit_db_exchange:get_all()), - create([Exchange1_0, Exchange2_0]), + create([Exchange1, Exchange2]), ?assertEqual(All, lists:sort(rabbit_db_exchange:get_all())), passed. @@ -148,7 +148,7 @@ get_all_by_vhost1(_Config) -> Exchange2 = rabbit_exchange_decorator:set(Exchange2_0), All = lists:sort([Exchange1, Exchange2]), ?assertEqual([], rabbit_db_exchange:get_all(?VHOST)), - create([Exchange1_0, Exchange2_0]), + create([Exchange1, Exchange2]), ?assertEqual(All, lists:sort(rabbit_db_exchange:get_all(?VHOST))), ?assertEqual([], lists:sort(rabbit_db_exchange:get_all(<<"other-vhost">>))), passed. @@ -216,7 +216,6 @@ set1(_Config) -> XName = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), Exchange = #exchange{name = XName, durable = true}, ?assertEqual(ok, rabbit_db_exchange:set([Exchange])), - ?assertEqual({error, not_found}, rabbit_db_exchange:get(XName)), ?assertEqual([Exchange], rabbit_db_exchange:get_all_durable()), passed. @@ -275,8 +274,8 @@ delete_if_unused(Config) -> delete_if_unused1(_Config) -> XName1 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange1">>), XName2 = rabbit_misc:r(?VHOST, exchange, <<"test-exchange2">>), - Exchange1 = #exchange{name = XName1, durable = true}, - Exchange2 = #exchange{name = XName2, durable = true}, + Exchange1 = rabbit_exchange_decorator:set(#exchange{name = XName1, durable = true}), + Exchange2 = rabbit_exchange_decorator:set(#exchange{name = XName2, durable = true}), Binding = #binding{source = XName1, key = <<"">>, destination = XName2, args = #{}}, ?assertMatch({error, not_found}, rabbit_db_exchange:delete(XName1, true)), create([Exchange1, Exchange2]), @@ -316,9 +315,8 @@ recover(Config) -> recover1(_Config) -> XName = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), - Exchange = #exchange{name = XName, durable = true}, + Exchange = rabbit_exchange_decorator:set(#exchange{name = XName, durable = true}), ?assertEqual(ok, rabbit_db_exchange:set([Exchange])), - ?assertEqual({error, not_found}, rabbit_db_exchange:get(XName)), ?assertEqual([Exchange], rabbit_db_exchange:get_all_durable()), ?assertMatch([Exchange], rabbit_db_exchange:recover(?VHOST)), ?assertMatch({ok, #exchange{name = XName}}, rabbit_db_exchange:get(XName)), diff --git a/deps/rabbit/test/rabbit_db_maintenance_SUITE.erl b/deps/rabbit/test/rabbit_db_maintenance_SUITE.erl index 491cdfb9a34a..b2b26da1d937 100644 --- a/deps/rabbit/test/rabbit_db_maintenance_SUITE.erl +++ b/deps/rabbit/test/rabbit_db_maintenance_SUITE.erl @@ -24,7 +24,6 @@ groups() -> all_tests() -> [ - setup_schema, set_and_get, set_and_get_consistent ]. @@ -64,14 +63,6 @@ end_per_testcase(Testcase, Config) -> %% Test Cases %% --------------------------------------------------------------------------- -setup_schema(Config) -> - passed = rabbit_ct_broker_helpers:rpc( - Config, 0, ?MODULE, setup_schema1, [Config]). - -setup_schema1(_Config) -> - ?assertEqual(ok, rabbit_db_maintenance:setup_schema()), - passed. - set_and_get(Config) -> passed = rabbit_ct_broker_helpers:rpc( Config, 0, ?MODULE, set_and_get1, [Config]). diff --git a/deps/rabbit/test/rabbit_db_msup_SUITE.erl b/deps/rabbit/test/rabbit_db_msup_SUITE.erl index 20ca9f1fbfd9..46e8caf1d45d 100644 --- a/deps/rabbit/test/rabbit_db_msup_SUITE.erl +++ b/deps/rabbit/test/rabbit_db_msup_SUITE.erl @@ -83,9 +83,12 @@ create_or_update1(_Config) -> Overall = spawn(fun() -> ok end), Spec = #{id => id, start => {m, f, args}}, ?assertEqual(start, - rabbit_db_msup:create_or_update(group, Overall, undefined, Spec, id)), + rabbit_db_msup:create_or_update(group, Overall, undefined, Spec, id(id))), passed. +id(Id) -> + {[Id], Id}. + find_mirror(Config) -> passed = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, find_mirror1, [Config]). @@ -93,8 +96,8 @@ find_mirror1(_Config) -> Overall = spawn(fun() -> ok end), Spec = #{id => id, start => {m, f, args}}, ?assertEqual(start, rabbit_db_msup:create_or_update(group, Overall, undefined, - Spec, id)), - ?assertEqual({ok, Overall}, rabbit_db_msup:find_mirror(group, id)), + Spec, id(id))), + ?assertEqual({ok, Overall}, rabbit_db_msup:find_mirror(group, id(id))), passed. delete(Config) -> @@ -104,9 +107,9 @@ delete1(_Config) -> Overall = spawn(fun() -> ok end), Spec = #{id => id, start => {m, f, args}}, ?assertEqual(start, rabbit_db_msup:create_or_update(group, Overall, undefined, - Spec, id)), - ?assertEqual(ok, rabbit_db_msup:delete(group, id)), - ?assertEqual({error, not_found}, rabbit_db_msup:find_mirror(group, id)), + Spec, id(id))), + ?assertEqual(ok, rabbit_db_msup:delete(group, id(id))), + ?assertEqual({error, not_found}, rabbit_db_msup:find_mirror(group, id(id))), passed. delete_all(Config) -> @@ -116,9 +119,9 @@ delete_all1(_Config) -> Overall = spawn(fun() -> ok end), Spec = #{id => id, start => {m, f, args}}, ?assertEqual(start, rabbit_db_msup:create_or_update(group, Overall, undefined, - Spec, id)), + Spec, id(id))), ?assertEqual(ok, rabbit_db_msup:delete_all(group)), - ?assertEqual({error, not_found}, rabbit_db_msup:find_mirror(group, id)), + ?assertEqual({error, not_found}, rabbit_db_msup:find_mirror(group, id(id))), passed. update_all(Config) -> @@ -129,8 +132,8 @@ update_all1(_Config) -> Overall = spawn(fun() -> ok end), Spec = #{id => id, start => {m, f, args}}, ?assertEqual(start, rabbit_db_msup:create_or_update(group, OldOverall, undefined, - Spec, id)), - ?assertEqual({ok, OldOverall}, rabbit_db_msup:find_mirror(group, id)), + Spec, id(id))), + ?assertEqual({ok, OldOverall}, rabbit_db_msup:find_mirror(group, id(id))), ?assertEqual([Spec], rabbit_db_msup:update_all(Overall, OldOverall)), - ?assertEqual({ok, Overall}, rabbit_db_msup:find_mirror(group, id)), + ?assertEqual({ok, Overall}, rabbit_db_msup:find_mirror(group, id(id))), passed. diff --git a/deps/rabbit/test/rabbit_db_queue_SUITE.erl b/deps/rabbit/test/rabbit_db_queue_SUITE.erl index 3cafb91443d0..03e163f3222c 100644 --- a/deps/rabbit/test/rabbit_db_queue_SUITE.erl +++ b/deps/rabbit/test/rabbit_db_queue_SUITE.erl @@ -50,14 +50,14 @@ all_tests() -> filter_all_durable, get_durable, get_many_durable, - set_dirty, - internal_delete, - update_durable + update_durable, + foreach_durable, + internal_delete ]. mnesia_tests() -> [ - foreach_durable, + set_dirty, foreach_transient, delete_transient, update_in_mnesia_tx, @@ -75,7 +75,13 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). +init_per_group(mnesia_store = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, mnesia}]), + init_per_group_common(Group, Config); init_per_group(Group, Config) -> + init_per_group_common(Group, Config). + +init_per_group_common(Group, Config) -> Config1 = rabbit_ct_helpers:set_config(Config, [ {rmq_nodename_suffix, Group}, {rmq_nodes_count, 1} @@ -137,7 +143,7 @@ get_many1(_Config) -> ?assertEqual([Q], rabbit_db_queue:get_many([QName, QName2])), ?assertEqual([], rabbit_db_queue:get_many([QName2])), ok = rabbit_db_queue:set(Q2), - ?assertEqual([Q, Q2], rabbit_db_queue:get_many([QName, QName2])), + ?assertEqual(lists:sort([Q, Q2]), lists:sort(rabbit_db_queue:get_many([QName, QName2]))), passed. get_all(Config) -> @@ -455,8 +461,6 @@ update_durable1(_Config) -> fun(Q0) when ?is_amqqueue(Q0) -> true end)), {ok, Q0} = rabbit_db_queue:get_durable(QName1), ?assertMatch(my_policy, amqqueue:get_policy(Q0)), - {ok, Q00} = rabbit_db_queue:get(QName1), - ?assertMatch(undefined, amqqueue:get_policy(Q00)), passed. foreach_durable(Config) -> @@ -464,11 +468,8 @@ foreach_durable(Config) -> foreach_durable1(_Config) -> QName1 = rabbit_misc:r(?VHOST, queue, <<"test-queue1">>), - QName2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), Q1 = new_queue(QName1, rabbit_classic_queue), - Q2 = new_queue(QName2, rabbit_classic_queue), ?assertEqual(ok, rabbit_db_queue:set(Q1)), - ?assertEqual(ok, rabbit_db_queue:set_dirty(Q2)), ?assertEqual(ok, rabbit_db_queue:foreach_durable( fun(Q0) -> rabbit_db_queue:internal_delete(amqqueue:get_name(Q0), true, normal) @@ -476,7 +477,6 @@ foreach_durable1(_Config) -> fun(Q0) when ?is_amqqueue(Q0) -> true end)), ?assertEqual({error, not_found}, rabbit_db_queue:get(QName1)), ?assertEqual({error, not_found}, rabbit_db_queue:get_durable(QName1)), - ?assertMatch({ok, _}, rabbit_db_queue:get(QName2)), passed. foreach_transient(Config) -> diff --git a/deps/rabbit/test/rabbit_db_topic_exchange_SUITE.erl b/deps/rabbit/test/rabbit_db_topic_exchange_SUITE.erl index 1bde941840f2..b1331b7a8f2f 100644 --- a/deps/rabbit/test/rabbit_db_topic_exchange_SUITE.erl +++ b/deps/rabbit/test/rabbit_db_topic_exchange_SUITE.erl @@ -8,6 +8,7 @@ -module(rabbit_db_topic_exchange_SUITE). -include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). -compile([nowarn_export_all, export_all]). @@ -16,27 +17,33 @@ all() -> [ - {group, tests} + {group, mnesia_store} ]. groups() -> [ - {tests, [shuffle], all_tests()} + {mnesia_store, [], mnesia_tests()}, + {benchmarks, [], benchmarks()} ]. -all_tests() -> +mnesia_tests() -> [ set, delete, delete_all_for_exchange, match, match_return_binding_keys_many_destinations, - match_return_binding_keys_single_destination + match_return_binding_keys_single_destination, + build_key_from_topic_trie_binding_record, + build_key_from_deletion_events, + build_key_from_binding_deletion_event, + build_multiple_key_from_deletion_events ]. -%% ------------------------------------------------------------------- -%% Test suite setup/teardown. -%% ------------------------------------------------------------------- +benchmarks() -> + [ + match_benchmark + ]. init_per_suite(Config) -> rabbit_ct_helpers:log_environment(), @@ -45,7 +52,13 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). +init_per_group(mnesia_store = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, mnesia}]), + init_per_group_common(Group, Config); init_per_group(Group, Config) -> + init_per_group_common(Group, Config). + +init_per_group_common(Group, Config) -> Config1 = rabbit_ct_helpers:set_config(Config, [ {rmq_nodename_suffix, Group}, {rmq_nodes_count, 1} @@ -60,7 +73,11 @@ end_per_group(_Group, Config) -> rabbit_ct_broker_helpers:teardown_steps()). init_per_testcase(Testcase, Config) -> - rabbit_ct_helpers:testcase_started(Config, Testcase). + XName = rabbit_misc:r(<<"/">>, exchange, <<"amq.topic">>), + {ok, X} = rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_exchange, lookup, [XName]), + Config1 = rabbit_ct_helpers:set_config(Config, [{exchange_name, XName}, + {exchange, X}]), + rabbit_ct_helpers:testcase_started(Config1, Testcase). end_per_testcase(Testcase, Config) -> rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_db_topic_exchange, clear, []), @@ -247,3 +264,224 @@ set(Src, BindingKey, Dst, Args) when is_list(Args) -> Binding = #binding{source = Src, key = BindingKey, destination = Dst, args = Args}, ok = rabbit_db_topic_exchange:set(Binding). + +%% --------------------------------------------------------------------------- +%% Functional tests +%% --------------------------------------------------------------------------- + +build_key_from_topic_trie_binding_record(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, build_key_from_topic_trie_binding_record1, [Config]). + +build_key_from_topic_trie_binding_record1(Config) -> + XName = ?config(exchange_name, Config), + X = ?config(exchange, Config), + QName = rabbit_misc:r(<<"/">>, queue, <<"q1">>), + RK = <<"a.b.c.d.e.f">>, + ok = rabbit_exchange_type_topic:add_binding(none, X, #binding{source = XName, + destination = QName, + key = RK, + args = []}), + SplitRK = rabbit_db_topic_exchange:split_topic_key(RK), + [TopicTrieBinding] = ets:tab2list(rabbit_topic_trie_binding), + ?assertEqual(SplitRK, rabbit_db_topic_exchange:trie_binding_to_key(TopicTrieBinding)), + passed. + +build_key_from_deletion_events(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, build_key_from_deletion_events1, [Config]). + +build_key_from_deletion_events1(Config) -> + XName = ?config(exchange_name, Config), + X = ?config(exchange, Config), + QName = rabbit_misc:r(<<"/">>, queue, <<"q1">>), + RK = <<"a.b.c.d.e.f">>, + Binding = #binding{source = XName, + destination = QName, + key = RK, + args = []}, + ok = rabbit_exchange_type_topic:add_binding(none, X, Binding), + SplitRK = rabbit_db_topic_exchange:split_topic_key(RK), + Tables = [rabbit_topic_trie_binding, rabbit_topic_trie_edge], + subscribe_to_mnesia_changes(Tables), + rabbit_exchange_type_topic:remove_bindings(none, X, [Binding]), + Records = receive_delete_events(7), + unsubscribe_to_mnesia_changes(Tables), + ?assertMatch([{_, SplitRK}], + rabbit_db_topic_exchange:trie_records_to_key(Records)), + passed. + +build_key_from_binding_deletion_event(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, build_key_from_binding_deletion_event1, [Config]). + +build_key_from_binding_deletion_event1(Config) -> + XName = ?config(exchange_name, Config), + X = ?config(exchange, Config), + QName = rabbit_misc:r(<<"/">>, queue, <<"q1">>), + RK = <<"a.b.c.d.e.f">>, + Binding0 = #binding{source = XName, + destination = QName, + key = RK, + args = [some_args]}, + Binding = #binding{source = XName, + destination = QName, + key = RK, + args = []}, + ok = rabbit_exchange_type_topic:add_binding(none, X, Binding0), + ok = rabbit_exchange_type_topic:add_binding(none, X, Binding), + SplitRK = rabbit_db_topic_exchange:split_topic_key(RK), + Tables = [rabbit_topic_trie_binding, rabbit_topic_trie_edge], + subscribe_to_mnesia_changes(Tables), + rabbit_exchange_type_topic:remove_bindings(none, X, [Binding]), + Records = receive_delete_events(7), + unsubscribe_to_mnesia_changes(Tables), + ?assertMatch([{_, SplitRK}], + rabbit_db_topic_exchange:trie_records_to_key(Records)), + passed. + +build_multiple_key_from_deletion_events(Config) -> + passed = rabbit_ct_broker_helpers:rpc( + Config, 0, ?MODULE, build_multiple_key_from_deletion_events1, [Config]). + +build_multiple_key_from_deletion_events1(Config) -> + XName = ?config(exchange_name, Config), + X = ?config(exchange, Config), + QName = rabbit_misc:r(<<"/">>, queue, <<"q1">>), + RK0 = <<"a.b.c.d.e.f">>, + RK1 = <<"a.b.c.d">>, + RK2 = <<"a.b.c.g.e.f">>, + RK3 = <<"hare.rabbit.ho">>, + Binding0 = #binding{source = XName, destination = QName, key = RK0, args = []}, + Binding1 = #binding{source = XName, destination = QName, key = RK1, args = []}, + Binding2 = #binding{source = XName, destination = QName, key = RK2, args = []}, + Binding3 = #binding{source = XName, destination = QName, key = RK3, args = []}, + ok = rabbit_exchange_type_topic:add_binding(none, X, Binding0), + ok = rabbit_exchange_type_topic:add_binding(none, X, Binding1), + ok = rabbit_exchange_type_topic:add_binding(none, X, Binding2), + ok = rabbit_exchange_type_topic:add_binding(none, X, Binding3), + SplitRK0 = rabbit_db_topic_exchange:split_topic_key(RK0), + SplitRK1 = rabbit_db_topic_exchange:split_topic_key(RK1), + SplitRK2 = rabbit_db_topic_exchange:split_topic_key(RK2), + SplitRK3 = rabbit_db_topic_exchange:split_topic_key(RK3), + Tables = [rabbit_topic_trie_binding, rabbit_topic_trie_edge], + subscribe_to_mnesia_changes(Tables), + rabbit_exchange_type_topic:delete(none, X), + Records = receive_delete_events(7), + unsubscribe_to_mnesia_changes(Tables), + RKs = lists:sort([SplitRK0, SplitRK1, SplitRK2, SplitRK3]), + ?assertMatch( + RKs, + lists:sort([RK || {_, RK} <- rabbit_db_topic_exchange:trie_records_to_key(Records)])), + passed. + +%% --------------------------------------------------------------------------- +%% Benchmarks +%% --------------------------------------------------------------------------- + +match_benchmark(Config) -> + %% run the benchmark with Mnesia first + MnesiaResults = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, match_benchmark1, [Config]), + + %% migrate to Khepri + Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + {T, ok} = timer:tc(fun() -> + rabbit_ct_broker_helpers:enable_feature_flag(Config, Servers, khepri_db) + end), + ct:pal("~p: time to migrate to Khepri: ~.2fs", [?FUNCTION_NAME, T/1000000]), + + %% run the same same benchmark with Khepri enabled + KhepriResults = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, match_benchmark1, [Config]), + + %% print all the results first + maps:foreach(fun(Test, KhepriResult) -> + MnesiaResult = maps:get(Test, MnesiaResults), + ct:pal("~p: Test: ~p, Mnesia: ~.2fus, Khepri: ~.2fus", [?FUNCTION_NAME, Test, MnesiaResult, KhepriResult]) + end, KhepriResults), + + %% fail the test if needed + maps:foreach(fun(Test, KhepriResult) -> + MnesiaResult = maps:get(Test, MnesiaResults), + ?assert(KhepriResult < MnesiaResult * 1.5, "Khepri can't be significantly slower than Mnesia") + end, KhepriResults). + +match_benchmark1(_Config) -> + Src = rabbit_misc:r(?VHOST, exchange, <<"test-exchange">>), + Dst1 = rabbit_misc:r(?VHOST, queue, <<"test-queue1">>), + Dst2 = rabbit_misc:r(?VHOST, queue, <<"test-queue2">>), + Dst3 = rabbit_misc:r(?VHOST, queue, <<"test-queue3">>), + Dst4 = rabbit_misc:r(?VHOST, queue, <<"test-queue4">>), + Dst5 = rabbit_misc:r(?VHOST, queue, <<"test-queue5">>), + Dst6 = rabbit_misc:r(?VHOST, queue, <<"test-queue6">>), + + SimpleTopics = [list_to_binary("a.b." ++ integer_to_list(N)) || N <- lists:seq(1,1000)], + Bindings = [#binding{source = Src, key = RoutingKey, destination = Dst1, args = #{}} || RoutingKey <- SimpleTopics], + BindingRes = [rabbit_db_topic_exchange:set(Binding) || Binding <- Bindings], + ?assertMatch([ok], lists:uniq(BindingRes)), + ok = rabbit_db_topic_exchange:set(#binding{source = Src, key = <<"a.b.*">>, destination = Dst2, args = #{}}), + ok = rabbit_db_topic_exchange:set(#binding{source = Src, key = <<"a.b.*">>, destination = Dst3, args = #{}}), + ok = rabbit_db_topic_exchange:set(#binding{source = Src, key = <<"a.#">>, destination = Dst4, args = #{}}), + ok = rabbit_db_topic_exchange:set(#binding{source = Src, key = <<"*.b.42">>, destination = Dst5, args = #{}}), + ok = rabbit_db_topic_exchange:set(#binding{source = Src, key = <<"#">>, destination = Dst6, args = #{}}), + + {Tany, _} = timer:tc(fun() -> + [rabbit_db_topic_exchange:match(Src, <<"foo">>) || _ <- lists:seq(1, 100)] + end), + ?assertMatch([Dst6], rabbit_db_topic_exchange:match(Src, <<"foo">>)), + + {Tbar, _} = timer:tc(fun() -> + [rabbit_db_topic_exchange:match(Src, <<"a.b.bar">>) || _ <- lists:seq(1, 100)] + end), + ?assertEqual(lists:sort([Dst2,Dst3,Dst4,Dst6]), lists:sort(rabbit_db_topic_exchange:match(Src, <<"a.b.bar">>))), + + {Tbaz, _} = timer:tc(fun() -> + [rabbit_db_topic_exchange:match(Src, <<"baz.b.42">>) || _ <- lists:seq(1, 100)] + end), + ?assertEqual(lists:sort([Dst5,Dst6]), lists:sort(rabbit_db_topic_exchange:match(Src, <<"baz.b.42">>))), + + {Tsimple, Rsimple} = timer:tc(fun() -> + [rabbit_db_topic_exchange:match(Src, RoutingKey) + || RoutingKey <- SimpleTopics, RoutingKey =/= <<"a.b.123">>] + end), + ?assertEqual([Dst1,Dst2,Dst3,Dst4,Dst6], lists:sort(lists:uniq(hd(Rsimple)))), + + #{ + "average time to match `foo`" => Tany/100, + "average time to match `a.b.bar`" => Tbar/100, + "average time to match `baz.b.42`" => Tbaz/100, + "average time to match a simple topic" => Tsimple/length(SimpleTopics) + }. + +subscribe_to_mnesia_changes([Table | Rest]) -> + case mnesia:subscribe({table, Table, detailed}) of + {ok, _} -> subscribe_to_mnesia_changes(Rest); + Error -> Error + end; +subscribe_to_mnesia_changes([]) -> + ok. + +unsubscribe_to_mnesia_changes([Table | Rest]) -> + case mnesia:unsubscribe({table, Table, detailed}) of + {ok, _} -> unsubscribe_to_mnesia_changes(Rest); + Error -> Error + end; +unsubscribe_to_mnesia_changes([]) -> + ok. + +receive_delete_events(Num) -> + receive_delete_events(Num, []). + +receive_delete_events(0, Evts) -> + receive + {mnesia_table_event, {delete, _, Record, _, _}} -> + receive_delete_events(0, [Record | Evts]) + after 0 -> + Evts + end; +receive_delete_events(N, Evts) -> + receive + {mnesia_table_event, {delete, _, Record, _, _}} -> + receive_delete_events(N - 1, [Record | Evts]) + after 10000 -> + Evts + end. diff --git a/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl b/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl index d4e3f4e2954d..d3519c7a4c3b 100644 --- a/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl +++ b/deps/rabbit/test/rabbit_fifo_dlx_integration_SUITE.erl @@ -97,10 +97,15 @@ init_per_group(Group, Config, NodesCount) -> Config2 = rabbit_ct_helpers:run_steps(Config1, [fun merge_app_env/1 ] ++ rabbit_ct_broker_helpers:setup_steps()), - _ = rabbit_ct_broker_helpers:enable_feature_flag(Config2, message_containers), - ok = rpc(Config2, 0, application, set_env, - [rabbit, channel_tick_interval, 100]), - Config2. + case Config2 of + {skip, _Reason} = Skip -> + Skip; + _ -> + _ = rabbit_ct_broker_helpers:enable_feature_flag(Config2, message_containers), + ok = rpc(Config2, 0, application, set_env, + [rabbit, channel_tick_interval, 100]), + Config2 + end. end_per_group(_, Config) -> rabbit_ct_helpers:run_steps(Config, @@ -113,10 +118,16 @@ merge_app_env(Config) -> {ra, [{min_wal_roll_over_interval, 30000}]}). init_per_testcase(Testcase, Config) -> - case {Testcase, rabbit_ct_helpers:is_mixed_versions()} of - {single_dlx_worker, true} -> + IsKhepriEnabled = lists:any(fun(B) -> B end, + rabbit_ct_broker_helpers:rpc_all( + Config, rabbit_feature_flags, is_enabled, + [khepri_db])), + case {Testcase, rabbit_ct_helpers:is_mixed_versions(), IsKhepriEnabled} of + {single_dlx_worker, true, _} -> {skip, "single_dlx_worker is not mixed version compatible because process " "rabbit_fifo_dlx_sup does not exist in 3.9"}; + {many_target_queues, _, true} -> + {skip, "Classic queue mirroring not supported by Khepri"}; _ -> Config1 = rabbit_ct_helpers:testcase_started(Config, Testcase), T = rabbit_data_coercion:to_binary(Testcase), diff --git a/deps/rabbit/test/rabbit_stream_queue_SUITE.erl b/deps/rabbit/test/rabbit_stream_queue_SUITE.erl index c31327b9b469..6dcc26254f63 100644 --- a/deps/rabbit/test/rabbit_stream_queue_SUITE.erl +++ b/deps/rabbit/test/rabbit_stream_queue_SUITE.erl @@ -283,6 +283,14 @@ init_per_testcase(TestCase, Config) _ -> init_test_case(TestCase, Config) end; +init_per_testcase(TestCase, Config) + when TestCase == publish_coordinator_unavailable -> + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + {khepri, []} -> + {skip, "test case should not run with Khepri"}; + _ -> + init_test_case(TestCase, Config) + end; init_per_testcase(TestCase, Config) -> init_test_case(TestCase, Config). @@ -445,12 +453,10 @@ find_queue_info(Config, Node, Keys) -> delete_queue(Config) -> [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - Ch = rabbit_ct_client_helpers:open_channel(Config, Server), Q = ?config(queue_name, Config), ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Config, Server, Q, [{<<"x-queue-type">>, longstr, <<"stream">>}])), - ?assertMatch(#'queue.delete_ok'{}, - amqp_channel:call(Ch, #'queue.delete'{queue = Q})). + ?assertMatch(#'queue.delete_ok'{}, delete(Config, Server, Q)). add_replicas(Config) -> [Server0, Server1, Server2] = @@ -510,14 +516,14 @@ add_replicas(Config) -> after 60000 -> flush(), ?assertMatch(#'queue.delete_ok'{}, - amqp_channel:call(Ch, #'queue.delete'{queue = Q})), + delete(Config, Server0, Q)), exit(deliver_timeout) end, % ?assertMatch({error, {disallowed, out_of_sync_replica}} , % rpc:call(Server0, rabbit_stream_queue, add_replica, % [<<"/">>, Q, Server2])), ?assertMatch(#'queue.delete_ok'{}, - amqp_channel:call(Ch, #'queue.delete'{queue = Q})), + delete(Config, Server0, Q)), rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_testcase_queue, [Q]). add_replica(Config) -> @@ -744,8 +750,14 @@ delete_down_replica(Config) -> rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_testcase_queue, [Q]). publish_coordinator_unavailable(Config) -> - [Server0, Server1, Server2] = - rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + %% This testcase leaves Khepri on minority when it is enabled. + %% If the Khepri leader is in one of the nodes being stopped, the + %% remaining node won't be able to reply to any channel query. + %% Let's remove it from the Khepri test suite, as it does not make + %% much sense to test something that will randomly work + %% depending on where the leader is placed - even though we could + %% always select as running node the Khepri leader + [Server0, Server1, Server2] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), Ch = rabbit_ct_client_helpers:open_channel(Config, Server0), Q = ?config(queue_name, Config), ?assertEqual({'queue.declare_ok', Q, 0, 0}, @@ -755,7 +767,7 @@ publish_coordinator_unavailable(Config) -> ok = rabbit_ct_broker_helpers:stop_node(Config, Server2), rabbit_ct_helpers:await_condition( fun () -> - N = rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_nodes, list_running, []), + N = rabbit_ct_broker_helpers:rpc(Config, Server0, rabbit_nodes, list_running, []), length(N) == 1 end), #'confirm.select_ok'{} = amqp_channel:call(Ch, #'confirm.select'{}), @@ -763,11 +775,13 @@ publish_coordinator_unavailable(Config) -> publish(Ch, Q), ?assertExit({{shutdown, {connection_closing, {server_initiated_close, 506, _}}}, _}, amqp_channel:wait_for_confirms(Ch, 60)), - ok = rabbit_ct_broker_helpers:start_node(Config, Server1), - ok = rabbit_ct_broker_helpers:start_node(Config, Server2), + ok = rabbit_ct_broker_helpers:async_start_node(Config, Server1), + ok = rabbit_ct_broker_helpers:async_start_node(Config, Server2), + ok = rabbit_ct_broker_helpers:wait_for_async_start_node(Server1), + ok = rabbit_ct_broker_helpers:wait_for_async_start_node(Server2), rabbit_ct_helpers:await_condition( fun () -> - Info = find_queue_info(Config, 0, [online]), + Info = find_queue_info(Config, Server0, [online]), length(proplists:get_value(online, Info)) == 3 end), Ch1 = rabbit_ct_client_helpers:open_channel(Config, Server0), @@ -875,7 +889,9 @@ recover(Config) -> ct:pal("recover: running stop start for permutation ~w", [Servers]), [rabbit_ct_broker_helpers:stop_node(Config, S) || S <- Servers], - [rabbit_ct_broker_helpers:start_node(Config, S) || S <- lists:reverse(Servers)], + [rabbit_ct_broker_helpers:async_start_node(Config, S) || S <- lists:reverse(Servers)], + [ok = rabbit_ct_broker_helpers:wait_for_async_start_node(S) || S <- lists:reverse(Servers)], + ct:pal("recover: running stop waiting for messages ~w", [Servers]), check_leader_and_replicas(Config, Servers0), queue_utils:wait_for_messages(Config, [[Q, <<"1">>, <<"1">>, <<"0">>]], 60), @@ -886,7 +902,10 @@ recover(Config) -> ct:pal("recover: running app stop start for permuation ~w", [Servers1]), [rabbit_control_helper:command(stop_app, S) || S <- Servers1], - [rabbit_control_helper:command(start_app, S) || S <- lists:reverse(Servers1)], + [rabbit_control_helper:async_command(start_app, S, [], []) + || S <- lists:reverse(Servers1)], + [rabbit_control_helper:wait_for_async_command(S) || S <- lists:reverse(Servers1)], + ct:pal("recover: running app stop waiting for messages ~w", [Servers1]), check_leader_and_replicas(Config, Servers0), queue_utils:wait_for_messages(Config, [[Q, <<"1">>, <<"1">>, <<"0">>]], 60), @@ -907,10 +926,12 @@ restart_coordinator_without_queues(Config) -> ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Config, Server, Q, [{<<"x-queue-type">>, longstr, <<"stream">>}])), publish_confirm(Ch, Q, [<<"msg">>]), - ?assertMatch(#'queue.delete_ok'{}, amqp_channel:call(Ch, #'queue.delete'{queue = Q})), + ?assertMatch(#'queue.delete_ok'{}, delete(Config, Server, Q)), [rabbit_ct_broker_helpers:stop_node(Config, S) || S <- Servers0], - [rabbit_ct_broker_helpers:start_node(Config, S) || S <- lists:reverse(Servers0)], + [rabbit_ct_broker_helpers:async_start_node(Config, S) || S <- lists:reverse(tl(Servers0))], + [rabbit_ct_broker_helpers:wait_for_async_start_node(S) || S <- lists:reverse(tl(Servers0))], + rabbit_ct_broker_helpers:start_node(Config, hd(Servers0)), ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Config, Server, Q, [{<<"x-queue-type">>, longstr, <<"stream">>}])), @@ -1933,7 +1954,7 @@ leader_failover_dedupe(Config) -> declare(Config, DownNode, Q, [{<<"x-queue-type">>, longstr, <<"stream">>}])), check_leader_and_replicas(Config, Nodes), - + timer:sleep(5000), Ch2 = rabbit_ct_client_helpers:open_channel(Config, PubNode), #'confirm.select_ok'{} = amqp_channel:call(Ch2, #'confirm.select'{}), @@ -1993,7 +2014,6 @@ leader_failover_dedupe(Config) -> initial_cluster_size_one(Config) -> [Server1 | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - Ch = rabbit_ct_client_helpers:open_channel(Config, Server1), Q = ?config(queue_name, Config), ?assertEqual({'queue.declare_ok', Q, 0, 0}, @@ -2002,13 +2022,12 @@ initial_cluster_size_one(Config) -> check_leader_and_replicas(Config, [Server1]), ?assertMatch(#'queue.delete_ok'{}, - amqp_channel:call(Ch, #'queue.delete'{queue = Q})), + delete(Config, Server1, Q)), rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_testcase_queue, [Q]). initial_cluster_size_two(Config) -> [Server1 | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - Ch = rabbit_ct_client_helpers:open_channel(Config, Server1), Q = ?config(queue_name, Config), ?assertEqual({'queue.declare_ok', Q, 0, 0}, @@ -2021,7 +2040,7 @@ initial_cluster_size_two(Config) -> ?assertEqual(2, length(proplists:get_value(members, Info))), ?assertMatch(#'queue.delete_ok'{}, - amqp_channel:call(Ch, #'queue.delete'{queue = Q})), + delete(Config, Server1, Q)), rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_testcase_queue, [Q]). initial_cluster_size_one_policy(Config) -> @@ -2033,7 +2052,6 @@ initial_cluster_size_one_policy(Config) -> <<"queues">>, [{<<"initial-cluster-size">>, 1}]), - Ch = rabbit_ct_client_helpers:open_channel(Config, Server1), Q = ?config(queue_name, Config), ?assertEqual({'queue.declare_ok', Q, 0, 0}, @@ -2042,7 +2060,7 @@ initial_cluster_size_one_policy(Config) -> check_leader_and_replicas(Config, [Server1]), ?assertMatch(#'queue.delete_ok'{}, - amqp_channel:call(Ch, #'queue.delete'{queue = Q})), + delete(Config, Server1, Q)), ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, PolicyName), rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_testcase_queue, [Q]). @@ -2050,14 +2068,11 @@ initial_cluster_size_one_policy(Config) -> declare_delete_same_stream(Config) -> Servers = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), Q = ?config(queue_name, Config), - [begin - Ch = rabbit_ct_client_helpers:open_channel(Config, S), ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Config, S, Q, [{<<"x-queue-type">>, longstr, <<"stream">>}])), ?assertMatch(#'queue.delete_ok'{}, - amqp_channel:call(Ch, #'queue.delete'{queue = Q})), - rabbit_ct_client_helpers:close_channel(Ch) + delete(Config, S, Q)) end || _ <- lists:seq(1, 20), S <- Servers], ok. @@ -2065,7 +2080,6 @@ declare_delete_same_stream(Config) -> leader_locator_client_local(Config) -> [Server1, Server2, Server3] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - Ch = rabbit_ct_client_helpers:open_channel(Config, Server1), Q = ?config(queue_name, Config), ?assertEqual({'queue.declare_ok', Q, 0, 0}, @@ -2077,10 +2091,9 @@ leader_locator_client_local(Config) -> ?assertEqual(Server1, proplists:get_value(leader, Info)), ?assertMatch(#'queue.delete_ok'{}, - amqp_channel:call(Ch, #'queue.delete'{queue = Q})), + delete(Config, Server1, Q)), %% Try second node - Ch2 = rabbit_ct_client_helpers:open_channel(Config, Server2), ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Config, Server2, Q, [{<<"x-queue-type">>, longstr, <<"stream">>}, {<<"x-queue-leader-locator">>, longstr, <<"client-local">>}])), @@ -2088,11 +2101,9 @@ leader_locator_client_local(Config) -> Info2 = find_queue_info(Config, [leader]), ?assertEqual(Server2, proplists:get_value(leader, Info2)), - ?assertMatch(#'queue.delete_ok'{}, - amqp_channel:call(Ch2, #'queue.delete'{queue = Q})), + ?assertMatch(#'queue.delete_ok'{}, delete(Config, Server2, Q)), %% Try third node - Ch3 = rabbit_ct_client_helpers:open_channel(Config, Server3), ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Config, Server3, Q, [{<<"x-queue-type">>, longstr, <<"stream">>}, {<<"x-queue-leader-locator">>, longstr, <<"client-local">>}])), @@ -2101,8 +2112,7 @@ leader_locator_client_local(Config) -> Info3 = find_queue_info(Config, [leader]), ?assertEqual(Server3, proplists:get_value(leader, Info3)), - ?assertMatch(#'queue.delete_ok'{}, - amqp_channel:call(Ch3, #'queue.delete'{queue = Q})), + ?assertMatch(#'queue.delete_ok'{}, delete(Config, Server3, Q)), rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_testcase_queue, [Q]). leader_locator_balanced(Config) -> @@ -2246,9 +2256,6 @@ invalid_policy(Config) -> Q = ?config(queue_name, Config), ?assertEqual({'queue.declare_ok', Q, 0, 0}, declare(Config, Server, Q, [{<<"x-queue-type">>, longstr, <<"stream">>}])), - ok = rabbit_ct_broker_helpers:set_policy( - Config, 0, <<"ha">>, <<"invalid_policy.*">>, <<"queues">>, - [{<<"ha-mode">>, <<"all">>}]), ok = rabbit_ct_broker_helpers:set_policy( Config, 0, <<"ttl">>, <<"invalid_policy.*">>, <<"queues">>, [{<<"message-ttl">>, 5}]), @@ -2258,7 +2265,6 @@ invalid_policy(Config) -> ?assertEqual('', proplists:get_value(policy, Info)), ?assertEqual('', proplists:get_value(operator_policy, Info)), ?assertEqual([], proplists:get_value(effective_policy_definition, Info)), - ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, <<"ha">>), ok = rabbit_ct_broker_helpers:clear_policy(Config, 0, <<"ttl">>), rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_testcase_queue, [Q]). @@ -2542,6 +2548,9 @@ declare(Config, Server, Q, Args) -> auto_delete = false, arguments = Args}). +delete(Config, Server, Q) -> + retry_if_coordinator_unavailable(Config, Server, #'queue.delete'{queue = Q}). + retry_if_coordinator_unavailable(Config, Server, Cmd) -> Props = ?config(tc_group_properties, Config), %% Running parallel tests the coordinator could be busy answering other @@ -2566,7 +2575,7 @@ retry_if_coordinator_unavailable(Config, Server, Cmd, Retry) -> case re:run(Msg, ".*coordinator_unavailable.*", [{capture, none}]) of match -> ct:pal("Attempt to execute command ~p failed, coordinator unavailable", [Cmd]), - retry_if_coordinator_unavailable(Ch, Cmd, Retry - 1); + retry_if_coordinator_unavailable(Config, Ch, Cmd, Retry - 1); _ -> exit(Error) end diff --git a/deps/rabbit/test/rabbitmq_4_0_deprecations_SUITE.erl b/deps/rabbit/test/rabbitmq_4_0_deprecations_SUITE.erl index 5201d29b85a2..6668c170dca5 100644 --- a/deps/rabbit/test/rabbitmq_4_0_deprecations_SUITE.erl +++ b/deps/rabbit/test/rabbitmq_4_0_deprecations_SUITE.erl @@ -40,27 +40,27 @@ suite() -> all() -> [ - {group, global_qos}, - {group, ram_node_type}, - {group, classic_queue_mirroring}, - {group, transient_nonexcl_queues} + {group, mnesia_store}, + {group, khepri_store} ]. groups() -> - [ - {global_qos, [], - [when_global_qos_is_permitted_by_default, - when_global_qos_is_not_permitted_from_conf]}, - {ram_node_type, [], - [join_when_ram_node_type_is_permitted_by_default, - join_when_ram_node_type_is_not_permitted_from_conf]}, - {classic_queue_mirroring, [], - [set_policy_when_cmq_is_permitted_by_default, - set_policy_when_cmq_is_not_permitted_from_conf]}, - {transient_nonexcl_queues, [], - [when_transient_nonexcl_is_permitted_by_default, - when_transient_nonexcl_is_not_permitted_from_conf]} - ]. + Groups = [ + {global_qos, [], + [when_global_qos_is_permitted_by_default, + when_global_qos_is_not_permitted_from_conf]}, + {ram_node_type, [], + [join_when_ram_node_type_is_permitted_by_default, + join_when_ram_node_type_is_not_permitted_from_conf]}, + {classic_queue_mirroring, [], + [set_policy_when_cmq_is_permitted_by_default, + set_policy_when_cmq_is_not_permitted_from_conf]}, + {transient_nonexcl_queues, [], + [when_transient_nonexcl_is_permitted_by_default, + when_transient_nonexcl_is_not_permitted_from_conf]} + ], + [{mnesia_store, [], Groups}, + {khepri_store, [], Groups}]. %% ------------------------------------------------------------------- %% Testsuite setup/teardown. @@ -76,6 +76,10 @@ init_per_suite(Config) -> end_per_suite(Config) -> Config. +init_per_group(mnesia_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, mnesia}]); +init_per_group(khepri_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, khepri}]); init_per_group(global_qos, Config) -> rabbit_ct_helpers:set_config(Config, {rmq_nodes_count, 1}); init_per_group(ram_node_type, Config) -> @@ -208,6 +212,14 @@ is_prefetch_limited(ServerCh) -> %% ------------------------------------------------------------------- join_when_ram_node_type_is_permitted_by_default(Config) -> + case ?config(metadata_store, Config) of + mnesia -> + join_when_ram_node_type_is_permitted_by_default_mnesia(Config); + khepri -> + join_when_ram_node_type_is_permitted_by_default_khepri(Config) + end. + +join_when_ram_node_type_is_permitted_by_default_mnesia(Config) -> [NodeA, NodeB] = rabbit_ct_broker_helpers:get_node_configs( Config, nodename), @@ -226,7 +238,8 @@ join_when_ram_node_type_is_permitted_by_default(Config) -> ?assert( log_file_contains_message( Config, NodeA, - ["Deprecated features: `ram_node_type`: Feature `ram_node_type` is deprecated", + ["Deprecated features: `ram_node_type`: Feature `ram_node_type` is " + "deprecated", "By default, this feature can still be used for now."])), %% Change the advanced configuration file to turn off RAM node type. @@ -265,7 +278,33 @@ join_when_ram_node_type_is_permitted_by_default(Config) -> end end. +join_when_ram_node_type_is_permitted_by_default_khepri(Config) -> + [NodeA, NodeB] = rabbit_ct_broker_helpers:get_node_configs( + Config, nodename), + + ok = rabbit_control_helper:command(stop_app, NodeA), + ?assertMatch( + {error, 70, + <<"Error:\nError: `ram` node type is unsupported", _/binary>>}, + rabbit_control_helper:command_with_output( + join_cluster, NodeA, + [atom_to_list(NodeB)], [{"--ram", true}])), + ok = rabbit_control_helper:command(start_app, NodeA), + + ?assertEqual([NodeA], get_all_nodes(Config, NodeA)), + ?assertEqual([NodeB], get_all_nodes(Config, NodeB)), + ?assertEqual([NodeA], get_disc_nodes(Config, NodeA)), + ?assertEqual([NodeB], get_disc_nodes(Config, NodeB)). + join_when_ram_node_type_is_not_permitted_from_conf(Config) -> + case ?config(metadata_store, Config) of + mnesia -> + join_when_ram_node_type_is_not_permitted_from_conf_mnesia(Config); + khepri -> + join_when_ram_node_type_is_not_permitted_from_conf_khepri(Config) + end. + +join_when_ram_node_type_is_not_permitted_from_conf_mnesia(Config) -> [NodeA, NodeB] = rabbit_ct_broker_helpers:get_node_configs( Config, nodename), @@ -300,6 +339,24 @@ join_when_ram_node_type_is_not_permitted_from_conf(Config) -> end end. +join_when_ram_node_type_is_not_permitted_from_conf_khepri(Config) -> + [NodeA, NodeB] = rabbit_ct_broker_helpers:get_node_configs( + Config, nodename), + + ok = rabbit_control_helper:command(stop_app, NodeA), + ?assertMatch( + {error, 70, + <<"Error:\nError: `ram` node type is unsupported", _/binary>>}, + rabbit_control_helper:command_with_output( + join_cluster, NodeA, + [atom_to_list(NodeB)], [{"--ram", true}])), + ok = rabbit_control_helper:command(start_app, NodeA), + + ?assertEqual([NodeA], get_all_nodes(Config, NodeA)), + ?assertEqual([NodeB], get_all_nodes(Config, NodeB)), + ?assertEqual([NodeA], get_disc_nodes(Config, NodeA)), + ?assertEqual([NodeB], get_disc_nodes(Config, NodeB)). + get_all_nodes(Config, Node) -> lists:sort( rabbit_ct_broker_helpers:rpc( @@ -315,6 +372,14 @@ get_disc_nodes(Config, Node) -> %% ------------------------------------------------------------------- set_policy_when_cmq_is_permitted_by_default(Config) -> + case ?config(metadata_store, Config) of + mnesia -> + set_policy_when_cmq_is_permitted_by_default_mnesia(Config); + khepri -> + set_policy_when_cmq_is_permitted_by_default_khepri(Config) + end. + +set_policy_when_cmq_is_permitted_by_default_mnesia(Config) -> ?assertEqual( ok, rabbit_ct_broker_helpers:set_ha_policy( @@ -325,7 +390,8 @@ set_policy_when_cmq_is_permitted_by_default(Config) -> ?assert( log_file_contains_message( Config, NodeA, - ["Deprecated features: `classic_queue_mirroring`: Classic mirrored queues are deprecated.", + ["Deprecated features: `classic_queue_mirroring`: Classic mirrored " + "queues are deprecated.", "By default, they can still be used for now."])), %% Change the advanced configuration file to turn off classic queue @@ -354,6 +420,14 @@ set_policy_when_cmq_is_permitted_by_default(Config) -> [{capture, none}]), ?assertEqual(match, Ret). +set_policy_when_cmq_is_permitted_by_default_khepri(Config) -> + ?assertError( + {badmatch, + {error_string, + "Validation failed\n\nClassic mirrored queues are deprecated." ++ _}}, + rabbit_ct_broker_helpers:set_ha_policy( + Config, 0, <<".*">>, <<"all">>)). + set_policy_when_cmq_is_not_permitted_from_conf(Config) -> ?assertError( {badmatch, diff --git a/deps/rabbit/test/rabbitmqctl_integration_SUITE.erl b/deps/rabbit/test/rabbitmqctl_integration_SUITE.erl index 38d53a140ead..c691a4c595f2 100644 --- a/deps/rabbit/test/rabbitmqctl_integration_SUITE.erl +++ b/deps/rabbit/test/rabbitmqctl_integration_SUITE.erl @@ -52,9 +52,14 @@ end_per_suite(Config) -> init_per_group(list_queues, Config0) -> NumNodes = 3, Config = create_n_node_cluster(Config0, NumNodes), - Config1 = declare_some_queues(Config), - rabbit_ct_broker_helpers:stop_node(Config1, NumNodes - 1), - Config1; + case Config of + {skip, _Reason} = Skip -> + Skip; + _ -> + Config1 = declare_some_queues(Config), + rabbit_ct_broker_helpers:stop_node(Config1, NumNodes - 1), + Config1 + end; init_per_group(_, Config) -> Config. diff --git a/deps/rabbit/test/routing_SUITE.erl b/deps/rabbit/test/routing_SUITE.erl index 4a4e8f749107..c7baa8681d4e 100644 --- a/deps/rabbit/test/routing_SUITE.erl +++ b/deps/rabbit/test/routing_SUITE.erl @@ -19,7 +19,8 @@ all() -> [ - {group, mnesia_store} + {group, mnesia_store}, + {group, khepri_store} ]. suite() -> @@ -27,7 +28,8 @@ suite() -> groups() -> [ - {mnesia_store, [], all_tests()} + {mnesia_store, [], all_tests()}, + {khepri_store, [], all_tests()} ]. all_tests() -> @@ -46,10 +48,17 @@ init_per_suite(Config) -> end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). -init_per_group(mnesia_store = Group, Config) -> +init_per_group(mnesia_store = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, mnesia}]), + init_per_group_common(Group, Config, 1); +init_per_group(khepri_store = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, khepri}]), + init_per_group_common(Group, Config, 1). + +init_per_group_common(Group, Config, Size) -> Config1 = rabbit_ct_helpers:set_config(Config, [ {rmq_nodename_suffix, Group}, - {rmq_nodes_count, 1} + {rmq_nodes_count, Size} ]), rabbit_ct_helpers:run_steps(Config1, rabbit_ct_broker_helpers:setup_steps() ++ diff --git a/deps/rabbit/test/simple_ha_SUITE.erl b/deps/rabbit/test/simple_ha_SUITE.erl index 920d67bbc1aa..10ba9f4356ee 100644 --- a/deps/rabbit/test/simple_ha_SUITE.erl +++ b/deps/rabbit/test/simple_ha_SUITE.erl @@ -50,7 +50,12 @@ groups() -> init_per_suite(Config) -> rabbit_ct_helpers:log_environment(), - rabbit_ct_helpers:run_setup_steps(Config). + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + mnesia -> + rabbit_ct_helpers:run_setup_steps(Config); + {khepri, _} -> + {skip, "Classic queue mirroring not supported by Khepri"} + end. end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). diff --git a/deps/rabbit/test/sync_detection_SUITE.erl b/deps/rabbit/test/sync_detection_SUITE.erl index 4bd14191b37b..bf22a6eb46e7 100644 --- a/deps/rabbit/test/sync_detection_SUITE.erl +++ b/deps/rabbit/test/sync_detection_SUITE.erl @@ -36,7 +36,12 @@ groups() -> init_per_suite(Config) -> rabbit_ct_helpers:log_environment(), - rabbit_ct_helpers:run_setup_steps(Config). + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + mnesia -> + rabbit_ct_helpers:run_setup_steps(Config); + {khepri, _} -> + {skip, "Classic queue mirroring not supported by Khepri"} + end. end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). diff --git a/deps/rabbit/test/topic_permission_SUITE.erl b/deps/rabbit/test/topic_permission_SUITE.erl index b39b3e8b77ad..0aa2f6b9f212 100644 --- a/deps/rabbit/test/topic_permission_SUITE.erl +++ b/deps/rabbit/test/topic_permission_SUITE.erl @@ -58,16 +58,14 @@ topic_permission_database_access(Config) -> ?MODULE, topic_permission_database_access1, [Config]). topic_permission_database_access1(_Config) -> - 0 = length(ets:tab2list(rabbit_topic_permission)), rabbit_vhost:add(<<"/">>, <<"acting-user">>), rabbit_vhost:add(<<"other-vhost">>, <<"acting-user">>), rabbit_auth_backend_internal:add_user(<<"guest">>, <<"guest">>, <<"acting-user">>), rabbit_auth_backend_internal:add_user(<<"dummy">>, <<"dummy">>, <<"acting-user">>), - rabbit_auth_backend_internal:set_topic_permissions( - <<"guest">>, <<"/">>, <<"amq.topic">>, "^a", "^a", <<"acting-user">> - ), - 1 = length(ets:tab2list(rabbit_topic_permission)), + ok = rabbit_auth_backend_internal:set_topic_permissions( + <<"guest">>, <<"/">>, <<"amq.topic">>, "^a", "^a", <<"acting-user">> + ), 1 = length(rabbit_auth_backend_internal:list_user_topic_permissions(<<"guest">>)), 0 = length(rabbit_auth_backend_internal:list_user_topic_permissions(<<"dummy">>)), 1 = length(rabbit_auth_backend_internal:list_vhost_topic_permissions(<<"/">>)), @@ -79,7 +77,6 @@ topic_permission_database_access1(_Config) -> rabbit_auth_backend_internal:set_topic_permissions( <<"guest">>, <<"other-vhost">>, <<"amq.topic">>, ".*", ".*", <<"acting-user">> ), - 2 = length(ets:tab2list(rabbit_topic_permission)), 2 = length(rabbit_auth_backend_internal:list_user_topic_permissions(<<"guest">>)), 0 = length(rabbit_auth_backend_internal:list_user_topic_permissions(<<"dummy">>)), 1 = length(rabbit_auth_backend_internal:list_vhost_topic_permissions(<<"/">>)), @@ -142,16 +139,15 @@ topic_permission_checks(Config) -> ?MODULE, topic_permission_checks1, [Config]). topic_permission_checks1(_Config) -> - 0 = length(ets:tab2list(rabbit_topic_permission)), - rabbit_db_vhost:create_or_get(<<"/">>, [], #{}), - rabbit_db_vhost:create_or_get(<<"other-vhost">>, [], #{}), + rabbit_vhost:add(<<"/">>, <<"">>), + rabbit_vhost:add(<<"other-vhost">>, <<"">>), + rabbit_auth_backend_internal:add_user(<<"guest">>, <<"guest">>, <<"acting-user">>), rabbit_auth_backend_internal:add_user(<<"dummy">>, <<"dummy">>, <<"acting-user">>), rabbit_auth_backend_internal:set_topic_permissions( <<"guest">>, <<"/">>, <<"amq.topic">>, "^a", "^a", <<"acting-user">> ), - 1 = length(ets:tab2list(rabbit_topic_permission)), 1 = length(rabbit_auth_backend_internal:list_user_topic_permissions(<<"guest">>)), 0 = length(rabbit_auth_backend_internal:list_user_topic_permissions(<<"dummy">>)), 1 = length(rabbit_auth_backend_internal:list_vhost_topic_permissions(<<"/">>)), @@ -160,7 +156,6 @@ topic_permission_checks1(_Config) -> rabbit_auth_backend_internal:set_topic_permissions( <<"guest">>, <<"other-vhost">>, <<"amq.topic">>, ".*", ".*", <<"acting-user">> ), - 2 = length(ets:tab2list(rabbit_topic_permission)), 2 = length(rabbit_auth_backend_internal:list_user_topic_permissions(<<"guest">>)), 0 = length(rabbit_auth_backend_internal:list_user_topic_permissions(<<"dummy">>)), 1 = length(rabbit_auth_backend_internal:list_vhost_topic_permissions(<<"/">>)), diff --git a/deps/rabbit/test/unit_policy_validators_SUITE.erl b/deps/rabbit/test/unit_policy_validators_SUITE.erl index d6b8cfe7eb31..c1ab09eae7c8 100644 --- a/deps/rabbit/test/unit_policy_validators_SUITE.erl +++ b/deps/rabbit/test/unit_policy_validators_SUITE.erl @@ -53,12 +53,19 @@ end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). init_per_group(Group = classic_queue_mirroring_validators, Config) -> - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodename_suffix, Group}, - {rmq_nodes_count, 1} - ]), - rabbit_ct_helpers:run_steps(Config1, - rabbit_ct_broker_helpers:setup_steps()); + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + mnesia -> + Config1 = rabbit_ct_helpers:set_config( + Config, [ + {rmq_nodename_suffix, Group}, + {rmq_nodes_count, 1} + ]), + rabbit_ct_helpers:run_steps( + Config1, + rabbit_ct_broker_helpers:setup_steps()); + {khepri, _} -> + {skip, "Classic queue mirroring not supported by Khepri"} + end; init_per_group(_, Config) -> Config. diff --git a/deps/rabbit/test/vhost_SUITE.erl b/deps/rabbit/test/vhost_SUITE.erl index b32bb83b2142..31f628b3ba2d 100644 --- a/deps/rabbit/test/vhost_SUITE.erl +++ b/deps/rabbit/test/vhost_SUITE.erl @@ -99,6 +99,17 @@ end_per_group(_Group, Config) -> rabbit_ct_client_helpers:teardown_steps() ++ rabbit_ct_broker_helpers:teardown_steps()). +init_per_testcase(node_starts_with_dead_vhosts_with_mirrors = Testcase, Config) -> + case lists:any(fun(B) -> B end, + rabbit_ct_broker_helpers:rpc_all( + Config, rabbit_feature_flags, is_enabled, + [khepri_db])) of + true -> + {skip, "Classic queue mirroring not supported by Khepri"}; + false -> + rabbit_ct_helpers:testcase_started(Config, Testcase), + Config + end; init_per_testcase(Testcase, Config) -> rabbit_ct_helpers:testcase_started(Config, Testcase), Config. diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/force_boot_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/force_boot_command.ex index 00c4101d6a85..237cadf00772 100644 --- a/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/force_boot_command.ex +++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/force_boot_command.ex @@ -43,6 +43,10 @@ defmodule RabbitMQ.CLI.Ctl.Commands.ForceBootCommand do File.write(Path.join(dir, "force_load"), "") end + {:error, :not_supported} -> + {:error, RabbitMQ.CLI.Core.ExitCodes.exit_usage(), + "This command is not supported by node #{node_name}"} + _ -> :ok end diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/force_standalone_khepri_boot.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/force_standalone_khepri_boot.ex new file mode 100644 index 000000000000..aaea2c51d7d8 --- /dev/null +++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/force_standalone_khepri_boot.ex @@ -0,0 +1,45 @@ +## This Source Code Form is subject to the terms of the Mozilla Public +## License, v. 2.0. If a copy of the MPL was not distributed with this +## file, You can obtain one at https://mozilla.org/MPL/2.0/. +## +## Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved. + +defmodule RabbitMQ.CLI.Ctl.Commands.ForceStandaloneKhepriBootCommand do + alias RabbitMQ.CLI.Core.{Config, DocGuide} + + @behaviour RabbitMQ.CLI.CommandBehaviour + + use RabbitMQ.CLI.Core.MergesNoDefaults + use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments + + def run([], %{node: node_name} = opts) do + ret = + :rabbit_misc.rpc_call(node_name, :rabbit_khepri, :force_shrink_member_to_current_member, []) + + case ret do + {:badrpc, {:EXIT, {:undef, _}}} -> + {:error, RabbitMQ.CLI.Core.ExitCodes.exit_usage(), + "This command is not supported by node #{node_name}"} + + _ -> + ret + end + end + + use RabbitMQ.CLI.DefaultOutput + + def usage, do: "force_standalone_khepri_boot" + + def usage_doc_guides() do + [ + DocGuide.clustering() + ] + end + + def help_section(), do: :cluster_management + + def description(), + do: "Forces node to start as a standalone node" + + def banner(_, _), do: nil +end diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/join_cluster_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/join_cluster_command.ex index 11b9695ac508..4da60181927b 100644 --- a/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/join_cluster_command.ex +++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/join_cluster_command.ex @@ -78,6 +78,21 @@ defmodule RabbitMQ.CLI.Ctl.Commands.JoinClusterCommand do "Error: cannot cluster node with itself: #{node_name}"} end + def output({:error, {:node_type_unsupported, db, node_type}}, %{node: node_name}) do + {:error, RabbitMQ.CLI.Core.ExitCodes.exit_software(), + "Error: `#{node_type}` node type is unsupported by the #{db} by database engine"} + end + + def output( + {:error, + {:khepri_mnesia_migration_ex, :all_mnesia_nodes_must_run, + %{all_nodes: nodes, running_nodes: running}}}, + _opts + ) do + {:error, RabbitMQ.CLI.Core.ExitCodes.exit_software(), + "Error: all mnesia nodes must run to join the cluster, mnesia nodes: #{inspect(nodes)}, running nodes: #{inspect(running)}"} + end + use RabbitMQ.CLI.DefaultOutput def banner([target_node], %{node: node_name}) do diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/rename_cluster_node_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/rename_cluster_node_command.ex index 1d9afcb1e7fc..23f945071d59 100644 --- a/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/rename_cluster_node_command.ex +++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/rename_cluster_node_command.ex @@ -38,10 +38,15 @@ defmodule RabbitMQ.CLI.Ctl.Commands.RenameClusterNodeCommand do node_pairs = make_node_pairs(nodes) try do - :rabbit_mnesia_rename.rename(node_name, node_pairs) + :rabbit_db_cluster.rename(node_name, node_pairs) catch - _, reason -> - {:rename_failed, reason} + :error, :undef -> + try do + :rabbit_mnesia_rename.rename(node_name, node_pairs) + catch + _, reason -> + {:rename_failed, reason} + end end end diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/default_output.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/default_output.ex index 81199ce76a98..8e77a622558f 100644 --- a/deps/rabbitmq_cli/lib/rabbitmq/cli/default_output.ex +++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/default_output.ex @@ -17,7 +17,7 @@ defmodule RabbitMQ.CLI.DefaultOutput do end def output(result, opts \\ %{}) do - format_output(normalize_output(result, opts)) + format_output(format_khepri_output(normalize_output(result, opts), opts)) end def mnesia_running_error(node_name) do @@ -25,6 +25,11 @@ defmodule RabbitMQ.CLI.DefaultOutput do "Please stop RabbitMQ with 'rabbitmqctl stop_app' first." end + def khepri_timeout_error(node_name) do + "Khepri has timed out on node #{node_name}.\n" <> + "Khepri cluster could be in minority." + end + defp normalize_output(:ok, %{node: node_name, formatter: "json"}) do {:ok, %{"result" => "ok", "node" => node_name}} end @@ -63,6 +68,18 @@ defmodule RabbitMQ.CLI.DefaultOutput do defp normalize_output({unknown, _} = input, _opts) when is_atom(unknown), do: {:error, input} defp normalize_output(result, _opts) when not is_atom(result), do: {:ok, result} + defp format_khepri_output({:error, {:timeout, {:metadata_store, _}}}, %{node: node_name}) do + {:error, RabbitMQ.CLI.Core.ExitCodes.exit_tempfail(), khepri_timeout_error(node_name)} + end + + defp format_khepri_output({:error, :timeout_waiting_for_leader}, %{node: node_name}) do + {:error, RabbitMQ.CLI.Core.ExitCodes.exit_tempfail(), khepri_timeout_error(node_name)} + end + + defp format_khepri_output(result, _opts) do + result + end + defp format_output({:error, _} = result) do result end diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/metadata_store_status_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/metadata_store_status_command.ex new file mode 100644 index 000000000000..49ebfc47825f --- /dev/null +++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/metadata_store_status_command.ex @@ -0,0 +1,36 @@ +## This Source Code Form is subject to the terms of the Mozilla Public +## License, v. 2.0. If a copy of the MPL was not distributed with this +## file, You can obtain one at https://mozilla.org/MPL/2.0/. +## +## Copyright (c) 2022 VMware, Inc. or its affiliates. All rights reserved. + +defmodule RabbitMQ.CLI.Diagnostics.Commands.MetadataStoreStatusCommand do + alias RabbitMQ.CLI.Core.DocGuide + + @behaviour RabbitMQ.CLI.CommandBehaviour + def scopes(), do: [:diagnostics] + + def merge_defaults(args, opts), do: {args, Map.merge(%{vhost: "/"}, opts)} + + use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments + use RabbitMQ.CLI.Core.RequiresRabbitAppRunning + + def run([] = _args, %{node: node_name}) do + :rabbit_misc.rpc_call(node_name, :rabbit_khepri, :status, []) + end + + use RabbitMQ.CLI.DefaultOutput + + def formatter(), do: RabbitMQ.CLI.Formatters.PrettyTable + + def usage() do + "metadata_store_status" + end + + def help_section(), do: :observability_and_health_checks + + def description(), do: "Displays quorum status of Khepri metadata store" + + def banner([], %{node: node_name}), + do: "Status of metadata store on node #{node_name} ..." +end diff --git a/deps/rabbitmq_cli/test/ctl/change_cluster_node_type_command_test.exs b/deps/rabbitmq_cli/test/ctl/change_cluster_node_type_command_test.exs index 834a0f8712a8..b11b2e912d6c 100644 --- a/deps/rabbitmq_cli/test/ctl/change_cluster_node_type_command_test.exs +++ b/deps/rabbitmq_cli/test/ctl/change_cluster_node_type_command_test.exs @@ -59,10 +59,21 @@ defmodule ChangeClusterNodeTypeCommandTest do # end test "run: request to a node with running RabbitMQ app fails", context do - assert match?( - {:error, :mnesia_unexpectedly_running}, - @command.run(["ram"], context[:opts]) - ) + node = RabbitMQ.CLI.Core.Helpers.normalise_node(context[:node], :shortnames) + + case :rabbit_misc.rpc_call(node, :rabbit_khepri, :is_enabled, []) do + true -> + assert match?( + :ok, + @command.run(["ram"], context[:opts]) + ) + + false -> + assert match?( + {:error, :mnesia_unexpectedly_running}, + @command.run(["ram"], context[:opts]) + ) + end end test "run: request to an unreachable node returns a badrpc", _context do diff --git a/deps/rabbitmq_cli/test/ctl/list_global_parameters_command_test.exs b/deps/rabbitmq_cli/test/ctl/list_global_parameters_command_test.exs index 272b7cfd1e09..0b3e45a78fbc 100644 --- a/deps/rabbitmq_cli/test/ctl/list_global_parameters_command_test.exs +++ b/deps/rabbitmq_cli/test/ctl/list_global_parameters_command_test.exs @@ -81,12 +81,12 @@ defmodule ListGlobalParametersCommandTest do # Checks each element of the first parameter against the expected context values defp assert_parameter_list(params, context) do - [param | _] = params + exp = + MapSet.new( + name: context[:key], + value: context[:value] + ) - assert MapSet.new(param) == - MapSet.new( - name: context[:key], - value: context[:value] - ) + assert List.foldl(params, false, fn param, acc -> MapSet.new(param) == exp or acc end) end end diff --git a/deps/rabbitmq_cli/test/ctl/set_global_parameter_command_test.exs b/deps/rabbitmq_cli/test/ctl/set_global_parameter_command_test.exs index 683ee1397aac..6f03f9ede1e5 100644 --- a/deps/rabbitmq_cli/test/ctl/set_global_parameter_command_test.exs +++ b/deps/rabbitmq_cli/test/ctl/set_global_parameter_command_test.exs @@ -79,9 +79,14 @@ defmodule SetGlobalParameterCommandTest do # Checks each element of the first parameter against the expected context values defp assert_parameter_fields(context) do - result_param = list_global_parameters() |> List.first() + result_params = list_global_parameters() - assert result_param[:value] == context[:value] - assert result_param[:name] == context[:key] + exp = + MapSet.new( + name: context[:key], + value: context[:value] + ) + + assert List.foldl(result_params, false, fn param, acc -> MapSet.new(param) == exp or acc end) end end diff --git a/deps/rabbitmq_consistent_hash_exchange/BUILD.bazel b/deps/rabbitmq_consistent_hash_exchange/BUILD.bazel index 933db1f24920..abeda2bf34d1 100644 --- a/deps/rabbitmq_consistent_hash_exchange/BUILD.bazel +++ b/deps/rabbitmq_consistent_hash_exchange/BUILD.bazel @@ -24,8 +24,6 @@ APP_DESCRIPTION = "Consistent Hash Exchange Type" all_beam_files(name = "all_beam_files") -all_test_beam_files(name = "all_test_beam_files") - all_srcs(name = "all_srcs") test_suite_beam_files(name = "test_suite_beam_files") @@ -43,6 +41,8 @@ rabbitmq_app( deps = [ "//deps/rabbit:erlang_app", "//deps/rabbit_common:erlang_app", + "@khepri//:erlang_app", + "@khepri_mnesia_migration//:erlang_app", ], ) @@ -82,7 +82,7 @@ broker_for_integration_suites() rabbitmq_integration_suite( name = "rabbit_exchange_type_consistent_hash_SUITE", - shard_count = 3, + shard_count = 7, ) assert_suites() @@ -92,3 +92,5 @@ alias( actual = ":erlang_app", visibility = ["//visibility:public"], ) + +all_test_beam_files(name = "all_test_beam_files") diff --git a/deps/rabbitmq_consistent_hash_exchange/app.bzl b/deps/rabbitmq_consistent_hash_exchange/app.bzl index ba5eb88887cb..e6a43a75079f 100644 --- a/deps/rabbitmq_consistent_hash_exchange/app.bzl +++ b/deps/rabbitmq_consistent_hash_exchange/app.bzl @@ -11,6 +11,7 @@ def all_beam_files(name = "all_beam_files"): srcs = [ "src/Elixir.RabbitMQ.CLI.Diagnostics.Commands.ConsistentHashExchangeRingStateCommand.erl", "src/rabbit_db_ch_exchange.erl", + "src/rabbit_db_ch_exchange_m2k_converter.erl", "src/rabbit_exchange_type_consistent_hash.erl", ], hdrs = [":public_and_private_hdrs"], @@ -21,6 +22,8 @@ def all_beam_files(name = "all_beam_files"): "//deps/rabbit:erlang_app", "//deps/rabbit_common:erlang_app", "//deps/rabbitmq_cli:erlang_app", + "@khepri//:erlang_app", + "@khepri_mnesia_migration//:erlang_app", ], ) @@ -36,6 +39,7 @@ def all_test_beam_files(name = "all_test_beam_files"): srcs = [ "src/Elixir.RabbitMQ.CLI.Diagnostics.Commands.ConsistentHashExchangeRingStateCommand.erl", "src/rabbit_db_ch_exchange.erl", + "src/rabbit_db_ch_exchange_m2k_converter.erl", "src/rabbit_exchange_type_consistent_hash.erl", ], hdrs = [":public_and_private_hdrs"], @@ -46,6 +50,8 @@ def all_test_beam_files(name = "all_test_beam_files"): "//deps/rabbit:erlang_app", "//deps/rabbit_common:erlang_app", "//deps/rabbitmq_cli:erlang_app", + "@khepri//:erlang_app", + "@khepri_mnesia_migration//:erlang_app", ], ) @@ -68,6 +74,7 @@ def all_srcs(name = "all_srcs"): srcs = [ "src/Elixir.RabbitMQ.CLI.Diagnostics.Commands.ConsistentHashExchangeRingStateCommand.erl", "src/rabbit_db_ch_exchange.erl", + "src/rabbit_db_ch_exchange_m2k_converter.erl", "src/rabbit_exchange_type_consistent_hash.erl", ], ) diff --git a/deps/rabbitmq_consistent_hash_exchange/src/rabbit_db_ch_exchange.erl b/deps/rabbitmq_consistent_hash_exchange/src/rabbit_db_ch_exchange.erl index 5af7a53a3ae7..1c7773034ebb 100644 --- a/deps/rabbitmq_consistent_hash_exchange/src/rabbit_db_ch_exchange.erl +++ b/deps/rabbitmq_consistent_hash_exchange/src/rabbit_db_ch_exchange.erl @@ -6,6 +6,10 @@ %% -module(rabbit_db_ch_exchange). +-include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("khepri/include/khepri.hrl"). +-include("rabbitmq_consistent_hash_exchange.hrl"). + -export([ setup_schema/0, create/1, @@ -15,13 +19,21 @@ delete_bindings/2 ]). --include_lib("rabbit_common/include/rabbit.hrl"). --include("rabbitmq_consistent_hash_exchange.hrl"). +-export([ + khepri_consistent_hash_path/0, + khepri_consistent_hash_path/1 + ]). -define(HASH_RING_STATE_TABLE, rabbit_exchange_type_consistent_hash_ring_state). +-rabbit_mnesia_tables_to_khepri_db( + [{?HASH_RING_STATE_TABLE, rabbit_db_ch_exchange_m2k_converter}]). + setup_schema() -> - setup_schema_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> setup_schema_in_mnesia() end, + khepri => ok + }). setup_schema_in_mnesia() -> _ = mnesia:create_table(?HASH_RING_STATE_TABLE, [{record_name, chx_hash_ring}, @@ -31,7 +43,10 @@ setup_schema_in_mnesia() -> rabbit_table:wait([?HASH_RING_STATE_TABLE]). create(X) -> - create_in_mnesia(X). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> create_in_mnesia(X) end, + khepri => fun() -> create_in_khepri(X) end + }). create_in_mnesia(X) -> rabbit_mnesia:execute_mnesia_transaction( @@ -49,8 +64,21 @@ create_in_mnesia_tx(X) -> bucket_map = #{}}, write) end. +create_in_khepri(X) -> + Path = khepri_consistent_hash_path(X), + case rabbit_khepri:create(Path, #chx_hash_ring{exchange = X, + next_bucket_number = 0, + bucket_map = #{}}) of + ok -> ok; + {error, {khepri, mismatching_node, _}} -> ok; + Error -> Error + end. + create_binding(Src, Dst, Weight, UpdateFun) -> - create_binding_in_mnesia(Src, Dst, Weight, UpdateFun). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> create_binding_in_mnesia(Src, Dst, Weight, UpdateFun) end, + khepri => fun() -> create_binding_in_khepri(Src, Dst, Weight, UpdateFun) end + }). create_binding_in_mnesia(Src, Dst, Weight, UpdateFun) -> rabbit_mnesia:execute_mnesia_transaction( @@ -73,8 +101,42 @@ create_binding_in_mnesia_tx(Src, Dst, Weight, UpdateFun) -> create_binding_in_mnesia_tx(Src, Dst, Weight, UpdateFun) end. +create_binding_in_khepri(Src, Dst, Weight, UpdateFun) -> + Path = khepri_consistent_hash_path(Src), + case rabbit_khepri:adv_get(Path) of + {ok, #{data := Chx0, payload_version := DVersion}} -> + case UpdateFun(Chx0, Dst, Weight) of + already_exists -> + already_exists; + Chx -> + Path1 = khepri_path:combine_with_conditions( + Path, [#if_payload_version{version = DVersion}]), + Ret2 = rabbit_khepri:put(Path1, Chx), + case Ret2 of + ok -> + created; + {error, {khepri, mismatching_node, _}} -> + create_binding_in_khepri(Src, Dst, Weight, UpdateFun); + {error, _} = Error -> + Error + end + end; + _ -> + case rabbit_khepri:create(Path, #chx_hash_ring{exchange = Src, + next_bucket_number = 0, + bucket_map = #{}}) of + ok -> ok; + {error, {khepri, mismatching_node, _}} -> + create_binding_in_khepri(Src, Dst, Weight, UpdateFun); + Error -> throw(Error) + end + end. + get(XName) -> - get_in_mnesia(XName). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_in_mnesia(XName) end, + khepri => fun() -> get_in_khepri(XName) end + }). get_in_mnesia(XName) -> case ets:lookup(?HASH_RING_STATE_TABLE, XName) of @@ -84,8 +146,20 @@ get_in_mnesia(XName) -> Chx end. +get_in_khepri(XName) -> + Path = khepri_consistent_hash_path(XName), + case rabbit_khepri:get(Path) of + {ok, Chx} -> + Chx; + _ -> + undefined + end. + delete(XName) -> - delete_in_mnesia(XName). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> delete_in_mnesia(XName) end, + khepri => fun() -> delete_in_khepri(XName) end + }). delete_in_mnesia(XName) -> rabbit_mnesia:execute_mnesia_transaction( @@ -94,8 +168,14 @@ delete_in_mnesia(XName) -> mnesia:delete({?HASH_RING_STATE_TABLE, XName}) end). +delete_in_khepri(XName) -> + rabbit_khepri:delete(khepri_consistent_hash_path(XName)). + delete_bindings(Bindings, DeleteFun) -> - delete_bindings_in_mnesia(Bindings, DeleteFun). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> delete_bindings_in_mnesia(Bindings, DeleteFun) end, + khepri => fun() -> delete_bindings_in_khepri(Bindings, DeleteFun) end + }). delete_bindings_in_mnesia(Bindings, DeleteFun) -> rabbit_mnesia:execute_mnesia_transaction( @@ -118,3 +198,31 @@ delete_binding_in_mnesia(#binding{source = S, destination = D, key = RK}, Delete [] -> {not_found, S} end. + +delete_bindings_in_khepri(Bindings, DeleteFun) -> + rabbit_khepri:transaction( + fun() -> + [delete_binding_in_khepri(Binding, DeleteFun) || Binding <- Bindings] + end). + +delete_binding_in_khepri(#binding{source = S, destination = D}, DeleteFun) -> + Path = khepri_consistent_hash_path(S), + case khepri_tx:get(Path) of + {ok, Chx0} -> + case DeleteFun(Chx0, D) of + not_found -> + ok; + Chx -> + ok = khepri_tx:put(Path, Chx) + end; + _ -> + {not_found, S} + end. + +khepri_consistent_hash_path(#exchange{name = Name}) -> + khepri_consistent_hash_path(Name); +khepri_consistent_hash_path(#resource{virtual_host = VHost, name = Name}) -> + [?MODULE, exchange_type_consistent_hash_ring_state, VHost, Name]. + +khepri_consistent_hash_path() -> + [?MODULE, exchange_type_consistent_hash_ring_state]. diff --git a/deps/rabbitmq_consistent_hash_exchange/src/rabbit_db_ch_exchange_m2k_converter.erl b/deps/rabbitmq_consistent_hash_exchange/src/rabbit_db_ch_exchange_m2k_converter.erl new file mode 100644 index 000000000000..a1660fa2652c --- /dev/null +++ b/deps/rabbitmq_consistent_hash_exchange/src/rabbit_db_ch_exchange_m2k_converter.erl @@ -0,0 +1,102 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright © 2022-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_ch_exchange_m2k_converter). + +-behaviour(mnesia_to_khepri_converter). + +-include_lib("kernel/include/logger.hrl"). +-include_lib("khepri/include/khepri.hrl"). +-include_lib("khepri_mnesia_migration/src/kmm_logging.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). + +-include("rabbitmq_consistent_hash_exchange.hrl"). + +-export([init_copy_to_khepri/3, + copy_to_khepri/3, + delete_from_khepri/3, + clear_data_in_khepri/1]). + +-record(?MODULE, {store_id :: khepri:store_id()}). + +-define(HASH_RING_STATE_TABLE, rabbit_exchange_type_consistent_hash_ring_state). + +-spec init_copy_to_khepri(StoreId, MigrationId, Tables) -> Ret when + StoreId :: khepri:store_id(), + MigrationId :: mnesia_to_khepri:migration_id(), + Tables :: [mnesia_to_khepri:mnesia_table()], + Ret :: {ok, Priv}, + Priv :: #?MODULE{}. +%% @private + +init_copy_to_khepri(StoreId, _MigrationId, _Tables) -> + State = #?MODULE{store_id = StoreId}, + {ok, State}. + +-spec copy_to_khepri(Table, Record, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Record :: tuple(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +copy_to_khepri(?HASH_RING_STATE_TABLE = Table, + #chx_hash_ring{exchange = XName} = Record, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] key: ~0p", + [Table, XName], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_ch_exchange:khepri_consistent_hash_path(XName), + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:put(StoreId, Path, Record) of + ok -> {ok, State}; + Error -> Error + end; +copy_to_khepri(Table, Record, State) -> + ?LOG_DEBUG("Mnesia->Khepri unexpected record table ~0p record ~0p state ~0p", + [Table, Record, State]), + {error, unexpected_record}. + +-spec delete_from_khepri(Table, Key, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Key :: any(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +delete_from_khepri(?HASH_RING_STATE_TABLE = Table, Key, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] key: ~0p", + [Table, Key], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_ch_exchange:khepri_consistent_hash_path(Key), + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:delete(StoreId, Path) of + ok -> {ok, State}; + Error -> Error + end. + +clear_data_in_khepri(?HASH_RING_STATE_TABLE) -> + Path = rabbit_db_ch_exchange:khepri_consistent_hash_path(), + case rabbit_khepri:delete(Path) of + ok -> + ok; + Error -> + throw(Error) + end. diff --git a/deps/rabbitmq_consistent_hash_exchange/test/rabbit_exchange_type_consistent_hash_SUITE.erl b/deps/rabbitmq_consistent_hash_exchange/test/rabbit_exchange_type_consistent_hash_SUITE.erl index cdfe70bc30cb..80e02996a805 100644 --- a/deps/rabbitmq_consistent_hash_exchange/test/rabbit_exchange_type_consistent_hash_SUITE.erl +++ b/deps/rabbitmq_consistent_hash_exchange/test/rabbit_exchange_type_consistent_hash_SUITE.erl @@ -20,37 +20,53 @@ all() -> [ - {group, routing_tests}, - {group, hash_ring_management_tests}, - {group, clustered} + {group, mnesia_store}, + {group, khepri_store}, + {group, khepri_migration} ]. groups() -> [ - {routing_tests, [], [ - routing_key_hashing_test, - custom_header_hashing_test, - message_id_hashing_test, - correlation_id_hashing_test, - timestamp_hashing_test, - other_routing_test - ]}, - {hash_ring_management_tests, [], [ - test_durable_exchange_hash_ring_recovery_between_node_restarts, - test_hash_ring_updates_when_queue_is_deleted, - test_hash_ring_updates_when_multiple_queues_are_deleted, - test_hash_ring_updates_when_exclusive_queues_are_deleted_due_to_connection_closure, - test_hash_ring_updates_when_exclusive_queues_are_deleted_due_to_connection_closure_case2, - test_hash_ring_updates_when_exclusive_queues_are_deleted_due_to_connection_closure_case3, - test_hash_ring_updates_when_exclusive_queues_are_deleted_due_to_connection_closure_case4, - test_hash_ring_updates_when_exclusive_queues_are_deleted_due_to_connection_closure_case5, - test_hash_ring_updates_when_exclusive_queues_are_deleted_due_to_connection_closure_case6, - test_hash_ring_updates_when_exchange_is_deleted, - test_hash_ring_updates_when_queue_is_unbound, - test_hash_ring_updates_when_duplicate_binding_is_created_and_queue_is_deleted, - test_hash_ring_updates_when_duplicate_binding_is_created_and_binding_is_deleted - ]}, - {clustered, [], [node_restart]} + {mnesia_store, [], [ + {routing_tests, [], routing_tests()}, + {hash_ring_management_tests, [], hash_ring_management_tests()}, + {clustered, [], [node_restart]} + ]}, + {khepri_store, [], [ + {routing_tests, [], routing_tests()}, + {hash_ring_management_tests, [], hash_ring_management_tests()}, + {clustered, [], [node_restart]} + ]}, + {khepri_migration, [], [ + from_mnesia_to_khepri + ]} + ]. + +routing_tests() -> + [ + routing_key_hashing_test, + custom_header_hashing_test, + message_id_hashing_test, + correlation_id_hashing_test, + timestamp_hashing_test, + other_routing_test + ]. + +hash_ring_management_tests() -> + [ + test_durable_exchange_hash_ring_recovery_between_node_restarts, + test_hash_ring_updates_when_queue_is_deleted, + test_hash_ring_updates_when_multiple_queues_are_deleted, + test_hash_ring_updates_when_exclusive_queues_are_deleted_due_to_connection_closure, + test_hash_ring_updates_when_exclusive_queues_are_deleted_due_to_connection_closure_case2, + test_hash_ring_updates_when_exclusive_queues_are_deleted_due_to_connection_closure_case3, + test_hash_ring_updates_when_exclusive_queues_are_deleted_due_to_connection_closure_case4, + test_hash_ring_updates_when_exclusive_queues_are_deleted_due_to_connection_closure_case5, + test_hash_ring_updates_when_exclusive_queues_are_deleted_due_to_connection_closure_case6, + test_hash_ring_updates_when_exchange_is_deleted, + test_hash_ring_updates_when_queue_is_unbound, + test_hash_ring_updates_when_duplicate_binding_is_created_and_queue_is_deleted, + test_hash_ring_updates_when_duplicate_binding_is_created_and_binding_is_deleted ]. %% ------------------------------------------------------------------- @@ -59,11 +75,19 @@ groups() -> init_per_suite(Config) -> rabbit_ct_helpers:log_environment(), - rabbit_ct_helpers:run_setup_steps(Config, []). + rabbit_ct_helpers:run_setup_steps(Config). end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). +init_per_group(mnesia_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, mnesia}]); +init_per_group(khepri_store, Config) -> + FFs = [khepri_db], + rabbit_ct_helpers:set_config(Config, [{metadata_store, {khepri, FFs}}]); +init_per_group(khepri_migration = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, mnesia}]), + init_per_group(Group, Config, 1); init_per_group(clustered = Group, Config) -> case rabbit_ct_helpers:is_mixed_versions() of false -> @@ -87,10 +111,16 @@ init_per_group(Group, Config, NodesCount) -> rabbit_ct_broker_helpers:setup_steps() ++ rabbit_ct_client_helpers:setup_steps()). +end_per_group(mnesia_store, Config) -> + Config; +end_per_group(khepri_store, Config) -> + Config; +end_per_group(khepri_migration, Config) -> + Config; end_per_group(_, Config) -> - rabbit_ct_helpers:run_steps(Config, - rabbit_ct_client_helpers:teardown_steps() ++ - rabbit_ct_broker_helpers:teardown_steps()). + rabbit_ct_helpers:run_teardown_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). init_per_testcase(Testcase, Config) -> clean_up_test_topology(Config), @@ -443,14 +473,14 @@ test_hash_ring_updates_when_exclusive_queues_are_deleted_due_to_connection_closu routing_key = <<"3">>}) || Q <- Queues], - ct:pal("all hash ring rows: ~tp", [hash_ring_rows(Config)]), + ct:pal("hash ring state: ~tp", [hash_ring_state(Config, X)]), ?assertEqual(18, count_buckets_of_exchange(Config, X)), assert_ring_consistency(Config, X), ok = amqp_connection:close(Conn), timer:sleep(500), - ct:pal("all hash ring rows after connection closure: ~tp", [hash_ring_rows(Config)]), + ct:pal("hash ring state after connection closure: ~tp", [hash_ring_state(Config, X)]), ?awaitMatch(0, count_buckets_of_exchange(Config, X), ?DEFAULT_WAIT, ?DEFAULT_INTERVAL), clean_up_test_topology(Config, X, []), @@ -499,7 +529,7 @@ test_hash_ring_updates_when_exclusive_queues_are_deleted_due_to_connection_closu routing_key = integer_to_binary(Key)}) || Q <- Queues], - ct:pal("all hash ring rows: ~tp", [hash_ring_rows(Config)]), + ct:pal("hash ring state: ~tp", [hash_ring_state(Config, X)]), %% NumQueues x 'Key' buckets per binding ?assertEqual(NumQueues * Key, count_buckets_of_exchange(Config, X)), @@ -507,7 +537,7 @@ test_hash_ring_updates_when_exclusive_queues_are_deleted_due_to_connection_closu ok = amqp_connection:close(Conn), timer:sleep(1000), - ct:pal("all hash ring rows after connection closure (~tp): ~tp", [XAsList, hash_ring_rows(Config)]), + ct:pal("hash ring state after connection closure (~tp): ~tp", [XAsList, hash_ring_state(Config, X)]), ?awaitMatch(0, count_buckets_of_exchange(Config, X), ?DEFAULT_WAIT, ?DEFAULT_INTERVAL), clean_up_test_topology(Config, X, []), @@ -717,7 +747,7 @@ node_restart(Config) -> rabbit_ct_broker_helpers:restart_node(Config, 1), rabbit_ct_broker_helpers:restart_node(Config, 2), - ?assertEqual(4, count_all_hash_ring_buckets(Config)), + ?assertEqual(4, count_buckets_of_exchange(Config, X)), assert_ring_consistency(Config, X), clean_up_test_topology(Config, X, QsNode1 ++ QsNode2), @@ -729,16 +759,10 @@ node_restart(Config) -> hash_ring_state(Config, X) -> rabbit_ct_broker_helpers:rpc( - Config, 0, ets, lookup, - [rabbit_exchange_type_consistent_hash_ring_state, - rabbit_misc:r(<<"/">>, exchange, X)]). - -hash_ring_rows(Config) -> - rabbit_ct_broker_helpers:rpc( - Config, 0, ets, tab2list, [rabbit_exchange_type_consistent_hash_ring_state]). + Config, 0, rabbit_exchange_type_consistent_hash, ring_state, [<<"/">>, X]). assert_ring_consistency(Config, X) -> - [#chx_hash_ring{bucket_map = M}] = hash_ring_state(Config, X), + {ok, #chx_hash_ring{bucket_map = M}} = hash_ring_state(Config, X), Buckets = lists:usort(maps:keys(M)), Hi = lists:last(Buckets), @@ -747,13 +771,78 @@ assert_ring_consistency(Config, X) -> count_buckets_of_exchange(Config, X) -> case hash_ring_state(Config, X) of - [#chx_hash_ring{bucket_map = M}] -> maps:size(M); - [] -> 0 + {ok, #chx_hash_ring{bucket_map = M}} -> + ct:pal("BUCKET MAP ~p", [M]), + maps:size(M); + {error, not_found} -> 0 end. -count_all_hash_ring_buckets(Config) -> - Rows = hash_ring_rows(Config), - lists:foldl(fun(#chx_hash_ring{bucket_map = M}, Acc) -> Acc + maps:size(M) end, 0, Rows). +from_mnesia_to_khepri(Config) -> + Queues = [Q1, Q2, Q3, Q4] = ?RoutingTestQs, + IterationCount = ?DEFAULT_SAMPLE_COUNT, + Chan = rabbit_ct_client_helpers:open_channel(Config, 0), + #'confirm.select_ok'{} = amqp_channel:call(Chan, #'confirm.select'{}), + + CHX = <<"e">>, + + clean_up_test_topology(Config, CHX, Queues), + + #'exchange.declare_ok'{} = + amqp_channel:call(Chan, + #'exchange.declare' { + exchange = CHX, + type = <<"x-consistent-hash">>, + auto_delete = true, + arguments = [] + }), + [#'queue.declare_ok'{} = + amqp_channel:call(Chan, #'queue.declare' { + queue = Q, exclusive = true }) || Q <- Queues], + [#'queue.bind_ok'{} = + amqp_channel:call(Chan, #'queue.bind' {queue = Q, + exchange = CHX, + routing_key = <<"1">>}) + || Q <- [Q1, Q2]], + [#'queue.bind_ok'{} = + amqp_channel:call(Chan, #'queue.bind' {queue = Q, + exchange = CHX, + routing_key = <<"2">>}) + || Q <- [Q3, Q4]], + + case rabbit_ct_broker_helpers:enable_feature_flag(Config, khepri_db) of + ok -> + case rabbit_ct_broker_helpers:enable_feature_flag(Config, rabbit_consistent_hash_exchange_raft_based_metadata_store) of + ok -> + [amqp_channel:call(Chan, + #'basic.publish'{exchange = CHX, routing_key = rnd()}, + #amqp_msg{props = #'P_basic'{}, payload = <<>>}) + || _ <- lists:duplicate(IterationCount, const)], + amqp_channel:wait_for_confirms(Chan, 300), + timer:sleep(500), + Counts = + [begin + #'queue.declare_ok'{message_count = M} = + amqp_channel:call(Chan, #'queue.declare' {queue = Q, + exclusive = true}), + M + end || Q <- Queues], + ?assertEqual(IterationCount, lists:sum(Counts)), %% All messages got routed + %% Chi-square test + %% H0: routing keys are not evenly distributed according to weight + Expected = [IterationCount div 6, IterationCount div 6, (IterationCount div 6) * 2, (IterationCount div 6) * 2], + Obs = lists:zip(Counts, Expected), + Chi = lists:sum([((O - E) * (O - E)) / E || {O, E} <- Obs]), + ct:pal("Chi-square test for 3 degrees of freedom is ~p, p = 0.01 is 11.35, observations (counts, expected): ~p", + [Chi, Obs]), + clean_up_test_topology(Config, CHX, Queues), + rabbit_ct_client_helpers:close_channel(Chan), + ok; + Skip -> + Skip + end; + Skip -> + Skip + end. clean_up_test_topology(Config) -> clean_up_test_topology(Config, none, ?AllQs). diff --git a/deps/rabbitmq_ct_helpers/src/rabbit_control_helper.erl b/deps/rabbitmq_ct_helpers/src/rabbit_control_helper.erl index eda2a6df1b1b..b850ec65137f 100644 --- a/deps/rabbitmq_ct_helpers/src/rabbit_control_helper.erl +++ b/deps/rabbitmq_ct_helpers/src/rabbit_control_helper.erl @@ -8,6 +8,7 @@ -module(rabbit_control_helper). -export([command/2, command/3, command/4, command_with_output/4, format_command/4]). +-export([async_command/4, wait_for_async_command/1]). command(Command, Node, Args) -> command(Command, Node, Args, []). @@ -22,6 +23,21 @@ command(Command, Node, Args, Opts) -> Error -> Error end. +async_command(Command, Node, Args, Opts) -> + Self = self(), + spawn(fun() -> + Reply = (catch command(Command, Node, Args, Opts)), + Self ! {async_command, Node, Reply} + end). + +wait_for_async_command(Node) -> + receive + {async_command, N, Reply} when N == Node -> + Reply + after 600000 -> + timeout + end. + command_with_output(Command, Node, Args, Opts) -> Formatted = format_command(Command, Node, Args, Opts), CommandResult = 'Elixir.RabbitMQCtl':exec_command( diff --git a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl index 31c7ade7eddc..09ef082fc4fa 100644 --- a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl +++ b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl @@ -42,6 +42,8 @@ rpc_all/4, rpc_all/5, start_node/2, + async_start_node/2, + wait_for_async_start_node/1, start_broker/2, restart_broker/2, stop_broker/2, @@ -169,7 +171,9 @@ test_channel/0, test_writer/1, - user/1 + user/1, + + configured_metadata_store/1 ]). %% Internal functions exported to be used by rpc:call/4. @@ -214,7 +218,8 @@ setup_steps() -> fun rabbit_ct_helpers:ensure_rabbitmq_plugins_cmd/1, fun set_lager_flood_limit/1, fun start_rabbitmq_nodes/1, - fun share_dist_and_proxy_ports_map/1 + fun share_dist_and_proxy_ports_map/1, + fun configure_metadata_store/1 ]; _ -> [ @@ -223,7 +228,8 @@ setup_steps() -> fun rabbit_ct_helpers:ensure_rabbitmq_plugins_cmd/1, fun set_lager_flood_limit/1, fun start_rabbitmq_nodes/1, - fun share_dist_and_proxy_ports_map/1 + fun share_dist_and_proxy_ports_map/1, + fun configure_metadata_store/1 ] end. @@ -930,6 +936,54 @@ share_dist_and_proxy_ports_map(Config) -> application, set_env, [kernel, dist_and_proxy_ports_map, Map]), Config. +configured_metadata_store(Config) -> + case rabbit_ct_helpers:get_config(Config, metadata_store) of + khepri -> + {khepri, []}; + {khepri, _FFs0} = Khepri -> + Khepri; + mnesia -> + mnesia; + _ -> + case os:getenv("RABBITMQ_METADATA_STORE") of + "khepri" -> + {khepri, []}; + _ -> + mnesia + end + end. + +configure_metadata_store(Config) -> + ct:pal("Configuring metadata store..."), + case configured_metadata_store(Config) of + {khepri, FFs0} -> + case enable_khepri_metadata_store(Config, FFs0) of + {skip, _} = Skip -> + _ = stop_rabbitmq_nodes(Config), + Skip; + Config1 -> + Config1 + end; + mnesia -> + ct:pal("Enabling Mnesia metadata store"), + Config + end. + +enable_khepri_metadata_store(Config, FFs0) -> + ct:pal("Enabling Khepri metadata store"), + FFs = [khepri_db | FFs0], + lists:foldl(fun(_FF, {skip, _Reason} = Skip) -> + Skip; + (FF, C) -> + case enable_feature_flag(C, FF) of + ok -> + C; + Skip -> + ct:pal("Enabling metadata store failed: ~p", [Skip]), + Skip + end + end, Config, FFs). + rewrite_node_config_file(Config, Node) -> NodeConfig = get_node_config(Config, Node), I = if @@ -1786,6 +1840,22 @@ start_node(Config, Node) -> _ -> ok end. +async_start_node(Config, Node) -> + Self = self(), + spawn(fun() -> + Reply = (catch start_node(Config, Node)), + Self ! {async_start_node, Node, Reply} + end), + ok. + +wait_for_async_start_node(Node) -> + receive + {async_start_node, N, Reply} when N == Node -> + Reply + after 600000 -> + timeout + end. + start_broker(Config, Node) -> ok = rpc(Config, Node, rabbit, start, []). @@ -1889,11 +1959,26 @@ enable_feature_flag(Config, FeatureName) -> Nodes = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), enable_feature_flag(Config, Nodes, FeatureName). -enable_feature_flag(Config, [Node1 | _] = Nodes, FeatureName) -> +enable_feature_flag(Config, Nodes, FeatureName) -> case is_feature_flag_supported(Config, Nodes, FeatureName) of true -> - rabbit_ct_broker_helpers:rpc( - Config, Node1, rabbit_feature_flags, enable, [FeatureName]); + %% Nodes might not be clustered for some test suites, so enabling + %% feature flags on the first one of the list is not enough + lists:foldl( + fun(N, ok) -> + case rabbit_ct_broker_helpers:rpc( + Config, N, rabbit_feature_flags, enable, [FeatureName]) of + {error, unsupported} -> + {skip, + lists:flatten( + io_lib:format("'~ts' feature flag is unsupported", + [FeatureName]))}; + Any -> + Any + end; + (_, Other) -> + Other + end, ok, Nodes); false -> {skip, lists:flatten( diff --git a/deps/rabbitmq_federation/src/rabbit_federation_exchange_link_sup_sup.erl b/deps/rabbitmq_federation/src/rabbit_federation_exchange_link_sup_sup.erl index dd4402e467b9..bcf3d4f28bc4 100644 --- a/deps/rabbitmq_federation/src/rabbit_federation_exchange_link_sup_sup.erl +++ b/deps/rabbitmq_federation/src/rabbit_federation_exchange_link_sup_sup.erl @@ -49,12 +49,12 @@ start_child(X) -> adjust({clear_upstream, VHost, UpstreamName}) -> _ = [rabbit_federation_link_sup:adjust(Pid, X, {clear_upstream, UpstreamName}) || - {#exchange{name = Name} = X, Pid, _, _} <- mirrored_supervisor:which_children(?SUPERVISOR), + {{_, #exchange{name = Name} = X}, Pid, _, _} <- mirrored_supervisor:which_children(?SUPERVISOR), Name#resource.virtual_host == VHost], ok; adjust(Reason) -> _ = [rabbit_federation_link_sup:adjust(Pid, X, Reason) || - {X, Pid, _, _} <- mirrored_supervisor:which_children(?SUPERVISOR)], + {{_, X}, Pid, _, _} <- mirrored_supervisor:which_children(?SUPERVISOR)], ok. stop_child(X) -> @@ -75,5 +75,9 @@ init([]) -> {ok, {{one_for_one, 1200, 60}, []}}. %% See comment in rabbit_federation_queue_link_sup_sup:id/1 -id(X = #exchange{policy = Policy}) -> X1 = rabbit_exchange:immutable(X), - X1#exchange{policy = Policy}. +id(X = #exchange{policy = Policy}) -> + X1 = rabbit_exchange:immutable(X), + {simple_id(X), X1#exchange{policy = Policy}}. + +simple_id(#exchange{name = #resource{virtual_host = VHost, name = Name}}) -> + [exchange, VHost, Name]. diff --git a/deps/rabbitmq_federation/src/rabbit_federation_queue_link_sup_sup.erl b/deps/rabbitmq_federation/src/rabbit_federation_queue_link_sup_sup.erl index fc7e8e36d489..8c477b11d051 100644 --- a/deps/rabbitmq_federation/src/rabbit_federation_queue_link_sup_sup.erl +++ b/deps/rabbitmq_federation/src/rabbit_federation_queue_link_sup_sup.erl @@ -51,12 +51,12 @@ start_child(Q) -> adjust({clear_upstream, VHost, UpstreamName}) -> _ = [rabbit_federation_link_sup:adjust(Pid, Q, {clear_upstream, UpstreamName}) || - {Q, Pid, _, _} <- mirrored_supervisor:which_children(?SUPERVISOR), + {{_, Q}, Pid, _, _} <- mirrored_supervisor:which_children(?SUPERVISOR), ?amqqueue_vhost_equals(Q, VHost)], ok; adjust(Reason) -> _ = [rabbit_federation_link_sup:adjust(Pid, Q, Reason) || - {Q, Pid, _, _} <- mirrored_supervisor:which_children(?SUPERVISOR)], + {{_, Q}, Pid, _, _} <- mirrored_supervisor:which_children(?SUPERVISOR)], ok. stop_child(Q) -> @@ -88,4 +88,8 @@ init([]) -> id(Q) when ?is_amqqueue(Q) -> Policy = amqqueue:get_policy(Q), Q1 = amqqueue:set_immutable(Q), - amqqueue:set_policy(Q1, Policy). + {simple_id(Q), amqqueue:set_policy(Q1, Policy)}. + +simple_id(Q) when ?is_amqqueue(Q) -> + #resource{virtual_host = VHost, name = Name} = amqqueue:get_name(Q), + [queue, VHost, Name]. diff --git a/deps/rabbitmq_federation/src/rabbit_federation_upstream_exchange.erl b/deps/rabbitmq_federation/src/rabbit_federation_upstream_exchange.erl index 17958c06f613..520d7d96302d 100644 --- a/deps/rabbitmq_federation/src/rabbit_federation_upstream_exchange.erl +++ b/deps/rabbitmq_federation/src/rabbit_federation_upstream_exchange.erl @@ -78,11 +78,11 @@ validate(#exchange{arguments = Args}) -> rabbit_federation_util:validate_arg(?MAX_HOPS_ARG, long, Args). validate_binding(_X, _B) -> ok. -create(_Tx, _X) -> ok. -delete(_Tx, _X) -> ok. +create(_Serial, _X) -> ok. +delete(_Serial, _X) -> ok. policy_changed(_X1, _X2) -> ok. -add_binding(_Tx, _X, _B) -> ok. -remove_bindings(_Tx, _X, _Bs) -> ok. +add_binding(_Serial, _X, _B) -> ok. +remove_bindings(_Serial, _X, _Bs) -> ok. assert_args_equivalence(X = #exchange{name = Name, arguments = Args}, ReqArgs) -> diff --git a/deps/rabbitmq_federation/test/exchange_SUITE.erl b/deps/rabbitmq_federation/test/exchange_SUITE.erl index 3857b020c0b0..9e9b4ed44233 100644 --- a/deps/rabbitmq_federation/test/exchange_SUITE.erl +++ b/deps/rabbitmq_federation/test/exchange_SUITE.erl @@ -31,12 +31,22 @@ all() -> [ {group, essential}, {group, cluster_size_3} - %% {group, channel_use_mode_single} ]. groups() -> - [ - {essential, [], [ + [ + {essential, [], essential()}, + {cluster_size_3, [], [max_hops]}, + {cycle_protection, [], [ + %% TBD: port from v3.10.x in an Erlang 25-compatible way + ]}, + {channel_use_mod_single, [], [ + %% TBD: port from v3.10.x in an Erlang 25-compatible way + ]} + ]. + +essential() -> + [ single_upstream, multiple_upstreams, multiple_upstreams_pattern, @@ -47,18 +57,10 @@ groups() -> unbind_on_client_unbind, exchange_federation_link_status, lookup_exchange_status - ]}, - {cluster_size_3, [], [ - max_hops - ]}, - {cycle_protection, [], [ - %% TBD: port from v3.10.x in an Erlang 25-compatible way - ]}, - {channel_use_mod_single, [], [ - %% TBD: port from v3.10.x in an Erlang 25-compatible way - ]} - ]. + ]. +suite() -> + [{timetrap, {minutes, 3}}]. %% ------------------------------------------------------------------- %% Setup/teardown. @@ -72,7 +74,7 @@ end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config). %% Some of the "regular" tests but in the single channel mode. -init_per_group(channel_use_mode_single, Config) -> +init_per_group(essential, Config) -> SetupFederation = [ fun(Config1) -> rabbit_federation_test_util:setup_federation_with_upstream_params(Config1, [ @@ -83,35 +85,12 @@ init_per_group(channel_use_mode_single, Config) -> Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"), Config1 = rabbit_ct_helpers:set_config(Config, [ {rmq_nodename_suffix, Suffix}, - {rmq_nodes_clustered, false} + {rmq_nodes_count, 1} ]), rabbit_ct_helpers:run_steps(Config1, rabbit_ct_broker_helpers:setup_steps() ++ rabbit_ct_client_helpers:setup_steps() ++ SetupFederation); -init_per_group(cycle_detection, Config) -> - Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"), - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodename_suffix, Suffix}, - {rmq_nodes_clustered, false}, - {rmq_nodes_count, 1} - ]), - rabbit_ct_helpers:run_steps(Config1, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()); -init_per_group(without_plugins, Config) -> - rabbit_ct_helpers:set_config(Config, - {broker_with_plugins, [true, false]}); -init_per_group(cluster_size_1 = Group, Config) -> - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodes_count, 1} - ]), - init_per_group1(Group, Config1); -init_per_group(cluster_size_2 = Group, Config) -> - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodes_count, 2} - ]), - init_per_group1(Group, Config1); init_per_group(cluster_size_3 = Group, Config) -> Config1 = rabbit_ct_helpers:set_config(Config, [ {rmq_nodes_count, 3} @@ -131,8 +110,6 @@ init_per_group1(_Group, Config) -> rabbit_ct_broker_helpers:setup_steps() ++ rabbit_ct_client_helpers:setup_steps()). -end_per_group(without_plugins, Config) -> - Config; end_per_group(_, Config) -> rabbit_ct_helpers:run_steps(Config, rabbit_ct_client_helpers:teardown_steps() ++ @@ -727,15 +704,15 @@ await_binding(Config, Node, Vhost, X, Key, ExpectedBindingCount) when is_integer await_binding(_Config, _Node, _Vhost, _X, _Key, ExpectedBindingCount, 0) -> {error, rabbit_misc:format("expected ~b bindings but they did not materialize in time", [ExpectedBindingCount])}; await_binding(Config, Node, Vhost, X, Key, ExpectedBindingCount, AttemptsLeft) when is_integer(ExpectedBindingCount) -> - case bound_keys_from(Config, Node, Vhost, X, Key) of - Bs when length(Bs) < ExpectedBindingCount -> - timer:sleep(100), - await_binding(Config, Node, Vhost, X, Key, ExpectedBindingCount, AttemptsLeft - 1); - Bs when length(Bs) =:= ExpectedBindingCount -> - ok; - Bs -> - {error, rabbit_misc:format("expected ~b bindings, got ~b", [ExpectedBindingCount, length(Bs)])} - end. + case bound_keys_from(Config, Node, Vhost, X, Key) of + Bs when length(Bs) < ExpectedBindingCount -> + timer:sleep(1000), + await_binding(Config, Node, Vhost, X, Key, ExpectedBindingCount, AttemptsLeft - 1); + Bs when length(Bs) =:= ExpectedBindingCount -> + ok; + Bs -> + {error, rabbit_misc:format("expected ~b bindings, got ~b", [ExpectedBindingCount, length(Bs)])} + end. await_bindings(Config, Node, X, Keys) -> [await_binding(Config, Node, X, Key) || Key <- Keys]. @@ -771,4 +748,4 @@ await_credentials_obfuscation_seeding_on_two_nodes(Config) -> rabbit_ct_broker_helpers:rpc(Config, 1, credentials_obfuscation, enabled, []) end), - timer:sleep(1000). \ No newline at end of file + timer:sleep(1000). diff --git a/deps/rabbitmq_federation/test/queue_SUITE.erl b/deps/rabbitmq_federation/test/queue_SUITE.erl index 795f25a7b1db..4ebdcadcfd85 100644 --- a/deps/rabbitmq_federation/test/queue_SUITE.erl +++ b/deps/rabbitmq_federation/test/queue_SUITE.erl @@ -32,39 +32,28 @@ all() -> ]. groups() -> - ClusterSize1 = [simple, - multiple_upstreams_pattern, - multiple_downstreams, - message_flow, - dynamic_reconfiguration, - federate_unfederate, - dynamic_plugin_stop_start - ], - ClusterSize2 = [restart_upstream], - [{classic_queue, [], [ - {without_disambiguate, [], [ - {cluster_size_1, [], ClusterSize1} - ]}, - {with_disambiguate, [], [ - {cluster_size_2, [], ClusterSize2} - ]} - ]}, - {quorum_queue, [], [ - {without_disambiguate, [], [ - {cluster_size_1, [], ClusterSize1} - ]}, - {with_disambiguate, [], [ - {cluster_size_2, [], ClusterSize2} - ]} - ]}, - {mixed, [], [ - {without_disambiguate, [], [ - {cluster_size_1, [], ClusterSize1} - ]}, - {with_disambiguate, [], [ - {cluster_size_2, [], ClusterSize2} - ]} - ]} + [ + {classic_queue, [], all_tests()}, + {quorum_queue, [], all_tests()}, + {mixed, [], all_tests()} + ]. + +all_tests() -> + [ + {without_disambiguate, [], [ + {cluster_size_1, [], [ + simple, + multiple_upstreams_pattern, + multiple_downstreams, + message_flow, + dynamic_reconfiguration, + federate_unfederate, + dynamic_plugin_stop_start + ]} + ]}, + {with_disambiguate, [], [ + {cluster_size_2, [], [restart_upstream]} + ]} ]. %% ------------------------------------------------------------------- diff --git a/deps/rabbitmq_federation/test/rabbit_federation_test_util.erl b/deps/rabbitmq_federation/test/rabbit_federation_test_util.erl index 29f54863b401..818d961c7653 100644 --- a/deps/rabbitmq_federation/test/rabbit_federation_test_util.erl +++ b/deps/rabbitmq_federation/test/rabbit_federation_test_util.erl @@ -94,13 +94,15 @@ setup_federation_with_upstream_params(Config, ExtraParams) -> ] ]), - rabbit_ct_broker_helpers:set_policy(Config, 0, - <<"fed">>, <<"^fed\.">>, <<"all">>, [ - {<<"federation-upstream-set">>, <<"upstream">>}]), - - rabbit_ct_broker_helpers:set_policy(Config, 0, - <<"fed12">>, <<"^fed12\.">>, <<"all">>, [ - {<<"federation-upstream-set">>, <<"upstream12">>}]), + rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_policy, set, + [<<"/">>, <<"fed">>, <<"^fed\.">>, [{<<"federation-upstream-set">>, <<"upstream">>}], + 0, <<"all">>, <<"acting-user">>]), + + rabbit_ct_broker_helpers:rpc( + Config, 0, rabbit_policy, set, + [<<"/">>, <<"fed12">>, <<"^fed12\.">>, [{<<"federation-upstream-set">>, <<"upstream12">>}], + 2, <<"all">>, <<"acting-user">>]), rabbit_ct_broker_helpers:set_policy(Config, 0, <<"one">>, <<"^two$">>, <<"all">>, [ diff --git a/deps/rabbitmq_jms_topic_exchange/BUILD.bazel b/deps/rabbitmq_jms_topic_exchange/BUILD.bazel index 39e3ff2eca48..e3e49612b060 100644 --- a/deps/rabbitmq_jms_topic_exchange/BUILD.bazel +++ b/deps/rabbitmq_jms_topic_exchange/BUILD.bazel @@ -26,8 +26,6 @@ APP_MODULE = "rabbit_federation_app" all_beam_files(name = "all_beam_files") -all_test_beam_files(name = "all_test_beam_files") - all_srcs(name = "all_srcs") test_suite_beam_files(name = "test_suite_beam_files") @@ -48,6 +46,8 @@ rabbitmq_app( deps = [ "//deps/rabbit:erlang_app", "//deps/rabbit_common:erlang_app", + "@khepri//:erlang_app", + "@khepri_mnesia_migration//:erlang_app", ], ) @@ -102,3 +102,5 @@ alias( actual = ":erlang_app", visibility = ["//visibility:public"], ) + +all_test_beam_files(name = "all_test_beam_files") diff --git a/deps/rabbitmq_jms_topic_exchange/Makefile b/deps/rabbitmq_jms_topic_exchange/Makefile index 199c42238f98..7d1dd52bfecf 100644 --- a/deps/rabbitmq_jms_topic_exchange/Makefile +++ b/deps/rabbitmq_jms_topic_exchange/Makefile @@ -1,7 +1,6 @@ PROJECT = rabbitmq_jms_topic_exchange PROJECT_DESCRIPTION = RabbitMQ JMS topic selector exchange plugin -LOCAL_DEPS = mnesia DEPS = rabbit_common rabbit TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers amqp_client diff --git a/deps/rabbitmq_jms_topic_exchange/app.bzl b/deps/rabbitmq_jms_topic_exchange/app.bzl index 3fe5ff18a359..5c73214ef386 100644 --- a/deps/rabbitmq_jms_topic_exchange/app.bzl +++ b/deps/rabbitmq_jms_topic_exchange/app.bzl @@ -10,6 +10,7 @@ def all_beam_files(name = "all_beam_files"): name = "other_beam", srcs = [ "src/rabbit_db_jms_exchange.erl", + "src/rabbit_db_jms_exchange_m2k_converter.erl", "src/rabbit_jms_topic_exchange.erl", "src/sjx_evaluator.erl", ], @@ -20,6 +21,8 @@ def all_beam_files(name = "all_beam_files"): deps = [ "//deps/rabbit:erlang_app", "//deps/rabbit_common:erlang_app", + "@khepri//:erlang_app", + "@khepri_mnesia_migration//:erlang_app", ], ) @@ -34,6 +37,7 @@ def all_test_beam_files(name = "all_test_beam_files"): testonly = True, srcs = [ "src/rabbit_db_jms_exchange.erl", + "src/rabbit_db_jms_exchange_m2k_converter.erl", "src/rabbit_jms_topic_exchange.erl", "src/sjx_evaluator.erl", ], @@ -44,6 +48,8 @@ def all_test_beam_files(name = "all_test_beam_files"): deps = [ "//deps/rabbit:erlang_app", "//deps/rabbit_common:erlang_app", + "@khepri//:erlang_app", + "@khepri_mnesia_migration//:erlang_app", ], ) @@ -65,6 +71,7 @@ def all_srcs(name = "all_srcs"): name = "srcs", srcs = [ "src/rabbit_db_jms_exchange.erl", + "src/rabbit_db_jms_exchange_m2k_converter.erl", "src/rabbit_jms_topic_exchange.erl", "src/sjx_evaluator.erl", ], diff --git a/deps/rabbitmq_jms_topic_exchange/src/rabbit_db_jms_exchange.erl b/deps/rabbitmq_jms_topic_exchange/src/rabbit_db_jms_exchange.erl index b772093336b6..0403b7a64f69 100644 --- a/deps/rabbitmq_jms_topic_exchange/src/rabbit_db_jms_exchange.erl +++ b/deps/rabbitmq_jms_topic_exchange/src/rabbit_db_jms_exchange.erl @@ -7,6 +7,7 @@ -module(rabbit_db_jms_exchange). -include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("khepri/include/khepri.hrl"). -include("rabbit_jms_topic_exchange.hrl"). -export([ @@ -18,12 +19,23 @@ delete/3 ]). +-export([ + khepri_jms_topic_exchange_path/0, + khepri_jms_topic_exchange_path/1 + ]). + +-rabbit_mnesia_tables_to_khepri_db( + [{?JMS_TOPIC_TABLE, rabbit_db_jms_exchange_m2k_converter}]). + %% ------------------------------------------------------------------- %% setup_schema() %% ------------------------------------------------------------------- setup_schema() -> - setup_schema_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> setup_schema_in_mnesia() end, + khepri => ok + }). setup_schema_in_mnesia() -> case mnesia:create_table( ?JMS_TOPIC_TABLE @@ -41,7 +53,12 @@ setup_schema_in_mnesia() -> %% ------------------------------------------------------------------- create_or_update(XName, BindingKeyAndFun, ErrorFun) -> - create_or_update_in_mnesia(XName, BindingKeyAndFun, ErrorFun). + rabbit_khepri:handle_fallback( + #{mnesia => + fun() -> create_or_update_in_mnesia(XName, BindingKeyAndFun, ErrorFun) end, + khepri => + fun() -> update_in_khepri(XName, BindingKeyAndFun, fun put_item/2, ErrorFun) end + }). create_or_update_in_mnesia(XName, BindingKeyAndFun, ErrorFun) -> rabbit_mnesia:execute_mnesia_transaction( @@ -51,12 +68,33 @@ create_or_update_in_mnesia(XName, BindingKeyAndFun, ErrorFun) -> write_state_fun_in_mnesia(XName, put_item(BindingFuns, BindingKeyAndFun)) end). +update_in_khepri(XName, BindingKeyAndFun, UpdateFun, ErrorFun) -> + Path = khepri_jms_topic_exchange_path(XName), + case rabbit_khepri:adv_get(Path) of + {ok, #{data := BindingFuns, payload_version := DVersion}} -> + Path1 = khepri_path:combine_with_conditions( + Path, [#if_payload_version{version = DVersion}]), + Ret = rabbit_khepri:put(Path1, UpdateFun(BindingFuns, BindingKeyAndFun)), + case Ret of + ok -> ok; + {error, {khepri, mismatching_node, _}} -> + update_in_khepri(XName, BindingKeyAndFun, UpdateFun, ErrorFun); + {error, _} -> + ErrorFun(XName) + end; + _Err -> + ErrorFun(XName) + end. + %% ------------------------------------------------------------------- %% insert(). %% ------------------------------------------------------------------- insert(XName, BFuns) -> - insert_in_mnesia(XName, BFuns). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> insert_in_mnesia(XName, BFuns) end, + khepri => fun() -> insert_in_khepri(XName, BFuns) end + }). insert_in_mnesia(XName, BFuns) -> rabbit_mnesia:execute_mnesia_transaction( @@ -64,12 +102,18 @@ insert_in_mnesia(XName, BFuns) -> write_state_fun_in_mnesia(XName, BFuns) end). +insert_in_khepri(XName, BFuns) -> + ok = rabbit_khepri:put(khepri_jms_topic_exchange_path(XName), BFuns). + %% ------------------------------------------------------------------- %% get(). %% ------------------------------------------------------------------- get(XName) -> - get_in_mnesia(XName). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_in_mnesia(XName) end, + khepri => fun() -> get_in_khepri(XName) end + }). get_in_mnesia(XName) -> mnesia:async_dirty( @@ -84,19 +128,38 @@ get_in_mnesia(XName) -> [] ). +get_in_khepri(XName) -> + case rabbit_khepri:get(khepri_jms_topic_exchange_path(XName)) of + {ok, BindingFuns} -> + BindingFuns; + _ -> + not_found + end. + %% ------------------------------------------------------------------- %% delete(). %% ------------------------------------------------------------------- delete(XName) -> - delete_in_mnesia(XName). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> delete_in_mnesia(XName) end, + khepri => fun() -> delete_in_khepri(XName) end + }). delete_in_mnesia(XName) -> rabbit_mnesia:execute_mnesia_transaction( fun() -> mnesia:delete(?JMS_TOPIC_TABLE, XName, write) end). +delete_in_khepri(XName) -> + rabbit_khepri:delete(khepri_jms_topic_exchange_path(XName)). + delete(XName, BindingKeys, ErrorFun) -> - delete_in_mnesia(XName, BindingKeys, ErrorFun). + rabbit_khepri:handle_fallback( + #{mnesia => + fun() -> delete_in_mnesia(XName, BindingKeys, ErrorFun) end, + khepri => + fun() -> update_in_khepri(XName, BindingKeys, fun remove_items/2, ErrorFun) end + }). delete_in_mnesia(XName, BindingKeys, ErrorFun) -> rabbit_mnesia:execute_mnesia_transaction( @@ -127,3 +190,13 @@ put_item(Dict, {Key, Item}) -> dict:store(Key, Item, Dict). % remove a list of keyed items from the dictionary, by key remove_items(Dict, []) -> Dict; remove_items(Dict, [Key | Keys]) -> remove_items(dict:erase(Key, Dict), Keys). + +%% ------------------------------------------------------------------- +%% paths +%% ------------------------------------------------------------------- + +khepri_jms_topic_exchange_path(#resource{virtual_host = VHost, name = Name}) -> + [?MODULE, jms_topic_exchange, VHost, Name]. + +khepri_jms_topic_exchange_path() -> + [?MODULE, jms_topic_exchange]. diff --git a/deps/rabbitmq_jms_topic_exchange/src/rabbit_db_jms_exchange_m2k_converter.erl b/deps/rabbitmq_jms_topic_exchange/src/rabbit_db_jms_exchange_m2k_converter.erl new file mode 100644 index 000000000000..6bbb09b6b383 --- /dev/null +++ b/deps/rabbitmq_jms_topic_exchange/src/rabbit_db_jms_exchange_m2k_converter.erl @@ -0,0 +1,100 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright © 2022-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_jms_exchange_m2k_converter). + +-behaviour(mnesia_to_khepri_converter). + +-include_lib("kernel/include/logger.hrl"). +-include_lib("khepri/include/khepri.hrl"). +-include_lib("khepri_mnesia_migration/src/kmm_logging.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). + +-include("rabbit_jms_topic_exchange.hrl"). + +-export([init_copy_to_khepri/3, + copy_to_khepri/3, + delete_from_khepri/3, + clear_data_in_khepri/1]). + +-record(?MODULE, {store_id :: khepri:store_id()}). + +-spec init_copy_to_khepri(StoreId, MigrationId, Tables) -> Ret when + StoreId :: khepri:store_id(), + MigrationId :: mnesia_to_khepri:migration_id(), + Tables :: [mnesia_to_khepri:mnesia_table()], + Ret :: {ok, Priv}, + Priv :: #?MODULE{}. +%% @private + +init_copy_to_khepri(StoreId, _MigrationId, _Tables) -> + State = #?MODULE{store_id = StoreId}, + {ok, State}. + +-spec copy_to_khepri(Table, Record, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Record :: tuple(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +copy_to_khepri( + ?JMS_TOPIC_TABLE = Table, #?JMS_TOPIC_RECORD{x_name = XName, x_selector_funs = BFuns}, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] key: ~0p", + [Table, XName], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_jms_exchange:khepri_jms_topic_exchange_path(XName), + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:put(StoreId, Path, BFuns) of + ok -> {ok, State}; + Error -> Error + end; +copy_to_khepri(Table, Record, State) -> + ?LOG_DEBUG("Mnesia->Khepri unexpected record table ~0p record ~0p state ~0p", + [Table, Record, State]), + {error, unexpected_record}. + +-spec delete_from_khepri(Table, Key, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Key :: any(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +delete_from_khepri(?JMS_TOPIC_TABLE = Table, Key, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] key: ~0p", + [Table, Key], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_jms_exchange:khepri_jms_topic_exchange_path(Key), + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:delete(StoreId, Path) of + ok -> {ok, State}; + Error -> Error + end. + +clear_data_in_khepri(?JMS_TOPIC_TABLE) -> + Path = rabbit_db_jms_exchange:khepri_jms_topic_exchange_path(), + case rabbit_khepri:delete(Path) of + ok -> + ok; + Error -> + throw(Error) + end. diff --git a/deps/rabbitmq_jms_topic_exchange/src/rabbit_jms_topic_exchange.erl b/deps/rabbitmq_jms_topic_exchange/src/rabbit_jms_topic_exchange.erl index ab061be6efa5..56532ae0e4b5 100644 --- a/deps/rabbitmq_jms_topic_exchange/src/rabbit_jms_topic_exchange.erl +++ b/deps/rabbitmq_jms_topic_exchange/src/rabbit_jms_topic_exchange.erl @@ -109,7 +109,8 @@ create(_Tx, #exchange{name = XName}) -> % Delete an exchange delete(_Tx, #exchange{name = XName}) -> - delete_state(XName). + delete_state(XName), + ok. % Before add binding validate_binding(_X, _B) -> ok. diff --git a/deps/rabbitmq_jms_topic_exchange/test/rjms_topic_selector_SUITE.erl b/deps/rabbitmq_jms_topic_exchange/test/rjms_topic_selector_SUITE.erl index da9b365bf19b..c830b6c4d632 100644 --- a/deps/rabbitmq_jms_topic_exchange/test/rjms_topic_selector_SUITE.erl +++ b/deps/rabbitmq_jms_topic_exchange/test/rjms_topic_selector_SUITE.erl @@ -22,15 +22,22 @@ all() -> [ - {group, all_tests} + {group, mnesia_store}, + {group, khepri_store}, + {group, khepri_migration} ]. groups() -> [ - {all_tests, [], [ - test_topic_selection, - restart_with_auto_delete_topic_exchange - ]} + {mnesia_store, [], [ + test_topic_selection, + restart_with_auto_delete_topic_exchange + ]}, + {khepri_store, [], [ + test_topic_selection, + restart_with_auto_delete_topic_exchange + ]}, + {khepri_migration, [], [from_mnesia_to_khepri]} ]. %% ------------------------------------------------------------------- @@ -39,23 +46,35 @@ groups() -> init_per_suite(Config) -> rabbit_ct_helpers:log_environment(), - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodename_suffix, ?MODULE} - ]), - rabbit_ct_helpers:run_setup_steps(Config1, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()). + rabbit_ct_helpers:run_setup_steps(Config). end_per_suite(Config) -> - rabbit_ct_helpers:run_teardown_steps(Config, - rabbit_ct_client_helpers:teardown_steps() ++ - rabbit_ct_broker_helpers:teardown_steps()). - -init_per_group(_, Config) -> - Config. + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(mnesia_store = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, mnesia}]), + init_per_group_common(Group, Config); +init_per_group(khepri_store = Group, Config0) -> + Config = rabbit_ct_helpers:set_config( + Config0, + [{metadata_store, {khepri, [khepri_db]}}]), + init_per_group_common(Group, Config); +init_per_group(khepri_migration = Group, Config0) -> + Config = rabbit_ct_helpers:set_config(Config0, [{metadata_store, mnesia}]), + init_per_group_common(Group, Config). + +init_per_group_common(Group, Config) -> + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, Group} + ]), + rabbit_ct_helpers:run_setup_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). end_per_group(_, Config) -> - Config. + rabbit_ct_helpers:run_teardown_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). init_per_testcase(Testcase, Config) -> rabbit_ct_helpers:testcase_started(Config, Testcase). @@ -89,7 +108,7 @@ test_topic_selection(Config) -> restart_with_auto_delete_topic_exchange(Config) -> [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - {Connection, Channel} = open_connection_and_channel(Config), + {_Connection, Channel} = open_connection_and_channel(Config), #'confirm.select_ok'{} = amqp_channel:call(Channel, #'confirm.select'{}), Exchange = declare_rjms_exchange(Channel, "restart_with_auto_delete_topic_exchange", true, true, []), @@ -100,6 +119,31 @@ restart_with_auto_delete_topic_exchange(Config) -> ok = rabbit_control_helper:command(stop_app, Server), ok = rabbit_control_helper:command(start_app, Server). +from_mnesia_to_khepri(Config) -> + {Connection, Channel} = open_connection_and_channel(Config), + #'confirm.select_ok'{} = amqp_channel:call(Channel, #'confirm.select'{}), + + Exchange = declare_rjms_exchange(Channel, "rjms_test_topic_selector_exchange", false, false, []), + + %% Declare a queue and bind it + Q = declare_queue(Channel), + bind_queue(Channel, Q, Exchange, <<"select-key">>, [?BSELECTARG(<<"{ident, <<\"boolVal\">>}.">>)]), + + case rabbit_ct_broker_helpers:enable_feature_flag(Config, khepri_db) of + ok -> + case rabbit_ct_broker_helpers:enable_feature_flag(Config, rabbit_jms_topic_exchange_raft_based_metadata_store) of + ok -> + publish_two_messages(Channel, Exchange, <<"select-key">>), + amqp_channel:wait_for_confirms(Channel, 5), + get_and_check(Channel, Q, 0, <<"true">>), + close_connection_and_channel(Connection, Channel), + ok; + Skip -> + Skip + end; + Skip -> + Skip + end. %% ------------------------------------------------------------------- %% Helpers. diff --git a/deps/rabbitmq_management/Makefile b/deps/rabbitmq_management/Makefile index c092c90f5e77..dea06b0000be 100644 --- a/deps/rabbitmq_management/Makefile +++ b/deps/rabbitmq_management/Makefile @@ -22,7 +22,7 @@ endef DEPS = rabbit_common rabbit amqp_client cowboy cowlib rabbitmq_web_dispatch rabbitmq_management_agent TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers proper -LOCAL_DEPS += mnesia ranch ssl crypto public_key +LOCAL_DEPS += ranch ssl crypto public_key # FIXME: Add Ranch as a BUILD_DEPS to be sure the correct version is picked. # See rabbitmq-components.mk. diff --git a/deps/rabbitmq_management/priv/www/js/tmpl/binary.ejs b/deps/rabbitmq_management/priv/www/js/tmpl/binary.ejs index 666adc99b05b..a2ecb9ab4da0 100644 --- a/deps/rabbitmq_management/priv/www/js/tmpl/binary.ejs +++ b/deps/rabbitmq_management/priv/www/js/tmpl/binary.ejs @@ -20,6 +20,7 @@ 'msg_index' : ['table', 'Message store index'], 'mgmt_db' : ['table', 'Management database'], 'plugins' : ['proc', 'Plugins'], + 'metadata_store' : ['metadata_store', 'Metadata store'], 'other' : ['system', 'Other binary references']}; var total_out = []; %> @@ -49,7 +50,8 @@ var key = [[{name: 'Classic Queues', colour: 'classic', ['mgmt_db', 'management database']]}], [{name: 'Processes', colour: 'proc', - keys: [['plugins', 'plugins']]}, + keys: [['plugins', 'plugins'], + ['metadata_store', 'metadata store']]}, {name: 'System', colour: 'system', keys: [['other', 'other']]}]]; %> diff --git a/deps/rabbitmq_management/priv/www/js/tmpl/memory.ejs b/deps/rabbitmq_management/priv/www/js/tmpl/memory.ejs index df7b33eb070f..fbe5f6308b6e 100644 --- a/deps/rabbitmq_management/priv/www/js/tmpl/memory.ejs +++ b/deps/rabbitmq_management/priv/www/js/tmpl/memory.ejs @@ -63,6 +63,7 @@ var key = [[{name: 'Classic Queues', colour: 'classic', [{name: 'Processes', colour: 'proc', keys: [['plugins', 'plugins'], + ['metadata_store', 'metadata store'], ['other_proc', 'other']]}, {name: 'System', colour: 'system', keys: [['code', 'code'], diff --git a/deps/rabbitmq_management/test/clustering_SUITE.erl b/deps/rabbitmq_management/test/clustering_SUITE.erl index 92a1e3456581..abaf45ea2f44 100644 --- a/deps/rabbitmq_management/test/clustering_SUITE.erl +++ b/deps/rabbitmq_management/test/clustering_SUITE.erl @@ -23,15 +23,13 @@ all() -> [ - {group, non_parallel_tests} + {group, non_parallel_tests}, + {group, non_parallel_tests_mirroring} ]. groups() -> [{non_parallel_tests, [], [ list_cluster_nodes_test, - multi_node_case1_test, - ha_queue_hosted_on_other_node, - ha_queue_with_multiple_consumers, queue_on_other_node, queue_with_multiple_consumers, queue_consumer_cancelled, @@ -58,7 +56,12 @@ groups() -> qq_replicas_delete, qq_replicas_grow, qq_replicas_shrink - ]} + ]}, + {non_parallel_tests_mirroring, [ + multi_node_case1_test, + ha_queue_hosted_on_other_node, + ha_queue_with_multiple_consumers + ]} ]. %% ------------------------------------------------------------------- @@ -95,6 +98,13 @@ end_per_suite(Config) -> rabbit_ct_helpers:run_teardown_steps(Config, rabbit_ct_broker_helpers:teardown_steps()). +init_per_group(non_parallel_tests_mirroring, Config) -> + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + mnesia -> + Config; + {khepri, _} -> + {skip, "Classic queue mirroring not supported by Khepri"} + end; init_per_group(_, Config) -> Config. diff --git a/deps/rabbitmq_management/test/rabbit_mgmt_http_SUITE.erl b/deps/rabbitmq_management/test/rabbit_mgmt_http_SUITE.erl index 2b78a3bdd262..de37d83da0c8 100644 --- a/deps/rabbitmq_management/test/rabbit_mgmt_http_SUITE.erl +++ b/deps/rabbitmq_management/test/rabbit_mgmt_http_SUITE.erl @@ -359,7 +359,7 @@ ets_tables_memory_test(Config) -> Path = "/nodes/" ++ binary_to_list(maps:get(name, Node)) ++ "/memory/ets", Result = http_get(Config, Path, ?OK), assert_keys([ets_tables_memory], Result), - NonMgmtKeys = [rabbit_vhost,rabbit_user_permission], + NonMgmtKeys = [tracked_connection, tracked_channel], Keys = [queue_stats, vhost_stats_coarse_conn_stats, connection_created_stats, channel_process_stats, consumer_stats, queue_msg_rates], @@ -2941,7 +2941,7 @@ policy_permissions_test(Config) -> http_put(Config, "/permissions/v/mgmt", Perms, {group, '2xx'}), Policy = [{pattern, <<".*">>}, - {definition, [{<<"ha-mode">>, <<"all">>}]}], + {definition, [{<<"max-length-bytes">>, 3000000}]}], Param = [{value, <<"">>}], http_put(Config, "/policies/%2F/HA", Policy, {group, '2xx'}), diff --git a/deps/rabbitmq_management/test/rabbit_mgmt_http_health_checks_SUITE.erl b/deps/rabbitmq_management/test/rabbit_mgmt_http_health_checks_SUITE.erl index a66f94fb31c3..553bb27b5680 100644 --- a/deps/rabbitmq_management/test/rabbit_mgmt_http_health_checks_SUITE.erl +++ b/deps/rabbitmq_management/test/rabbit_mgmt_http_health_checks_SUITE.erl @@ -24,13 +24,15 @@ all() -> [ - {group, all_tests}, + {group, cluster_size_3}, + {group, cluster_size_5}, {group, single_node} ]. groups() -> [ - {all_tests, [], all_tests()}, + {cluster_size_3, [], all_tests()}, + {cluster_size_5, [], [is_quorum_critical_test]}, {single_node, [], [ alarms_test, local_alarms_test, @@ -40,7 +42,6 @@ groups() -> all_tests() -> [ health_checks_test, - is_quorum_critical_test, is_mirror_sync_critical_test, virtual_hosts_test, protocol_listener_test, @@ -58,7 +59,8 @@ init_per_group(Group, Config0) -> rabbit_ct_helpers:log_environment(), inets:start(), ClusterSize = case Group of - all_tests -> 3; + cluster_size_3 -> 3; + cluster_size_5 -> 5; single_node -> 1 end, NodeConf = [{rmq_nodename_suffix, Group}, @@ -77,24 +79,30 @@ end_per_group(_, Config) -> Steps = Teardown0 ++ Teardown1, rabbit_ct_helpers:run_teardown_steps(Config, Steps). -init_per_testcase(Testcase, Config) - when Testcase == is_quorum_critical_test - orelse Testcase == is_mirror_sync_critical_test -> +init_per_testcase(Testcase, Config) when Testcase == is_quorum_critical_test -> case rabbit_ct_helpers:is_mixed_versions() of true -> {skip, "not mixed versions compatible"}; _ -> rabbit_ct_helpers:testcase_started(Config, Testcase) end; +init_per_testcase(Testcase, Config) + when Testcase == is_mirror_sync_critical_single_node_test + orelse Testcase == is_mirror_sync_critical_test -> + case rabbit_ct_helpers:is_mixed_versions() of + true -> + {skip, "not mixed versions compatible"}; + _ -> + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + mnesia -> + rabbit_ct_helpers:testcase_started(Config, Testcase); + {khepri, _} -> + {skip, "Classic queue mirroring not supported by Khepri"} + end + end; init_per_testcase(Testcase, Config) -> rabbit_ct_helpers:testcase_started(Config, Testcase). -end_per_testcase(is_quorum_critical_test = Testcase, Config) -> - [_, Server2, Server3] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - _ = rabbit_ct_broker_helpers:start_node(Config, Server2), - _ = rabbit_ct_broker_helpers:start_node(Config, Server3), - rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_queues, []), - rabbit_ct_helpers:testcase_finished(Config, Testcase); end_per_testcase(is_mirror_sync_critical_test = Testcase, Config) -> [_, Server2, Server3] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), _ = rabbit_ct_broker_helpers:start_node(Config, Server2), @@ -193,9 +201,10 @@ is_quorum_critical_test(Config) -> ?assertEqual(false, maps:is_key(reason, Check0)), ?assertEqual(<<"ok">>, maps:get(status, Check0)), - [Server1, Server2, Server3] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), - Ch = rabbit_ct_client_helpers:open_channel(Config, Server1), - Args = [{<<"x-queue-type">>, longstr, <<"quorum">>}], + [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename), + Ch = rabbit_ct_client_helpers:open_channel(Config, Server), + Args = [{<<"x-queue-type">>, longstr, <<"quorum">>}, + {<<"x-quorum-initial-group-size">>, long, 3}], QName = <<"is_quorum_critical_test">>, ?assertEqual({'queue.declare_ok', QName, 0, 0}, amqp_channel:call(Ch, #'queue.declare'{queue = QName, @@ -205,6 +214,9 @@ is_quorum_critical_test(Config) -> Check1 = http_get(Config, "/health/checks/node-is-quorum-critical", ?OK), ?assertEqual(false, maps:is_key(reason, Check1)), + RaName = binary_to_atom(<<"%2F_", QName/binary>>, utf8), + {ok, [_, {_, Server2}, {_, Server3}], _} = ra:members({RaName, Server}), + ok = rabbit_ct_broker_helpers:stop_node(Config, Server2), ok = rabbit_ct_broker_helpers:stop_node(Config, Server3), diff --git a/deps/rabbitmq_management/test/rabbit_mgmt_only_http_SUITE.erl b/deps/rabbitmq_management/test/rabbit_mgmt_only_http_SUITE.erl index bc08cb48ef4b..f9ddcea4e13b 100644 --- a/deps/rabbitmq_management/test/rabbit_mgmt_only_http_SUITE.erl +++ b/deps/rabbitmq_management/test/rabbit_mgmt_only_http_SUITE.erl @@ -122,12 +122,10 @@ init_per_group(all_tests_with_prefix = Group, Config0) -> PathConfig = {rabbitmq_management, [{path_prefix, ?PATH_PREFIX}]}, Config1 = rabbit_ct_helpers:merge_app_env(Config0, PathConfig), Config2 = finish_init(Group, Config1), - Config3 = start_broker(Config2), - Config3; + start_broker(Config2); init_per_group(Group, Config0) -> Config1 = finish_init(Group, Config0), - Config2 = start_broker(Config1), - Config2. + start_broker(Config1). end_per_group(_, Config) -> inets:stop(), @@ -142,6 +140,14 @@ init_per_testcase(Testcase = permissions_vhost_test, Config) -> rabbit_ct_broker_helpers:delete_vhost(Config, <<"myvhost2">>), rabbit_ct_helpers:testcase_started(Config, Testcase); +init_per_testcase(mirrored_queues_test = Testcase, Config) -> + case rabbit_ct_broker_helpers:configured_metadata_store(Config) of + mnesia -> + rabbit_ct_broker_helpers:close_all_connections(Config, 0, <<"rabbit_mgmt_only_http_SUITE:init_per_testcase">>), + rabbit_ct_helpers:testcase_started(Config, Testcase); + {khepri, _} -> + {skip, "Classic queue mirroring not supported by Khepri"} + end; init_per_testcase(Testcase, Config) -> rabbit_ct_broker_helpers:close_all_connections(Config, 0, <<"rabbit_mgmt_only_http_SUITE:init_per_testcase">>), rabbit_ct_helpers:testcase_started(Config, Testcase). @@ -401,8 +407,8 @@ queues_test(Config) -> ?BAD_REQUEST), Policy = [{pattern, <<"baz">>}, - {definition, [{<<"ha-mode">>, <<"all">>}]}], - http_put(Config, "/policies/%2F/HA", Policy, {group, '2xx'}), + {definition, [{<<"max-length">>, 100}]}], + http_put(Config, "/policies/%2F/length", Policy, {group, '2xx'}), http_put(Config, "/queues/%2F/baz", Good, {group, '2xx'}), Queues = http_get(Config, "/queues/%2F"), Queue = http_get(Config, "/queues/%2F/foo"), @@ -414,9 +420,7 @@ queues_test(Config) -> auto_delete => false, exclusive => false, arguments => #{}, - node => NodeBin, - slave_nodes => [], - synchronised_slave_nodes => []}, + node => NodeBin}, #{name => <<"foo">>, vhost => <<"/">>, durable => true, @@ -461,8 +465,8 @@ queues_enable_totals_test(Config) -> http_put(Config, "/queues/%2F/foo", GoodQQ, {group, '2xx'}), Policy = [{pattern, <<"baz">>}, - {definition, [{<<"ha-mode">>, <<"all">>}]}], - http_put(Config, "/policies/%2F/HA", Policy, {group, '2xx'}), + {definition, [{<<"max-length">>, 100}]}], + http_put(Config, "/policies/%2F/length", Policy, {group, '2xx'}), http_put(Config, "/queues/%2F/baz", Good, {group, '2xx'}), {Conn, Ch} = open_connection_and_channel(Config), @@ -494,11 +498,9 @@ queues_enable_totals_test(Config) -> exclusive => false, arguments => #{}, node => NodeBin, - slave_nodes => [], messages => 1, messages_ready => 1, - messages_unacknowledged => 0, - synchronised_slave_nodes => []}, + messages_unacknowledged => 0}, #{name => <<"foo">>, vhost => <<"/">>, durable => true, diff --git a/deps/rabbitmq_management/test/rabbit_mgmt_rabbitmqadmin_SUITE.erl b/deps/rabbitmq_management/test/rabbit_mgmt_rabbitmqadmin_SUITE.erl index 4c7852a34c6f..d4e9f5a4913a 100644 --- a/deps/rabbitmq_management/test/rabbit_mgmt_rabbitmqadmin_SUITE.erl +++ b/deps/rabbitmq_management/test/rabbit_mgmt_rabbitmqadmin_SUITE.erl @@ -297,12 +297,12 @@ bindings(Config) -> {ok, _} = run(Config, ["delete", "queue", "name=foo"]). policies(Config) -> - {ok, _} = run(Config, ["declare", "policy", "name=ha", - "pattern=.*", "definition={\"ha-mode\":\"all\"}"]), - {ok, [["ha", "/", ".*", "{\"ha-mode\": \"all\"}"]]} = + {ok, _} = run(Config, ["declare", "policy", "name=max-length-bytes", + "pattern=.*", "definition={\"max-length-bytes\":10000}"]), + {ok, [["max-length-bytes", "/", ".*", "{\"max-length-bytes\": 10000}"]]} = run_table(Config, ["list", "policies", "name", "vhost", "pattern", "definition"]), - {ok, _} = run(Config, ["delete", "policy", "name=ha"]). + {ok, _} = run(Config, ["delete", "policy", "name=max-length-bytes"]). operator_policies(Config) -> {ok, _} = run(Config, ["declare", "operator_policy", "name=len", diff --git a/deps/rabbitmq_management_agent/Makefile b/deps/rabbitmq_management_agent/Makefile index 545880b17469..05906c2ab53e 100644 --- a/deps/rabbitmq_management_agent/Makefile +++ b/deps/rabbitmq_management_agent/Makefile @@ -19,7 +19,7 @@ endef DEPS = rabbit_common rabbit rabbitmq_web_dispatch TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers -LOCAL_DEPS += xmerl mnesia ranch ssl crypto public_key +LOCAL_DEPS += xmerl ranch ssl crypto public_key DEP_EARLY_PLUGINS = rabbit_common/mk/rabbitmq-early-plugin.mk DEP_PLUGINS = rabbit_common/mk/rabbitmq-plugin.mk diff --git a/deps/rabbitmq_management_agent/src/rabbit_mgmt_metrics_collector.erl b/deps/rabbitmq_management_agent/src/rabbit_mgmt_metrics_collector.erl index 4e2d6d2496a5..b1e5836ae2ae 100644 --- a/deps/rabbitmq_management_agent/src/rabbit_mgmt_metrics_collector.erl +++ b/deps/rabbitmq_management_agent/src/rabbit_mgmt_metrics_collector.erl @@ -447,7 +447,7 @@ aggregate_entry({Id, Metrics, 0}, NextStats, Ops0, Metrics, {fun rabbit_mgmt_format:format_queue_stats/1, false}), insert_op(queue_stats, Id, ?queue_stats(Id, Fmt), O); - false -> + _ -> Ops1 end, {insert_old_aggr_stats(NextStats, Id, Stats), Ops2, State}; diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel index 7c2272a82621..13c1dfb0d971 100644 --- a/deps/rabbitmq_mqtt/BUILD.bazel +++ b/deps/rabbitmq_mqtt/BUILD.bazel @@ -208,6 +208,9 @@ rabbitmq_suite( rabbitmq_suite( name = "processor_SUITE", size = "small", + runtime_deps = [ + "@meck//:erlang_app", + ], deps = [ "//deps/amqp_client:erlang_app", "//deps/rabbit_common:erlang_app", @@ -250,7 +253,7 @@ rabbitmq_integration_suite( ":test_util_beam", ":test_event_recorder_beam", ], - shard_count = 12, + shard_count = 18, runtime_deps = [ "//deps/rabbitmq_management_agent:erlang_app", "@emqtt//:erlang_app", diff --git a/deps/rabbitmq_mqtt/test/processor_SUITE.erl b/deps/rabbitmq_mqtt/test/processor_SUITE.erl index 15ae0dd5374c..bf7ff1839450 100644 --- a/deps/rabbitmq_mqtt/test/processor_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/processor_SUITE.erl @@ -43,9 +43,17 @@ init_per_testcase(get_vhost, Config) -> mnesia:create_table(rabbit_runtime_parameters, [ {attributes, record_info(fields, runtime_parameters)}, {record_name, runtime_parameters}]), + meck:new(rabbit_feature_flags, [passthrough, no_link]), + meck:expect( + rabbit_feature_flags, is_enabled, + fun + (khepri_db, _) -> false; + (FeatureNames, _) -> meck:passthrough([FeatureNames]) + end), Config; init_per_testcase(_, Config) -> Config. end_per_testcase(get_vhost, Config) -> + meck:unload(rabbit_feature_flags), mnesia:stop(), Config; end_per_testcase(_, Config) -> Config. diff --git a/deps/rabbitmq_mqtt/test/shared_SUITE.erl b/deps/rabbitmq_mqtt/test/shared_SUITE.erl index 70821a27a3a1..248d91dd3b89 100644 --- a/deps/rabbitmq_mqtt/test/shared_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/shared_SUITE.erl @@ -61,24 +61,30 @@ groups() -> {mqtt, [], [{v3, [], [{cluster_size_1, [], cluster_size_1_tests()}, - {cluster_size_3, [], cluster_size_3_tests()}]}, + {cluster_size_3, [], cluster_size_3_tests()}, + {mnesia_store, [], mnesia_store_tests()}]}, {v4, [], [{cluster_size_1, [], cluster_size_1_tests()}, - {cluster_size_3, [], cluster_size_3_tests()}]}, + {cluster_size_3, [], cluster_size_3_tests()}, + {mnesia_store, [], mnesia_store_tests()}]}, {v5, [], [{cluster_size_1, [], cluster_size_1_tests()}, - {cluster_size_3, [], cluster_size_3_tests()}]} + {cluster_size_3, [], cluster_size_3_tests()}, + {mnesia_store, [], mnesia_store_tests()}]} ]}, {web_mqtt, [], [{v3, [], [{cluster_size_1, [], cluster_size_1_tests()}, - {cluster_size_3, [], cluster_size_3_tests()}]}, + {cluster_size_3, [], cluster_size_3_tests()}, + {mnesia_store, [], mnesia_store_tests()}]}, {v4, [], [{cluster_size_1, [], cluster_size_1_tests()}, - {cluster_size_3, [], cluster_size_3_tests()}]}, + {cluster_size_3, [], cluster_size_3_tests()}, + {mnesia_store, [], mnesia_store_tests()}]}, {v5, [], [{cluster_size_1, [], cluster_size_1_tests()}, - {cluster_size_3, [], cluster_size_3_tests()}]} + {cluster_size_3, [], cluster_size_3_tests()}, + {mnesia_store, [], mnesia_store_tests()}]} ]} ]. @@ -132,22 +138,26 @@ cluster_size_3_tests() -> [ pubsub, queue_down_qos1, - consuming_classic_mirrored_queue_down, consuming_classic_queue_down, - flow_classic_mirrored_queue, flow_quorum_queue, flow_stream, rabbit_mqtt_qos0_queue, cli_list_queues, maintenance, delete_create_queue, - publish_to_all_queue_types_qos0, - publish_to_all_queue_types_qos1, - duplicate_client_id, session_reconnect, session_takeover ]. +mnesia_store_tests() -> + [ + consuming_classic_mirrored_queue_down, + flow_classic_mirrored_queue, + publish_to_all_queue_types_qos0, + publish_to_all_queue_types_qos1, + duplicate_client_id + ]. + suite() -> [{timetrap, {minutes, 10}}]. @@ -173,12 +183,19 @@ init_per_group(Group, Config) Group =:= v5 -> rabbit_ct_helpers:set_config(Config, {mqtt_version, Group}); -init_per_group(Group, Config0) -> +init_per_group(Group, Config00) -> Nodes = case Group of cluster_size_1 -> 1; - cluster_size_3 -> 3 + cluster_size_3 -> 3; + mnesia_store -> 3 end, - Suffix = rabbit_ct_helpers:testcase_absname(Config0, "", "-"), + Suffix = rabbit_ct_helpers:testcase_absname(Config00, "", "-"), + Config0 = case Group of + mnesia_store -> + rabbit_ct_helpers:set_config(Config00, [{metadata_store, mnesia}]); + _ -> + Config00 + end, Config1 = rabbit_ct_helpers:set_config( Config0, [{rmq_nodes_count, Nodes}, @@ -438,6 +455,65 @@ publish_to_all_queue_types(Config, QoS) -> ?awaitMatch([], all_connection_pids(Config), 10_000, 1000). +publish_to_all_non_deprecated_queue_types_qos0(Config) -> + publish_to_all_non_deprecated_queue_types(Config, qos0). + +publish_to_all_non_deprecated_queue_types_qos1(Config) -> + publish_to_all_non_deprecated_queue_types(Config, qos1). + +publish_to_all_non_deprecated_queue_types(Config, QoS) -> + Ch = rabbit_ct_client_helpers:open_channel(Config), + + CQ = <<"classic-queue">>, + QQ = <<"quorum-queue">>, + SQ = <<"stream-queue">>, + Topic = <<"mytopic">>, + + declare_queue(Ch, CQ, []), + bind(Ch, CQ, Topic), + + declare_queue(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}]), + bind(Ch, QQ, Topic), + + declare_queue(Ch, SQ, [{<<"x-queue-type">>, longstr, <<"stream">>}]), + bind(Ch, SQ, Topic), + + NumMsgs = 2000, + C = connect(?FUNCTION_NAME, Config, [{retry_interval, 2}]), + lists:foreach(fun(N) -> + case emqtt:publish(C, Topic, integer_to_binary(N), QoS) of + ok -> + ok; + {ok, _} -> + ok; + Other -> + ct:fail("Failed to publish: ~p", [Other]) + end + end, lists:seq(1, NumMsgs)), + + eventually(?_assert( + begin + L = rabbitmqctl_list(Config, 0, ["list_queues", "messages", "--no-table-headers"]), + length(L) =:= 3 andalso + lists:all(fun([Bin]) -> + N = binary_to_integer(Bin), + case QoS of + qos0 -> + N =:= NumMsgs; + qos1 -> + %% Allow for some duplicates when client resends + %% a message that gets acked at roughly the same time. + N >= NumMsgs andalso + N < NumMsgs * 2 + end + end, L) + end), 2000, 10), + + delete_queue(Ch, [CQ, QQ, SQ]), + ok = emqtt:disconnect(C), + ?awaitMatch([], + all_connection_pids(Config), 10_000, 1000). + flow_classic_mirrored_queue(Config) -> QueueName = <<"flow">>, ok = rabbit_ct_broker_helpers:set_ha_policy(Config, 0, QueueName, <<"all">>), @@ -1461,24 +1537,15 @@ clean_session_node_down(NodeDown, Config) -> ?assertEqual(0, length(QsQos0)), ?assertEqual(2, length(QsClassic)) end, - Tables = [rabbit_durable_queue, - rabbit_queue, - rabbit_durable_route, - rabbit_semi_durable_route, - rabbit_route, - rabbit_reverse_route, - rabbit_topic_trie_node, - rabbit_topic_trie_edge, - rabbit_topic_trie_binding], - [?assertNotEqual(0, rpc(Config, ets, info, [T, size])) || T <- Tables], + ?assertEqual(2, rpc(Config, rabbit_amqqueue, count, [])), unlink(C), ok = rabbit_ct_broker_helpers:NodeDown(Config, 0), ok = rabbit_ct_broker_helpers:start_node(Config, 0), - %% After terminating a clean session by either node crash or graceful node shutdown, we - %% expect any session state to be cleaned up on the server once the server finished booting. - [?assertEqual(0, rpc(Config, ets, info, [T, size])) || T <- Tables]. + %% After terminating a clean session by a node crash, we expect any session + %% state to be cleaned up on the server once the server comes back up. + ?assertEqual(0, rpc(Config, rabbit_amqqueue, count, [])). rabbit_status_connection_count(Config) -> _Pid = rabbit_ct_client_helpers:open_connection(Config, 0), diff --git a/deps/rabbitmq_mqtt/test/util.erl b/deps/rabbitmq_mqtt/test/util.erl index 686be8083c40..df600459c173 100644 --- a/deps/rabbitmq_mqtt/test/util.erl +++ b/deps/rabbitmq_mqtt/test/util.erl @@ -141,6 +141,10 @@ await_exit(Pid, Reason) -> 20_000 -> ct:fail({missing_exit, Pid}) end. +maybe_skip_v5({skip, _Reason} = Skip) -> + %% Mixed-version can be skipped as `khepri_db` + %% is not supported + Skip; maybe_skip_v5(Config) -> case ?config(mqtt_version, Config) of v5 -> diff --git a/deps/rabbitmq_mqtt/test/v5_SUITE.erl b/deps/rabbitmq_mqtt/test/v5_SUITE.erl index d9c4663f837f..b078c8af1421 100644 --- a/deps/rabbitmq_mqtt/test/v5_SUITE.erl +++ b/deps/rabbitmq_mqtt/test/v5_SUITE.erl @@ -175,12 +175,19 @@ init_per_group(Group, Config0) -> Config2, rabbit_ct_broker_helpers:setup_steps() ++ rabbit_ct_client_helpers:setup_steps()), - case Group of - cluster_size_1 -> - ok = rabbit_ct_broker_helpers:enable_feature_flag(Config, mqtt_v5), - Config; - cluster_size_3 -> - util:maybe_skip_v5(Config) + %% Mixed-version is skipped as `khepri_db` + %% is not supported + case Config of + {skip, _Reason} = Skip -> + Skip; + _ -> + case Group of + cluster_size_1 -> + ok = rabbit_ct_broker_helpers:enable_feature_flag(Config, mqtt_v5), + Config; + cluster_size_3 -> + util:maybe_skip_v5(Config) + end end. end_per_group(G, Config) diff --git a/deps/rabbitmq_prometheus/src/collectors/prometheus_rabbitmq_core_metrics_collector.erl b/deps/rabbitmq_prometheus/src/collectors/prometheus_rabbitmq_core_metrics_collector.erl index 5e49c181dda9..8e3546150e94 100644 --- a/deps/rabbitmq_prometheus/src/collectors/prometheus_rabbitmq_core_metrics_collector.erl +++ b/deps/rabbitmq_prometheus/src/collectors/prometheus_rabbitmq_core_metrics_collector.erl @@ -634,16 +634,17 @@ get_data(exchange_bindings, _, _, _) -> (#exchange{name = EName, type = EType}, Acc) -> maps:put(EName, #{type => atom_to_binary(EType), binding_count => 0}, Acc) end, #{}, rabbit_exchange:list()), - WithCount = ets:foldl( - fun (#route{binding = #binding{source = EName}}, Acc) -> + WithCount = rabbit_db_binding:fold( + fun (#binding{source = EName}, Acc) -> case maps:is_key(EName, Acc) of false -> Acc; true -> - maps:update_with(EName, fun (R = #{binding_count := Cnt}) -> - R#{binding_count => Cnt + 1} - end, Acc) + maps:update_with(EName, + fun (R = #{binding_count := Cnt}) -> + R#{binding_count => Cnt + 1} + end, Acc) end - end, Exchanges, rabbit_route), + end, Exchanges), maps:fold(fun(#resource{virtual_host = VHost, name = Name}, #{type := Type, binding_count := Bindings}, Acc) -> [{<<"vhost=\"", VHost/binary, "\",exchange=\"", Name/binary, "\",type=\"", Type/binary, "\"">>, Bindings}|Acc] diff --git a/deps/rabbitmq_random_exchange/src/rabbit_exchange_type_random.erl b/deps/rabbitmq_random_exchange/src/rabbit_exchange_type_random.erl index 7b86152d2775..2852374978d2 100644 --- a/deps/rabbitmq_random_exchange/src/rabbit_exchange_type_random.erl +++ b/deps/rabbitmq_random_exchange/src/rabbit_exchange_type_random.erl @@ -19,8 +19,8 @@ -export([ add_binding/3, assert_args_equivalence/2, - create/2, - delete/2, + create/2, + delete/2, policy_changed/2, description/0, recover/2, diff --git a/deps/rabbitmq_recent_history_exchange/BUILD.bazel b/deps/rabbitmq_recent_history_exchange/BUILD.bazel index 47a8e881c4f3..73121ad44906 100644 --- a/deps/rabbitmq_recent_history_exchange/BUILD.bazel +++ b/deps/rabbitmq_recent_history_exchange/BUILD.bazel @@ -24,8 +24,6 @@ APP_DESCRIPTION = "RabbitMQ Recent History Exchange" all_beam_files(name = "all_beam_files") -all_test_beam_files(name = "all_test_beam_files") - all_srcs(name = "all_srcs") test_suite_beam_files(name = "test_suite_beam_files") @@ -43,6 +41,8 @@ rabbitmq_app( deps = [ "//deps/rabbit:erlang_app", "//deps/rabbit_common:erlang_app", + "@khepri//:erlang_app", + "@khepri_mnesia_migration//:erlang_app", ], ) @@ -86,3 +86,5 @@ alias( actual = ":erlang_app", visibility = ["//visibility:public"], ) + +all_test_beam_files(name = "all_test_beam_files") diff --git a/deps/rabbitmq_recent_history_exchange/app.bzl b/deps/rabbitmq_recent_history_exchange/app.bzl index db38feebd080..3bd05fe8ae54 100644 --- a/deps/rabbitmq_recent_history_exchange/app.bzl +++ b/deps/rabbitmq_recent_history_exchange/app.bzl @@ -10,6 +10,7 @@ def all_beam_files(name = "all_beam_files"): name = "other_beam", srcs = [ "src/rabbit_db_rh_exchange.erl", + "src/rabbit_db_rh_exchange_m2k_converter.erl", "src/rabbit_exchange_type_recent_history.erl", ], hdrs = [":public_and_private_hdrs"], @@ -19,6 +20,8 @@ def all_beam_files(name = "all_beam_files"): deps = [ "//deps/rabbit:erlang_app", "//deps/rabbit_common:erlang_app", + "@khepri//:erlang_app", + "@khepri_mnesia_migration//:erlang_app", ], ) @@ -33,6 +36,7 @@ def all_test_beam_files(name = "all_test_beam_files"): testonly = True, srcs = [ "src/rabbit_db_rh_exchange.erl", + "src/rabbit_db_rh_exchange_m2k_converter.erl", "src/rabbit_exchange_type_recent_history.erl", ], hdrs = [":public_and_private_hdrs"], @@ -42,6 +46,8 @@ def all_test_beam_files(name = "all_test_beam_files"): deps = [ "//deps/rabbit:erlang_app", "//deps/rabbit_common:erlang_app", + "@khepri//:erlang_app", + "@khepri_mnesia_migration//:erlang_app", ], ) @@ -63,6 +69,7 @@ def all_srcs(name = "all_srcs"): name = "srcs", srcs = [ "src/rabbit_db_rh_exchange.erl", + "src/rabbit_db_rh_exchange_m2k_converter.erl", "src/rabbit_exchange_type_recent_history.erl", ], ) diff --git a/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange.erl b/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange.erl index 915b27c885e3..e1064f74c9fb 100644 --- a/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange.erl +++ b/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange.erl @@ -8,6 +8,7 @@ -module(rabbit_db_rh_exchange). -include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("khepri/include/khepri.hrl"). -include("rabbit_recent_history.hrl"). -export([ @@ -18,12 +19,21 @@ delete/1 ]). +-export([khepri_recent_history_path/1, + khepri_recent_history_path/0]). + +-rabbit_mnesia_tables_to_khepri_db( + [{?RH_TABLE, rabbit_db_rh_exchange_m2k_converter}]). + %% ------------------------------------------------------------------- %% setup_schema(). %% ------------------------------------------------------------------- setup_schema() -> - setup_schema_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> setup_schema_in_mnesia() end, + khepri => fun() -> ok end + }). setup_schema_in_mnesia() -> _ = mnesia:create_table(?RH_TABLE, @@ -39,12 +49,24 @@ setup_schema_in_mnesia() -> %% ------------------------------------------------------------------- get(XName) -> - get_in_mnesia(XName). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> get_in_mnesia(XName) end, + khepri => fun() -> get_in_khepri(XName) end + }). get_in_mnesia(XName) -> rabbit_mnesia:execute_mnesia_transaction( fun() -> get_in_mnesia_tx(XName) end). +get_in_khepri(XName) -> + Path = khepri_recent_history_path(XName), + case rabbit_khepri:get(Path) of + {ok, Cached} -> + Cached; + _ -> + [] + end. + get_in_mnesia_tx(XName) -> case mnesia:read(?RH_TABLE, XName) of [] -> @@ -58,7 +80,10 @@ get_in_mnesia_tx(XName) -> %% ------------------------------------------------------------------- insert(XName, Message, Length) -> - insert_in_mnesia(XName, Message, Length). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> insert_in_mnesia(XName, Message, Length) end, + khepri => fun() -> insert_in_khepri(XName, Message, Length) end + }). insert_in_mnesia(XName, Message, Length) -> rabbit_mnesia:execute_mnesia_transaction( @@ -78,21 +103,71 @@ insert0_in_mnesia(Key, Cached, Message, Length) -> content = [Message|lists:sublist(Cached, Length-1)]}, write). +insert_in_khepri(XName, Message, Length) -> + Path = khepri_recent_history_path(XName), + case rabbit_khepri:adv_get(Path) of + {ok, #{data := Cached0, payload_version := DVersion}} -> + Cached = add_to_cache(Cached0, Message, Length), + Path1 = khepri_path:combine_with_conditions( + Path, [#if_payload_version{version = DVersion}]), + Ret = rabbit_khepri:put(Path1, Cached), + case Ret of + ok -> + ok; + {error, {khepri, mismatching_node, _}} -> + insert_in_khepri(XName, Message, Length); + {error, _} = Error -> + Error + end; + _ -> + Cached = add_to_cache([], Message, Length), + rabbit_khepri:put(Path, Cached) + end. + +add_to_cache(Cached, Message, undefined) -> + add_to_cache(Cached, Message, ?KEEP_NB); +add_to_cache(Cached, Message, {_Type, Length}) -> + add_to_cache(Cached, Message, Length); +add_to_cache(Cached, Message, Length) -> + [Message|lists:sublist(Cached, Length-1)]. + %% ------------------------------------------------------------------- %% delete(). %% ------------------------------------------------------------------- delete() -> - delete_in_mnesia(). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> delete_in_mnesia() end, + khepri => fun() -> delete_in_khepri() end + }). delete_in_mnesia() -> - _ = mnesia:delete_table(?RH_TABLE). + mnesia:delete_table(?RH_TABLE). + +delete_in_khepri() -> + rabbit_khepri:delete(khepri_recent_history_path()). delete(XName) -> - delete_in_mnesia(XName). + rabbit_khepri:handle_fallback( + #{mnesia => fun() -> delete_in_mnesia(XName) end, + khepri => fun() -> delete_in_khepri(XName) end + }). delete_in_mnesia(XName) -> rabbit_mnesia:execute_mnesia_transaction( fun() -> mnesia:delete(?RH_TABLE, XName, write) end). + +delete_in_khepri(XName) -> + rabbit_khepri:delete(khepri_recent_history_path(XName)). + +%% ------------------------------------------------------------------- +%% paths +%% ------------------------------------------------------------------- + +khepri_recent_history_path() -> + [?MODULE, recent_history_exchange]. + +khepri_recent_history_path(#resource{virtual_host = VHost, name = Name}) -> + [?MODULE, recent_history_exchange, VHost, Name]. diff --git a/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange_m2k_converter.erl b/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange_m2k_converter.erl new file mode 100644 index 000000000000..144ae6987a4b --- /dev/null +++ b/deps/rabbitmq_recent_history_exchange/src/rabbit_db_rh_exchange_m2k_converter.erl @@ -0,0 +1,99 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright © 2022-2023 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_db_rh_exchange_m2k_converter). + +-behaviour(mnesia_to_khepri_converter). + +-include_lib("kernel/include/logger.hrl"). +-include_lib("khepri/include/khepri.hrl"). +-include_lib("khepri_mnesia_migration/src/kmm_logging.hrl"). +-include_lib("rabbit_common/include/rabbit.hrl"). + +-include("rabbit_recent_history.hrl"). + +-export([init_copy_to_khepri/3, + copy_to_khepri/3, + delete_from_khepri/3, + clear_data_in_khepri/1]). + +-record(?MODULE, {store_id :: khepri:store_id()}). + +-spec init_copy_to_khepri(StoreId, MigrationId, Tables) -> Ret when + StoreId :: khepri:store_id(), + MigrationId :: mnesia_to_khepri:migration_id(), + Tables :: [mnesia_to_khepri:mnesia_table()], + Ret :: {ok, Priv}, + Priv :: #?MODULE{}. +%% @private + +init_copy_to_khepri(StoreId, _MigrationId, _Tables) -> + State = #?MODULE{store_id = StoreId}, + {ok, State}. + +-spec copy_to_khepri(Table, Record, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Record :: tuple(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +copy_to_khepri(?RH_TABLE = Table, #cached{key = Key, content = Content}, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] key: ~0p", + [Table, Key], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_rh_exchange:khepri_recent_history_path(Key), + ?LOG_DEBUG( + "Mnesia->Khepri data copy: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:put(StoreId, Path, Content) of + ok -> {ok, State}; + Error -> Error + end; +copy_to_khepri(Table, Record, State) -> + ?LOG_DEBUG("Mnesia->Khepri unexpected record table ~0p record ~0p state ~0p", + [Table, Record, State]), + {error, unexpected_record}. + +-spec delete_from_khepri(Table, Key, Priv) -> Ret when + Table :: mnesia_to_khepri:mnesia_table(), + Key :: any(), + Priv :: #?MODULE{}, + Ret :: {ok, NewPriv} | {error, Reason}, + NewPriv :: #?MODULE{}, + Reason :: any(). +%% @private + +delete_from_khepri(?RH_TABLE = Table, Key, + #?MODULE{store_id = StoreId} = State) -> + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] key: ~0p", + [Table, Key], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + Path = rabbit_db_rh_exchange:khepri_recent_history_path(Key), + ?LOG_DEBUG( + "Mnesia->Khepri data delete: [~0p] path: ~0p", + [Table, Path], + #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}), + case khepri:delete(StoreId, Path) of + ok -> {ok, State}; + Error -> Error + end. + +clear_data_in_khepri(?RH_TABLE) -> + Path = rabbit_db_rh_exchange:khepri_recent_history_path(), + case rabbit_khepri:delete(Path) of + ok -> + ok; + Error -> + throw(Error) + end. diff --git a/deps/rabbitmq_recent_history_exchange/src/rabbit_exchange_type_recent_history.erl b/deps/rabbitmq_recent_history_exchange/src/rabbit_exchange_type_recent_history.erl index 9aeea6ab7834..e37dbae15e5f 100644 --- a/deps/rabbitmq_recent_history_exchange/src/rabbit_exchange_type_recent_history.erl +++ b/deps/rabbitmq_recent_history_exchange/src/rabbit_exchange_type_recent_history.erl @@ -7,7 +7,6 @@ -module(rabbit_exchange_type_recent_history). -include_lib("rabbit_common/include/rabbit.hrl"). --include("rabbit_recent_history.hrl"). -behaviour(rabbit_exchange_type). @@ -68,7 +67,7 @@ validate(#exchange{arguments = Args}) -> end. validate_binding(_X, _B) -> ok. -create(_Tx, _X) -> ok. +create(_Serial, _X) -> ok. policy_changed(_X1, _X2) -> ok. delete(_Tx, #exchange{ name = XName }) -> @@ -101,11 +100,9 @@ add_binding(_Tx, #exchange{ name = XName }, end end || Msg <- Msgs] end, - ok; -add_binding(_Tx, _Exchange, _Binding) -> ok. -remove_bindings(_Tx, _X, _Bs) -> ok. +remove_bindings(_Serial, _X, _Bs) -> ok. assert_args_equivalence(X, Args) -> rabbit_exchange:assert_args_equivalence(X, Args). @@ -117,8 +114,7 @@ setup_schema() -> disable_plugin() -> rabbit_registry:unregister(exchange, <<"x-recent-history">>), - _ = rabbit_db_rh_exchange:delete(), - ok. + rabbit_db_rh_exchange:delete(). %%---------------------------------------------------------------------------- %%private diff --git a/deps/rabbitmq_recent_history_exchange/test/system_SUITE.erl b/deps/rabbitmq_recent_history_exchange/test/system_SUITE.erl index 61862ee288a0..3ceca523029a 100644 --- a/deps/rabbitmq_recent_history_exchange/test/system_SUITE.erl +++ b/deps/rabbitmq_recent_history_exchange/test/system_SUITE.erl @@ -16,19 +16,32 @@ all() -> [ - {group, non_parallel_tests} + {group, mnesia_store}, + {group, khepri_store}, + {group, khepri_migration} ]. groups() -> [ - {non_parallel_tests, [], [ - default_length_test, - length_argument_test, - wrong_argument_type_test, - no_store_test, - e2e_test, - multinode_test - ]} + {mnesia_store, [], [ + {non_parallel_tests, [], all_tests()} + ]}, + {khepri_store, [], [ + {non_parallel_tests, [], all_tests()} + ]}, + {khepri_migration, [], [ + from_mnesia_to_khepri + ]} + ]. + +all_tests() -> + [ + default_length_test, + length_argument_test, + wrong_argument_type_test, + no_store_test, + e2e_test, + multinode_test ]. %% ------------------------------------------------------------------- @@ -38,24 +51,34 @@ groups() -> init_per_suite(Config) -> inets:start(), rabbit_ct_helpers:log_environment(), - Config1 = rabbit_ct_helpers:set_config(Config, [ - {rmq_nodename_suffix, ?MODULE}, - {rmq_nodes_count, 2} - ]), - rabbit_ct_helpers:run_setup_steps(Config1, - rabbit_ct_broker_helpers:setup_steps() ++ - rabbit_ct_client_helpers:setup_steps()). + rabbit_ct_helpers:run_setup_steps(Config). end_per_suite(Config) -> - rabbit_ct_helpers:run_teardown_steps(Config, - rabbit_ct_client_helpers:teardown_steps() ++ - rabbit_ct_broker_helpers:teardown_steps()). - + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(mnesia_store, Config) -> + rabbit_ct_helpers:set_config(Config, [{metadata_store, mnesia}]); +init_per_group(khepri_store, Config) -> + rabbit_ct_helpers:set_config( + Config, + [{metadata_store, {khepri, [khepri_db]}}]); init_per_group(_, Config) -> - Config. + Config1 = rabbit_ct_helpers:set_config(Config, [ + {rmq_nodename_suffix, ?MODULE}, + {rmq_nodes_count, 2} + ]), + rabbit_ct_helpers:run_setup_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). +end_per_group(mnesia_store, Config) -> + Config; +end_per_group(khepri_store, Config) -> + Config; end_per_group(_, Config) -> - Config. + rabbit_ct_helpers:run_teardown_steps(Config, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()). init_per_testcase(Testcase, Config) -> TestCaseName = rabbit_ct_helpers:config_to_testcase_name(Config, Testcase), @@ -279,3 +302,74 @@ qs() -> make_exchange_name(Config, Suffix) -> B = rabbit_ct_helpers:get_config(Config, test_resource_name), erlang:list_to_binary("x-" ++ B ++ "-" ++ Suffix). + +from_mnesia_to_khepri(Config) -> + MsgCount = 10, + + {Conn, Chan} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0), + + #'exchange.declare_ok'{} = + amqp_channel:call(Chan, + #'exchange.declare' { + exchange = make_exchange_name(Config, "1"), + type = <<"x-recent-history">>, + auto_delete = true + }), + + #'exchange.declare_ok'{} = + amqp_channel:call(Chan, + #'exchange.declare' { + exchange = make_exchange_name(Config, "2"), + type = <<"direct">>, + auto_delete = true + }), + + #'queue.declare_ok'{queue = Q} = + amqp_channel:call(Chan, #'queue.declare' { + queue = <<"q">> + }), + + #'queue.bind_ok'{} = + amqp_channel:call(Chan, #'queue.bind' { + queue = Q, + exchange = make_exchange_name(Config, "2"), + routing_key = <<"">> + }), + + #'tx.select_ok'{} = amqp_channel:call(Chan, #'tx.select'{}), + [amqp_channel:call(Chan, + #'basic.publish'{exchange = make_exchange_name(Config, "1")}, + #amqp_msg{props = #'P_basic'{}, payload = <<>>}) || + _ <- lists:duplicate(MsgCount, const)], + amqp_channel:call(Chan, #'tx.commit'{}), + + amqp_channel:call(Chan, + #'exchange.bind' { + source = make_exchange_name(Config, "1"), + destination = make_exchange_name(Config, "2"), + routing_key = <<"">> + }), + + case rabbit_ct_broker_helpers:enable_feature_flag(Config, khepri_db) of + ok -> + case rabbit_ct_broker_helpers:enable_feature_flag(Config, rabbit_recent_history_exchange_raft_based_metadata_store) of + ok -> + #'queue.declare_ok'{message_count = Count, queue = Q} = + amqp_channel:call(Chan, #'queue.declare' { + passive = true, + queue = Q + }), + ?assertEqual(MsgCount, Count), + + amqp_channel:call(Chan, #'exchange.delete' { exchange = make_exchange_name(Config, "1") }), + amqp_channel:call(Chan, #'exchange.delete' { exchange = make_exchange_name(Config, "2") }), + amqp_channel:call(Chan, #'queue.delete' { queue = Q }), + + rabbit_ct_client_helpers:close_connection_and_channel(Conn, Chan), + ok; + Skip -> + Skip + end; + Skip -> + Skip + end. diff --git a/deps/rabbitmq_shovel/src/rabbit_amqp091_shovel.erl b/deps/rabbitmq_shovel/src/rabbit_amqp091_shovel.erl index e85f245f5880..cbee664040ee 100644 --- a/deps/rabbitmq_shovel/src/rabbit_amqp091_shovel.erl +++ b/deps/rabbitmq_shovel/src/rabbit_amqp091_shovel.erl @@ -34,6 +34,14 @@ forward/4 ]). +%% Function references should not be stored on the metadata store. +%% They are only valid for the version of the module they were created +%% from and can break with the next upgrade. It should not be used by +%% another one that the one who created it or survive a node restart. +%% Thus, function references have been replace by the following MFA. +-export([decl_fun/3, publish_fun/4, props_fun_timestamp_header/4, + props_fun_forward_header/5]). + -define(MAX_CONNECTION_CLOSE_TIMEOUT, 10000). parse(_Name, {source, Source}) -> @@ -77,9 +85,9 @@ init_source(Conf = #{ack_mode := AckMode, source := #{queue := Queue, current := {Conn, Chan, _}, prefetch_count := Prefetch, - resource_decl := Decl, + resource_decl := {M, F, MFArgs}, consumer_args := Args} = Src}) -> - Decl(Conn, Chan), + apply(M, F, MFArgs ++ [Conn, Chan]), NoAck = AckMode =:= no_ack, case NoAck of @@ -108,9 +116,9 @@ connect_dest(Conf = #{name := Name, dest := #{uris := Uris} = Dst}) -> init_dest(Conf = #{ack_mode := AckMode, dest := #{current := {Conn, Chan, _}, - resource_decl := Decl} = Dst}) -> + resource_decl := {M, F, MFArgs}} = Dst}) -> - Decl(Conn, Chan), + apply(M, F, MFArgs ++ [Conn, Chan]), case AckMode of on_confirm -> @@ -187,16 +195,16 @@ forward(IncomingTag, Props, Payload, State) -> end. do_forward(IncomingTag, Props, Payload, - State0 = #{dest := #{props_fun := PropsFun, + State0 = #{dest := #{props_fun := {M, F, Args}, current := {_, _, DstUri}, - fields_fun := FieldsFun}}) -> + fields_fun := {Mf, Ff, Argsf}}}) -> SrcUri = rabbit_shovel_behaviour:source_uri(State0), % do publish Exchange = maps:get(exchange, Props, undefined), RoutingKey = maps:get(routing_key, Props, undefined), Method = #'basic.publish'{exchange = Exchange, routing_key = RoutingKey}, - Method1 = FieldsFun(SrcUri, DstUri, Method), - Msg1 = #amqp_msg{props = PropsFun(SrcUri, DstUri, props_from_map(Props)), + Method1 = apply(Mf, Ff, Argsf ++ [SrcUri, DstUri, Method]), + Msg1 = #amqp_msg{props = apply(M, F, Args ++ [SrcUri, DstUri, props_from_map(Props)]), payload = Payload}, publish(IncomingTag, Method1, Msg1, State0). @@ -519,11 +527,7 @@ make_publish_fun(Fields, ValidFields) when is_list(Fields) -> case SuppliedFields -- ValidFields of [] -> FieldIndices = make_field_indices(ValidFields, Fields), - fun (_SrcUri, _DestUri, Publish) -> - lists:foldl(fun ({Pos1, Value}, Pub) -> - setelement(Pos1, Pub, Value) - end, Publish, FieldIndices) - end; + {?MODULE, publish_fun, [FieldIndices]}; Unexpected -> fail({invalid_parameter_value, publish_properties, {unexpected_fields, Unexpected, ValidFields}}) @@ -532,6 +536,11 @@ make_publish_fun(Fields, _) -> fail({invalid_parameter_value, publish_properties, {require_list, Fields}}). +publish_fun(FieldIndices, _SrcUri, _DestUri, Publish) -> + lists:foldl(fun ({Pos1, Value}, Pub) -> + setelement(Pos1, Pub, Value) + end, Publish, FieldIndices). + make_field_indices(Valid, Fields) -> make_field_indices(Fields, field_map(Valid, 2), []). @@ -551,23 +560,25 @@ field_map(Fields, Idx0) -> fail(Reason) -> throw({error, Reason}). add_forward_headers_fun(Name, true, PubProps) -> - fun(SrcUri, DestUri, Props) -> - rabbit_shovel_util:update_headers( - [{<<"shovelled-by">>, rabbit_nodes:cluster_name()}, - {<<"shovel-type">>, <<"static">>}, - {<<"shovel-name">>, list_to_binary(atom_to_list(Name))}], - [], SrcUri, DestUri, PubProps(SrcUri, DestUri, Props)) - end; + {?MODULE, props_fun_forward_header, [Name, PubProps]}; add_forward_headers_fun(_Name, false, PubProps) -> PubProps. +props_fun_forward_header(Name, {M, F, Args}, SrcUri, DestUri, Props) -> + rabbit_shovel_util:update_headers( + [{<<"shovelled-by">>, rabbit_nodes:cluster_name()}, + {<<"shovel-type">>, <<"static">>}, + {<<"shovel-name">>, list_to_binary(atom_to_list(Name))}], + [], SrcUri, DestUri, apply(M, F, Args ++ [SrcUri, DestUri, Props])). + add_timestamp_header_fun(true, PubProps) -> - fun(SrcUri, DestUri, Props) -> - rabbit_shovel_util:add_timestamp_header( - PubProps(SrcUri, DestUri, Props)) - end; + {?MODULE, props_fun_timestamp_header, [PubProps]}; add_timestamp_header_fun(false, PubProps) -> PubProps. +props_fun_timestamp_header({M, F, Args}, SrcUri, DestUri, Props) -> + rabbit_shovel_util:add_timestamp_header( + apply(M, F, Args ++ [SrcUri, DestUri, Props])). + parse_declaration({[], Acc}) -> Acc; parse_declaration({[{Method, Props} | Rest], Acc}) when is_list(Props) -> @@ -596,11 +607,12 @@ parse_declaration({[Method | Rest], Acc}) -> decl_fun(Endpoint) -> Decl = parse_declaration({proplists:get_value(declarations, Endpoint, []), []}), - fun (_Conn, Ch) -> - [begin - amqp_channel:call(Ch, M) - end || M <- lists:reverse(Decl)] - end. + {?MODULE, decl_fun, [Decl]}. + +decl_fun(Decl, _Conn, Ch) -> + [begin + amqp_channel:call(Ch, M) + end || M <- lists:reverse(Decl)]. parse_parameter(Param, Fun, Value) -> try diff --git a/deps/rabbitmq_shovel/src/rabbit_shovel_dyn_worker_sup_sup.erl b/deps/rabbitmq_shovel/src/rabbit_shovel_dyn_worker_sup_sup.erl index c9b82335f05d..0adeafc0a1ca 100644 --- a/deps/rabbitmq_shovel/src/rabbit_shovel_dyn_worker_sup_sup.erl +++ b/deps/rabbitmq_shovel/src/rabbit_shovel_dyn_worker_sup_sup.erl @@ -43,7 +43,7 @@ start_child({VHost, ShovelName} = Name, Def) -> rabbit_log_shovel:debug("Starting a mirrored supervisor named '~ts' in virtual host '~ts'", [ShovelName, VHost]), Result = case mirrored_supervisor:start_child( ?SUPERVISOR, - {Name, {rabbit_shovel_dyn_worker_sup, start_link, [Name, obfuscated_uris_parameters(Def)]}, + {id(Name), {rabbit_shovel_dyn_worker_sup, start_link, [Name, obfuscated_uris_parameters(Def)]}, transient, ?WORKER_WAIT, worker, [rabbit_shovel_dyn_worker_sup]}) of {ok, _Pid} -> ok; {error, {already_started, _Pid}} -> ok @@ -58,7 +58,7 @@ obfuscated_uris_parameters(Def) when is_list(Def) -> rabbit_shovel_parameters:obfuscate_uris_in_definition(Def). child_exists(Name) -> - lists:any(fun ({N, _, _, _}) -> N =:= Name end, + lists:any(fun ({{_, N}, _, _, _}) -> N =:= Name end, mirrored_supervisor:which_children(?SUPERVISOR)). stop_child({VHost, ShovelName} = Name) -> @@ -67,8 +67,8 @@ stop_child({VHost, ShovelName} = Name) -> case get({shovel_worker_autodelete, Name}) of true -> ok; %% [1] _ -> - ok = mirrored_supervisor:terminate_child(?SUPERVISOR, Name), - ok = mirrored_supervisor:delete_child(?SUPERVISOR, Name), + ok = mirrored_supervisor:terminate_child(?SUPERVISOR, id(Name)), + ok = mirrored_supervisor:delete_child(?SUPERVISOR, id(Name)), rabbit_shovel_status:remove(Name) end, rabbit_shovel_locks:unlock(LockId), @@ -83,10 +83,10 @@ stop_child({VHost, ShovelName} = Name) -> %% See rabbit_shovel_worker:terminate/2 cleanup_specs() -> - SpecsSet = sets:from_list([element(1, S) || S <- mirrored_supervisor:which_children(?SUPERVISOR)]), + SpecsSet = sets:from_list([element(2, element(1, S)) || S <- mirrored_supervisor:which_children(?SUPERVISOR)]), ParamsSet = sets:from_list(rabbit_runtime_parameters:list_component(<<"shovel">>)), - F = fun(Spec, ok) -> - _ = mirrored_supervisor:delete_child(?SUPERVISOR, Spec), + F = fun(Name, ok) -> + _ = mirrored_supervisor:delete_child(?SUPERVISOR, id(Name)), ok end, ok = sets:fold(F, ok, sets:subtract(SpecsSet, ParamsSet)). @@ -95,3 +95,6 @@ cleanup_specs() -> init([]) -> {ok, {{one_for_one, 3, 10}, []}}. + +id({V, S} = Name) -> + {[V, S], Name}. diff --git a/deps/rabbitmq_shovel/src/rabbit_shovel_parameters.erl b/deps/rabbitmq_shovel/src/rabbit_shovel_parameters.erl index b6682c25b8ed..df7df4f37573 100644 --- a/deps/rabbitmq_shovel/src/rabbit_shovel_parameters.erl +++ b/deps/rabbitmq_shovel/src/rabbit_shovel_parameters.erl @@ -15,6 +15,14 @@ -export([register/0, unregister/0, parse/3]). -export([obfuscate_uris_in_definition/1]). +%% Function references should not be stored on the metadata store. +%% They are only valid for the version of the module they were created +%% from and can break with the next upgrade. It should not be used by +%% another one that the one who created it or survive a node restart. +%% Thus, function references have been replace by the following MFA. +-export([dest_decl/4, src_decl_exchange/4, src_decl_queue/4, + fields_fun/5, props_fun/9]). + -import(rabbit_misc, [pget/2, pget/3, pset/3]). -rabbit_boot_step({?MODULE, @@ -321,12 +329,7 @@ parse_amqp091_dest({VHost, Name}, ClusterName, Def, SourceHeaders) -> DestXKey = pget(<<"dest-exchange-key">>, Def, none), DestQ = pget(<<"dest-queue">>, Def, none), DestQArgs = pget(<<"dest-queue-args">>, Def, #{}), - DestDeclFun = fun (Conn, _Ch) -> - case DestQ of - none -> ok; - _ -> ensure_queue(Conn, DestQ, rabbit_misc:to_amqp_table(DestQArgs)) - end - end, + DestDeclFun = {?MODULE, dest_decl, [DestQ, DestQArgs]}, {X, Key} = case DestQ of none -> {DestX, DestXKey}; _ -> {<<>>, DestQ} @@ -335,16 +338,6 @@ parse_amqp091_dest({VHost, Name}, ClusterName, Def, SourceHeaders) -> {<<"dest-exchange-key">>, DestXKey}, {<<"dest-queue">>, DestQ}], V =/= none], - PubFun = fun (_SrcURI, _DestURI, P0) -> - P1 = case X of - none -> P0; - _ -> P0#'basic.publish'{exchange = X} - end, - case Key of - none -> P1; - _ -> P1#'basic.publish'{routing_key = Key} - end - end, AddHeadersLegacy = pget(<<"add-forward-headers">>, Def, false), AddHeaders = pget(<<"dest-add-forward-headers">>, Def, AddHeadersLegacy), Table0 = [{<<"shovelled-by">>, ClusterName}, @@ -357,19 +350,6 @@ parse_amqp091_dest({VHost, Name}, ClusterName, Def, SourceHeaders) -> AddTimestampHeaderLegacy = pget(<<"add-timestamp-header">>, Def, false), AddTimestampHeader = pget(<<"dest-add-timestamp-header">>, Def, AddTimestampHeaderLegacy), - PubPropsFun = fun (SrcURI, DestURI, P0) -> - P = set_properties(P0, SetProps), - P1 = case AddHeaders of - true -> rabbit_shovel_util:update_headers( - Table0, SourceHeaders ++ Table2, - SrcURI, DestURI, P); - false -> P - end, - case AddTimestampHeader of - true -> rabbit_shovel_util:add_timestamp_header(P1); - false -> P1 - end - end, %% Details are only used for status report in rabbitmqctl, as vhost is not %% available to query the runtime parameters. Details = maps:from_list([{K, V} || {K, V} <- [{dest_exchange, DestX}, @@ -379,10 +359,42 @@ parse_amqp091_dest({VHost, Name}, ClusterName, Def, SourceHeaders) -> maps:merge(#{module => rabbit_amqp091_shovel, uris => DestURIs, resource_decl => DestDeclFun, - fields_fun => PubFun, - props_fun => PubPropsFun + fields_fun => {?MODULE, fields_fun, [X, Key]}, + props_fun => {?MODULE, props_fun, [Table0, Table2, SetProps, + AddHeaders, SourceHeaders, + AddTimestampHeader]} }, Details). +fields_fun(X, Key, _SrcURI, _DestURI, P0) -> + P1 = case X of + none -> P0; + _ -> P0#'basic.publish'{exchange = X} + end, + case Key of + none -> P1; + _ -> P1#'basic.publish'{routing_key = Key} + end. + +props_fun(Table0, Table2, SetProps, AddHeaders, SourceHeaders, AddTimestampHeader, + SrcURI, DestURI, P0) -> + P = set_properties(P0, SetProps), + P1 = case AddHeaders of + true -> rabbit_shovel_util:update_headers( + Table0, SourceHeaders ++ Table2, + SrcURI, DestURI, P); + false -> P + end, + case AddTimestampHeader of + true -> rabbit_shovel_util:add_timestamp_header(P1); + false -> P1 + end. + +dest_decl(DestQ, DestQArgs, Conn, _Ch) -> + case DestQ of + none -> ok; + _ -> ensure_queue(Conn, DestQ, rabbit_misc:to_amqp_table(DestQArgs)) + end. + parse_amqp10_source(Def) -> Uris = deobfuscated_uris(<<"src-uri">>, Def), Address = pget(<<"src-address">>, Def), @@ -405,16 +417,11 @@ parse_amqp091_source(Def) -> SrcCArgs = rabbit_misc:to_amqp_table(pget(<<"src-consumer-args">>, Def, [])), {SrcDeclFun, Queue, DestHeaders} = case SrcQ of - none -> {fun (_Conn, Ch) -> - Ms = [#'queue.declare'{exclusive = true}, - #'queue.bind'{routing_key = SrcXKey, - exchange = SrcX}], - [amqp_channel:call(Ch, M) || M <- Ms] - end, <<>>, [{<<"src-exchange">>, SrcX}, - {<<"src-exchange-key">>, SrcXKey}]}; - _ -> {fun (Conn, _Ch) -> - ensure_queue(Conn, SrcQ, rabbit_misc:to_amqp_table(SrcQArgs)) - end, SrcQ, [{<<"src-queue">>, SrcQ}]} + none -> {{?MODULE, src_decl_exchange, [SrcX, SrcXKey]}, <<>>, + [{<<"src-exchange">>, SrcX}, + {<<"src-exchange-key">>, SrcXKey}]}; + _ -> {{?MODULE, src_decl_queue, [SrcQ, SrcQArgs]}, + SrcQ, [{<<"src-queue">>, SrcQ}]} end, DeleteAfter = pget(<<"src-delete-after">>, Def, pget(<<"delete-after">>, Def, <<"never">>)), @@ -434,6 +441,15 @@ parse_amqp091_source(Def) -> consumer_args => SrcCArgs }, Details), DestHeaders}. +src_decl_exchange(SrcX, SrcXKey, _Conn, Ch) -> + Ms = [#'queue.declare'{exclusive = true}, + #'queue.bind'{routing_key = SrcXKey, + exchange = SrcX}], + [amqp_channel:call(Ch, M) || M <- Ms]. + +src_decl_queue(SrcQ, SrcQArgs, Conn, _Ch) -> + ensure_queue(Conn, SrcQ, rabbit_misc:to_amqp_table(SrcQArgs)). + get_uris(Key, Def) -> URIs = case pget(Key, Def) of B when is_binary(B) -> [B]; diff --git a/deps/rabbitmq_shovel/src/rabbit_shovel_worker_sup.erl b/deps/rabbitmq_shovel/src/rabbit_shovel_worker_sup.erl index 35764c628da0..66538eac481c 100644 --- a/deps/rabbitmq_shovel/src/rabbit_shovel_worker_sup.erl +++ b/deps/rabbitmq_shovel/src/rabbit_shovel_worker_sup.erl @@ -18,7 +18,7 @@ start_link(ShovelName, ShovelConfig) -> ?MODULE, [ShovelName, ShovelConfig]). init([Name, Config]) -> - ChildSpecs = [{Name, + ChildSpecs = [{id(Name), {rabbit_shovel_worker, start_link, [static, Name, Config]}, case Config of #{reconnect_delay := N} @@ -29,3 +29,6 @@ init([Name, Config]) -> worker, [rabbit_shovel_worker]}], {ok, {{one_for_one, 1, ?MAX_WAIT}, ChildSpecs}}. + +id(Name) -> + {[Name], Name}. diff --git a/deps/rabbitmq_shovel/test/parameters_SUITE.erl b/deps/rabbitmq_shovel/test/parameters_SUITE.erl index 837faea7bc3f..913ee24cec0b 100644 --- a/deps/rabbitmq_shovel/test/parameters_SUITE.erl +++ b/deps/rabbitmq_shovel/test/parameters_SUITE.erl @@ -159,7 +159,7 @@ test_parse_amqp091(Params) -> reconnect_delay := 1001, dest := #{module := rabbit_amqp091_shovel, uris := ["amqp://remotehost:5672"], - props_fun := PropsFun + props_fun := {M, F, Args} }, source := #{module := rabbit_amqp091_shovel, uris := ["amqp://localhost:5672"], @@ -170,9 +170,9 @@ test_parse_amqp091(Params) -> #'P_basic'{headers = ActualHeaders, delivery_mode = 2, - cluster_id = <<"x">>} = PropsFun("amqp://localhost:5672", - "amqp://remotehost:5672", - #'P_basic'{headers = undefined}), + cluster_id = <<"x">>} = apply(M, F, Args ++ ["amqp://localhost:5672", + "amqp://remotehost:5672", + #'P_basic'{headers = undefined}]), assert_amqp901_headers(ActualHeaders), ok. @@ -185,7 +185,7 @@ test_parse_amqp091_with_blank_proprties(Params) -> reconnect_delay := 1001, dest := #{module := rabbit_amqp091_shovel, uris := ["amqp://remotehost:5672"], - props_fun := PropsFun + props_fun := {M, F, Args} }, source := #{module := rabbit_amqp091_shovel, uris := ["amqp://localhost:5672"], @@ -194,9 +194,9 @@ test_parse_amqp091_with_blank_proprties(Params) -> delete_after := 'queue-length'} } = Result, - #'P_basic'{headers = ActualHeaders} = PropsFun("amqp://localhost:5672", - "amqp://remotehost:5672", - #'P_basic'{headers = undefined}), + #'P_basic'{headers = ActualHeaders} = apply(M, F, Args ++ ["amqp://localhost:5672", + "amqp://remotehost:5672", + #'P_basic'{headers = undefined}]), assert_amqp901_headers(ActualHeaders), ok. diff --git a/moduleindex.yaml b/moduleindex.yaml index 6de18fe66202..8207e2baff2a 100755 --- a/moduleindex.yaml +++ b/moduleindex.yaml @@ -122,6 +122,7 @@ credentials_obfuscation: ct_helper: - ct_helper - ct_helper_error_h +- erl_make_certs cth_styledout: - cth_styledout cuttlefish: @@ -208,6 +209,9 @@ erlc: - foo_worker eunit: - foo +eunit_formatters: +- binomial_heap +- eunit_progress eunit_surefire: - foo foo: @@ -267,6 +271,12 @@ hackney: - hackney_trace - hackney_url - hackney_util +horus: +- horus +- horus_cover +- horus_utils +host_triple: +- host_triple idna: - idna - idna_bidi @@ -390,6 +400,61 @@ jose: - jose_jws_alg_rsa_pkcs1_v1_5 - jose_jws_alg_rsa_pss - jose_jwt +khepri: +- khepri +- khepri_adv +- khepri_app +- khepri_cluster +- khepri_condition +- khepri_event_handler +- khepri_evf +- khepri_export_erlang +- khepri_import_export +- khepri_machine +- khepri_path +- khepri_pattern_tree +- khepri_payload +- khepri_projection +- khepri_sproc +- khepri_sup +- khepri_tree +- khepri_tx +- khepri_tx_adv +- khepri_utils +khepri_mnesia_migration: +- khepri_mnesia_migration_app +- khepri_mnesia_migration_sup +- kmm_utils +- m2k_cluster_sync +- m2k_cluster_sync_sup +- m2k_export +- m2k_subscriber +- m2k_table_copy +- m2k_table_copy_sup +- m2k_table_copy_sup_sup +- mnesia_to_khepri +- mnesia_to_khepri_converter +- mnesia_to_khepri_example_converter +looking_glass: +- lg +- lg_callgrind +- lg_file_reader +- lg_file_tracer +- lg_flame +- lg_messages +- lg_messages_seqdiag +- lg_rabbit_hole +- lg_raw_console_tracer +- lg_socket_client +- lg_socket_tracer +- lg_term +- lg_tracer +- lg_tracer_pool +- looking_glass_app +- looking_glass_sup +lz4: +- lz4_nif +- lz4f meck: - meck - meck_args_matcher @@ -624,17 +689,26 @@ rabbit: - rabbit_cuttlefish - rabbit_db - rabbit_db_binding +- rabbit_db_binding_m2k_converter - rabbit_db_cluster - rabbit_db_exchange +- rabbit_db_exchange_m2k_converter +- rabbit_db_m2k_converter - rabbit_db_maintenance +- rabbit_db_maintenance_m2k_converter - rabbit_db_msup +- rabbit_db_msup_m2k_converter - rabbit_db_policy - rabbit_db_queue +- rabbit_db_queue_m2k_converter - rabbit_db_rtparams +- rabbit_db_rtparams_m2k_converter - rabbit_db_topic_exchange - rabbit_db_user +- rabbit_db_user_m2k_converter - rabbit_db_vhost - rabbit_db_vhost_defaults +- rabbit_db_vhost_m2k_converter - rabbit_dead_letter - rabbit_definitions - rabbit_definitions_hashing @@ -676,6 +750,7 @@ rabbit: - rabbit_global_counters - rabbit_guid - rabbit_health_check +- rabbit_khepri - rabbit_limiter - rabbit_log_channel - rabbit_log_connection @@ -903,6 +978,7 @@ rabbitmq_aws: rabbitmq_consistent_hash_exchange: - Elixir.RabbitMQ.CLI.Diagnostics.Commands.ConsistentHashExchangeRingStateCommand - rabbit_db_ch_exchange +- rabbit_db_ch_exchange_m2k_converter - rabbit_exchange_type_consistent_hash rabbitmq_ct_client_helpers: - rabbit_ct_client_helpers @@ -944,6 +1020,7 @@ rabbitmq_federation_management: - rabbit_federation_mgmt rabbitmq_jms_topic_exchange: - rabbit_db_jms_exchange +- rabbit_db_jms_exchange_m2k_converter - rabbit_jms_topic_exchange - sjx_evaluator rabbitmq_management: @@ -1147,6 +1224,7 @@ rabbitmq_random_exchange: - rabbit_exchange_type_random rabbitmq_recent_history_exchange: - rabbit_db_rh_exchange +- rabbit_db_rh_exchange_m2k_converter - rabbit_exchange_type_recent_history rabbitmq_sharding: - rabbit_sharding_exchange_decorator