Allow to use Khepri database to store metadata instead of Mnesia

[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 <jean-sebastien@rabbitmq.com>
Co-authored-by: Diana Parra Corbacho <dparracorbac@vmware.com>
Co-authored-by: Michael Davis <mcarsondavis@gmail.com>
This commit is contained in:
Diana Parra Corbacho 2023-01-05 13:57:50 +01:00 committed by Jean-Sébastien Pédron
parent 35110d8261
commit 5f0981c5a3
No known key found for this signature in database
GPG Key ID: 39E99761A5FD94CC
184 changed files with 14604 additions and 1665 deletions

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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",

114
bazel/BUILD.horus Normal file
View File

@ -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"],
)

185
bazel/BUILD.khepri Normal file
View File

@ -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",
],
)

View File

@ -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*"]),
)

View File

@ -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(

View File

@ -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))))

84
deps/rabbit/app.bzl vendored
View File

@ -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"],
)

View File

@ -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.

View File

@ -78,6 +78,8 @@
pattern_match_all/0,
pattern_match_on_name/1,
pattern_match_on_type/1,
pattern_match_on_durable/1,
pattern_match_on_type_and_durable/2,
reset_mirroring_and_decorators/1,
set_immutable/1,
qnode/1,
@ -590,6 +592,17 @@ pattern_match_on_name(Name) ->
pattern_match_on_type(Type) ->
#amqqueue{type = Type, _ = '_'}.
-spec pattern_match_on_durable(boolean()) -> amqqueue_pattern().
pattern_match_on_durable(IsDurable) ->
#amqqueue{durable = IsDurable, _ = '_'}.
-spec pattern_match_on_type_and_durable(atom(), boolean()) ->
amqqueue_pattern().
pattern_match_on_type_and_durable(Type, IsDurable) ->
#amqqueue{type = Type, durable = IsDurable, _ = '_'}.
-spec reset_mirroring_and_decorators(amqqueue()) -> amqqueue().
reset_mirroring_and_decorators(#amqqueue{} = Queue) ->

View File

@ -0,0 +1 @@
-record(mirrored_sup_childspec, {key, mirroring_pid, childspec}).

View File

@ -356,14 +356,40 @@ run_prelaunch_second_phase() ->
%% 3. Logging.
ok = rabbit_prelaunch_logging:setup(Context),
%% 4. Clustering.
%% 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),
%% 4. Clustering checks. This covers the compatibility between nodes,
%% feature-flags-wise.
ok = rabbit_prelaunch_cluster:setup(Context),
%% Start Mnesia now that everything is ready.
?LOG_DEBUG("Starting Mnesia"),
ok = mnesia:start(),
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),

View File

@ -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)

View File

@ -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

View File

@ -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 ->

View File

@ -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;

View File

@ -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(

View File

@ -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}}
}}).

View File

@ -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.

View File

@ -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
%% --------------------------------------------------------------

View File

@ -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.

View File

@ -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,13 @@ 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}),
case rabbit_khepri:is_enabled() of
true -> forget_member_using_khepri(Node, RemoveWhenOffline);
false -> forget_member_using_mnesia(Node, RemoveWhenOffline)
end;
true ->
{error, {failed_to_remove_node, Node, rabbit_still_running}}
end.
@ -119,6 +145,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 +165,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),
case rabbit_khepri:is_enabled() of
true -> ok;
false -> change_node_type_using_mnesia(NodeType)
end.
change_node_type_using_mnesia(NodeType) ->
rabbit_mnesia:change_cluster_node_type(NodeType).
@ -144,27 +183,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 +225,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 +245,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 +260,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 +279,25 @@ check_consistency_using_mnesia() ->
%% command.
cli_cluster_status() ->
cli_cluster_status_using_mnesia().
case rabbit_khepri:is_enabled() of
true -> cli_cluster_status_using_khepri();
false -> cli_cluster_status_using_mnesia()
end.
cli_cluster_status_using_mnesia() ->
rabbit_mnesia:status().
cli_cluster_status_using_khepri() ->
rabbit_khepri:cli_cluster_status().
rename(Node, NodeMapList) ->
case rabbit_khepri:is_enabled() of
true -> {error, not_supported};
false -> rabbit_mnesia_rename:rename(Node, NodeMapList)
end.
update_cluster_nodes(DiscoveryNode) ->
case rabbit_khepri:is_enabled() of
true -> {error, not_supported};
false -> rabbit_mnesia:update_cluster_nodes(DiscoveryNode)
end.

View File

@ -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).

View File

@ -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.

View File

@ -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.

View File

@ -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
-spec table_definitions() -> [Def] when
Def :: {Name :: atom(), term()}.
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 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].

View File

@ -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.

View File

@ -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.

View File

@ -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.

View File

@ -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),

View File

@ -7,8 +7,11 @@
-module(rabbit_db_queue).
-include_lib("rabbit_common/include/rabbit.hrl").
-include_lib("kernel/include/logger.hrl").
-include_lib("stdlib/include/qlc.hrl").
-include_lib("khepri/include/khepri.hrl").
-include_lib("rabbit_common/include/rabbit.hrl").
-include("amqqueue.hrl").
-export([
@ -26,10 +29,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 +45,46 @@
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
]).
-dialyzer({nowarn_function, [foreach_transient/1,
foreach_transient_in_khepri/1]}).
-define(MNESIA_TABLE, rabbit_queue).
-define(MNESIA_DURABLE_TABLE, rabbit_durable_queue).
-define(KHEPRI_PROJECTION, rabbit_khepri_queue).
%% -------------------------------------------------------------------
%% get_all().
%% -------------------------------------------------------------------
@ -76,7 +99,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 +110,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 +127,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 +139,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 +160,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 +171,13 @@ 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() ->
Pattern = amqqueue:pattern_match_on_durable(true),
ets:match_object(?KHEPRI_PROJECTION, Pattern)
end).
-spec get_all_durable_by_type(Type) -> [Queue] when
Type :: atom(),
Queue :: amqqueue:amqqueue().
@ -137,12 +189,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_and_durable(Type, true),
ets:match_object(?KHEPRI_PROJECTION, Pattern).
%% -------------------------------------------------------------------
%% filter_all_durable().
%% -------------------------------------------------------------------
@ -158,7 +217,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 +230,16 @@ filter_all_durable_in_mnesia(FilterFun) ->
]))
end).
filter_all_durable_in_khepri(FilterFun) ->
ets:foldl(
fun(Q, Acc0) ->
case amqqueue:is_durable(Q) andalso FilterFun(Q) of
true -> [Q | Acc0];
false -> Acc0
end
end,
[], ?KHEPRI_PROJECTION).
%% -------------------------------------------------------------------
%% list().
%% -------------------------------------------------------------------
@ -182,11 +254,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 +280,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 +311,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 +329,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 +343,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 +364,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 +395,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 +424,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 +459,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,11 +482,25 @@ 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_durable_in_khepri(Name) end
}).
get_durable_in_mnesia(Name) ->
rabbit_mnesia:dirty_read({?MNESIA_DURABLE_TABLE, Name}).
get_durable_in_khepri(Name) ->
case get_in_khepri(Name) of
{ok, Queue} = Ret ->
case amqqueue:is_durable(Queue) of
true -> Ret;
false -> {error, not_found}
end;
Error ->
Error
end.
%% -------------------------------------------------------------------
%% get_many_durable().
%% -------------------------------------------------------------------
@ -372,7 +510,17 @@ 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_durable_in_mnesia(Names) end,
khepri => fun() -> get_many_durable_in_khepri(Names) end
}).
get_many_durable_in_mnesia(Names) ->
get_many_in_ets(?MNESIA_DURABLE_TABLE, Names).
get_many_durable_in_khepri(Names) ->
Queues = get_many_in_ets(?KHEPRI_PROJECTION, Names),
[Q || Q <- Queues, amqqueue:is_durable(Q)].
%% -------------------------------------------------------------------
%% update().
@ -382,13 +530,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 +547,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 +631,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 +650,23 @@ 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}) ->
DoUpdate = amqqueue:is_durable(Q)
andalso FilterFun(Q),
case DoUpdate of
true ->
khepri_tx:put(Path0, UpdateFun(Q));
false ->
ok
end
end)
end).
%% -------------------------------------------------------------------
%% exists().
%% -------------------------------------------------------------------
@ -460,11 +681,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 +708,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 +735,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 +763,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 +777,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 +792,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 +821,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 +846,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),
@ -597,7 +859,7 @@ set_in_mnesia(Q) ->
end).
set_in_mnesia_tx(DurableQ, Q) ->
case ?amqqueue_is_durable(Q) of
case amqqueue:is_durable(Q) of
true ->
ok = mnesia:write(?MNESIA_DURABLE_TABLE, DurableQ, write);
false ->
@ -605,6 +867,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().
%% -------------------------------------------------------------------
@ -612,24 +878,45 @@ set_in_mnesia_tx(DurableQ, Q) ->
-spec set_many([Queue]) -> ok when
Queue :: amqqueue:amqqueue().
%% @doc Writes a list of durable queue records.
%% It is responsibility of the calling function to ensure all records are durable.
%% Once transient entities are deprecated, this is a non-issue.
%%
%% It is responsibility of the calling function to ensure all records are
%% durable.
%%
%% @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} =
%% Just to be nested in forget_node_for_queue
mnesia:transaction(
fun() ->
[ok = mnesia:write(?MNESIA_DURABLE_TABLE, Q, write) || Q <- Qs],
[begin
true = amqqueue:is_durable(Q),
ok = mnesia:write(?MNESIA_DURABLE_TABLE, Q, write)
end || Q <- Qs],
ok
end),
ok.
set_many_in_khepri(Qs) ->
rabbit_khepri:transaction(
fun() ->
[begin
true = amqqueue:is_durable(Q),
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 +931,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() -> delete_transient_in_khepri(FilterFun) end
}).
delete_transient_in_mnesia(FilterFun) ->
Qs = rabbit_mnesia:execute_mnesia_transaction(
@ -685,6 +975,41 @@ partition_queues([Q0,Q1,Q2,Q3,Q4,Q5,Q6,Q7,Q8,Q9 | T]) ->
partition_queues(T) ->
[T].
delete_transient_in_khepri(FilterFun) ->
PathPattern = khepri_queues_path() ++
[?KHEPRI_WILDCARD_STAR,
#if_data_matches{
pattern = amqqueue:pattern_match_on_durable(false)}],
%% The `FilterFun' might try to determine if the queue's process is alive.
%% This can cause a `calling_self' exception if we use the `FilterFun'
%% within the function passed to `khepri:fold/5' since the Khepri server
%% process might call itself. Instead we can fetch all of the transient
%% queues with `get_many' and then filter and fold the results outside of
%% Khepri's Ra server process.
case rabbit_khepri:get_many(PathPattern) of
{ok, Qs} ->
Items = maps:fold(
fun(Path, Queue, Acc) when ?is_amqqueue(Queue) ->
case FilterFun(Queue) of
true ->
QueueName = khepri_queue_path_to_name(
Path),
case delete_in_khepri(QueueName, false) of
ok ->
Acc;
Deletions ->
[{QueueName, Deletions} | Acc]
end;
false ->
Acc
end
end, [], Qs),
{QueueNames, Deletions} = lists:unzip(Items),
{QueueNames, lists:flatten(Deletions)};
{error, _} = Error ->
Error
end.
%% -------------------------------------------------------------------
%% foreach_transient().
%% -------------------------------------------------------------------
@ -697,7 +1022,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() -> foreach_transient_in_khepri(UpdateFun) end
}).
foreach_transient_in_mnesia(UpdateFun) ->
Pattern = amqqueue:pattern_match_all(),
@ -708,6 +1036,27 @@ foreach_transient_in_mnesia(UpdateFun) ->
ok
end).
foreach_transient_in_khepri(UpdateFun) ->
PathPattern = khepri_queues_path() ++
[?KHEPRI_WILDCARD_STAR,
#if_data_matches{
pattern = amqqueue:pattern_match_on_durable(false)}],
%% The `UpdateFun' might try to determine if the queue's process is alive.
%% This can cause a `calling_self' exception if we use the `UpdateFun'
%% within the function passed to `khepri:fold/5' since the Khepri server
%% process might call itself. Instead we can fetch all of the transient
%% queues with `get_many' and then filter and fold the results outside of
%% Khepri's Ra server process.
case rabbit_khepri:get_many(PathPattern) of
{ok, Qs} ->
maps:foreach(
fun(_Path, Queue) when ?is_amqqueue(Queue) ->
UpdateFun(Queue)
end, Qs);
{error, _} = Error ->
Error
end.
%% -------------------------------------------------------------------
%% foreach_durable().
%% -------------------------------------------------------------------
@ -720,7 +1069,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 +1085,22 @@ 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() ++
[?KHEPRI_WILDCARD_STAR,
#if_data_matches{
pattern = amqqueue:pattern_match_on_durable(true)}],
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 +1113,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 +1146,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 +1181,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 +1198,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 +1249,37 @@ 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].
khepri_queue_path_to_name([?MODULE, queues, VHost, Name]) ->
rabbit_misc:r(VHost, queue, Name).

View File

@ -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.

View File

@ -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].

View File

@ -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.

View File

@ -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) ->
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(<<C:8, Rest/binary>>, RevWordAcc, RevResAcc) ->
split_topic_key(Rest, [C | RevWordAcc], RevResAcc).
%% --------------------------------------------------------------
%% 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() ->
FinalNode = follow_down_create(XName, split_topic_key(BindingKey)),
trie_add_binding(XName, FinalNode, Destination, Args),
ok
follow_up_get_path(mnesia, rabbit_topic_trie_edge, NodeId)
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).
%% --------------------------------------------------------------
%% trie_records_to_key().
%% --------------------------------------------------------------
match_in_mnesia(XName, RoutingKey, BKeys) ->
Words = split_topic_key(RoutingKey),
mnesia:async_dirty(fun trie_match/3, [XName, Words, BKeys]).
-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).
follow_up_get_path(Mod, Tab, Node) ->
follow_up_get_path(Mod, Tab, Node, []).
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(<<C:8, Rest/binary>>, RevWordAcc, RevResAcc) ->
split_topic_key(Rest, [C | RevWordAcc], RevResAcc).
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.

View File

@ -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].

View File

@ -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.

View File

@ -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].

View File

@ -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.

View File

@ -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])).

View File

@ -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) ->

View File

@ -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.

View File

@ -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.

View File

@ -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).

View File

@ -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).

View File

@ -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).

View File

@ -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).

View File

@ -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

1607
deps/rabbit/src/rabbit_khepri.erl vendored Normal file

File diff suppressed because it is too large Load Diff

View File

@ -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.

View File

@ -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,

View File

@ -29,7 +29,8 @@
-export([get_replicas/1, transfer_leadership/2, migrate_leadership_to_existing_replica/2]).
%% Deprecated feature callback.
-export([are_cmqs_used/1]).
-export([are_cmqs_permitted/0,
are_cmqs_used/1]).
%% for testing only
-export([module/1]).
@ -114,6 +115,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 +173,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 +186,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 +214,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 +296,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 +461,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 +490,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 +559,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 +579,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 +949,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(),

View File

@ -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) ->

View File

@ -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]).

View File

@ -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

View File

@ -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).
matches(Q, Policy) when ?is_amqqueue(Q) ->
match_all(NameOrQueue, Policies, Function) ->
lists:sort(fun priority_comparator/2, [P || P <- Policies, matches(NameOrQueue, P, Function)]).
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.
%%----------------------------------------------------------------------------

View File

@ -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}),

View File

@ -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.

View File

@ -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),

View File

@ -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

View File

@ -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()].

View File

@ -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}) ->

View File

@ -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

View File

@ -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,

View File

@ -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(),

View File

@ -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}].
%%----------------------------------------------------------------------------

View File

@ -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.

View File

@ -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
transient_queue_on_node_down
]},
{khepri_cluster, [], [
transient_queue_on_node_down
]}
].
@ -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,63 +759,63 @@ 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.
transient_queue_on_node_down_mnesia(Config) ->
transient_queue_on_node_down(Config) ->
Server = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
@ -759,27 +846,32 @@ transient_queue_on_node_down_mnesia(Config) ->
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([DefaultBinding, DirectBinding]),
?assertEqual([DirectBinding],
lists:sort(rabbit_ct_broker_helpers:rpc(Config, 1, rabbit_binding, list, [<<"/">>]))),
?assertMatch([],
rabbit_ct_broker_helpers:rpc(Config, 1, rabbit_amqqueue, list, [<<"/">>])),
rabbit_ct_broker_helpers:start_node(Config, Server),
Bindings1 = lists:sort([DefaultBinding, DirectBinding, DefaultAltBinding, 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, [<<"/">>]),
rabbit_ct_broker_helpers:stop_node(Config, Server),
?awaitMatch([DirectBinding],
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),
Bindings2 = lists:sort([DefaultBinding, DirectBinding]),
?awaitMatch(Bindings2,
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),
ok.

View File

@ -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).

View File

@ -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].

View File

@ -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) ->

View File

@ -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,189 @@ 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.
ct:pal("Initial Raft status: ~p", [RaftStatus]),
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
RaftStatus1 = get_raft_status(Config, Hare),
ct:pal("Latest Raft status: ~p", [RaftStatus1]),
case RaftStatus1 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 +826,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 +847,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 +863,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 +900,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 +953,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 +1018,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 +1098,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 +1166,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 +1430,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}).

View File

@ -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},

View File

@ -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]).

View File

@ -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 ->

View File

@ -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.

View File

@ -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.

View File

@ -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)]).

View File

@ -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) ->

View File

@ -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,

View File

@ -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),

View File

@ -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).

View File

@ -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() ->

View File

@ -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.

View File

@ -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,

View File

@ -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).

View File

@ -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).

View File

@ -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]),
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]),
rabbit_ct_broker_helpers:set_policy(Config2, 0,
<<"ha-policy">>, <<".*">>, <<"queues">>,
[{<<"ha-mode">>, <<"all">>}]),
Config2.
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(

View File

@ -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 = []}).

View File

@ -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.

File diff suppressed because it is too large Load Diff

View File

@ -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).

View File

@ -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
%% -------------------------------------------------------------------

View File

@ -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),

View File

@ -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() ++

View File

@ -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() ++

View File

@ -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) ->

View File

@ -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),

Some files were not shown because too many files have changed in this diff Show More