diff --git a/BUILD.khepri b/BUILD.khepri new file mode 100644 index 000000000000..d89fefdf8268 --- /dev/null +++ b/BUILD.khepri @@ -0,0 +1,15 @@ +load("@rules_erlang//:erlang_app.bzl", "erlang_app", "DEFAULT_ERLC_OPTS") + +erlang_app( + app_name = "khepri", + app_description = "ZooKeeper-inspired database library", + app_version = "main", + app_module = "khepri_app", + erlc_opts = ["-I", "external/khepri"] + DEFAULT_ERLC_OPTS, + extra_apps = [ + "sasl", + ], + deps = [ + "@ra//:erlang_app", + ], +) diff --git a/deps/rabbit/BUILD.bazel b/deps/rabbit/BUILD.bazel index 4e5b6ee4d216..c0c17e48945e 100644 --- a/deps/rabbit/BUILD.bazel +++ b/deps/rabbit/BUILD.bazel @@ -152,6 +152,7 @@ _APP_ENV = """[ DEPS = [ "//deps/amqp10_common:erlang_app", "//deps/rabbit_common:erlang_app", + "@khepri//:erlang_app", "@ra//:erlang_app", "@ranch//:erlang_app", "@stdout_formatter//:erlang_app", @@ -504,6 +505,18 @@ suites = [ ":rabbit_ha_test_producer", ], ), + rabbitmq_integration_suite( + PACKAGE, + name = "metadata_store_phase1_SUITE", + size = "small", + deps = [ + "@khepri//:erlang_app", + ], + runtime_deps = [ + "@meck//:erlang_app", + "@ra//:erlang_app", + ], + ), rabbitmq_integration_suite( PACKAGE, name = "message_size_limit_SUITE", diff --git a/deps/rabbit/Makefile b/deps/rabbit/Makefile index 334ed8b2bd29..e1ad5899b2f4 100644 --- a/deps/rabbit/Makefile +++ b/deps/rabbit/Makefile @@ -139,7 +139,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 TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers amqp_client meck proper PLT_APPS += mnesia @@ -148,6 +148,7 @@ dep_syslog = git https://github.com/schlagert/syslog 4.0.0 dep_osiris = git https://github.com/rabbitmq/osiris main dep_systemd = hex 0.6.1 dep_seshat = hex 0.3.2 +dep_khepri = git https://github.com/rabbitmq/khepri.git 15721dc37b84e14d9820b74b106cd56f763acd1c define usage_xml_to_erl $(subst __,_,$(patsubst $(DOCS_DIR)/rabbitmq%.1.xml, src/rabbit_%_usage.erl, $(subst -,_,$(1)))) diff --git a/deps/rabbit/include/internal_user.hrl b/deps/rabbit/include/internal_user.hrl new file mode 100644 index 000000000000..3bff8cfcb797 --- /dev/null +++ b/deps/rabbit/include/internal_user.hrl @@ -0,0 +1,13 @@ +%% 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. +%% + +-define(is_internal_user(V), + (?is_internal_user_v2(V) orelse + ?is_internal_user_v1(V))). + +-define(is_internal_user_v1(V), is_record(V, internal_user, 5)). +-define(is_internal_user_v2(V), is_record(V, internal_user, 6)). diff --git a/deps/rabbit/src/rabbit.erl b/deps/rabbit/src/rabbit.erl index 5f387412de00..758cbf475c6d 100644 --- a/deps/rabbit/src/rabbit.erl +++ b/deps/rabbit/src/rabbit.erl @@ -363,6 +363,10 @@ run_prelaunch_second_phase() -> ok = rabbit_ra_systems:setup(Context), + %% Khepri requires the "coordination" Ra system to be started by the + %% previous call, but will ensure it runs anyway. + ok = rabbit_khepri:setup(Context), + ?LOG_DEBUG(""), ?LOG_DEBUG("== Prelaunch DONE =="), diff --git a/deps/rabbit/src/rabbit_auth_backend_internal.erl b/deps/rabbit/src/rabbit_auth_backend_internal.erl index a49867efc39e..96bd6565d558 100644 --- a/deps/rabbit/src/rabbit_auth_backend_internal.erl +++ b/deps/rabbit/src/rabbit_auth_backend_internal.erl @@ -6,19 +6,34 @@ %% -module(rabbit_auth_backend_internal). + +-include_lib("kernel/include/logger.hrl"). + +-include_lib("khepri/include/khepri.hrl"). + -include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("rabbit_common/include/logging.hrl"). + +-include("internal_user.hrl"). -behaviour(rabbit_authn_backend). -behaviour(rabbit_authz_backend). -export([user_login_authentication/2, user_login_authorization/2, - check_vhost_access/3, check_resource_access/4, check_topic_access/4]). + check_vhost_access/3, check_resource_access/4, check_topic_access/4, + with_user/2, with_user_in_mnesia/2, with_user_in_khepri/2]). -export([add_user/3, add_user/4, add_user/5, delete_user/2, lookup_user/1, exists/1, change_password/3, clear_password/2, hash_password/2, change_password_hash/2, change_password_hash/3, set_tags/3, set_permissions/6, clear_permissions/3, + clear_vhost_permissions_in_khepri/2, + clear_permissions_in_mnesia/2, + clear_permissions_in_khepri/2, set_topic_permissions/6, clear_topic_permissions/3, clear_topic_permissions/4, + clear_vhost_topic_permissions_in_khepri/2, + clear_topic_permissions_in_mnesia/3, + clear_topic_permissions_in_khepri/3, clear_topic_permissions_in_khepri_tx_fun/3, add_user_sans_validation/3, put_user/2, put_user/3, update_user/5, update_user_with_hash/5, @@ -30,18 +45,64 @@ -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, list_vhost_permissions/1, list_vhost_permissions/3, + list_vhost_permissions_in_khepri_tx_fun/1, list_user_vhost_permissions/2, - list_user_topic_permissions/1, list_vhost_topic_permissions/1, list_user_vhost_topic_permissions/2]). + list_user_topic_permissions/1, list_vhost_topic_permissions/1, list_user_vhost_topic_permissions/2, + list_vhost_topic_permissions_in_khepri_tx_fun/1]). -export([state_can_expire/0]). +-export([clear_data_in_khepri/0, + mnesia_write_to_khepri/1, + mnesia_delete_to_khepri/1]). +-export([khepri_users_path/0, + khepri_user_path/1]). %% for testing -export([hashing_module_for_user/1, expand_topic_permission/2]). +-ifdef(TEST). +-export([lookup_user_in_mnesia/1, + lookup_user_in_khepri/1, + add_user_sans_validation_in_mnesia/2, + add_user_sans_validation_in_khepri/2, + update_user_in_mnesia/2, + update_user_in_khepri/2, + delete_user_in_mnesia/1, + delete_user_in_khepri/1, + all_users_in_mnesia/0, + all_users_in_khepri/0, + + check_vhost_access_in_mnesia/2, + check_vhost_access_in_khepri/2, + check_resource_access_in_mnesia/4, + check_resource_access_in_khepri/4, + set_permissions_in_mnesia/3, + set_permissions_in_khepri/3, + list_permissions_in_mnesia/1, + list_permissions_in_khepri/1, + match_user_vhost/2, + match_path_in_khepri/1, + + check_topic_access_in_mnesia/5, + check_topic_access_in_khepri/5, + set_topic_permissions_in_mnesia/4, + set_topic_permissions_in_khepri/4, + list_topic_permissions_in_mnesia/1, + list_topic_permissions_in_khepri/1, + match_user_vhost_topic_permission/3, + clear_topic_permissions_in_mnesia/2, + clear_topic_permissions_in_khepri/2, + + extract_user_permission_params/2, + extract_topic_permission_params/2]). +-endif. + -import(rabbit_data_coercion, [to_atom/1, to_list/1, to_binary/1]). %%---------------------------------------------------------------------------- @@ -64,6 +125,30 @@ hashing_module_for_user(User) -> "To use TLS/x509 certificate-based authentication, see the rabbitmq_auth_mechanism_ssl plugin and configure the client to use the EXTERNAL authentication mechanism. " "Alternatively change the password for the user to be non-blank."). +with_user(Username, Thunk) -> + fun() -> + rabbit_khepri:try_mnesia_or_khepri( + with_user_in_mnesia(Username, Thunk), + with_user_in_khepri(Username, Thunk)) + end. + +with_user_in_mnesia(Username, Thunk) -> + fun () -> + case mnesia:read({rabbit_user, Username}) of + [_U] -> Thunk(); + [] -> mnesia:abort({no_such_user, Username}) + end + end. + +with_user_in_khepri(Username, Thunk) -> + fun() -> + Path = khepri_user_path(Username), + case khepri_tx:exists(Path) of + true -> Thunk(); + false -> khepri_tx:abort({no_such_user, Username}) + end + end. + %% For cases when we do not have a set of credentials, %% namely when x509 (TLS) certificates are used. This should only be %% possible when the EXTERNAL authentication mechanism is used, see @@ -119,6 +204,11 @@ internal_check_user_login(Username, Fun) -> end. check_vhost_access(#auth_user{username = Username}, VHostPath, _AuthzData) -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> check_vhost_access_in_mnesia(Username, VHostPath) end, + fun() -> check_vhost_access_in_khepri(Username, VHostPath) end). + +check_vhost_access_in_mnesia(Username, VHostPath) -> case mnesia:dirty_read({rabbit_user_permission, #user_vhost{username = Username, virtual_host = VHostPath}}) of @@ -126,31 +216,67 @@ check_vhost_access(#auth_user{username = Username}, VHostPath, _AuthzData) -> [_R] -> true end. +check_vhost_access_in_khepri(Username, VHostPath) -> + Path = khepri_user_permission_path(Username, VHostPath), + rabbit_khepri:exists(Path). + check_resource_access(#auth_user{username = Username}, #resource{virtual_host = VHostPath, name = Name}, Permission, _AuthContext) -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> + check_resource_access_in_mnesia( + Username, VHostPath, Name, Permission) + end, + fun() -> + check_resource_access_in_khepri( + Username, VHostPath, Name, Permission) + end). + +check_resource_access_in_mnesia(Username, VHostPath, Name, Permission) -> case mnesia:dirty_read({rabbit_user_permission, #user_vhost{username = Username, virtual_host = VHostPath}}) of [] -> false; [#user_permission{permission = P}] -> - PermRegexp = case element(permission_index(Permission), P) of - %% <<"^$">> breaks Emacs' erlang mode - <<"">> -> <<$^, $$>>; - RE -> RE - end, - case re:run(Name, PermRegexp, [{capture, none}]) of - match -> true; - nomatch -> false - end + do_check_resource_access(Name, Permission, P) end. +check_resource_access_in_khepri(Username, VHostPath, Name, Permission) -> + Path = khepri_user_permission_path(Username, VHostPath), + case rabbit_khepri:get_data(Path) of + {ok, #user_permission{permission = P}} -> + do_check_resource_access(Name, Permission, P); + _ -> + false + end. + +do_check_resource_access(Name, Permission, P) -> + PermRegexp = case element(permission_index(Permission), P) of + %% <<"^$">> breaks Emacs' erlang mode + <<"">> -> <<$^, $$>>; + RE -> RE + end, + re:run(Name, PermRegexp, [{capture, none}]) =:= match. + check_topic_access(#auth_user{username = Username}, #resource{virtual_host = VHostPath, name = Name, kind = topic}, Permission, Context) -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> + check_topic_access_in_mnesia( + Username, VHostPath, Name, Permission, Context) + end, + fun() -> + check_topic_access_in_khepri( + Username, VHostPath, Name, Permission, Context) + end). + +check_topic_access_in_mnesia( + Username, VHostPath, Name, Permission, Context) -> case mnesia:dirty_read({rabbit_topic_permission, #topic_permission_key{user_vhost = #user_vhost{username = Username, virtual_host = VHostPath}, @@ -159,21 +285,32 @@ check_topic_access(#auth_user{username = Username}, [] -> true; [#topic_permission{permission = P}] -> - PermRegexp = case element(permission_index(Permission), P) of - %% <<"^$">> breaks Emacs' erlang mode - <<"">> -> <<$^, $$>>; - RE -> RE - end, - PermRegexpExpanded = expand_topic_permission( - PermRegexp, - maps:get(variable_map, Context, undefined) - ), - case re:run(maps:get(routing_key, Context), PermRegexpExpanded, [{capture, none}]) of - match -> true; - nomatch -> false - end + do_check_topic_access(Permission, Context, P) + end. + +check_topic_access_in_khepri( + Username, VHostPath, Name, Permission, Context) -> + Path = khepri_topic_permission_path(Username, VHostPath, Name), + case rabbit_khepri:get_data(Path) of + {ok, #topic_permission{permission = P}} -> + do_check_topic_access(Permission, Context, P); + _ -> + true end. +do_check_topic_access(Permission, Context, P) -> + PermRegexp = case element(permission_index(Permission), P) of + %% <<"^$">> breaks Emacs' erlang mode + <<"">> -> <<$^, $$>>; + RE -> RE + end, + PermRegexpExpanded = expand_topic_permission( + PermRegexp, + maps:get(variable_map, Context, undefined) + ), + re:run(maps:get(routing_key, Context), PermRegexpExpanded, [{capture, none}]) + =:= match. + expand_topic_permission(Permission, ToExpand) when is_map(ToExpand) -> Opening = <<"{">>, Closing = <<"}">>, @@ -262,15 +399,9 @@ add_user_sans_validation(Username, PasswordHash, HashingAlgorithm, Tags, Limits, add_user_sans_validation_in(Username, User, ConvertedTags, Limits, ActingUser) -> try - R = rabbit_misc:execute_mnesia_transaction( - fun () -> - case mnesia:wread({rabbit_user, Username}) of - [] -> - ok = mnesia:write(rabbit_user, User, write); - _ -> - mnesia:abort({user_already_exists, Username}) - end - end), + R = rabbit_khepri:try_mnesia_or_khepri( + fun() -> add_user_sans_validation_in_mnesia(Username, User) end, + fun() -> add_user_sans_validation_in_khepri(Username, User) end), rabbit_log:info("Created user '~s'", [Username]), rabbit_event:notify(user_created, [{name, Username}, {user_who_performed_action, ActingUser}]), @@ -292,30 +423,37 @@ add_user_sans_validation_in(Username, User, ConvertedTags, Limits, ActingUser) - erlang:raise(Class, Error, Stacktrace) end . +add_user_sans_validation_in_mnesia(Username, User) -> + rabbit_misc:execute_mnesia_transaction( + fun () -> + case mnesia:wread({rabbit_user, Username}) of + [] -> + ok = mnesia:write(rabbit_user, User, write); + _ -> + mnesia:abort({user_already_exists, Username}) + end + end), + ok. + +add_user_sans_validation_in_khepri(Username, User) -> + Path = khepri_user_path(Username), + case rabbit_khepri:create(Path, User) of + {ok, _} -> + ok; + {error, {mismatching_node, _}} -> + throw({error, {user_already_exists, Username}}); + {error, _} = Error -> + throw(Error) + end. + -spec delete_user(rabbit_types:username(), rabbit_types:username()) -> 'ok'. delete_user(Username, ActingUser) -> rabbit_log:debug("Asked to delete user '~s'", [Username]), try - R = rabbit_misc:execute_mnesia_transaction( - rabbit_misc:with_user( - Username, - fun () -> - ok = mnesia:delete({rabbit_user, Username}), - [ok = mnesia:delete_object( - rabbit_user_permission, R, write) || - R <- mnesia:match_object( - rabbit_user_permission, - #user_permission{user_vhost = #user_vhost{ - username = Username, - virtual_host = '_'}, - permission = '_'}, - write)], - UserTopicPermissionsQuery = match_user_vhost_topic_permission(Username, '_'), - UserTopicPermissions = UserTopicPermissionsQuery(), - [ok = mnesia:delete_object(rabbit_topic_permission, R, write) || R <- UserTopicPermissions], - ok - end)), + R = rabbit_khepri:try_mnesia_or_khepri( + fun() -> delete_user_in_mnesia(Username) end, + fun() -> delete_user_in_khepri(Username) end), rabbit_log:info("Deleted user '~s'", [Username]), rabbit_event:notify(user_deleted, [{name, Username}, @@ -330,14 +468,59 @@ delete_user(Username, ActingUser) -> erlang:raise(Class, Error, Stacktrace) end . +delete_user_in_mnesia(Username) -> + rabbit_misc:execute_mnesia_transaction( + with_user_in_mnesia( + Username, + fun () -> + ok = mnesia:delete({rabbit_user, Username}), + [ok = mnesia:delete_object( + rabbit_user_permission, R, write) || + R <- mnesia:match_object( + rabbit_user_permission, + #user_permission{user_vhost = #user_vhost{ + username = Username, + virtual_host = '_'}, + permission = '_'}, + write)], + UserTopicPermissionsQuery = match_user_vhost_topic_permission(Username, '_'), + UserTopicPermissions = UserTopicPermissionsQuery(), + [ok = mnesia:delete_object(rabbit_topic_permission, R, write) || R <- UserTopicPermissions], + ok + end)). + +delete_user_in_khepri(Username) -> + rabbit_khepri:transaction( + with_user_in_khepri( + Username, + fun() -> + Path = khepri_user_path(Username), + case khepri_tx:delete(Path) of + {ok, _} -> ok; + Error -> khepri_tx:abort(Error) + end + end)). + -spec lookup_user (rabbit_types:username()) -> rabbit_types:ok(internal_user:internal_user()) | rabbit_types:error('not_found'). lookup_user(Username) -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> lookup_user_in_mnesia(Username) end, + fun() -> lookup_user_in_khepri(Username) end). + +lookup_user_in_mnesia(Username) -> rabbit_misc:dirty_read({rabbit_user, Username}). +lookup_user_in_khepri(Username) -> + Path = khepri_user_path(Username), + case rabbit_khepri:get_data(Path) of + {ok, User} -> {ok, User}; + _ -> {error, not_found} + end. + -spec exists(rabbit_types:username()) -> boolean(). exists(Username) -> @@ -498,21 +681,24 @@ set_permissions(Username, VirtualHost, ConfigurePerm, WritePerm, ReadPerm, Actin throw({error, {invalid_regexp, Regexp, Reason}}) end end, [ConfigurePerm, WritePerm, ReadPerm]), + UserPermission = #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VirtualHost}, + permission = #permission{ + configure = ConfigurePerm, + write = WritePerm, + read = ReadPerm}}, try - R = rabbit_misc:execute_mnesia_transaction( - rabbit_vhost:with_user_and_vhost( - Username, VirtualHost, - fun () -> ok = mnesia:write( - rabbit_user_permission, - #user_permission{user_vhost = #user_vhost{ - username = Username, - virtual_host = VirtualHost}, - permission = #permission{ - configure = ConfigurePerm, - write = WritePerm, - read = ReadPerm}}, - write) - end)), + R = rabbit_khepri:try_mnesia_or_khepri( + fun() -> + set_permissions_in_mnesia( + Username, VirtualHost, UserPermission) + end, + fun() -> + set_permissions_in_khepri( + Username, VirtualHost, UserPermission) + end), rabbit_log:info("Successfully set permissions for " "'~s' in virtual host '~s' to '~s', '~s', '~s'", [Username, VirtualHost, ConfigurePerm, WritePerm, ReadPerm]), @@ -538,6 +724,40 @@ set_permissions(Username, VirtualHost, ConfigurePerm, WritePerm, ReadPerm, Actin erlang:raise(Class, Error, Stacktrace) end. +set_permissions_in_mnesia(Username, VirtualHost, UserPermission) -> + rabbit_misc:execute_mnesia_transaction( + rabbit_vhost:with_user_and_vhost_in_mnesia( + Username, VirtualHost, + fun () -> ok = mnesia:write( + rabbit_user_permission, + UserPermission, + write) + end)). + +set_permissions_in_khepri(Username, VirtualHost, UserPermission) -> + rabbit_khepri:transaction( + rabbit_vhost:with_user_and_vhost_in_khepri( + Username, VirtualHost, + fun() -> + Path = khepri_user_permission_path( + #if_all{conditions = + [Username, + #if_node_exists{exists = true}]}, + VirtualHost), + %% TODO: Add a keep_while for the intermediate + %% 'user_permissions' node so it is removed when its last + %% children is removed. + Extra = #{keep_while => + #{rabbit_vhost:khepri_vhost_path(VirtualHost) => + #if_node_exists{exists = true}}}, + Ret = khepri_tx:put( + Path, UserPermission, Extra), + case Ret of + {ok, _} -> ok; + Error -> khepri_tx:abort(Error) + end + end)). + -spec clear_permissions (rabbit_types:username(), rabbit_types:vhost(), rabbit_types:username()) -> 'ok'. @@ -545,19 +765,10 @@ clear_permissions(Username, VirtualHost, ActingUser) -> rabbit_log:debug("Asked to clear permissions for '~s' in virtual host '~s'", [Username, VirtualHost]), try - R = rabbit_misc:execute_mnesia_transaction( - rabbit_vhost:with_user_and_vhost( - Username, VirtualHost, - fun () -> - ok = mnesia:delete({rabbit_user_permission, - #user_vhost{username = Username, - virtual_host = VirtualHost}}) - end)), - rabbit_log:info("Successfully cleared permissions for '~s' in virtual host '~s'", - [Username, VirtualHost]), - rabbit_event:notify(permission_deleted, [{user, Username}, - {vhost, VirtualHost}, - {user_who_performed_action, ActingUser}]), + R = rabbit_khepri:try_mnesia_or_khepri( + fun() -> clear_permissions_in_mnesia(Username, VirtualHost) end, + fun() -> clear_permissions_in_khepri(Username, VirtualHost) end), + post_clear_permissions(Username, VirtualHost, ActingUser), R catch throw:{error, {no_such_vhost, _}} = Error -> @@ -574,16 +785,86 @@ clear_permissions(Username, VirtualHost, ActingUser) -> erlang:raise(Class, Error, Stacktrace) end. +clear_vhost_permissions_in_khepri(VirtualHost, ActingUser) -> + rabbit_log:debug("Asked to clear permissions for everyone in virtual host '~s'", + [VirtualHost]), + Path = khepri_user_permission_path(?STAR, VirtualHost), + case rabbit_khepri:delete(Path) of + {ok, Result} -> + _ = maps:fold( + fun(Path1, _NodeProps, Acc) -> + Username = khepri_path_to_user(Path1), + post_clear_permissions( + Username, VirtualHost, ActingUser), + Acc + end, ok, Result), + ok; + Error -> + Error + end. + +post_clear_permissions(Username, VirtualHost, ActingUser) -> + rabbit_log:info("Successfully cleared permissions for '~s' in virtual host '~s'", + [Username, VirtualHost]), + rabbit_event:notify(permission_deleted, [{user, Username}, + {vhost, VirtualHost}, + {user_who_performed_action, ActingUser}]). + +clear_permissions_in_mnesia(Username, VirtualHost) -> + rabbit_misc:execute_mnesia_transaction( + rabbit_vhost:with_user_and_vhost_in_mnesia( + Username, VirtualHost, + fun () -> + ok = mnesia:delete({rabbit_user_permission, + #user_vhost{username = Username, + virtual_host = VirtualHost}}) + end)). + +clear_permissions_in_khepri(Username, VirtualHost) -> + %% FIXME: We now check the vhost and user existence in an atomic manner. + %% + %% In the end, do we really need to have a more complex transaction + %% mechanism when we just want to delete something? The code and the + %% execution would be much simpler if we just deleted the permission and + %% be done with it. + rabbit_khepri:transaction( + rabbit_vhost:with_user_and_vhost_in_khepri( + Username, VirtualHost, + fun () -> + Path = khepri_user_permission_path(Username, VirtualHost), + case khepri_tx:delete(Path) of + {ok, _} -> ok; + Error -> khepri_tx:abort(Error) + end + end)). update_user(Username, Fun) -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> update_user_in_mnesia(Username, Fun) end, + fun() -> update_user_in_khepri(Username, Fun) end). + +update_user_in_mnesia(Username, Fun) -> rabbit_misc:execute_mnesia_transaction( - rabbit_misc:with_user( + with_user_in_mnesia( Username, fun () -> {ok, User} = lookup_user(Username), ok = mnesia:write(rabbit_user, Fun(User), write) end)). +update_user_in_khepri(Username, Fun) -> + rabbit_khepri:transaction( + with_user_in_khepri( + Username, + fun () -> + Path = khepri_user_path(Username), + {ok, #{Path := #{data := User}}} = khepri_tx:get(Path), + case khepri_tx:put(Path, Fun(User)) of + {ok, #{Path := #{data := User}}} -> ok; + Error -> khepri_tx:abort(Error) + end + end)). + set_topic_permissions(Username, VirtualHost, Exchange, WritePerm, ReadPerm, ActingUser) -> rabbit_log:debug("Asked to set topic permissions on exchange '~s' for " "user '~s' in virtual host '~s' to '~s', '~s'", @@ -601,26 +882,28 @@ set_topic_permissions(Username, VirtualHost, Exchange, WritePerm, ReadPerm, Acti throw({error, {invalid_regexp, RegexpBin, Reason}}) end end, [WritePerm, ReadPerm]), + TopicPermission = #topic_permission{ + topic_permission_key = #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VirtualHost}, + exchange = Exchange + }, + permission = #permission{ + write = WritePermRegex, + read = ReadPermRegex + } + }, try - R = rabbit_misc:execute_mnesia_transaction( - rabbit_vhost:with_user_and_vhost( - Username, VirtualHost, - fun () -> ok = mnesia:write( - rabbit_topic_permission, - #topic_permission{ - topic_permission_key = #topic_permission_key{ - user_vhost = #user_vhost{ - username = Username, - virtual_host = VirtualHost}, - exchange = Exchange - }, - permission = #permission{ - write = WritePermRegex, - read = ReadPermRegex - } - }, - write) - end)), + R = rabbit_khepri:try_mnesia_or_khepri( + fun() -> + set_topic_permissions_in_mnesia( + Username, VirtualHost, Exchange, TopicPermission) + end, + fun() -> + set_topic_permissions_in_khepri( + Username, VirtualHost, Exchange, TopicPermission) + end), rabbit_log:info("Successfully set topic permissions on exchange '~s' for " "'~s' in virtual host '~s' to '~s', '~s'", [Exchange, Username, VirtualHost, WritePerm, ReadPerm]), @@ -647,25 +930,55 @@ set_topic_permissions(Username, VirtualHost, Exchange, WritePerm, ReadPerm, Acti erlang:raise(Class, Error, Stacktrace) end . +set_topic_permissions_in_mnesia( + Username, VirtualHost, _Exchange, TopicPermission) -> + rabbit_misc:execute_mnesia_transaction( + rabbit_vhost:with_user_and_vhost_in_mnesia( + Username, VirtualHost, + fun () -> ok = mnesia:write( + rabbit_topic_permission, + TopicPermission, + write) + end)). + +set_topic_permissions_in_khepri( + Username, VirtualHost, Exchange, TopicPermission) -> + %% TODO: Add a keep_while for the intermediate 'topic_permissions' node so + %% it is removed when its last children is removed. + rabbit_khepri:transaction( + rabbit_vhost:with_user_and_vhost_in_khepri( + Username, VirtualHost, + fun () -> + Path = khepri_topic_permission_path( + #if_all{conditions = + [Username, + #if_node_exists{exists = true}]}, + VirtualHost, + Exchange), + Extra = #{keep_while => + #{rabbit_vhost:khepri_vhost_path(VirtualHost) => + #if_node_exists{exists = true}}}, + Ret = khepri_tx:put(Path, TopicPermission, Extra), + case Ret of + {ok, _} -> ok; + Error -> khepri_tx:abort(Error) + end + end)). + clear_topic_permissions(Username, VirtualHost, ActingUser) -> rabbit_log:debug("Asked to clear topic permissions for '~s' in virtual host '~s'", [Username, VirtualHost]), try - R = rabbit_misc:execute_mnesia_transaction( - rabbit_vhost:with_user_and_vhost( - Username, VirtualHost, - fun () -> - ListFunction = match_user_vhost_topic_permission(Username, VirtualHost), - List = ListFunction(), - lists:foreach(fun(X) -> - ok = mnesia:delete_object(rabbit_topic_permission, X, write) - end, List) - end)), - rabbit_log:info("Successfully cleared topic permissions for '~s' in virtual host '~s'", - [Username, VirtualHost]), - rabbit_event:notify(topic_permission_deleted, [{user, Username}, - {vhost, VirtualHost}, - {user_who_performed_action, ActingUser}]), + R = rabbit_khepri:try_mnesia_or_khepri( + fun() -> + clear_topic_permissions_in_mnesia( + Username, VirtualHost) + end, + fun() -> + clear_topic_permissions_in_khepri( + Username, VirtualHost) + end), + post_clear_topic_permissions(Username, VirtualHost, ActingUser), R catch throw:{error, {no_such_vhost, _}} = Error -> @@ -682,22 +995,59 @@ clear_topic_permissions(Username, VirtualHost, ActingUser) -> erlang:raise(Class, Error, Stacktrace) end. +clear_vhost_topic_permissions_in_khepri(VirtualHost, ActingUser) -> + rabbit_log:debug("Asked to clear topic permissions for everyone in virtual host '~s'", + [VirtualHost]), + Path = khepri_topic_permission_path(?STAR, VirtualHost, ?STAR), + case rabbit_khepri:delete(Path) of + {ok, Result} -> + _ = maps:fold( + fun(Path1, _NodeProps, Acc) -> + Username = khepri_path_to_user(Path1), + post_clear_topic_permissions( + Username, VirtualHost, ActingUser), + Acc + end, ok, Result), + ok; + Error -> + Error + end. + +post_clear_topic_permissions(Username, VirtualHost, ActingUser) -> + rabbit_log:info("Successfully cleared topic permissions for '~s' in virtual host '~s'", + [Username, VirtualHost]), + rabbit_event:notify(topic_permission_deleted, [{user, Username}, + {vhost, VirtualHost}, + {user_who_performed_action, ActingUser}]). + +clear_topic_permissions_in_mnesia(Username, VirtualHost) -> + rabbit_misc:execute_mnesia_transaction( + rabbit_vhost:with_user_and_vhost_in_mnesia( + Username, VirtualHost, + fun () -> + ListFunction = match_user_vhost_topic_permission(Username, VirtualHost), + List = ListFunction(), + lists:foreach(fun(X) -> + ok = mnesia:delete_object(rabbit_topic_permission, X, write) + end, List) + end)). + +clear_topic_permissions_in_khepri(Username, VirtualHost) -> + clear_topic_permissions_in_khepri(Username, VirtualHost, ?STAR). + clear_topic_permissions(Username, VirtualHost, Exchange, ActingUser) -> rabbit_log:debug("Asked to clear topic permissions on exchange '~s' for '~s' in virtual host '~s'", [Exchange, Username, VirtualHost]), try - R = rabbit_misc:execute_mnesia_transaction( - rabbit_vhost:with_user_and_vhost( - Username, VirtualHost, - fun () -> - ok = mnesia:delete(rabbit_topic_permission, - #topic_permission_key{ - user_vhost = #user_vhost{ - username = Username, - virtual_host = VirtualHost}, - exchange = Exchange - }, write) - end)), + R = rabbit_khepri:try_mnesia_or_khepri( + fun() -> + clear_topic_permissions_in_mnesia( + Username, VirtualHost, Exchange) + end, + fun() -> + clear_topic_permissions_in_khepri( + Username, VirtualHost, Exchange) + end), rabbit_log:info("Successfully cleared topic permissions on exchange '~s' for '~s' in virtual host '~s'", [Exchange, Username, VirtualHost]), rabbit_event:notify(permission_deleted, [{user, Username}, @@ -719,6 +1069,42 @@ clear_topic_permissions(Username, VirtualHost, Exchange, ActingUser) -> erlang:raise(Class, Error, Stacktrace) end. +clear_topic_permissions_in_mnesia(Username, VirtualHost, Exchange) -> + rabbit_misc:execute_mnesia_transaction( + rabbit_vhost:with_user_and_vhost_in_mnesia( + Username, VirtualHost, + fun () -> + ok = mnesia:delete(rabbit_topic_permission, + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VirtualHost}, + exchange = Exchange + }, write) + end)). + +clear_topic_permissions_in_khepri(Username, VirtualHost, Exchange) -> + %% FIXME: We now check the vhost and user existence in an atomic manner. + %% + %% In the end, do we really need to have a more complex transaction + %% mechanism when we just want to delete something? The code and the + %% execution would be much simpler if we just deleted the permission and + %% be done with it. + rabbit_khepri:transaction( + clear_topic_permissions_in_khepri_tx_fun(Username, VirtualHost, Exchange)). + +clear_topic_permissions_in_khepri_tx_fun(Username, VirtualHost, Exchange) -> + rabbit_vhost:with_user_and_vhost_in_khepri( + Username, VirtualHost, + fun () -> + Path = khepri_topic_permission_path( + Username, VirtualHost, Exchange), + case khepri_tx:delete(Path) of + {ok, _} -> ok; + Error -> khepri_tx:abort(Error) + end + end). + put_user(User, ActingUser) -> put_user(User, undefined, ActingUser). put_user(User, Version, ActingUser) -> @@ -936,7 +1322,20 @@ user_topic_perms_info_keys() -> [vhost, exchange, write, read]. vhost_topic_perms_info_keys() -> [user, exchange, write, read]. user_vhost_topic_perms_info_keys() -> [exchange, write, read]. -all_users() -> mnesia:dirty_match_object(rabbit_user, internal_user:pattern_match_all()). +all_users() -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> all_users_in_mnesia() end, + fun() -> all_users_in_khepri() end). + +all_users_in_mnesia() -> + mnesia:dirty_match_object(rabbit_user, internal_user:pattern_match_all()). + +all_users_in_khepri() -> + Path = khepri_users_path(), + case rabbit_khepri:list_child_data(Path) of + {ok, Users} -> maps:values(Users); + _ -> [] + end. -spec list_users() -> [rabbit_types:infos()]. @@ -955,16 +1354,33 @@ list_users(Ref, AggregatorPid) -> -spec list_permissions() -> [rabbit_types:infos()]. list_permissions() -> - list_permissions(perms_info_keys(), match_user_vhost('_', '_')). - -list_permissions(Keys, QueryThunk) -> - [extract_user_permission_params(Keys, U) || - U <- rabbit_misc:execute_mnesia_transaction(QueryThunk)]. + MnesiaThunk = match_user_vhost('_', '_'), + KhepriThunk = match_path_in_khepri( + khepri_user_permission_path(?STAR, ?STAR)), + list_permissions(perms_info_keys(), MnesiaThunk, KhepriThunk). + +list_permissions(Keys, MnesiaThunk, KhepriThunk) -> + UserPermissions = rabbit_khepri:try_mnesia_or_khepri( + fun() -> list_permissions_in_mnesia(MnesiaThunk) end, + fun() -> list_permissions_in_khepri(KhepriThunk) end), + [extract_user_permission_params(Keys, U) || U <- UserPermissions]. + +list_permissions_in_mnesia(QueryThunk) when is_function(QueryThunk) -> + rabbit_misc:execute_mnesia_transaction(QueryThunk). + +list_permissions_in_khepri(QueryThunk) when is_function(QueryThunk) -> + case rabbit_khepri:transaction(QueryThunk, ro) of + {ok, UserPermissions} -> maps:values(UserPermissions); + _ -> [] + end. -list_permissions(Keys, QueryThunk, Ref, AggregatorPid) -> +list_permissions(Keys, MnesiaThunk, KhepriThunk, Ref, AggregatorPid) -> + Users = rabbit_khepri:try_mnesia_or_khepri( + fun() -> list_permissions_in_mnesia(MnesiaThunk) end, + fun() -> list_permissions_in_khepri(KhepriThunk) end), rabbit_control_misc:emitting_map( AggregatorPid, Ref, fun(U) -> extract_user_permission_params(Keys, U) end, - rabbit_misc:execute_mnesia_transaction(QueryThunk)). + Users). filter_props(Keys, Props) -> [T || T = {K, _} <- Props, lists:member(K, Keys)]. @@ -972,44 +1388,77 @@ filter_props(Keys, Props) -> [T || T = {K, _} <- Props, lists:member(K, Keys)]. (rabbit_types:username()) -> [rabbit_types:infos()]. list_user_permissions(Username) -> - list_permissions( - user_perms_info_keys(), - rabbit_misc:with_user(Username, match_user_vhost(Username, '_'))). + MnesiaThunk = with_user_in_mnesia( + Username, match_user_vhost(Username, '_')), + KhepriThunk = with_user_in_khepri( + Username, + match_path_in_khepri( + khepri_user_permission_path(Username, ?STAR))), + list_permissions(user_perms_info_keys(), MnesiaThunk, KhepriThunk). -spec list_user_permissions (rabbit_types:username(), reference(), pid()) -> 'ok'. list_user_permissions(Username, Ref, AggregatorPid) -> + MnesiaThunk = with_user_in_mnesia( + Username, match_user_vhost(Username, '_')), + KhepriThunk = with_user_in_khepri( + Username, + match_path_in_khepri( + khepri_user_permission_path(Username, ?STAR))), list_permissions( - user_perms_info_keys(), - rabbit_misc:with_user(Username, match_user_vhost(Username, '_')), - Ref, AggregatorPid). + user_perms_info_keys(), MnesiaThunk, KhepriThunk, Ref, AggregatorPid). -spec list_vhost_permissions (rabbit_types:vhost()) -> [rabbit_types:infos()]. list_vhost_permissions(VHostPath) -> - list_permissions( - vhost_perms_info_keys(), - rabbit_vhost:with(VHostPath, match_user_vhost('_', VHostPath))). + MnesiaThunk = rabbit_vhost:with_in_mnesia( + VHostPath, match_user_vhost('_', VHostPath)), + KhepriThunk = rabbit_vhost:with_in_khepri( + VHostPath, + match_path_in_khepri( + khepri_user_permission_path(?STAR, VHostPath))), + list_permissions(vhost_perms_info_keys(), MnesiaThunk, KhepriThunk). + +list_vhost_permissions_in_khepri_tx_fun(VHostPath) -> + Fun = rabbit_vhost:with_in_khepri( + VHostPath, + match_path_in_khepri( + khepri_user_permission_path(?STAR, VHostPath))), + case Fun() of + {ok, UserPermissions} -> + [extract_user_permission_params(vhost_perms_info_keys(), U) + || U <- maps:values(UserPermissions)]; + _ -> + [] + end. -spec list_vhost_permissions (rabbit_types:vhost(), reference(), pid()) -> 'ok'. list_vhost_permissions(VHostPath, Ref, AggregatorPid) -> + MnesiaThunk = rabbit_vhost:with_in_mnesia( + VHostPath, match_user_vhost('_', VHostPath)), + KhepriThunk = rabbit_vhost:with_in_khepri( + VHostPath, + match_path_in_khepri( + khepri_user_permission_path(?STAR, VHostPath))), list_permissions( - vhost_perms_info_keys(), - rabbit_vhost:with(VHostPath, match_user_vhost('_', VHostPath)), - Ref, AggregatorPid). + vhost_perms_info_keys(), MnesiaThunk, KhepriThunk, Ref, AggregatorPid). -spec list_user_vhost_permissions (rabbit_types:username(), rabbit_types:vhost()) -> [rabbit_types:infos()]. list_user_vhost_permissions(Username, VHostPath) -> - list_permissions( - user_vhost_perms_info_keys(), - rabbit_vhost:with_user_and_vhost( - Username, VHostPath, match_user_vhost(Username, VHostPath))). + MnesiaThunk = rabbit_vhost:with_user_and_vhost_in_mnesia( + Username, VHostPath, + match_user_vhost(Username, VHostPath)), + KhepriThunk = rabbit_vhost:with_user_and_vhost_in_khepri( + Username, VHostPath, + match_path_in_khepri( + khepri_user_permission_path(Username, VHostPath))), + list_permissions(user_vhost_perms_info_keys(), MnesiaThunk, KhepriThunk). extract_user_permission_params(Keys, #user_permission{ user_vhost = @@ -1039,24 +1488,77 @@ match_user_vhost(Username, VHostPath) -> read) end. +match_path_in_khepri(Path) -> + fun() -> rabbit_khepri:tx_match_and_get_data(Path) end. + list_topic_permissions() -> - list_topic_permissions(topic_perms_info_keys(), match_user_vhost_topic_permission('_', '_')). + QueryThunk = match_user_vhost_topic_permission('_', '_'), + Path = match_path_in_khepri(khepri_topic_permission_path(?STAR, ?STAR, ?STAR)), + list_topic_permissions(topic_perms_info_keys(), QueryThunk, Path). list_user_topic_permissions(Username) -> - list_topic_permissions(user_topic_perms_info_keys(), - rabbit_misc:with_user(Username, match_user_vhost_topic_permission(Username, '_'))). + MnesiaThunk = with_user_in_mnesia( + Username, + match_user_vhost_topic_permission(Username, '_')), + KhepriThunk = with_user_in_khepri( + Username, + match_path_in_khepri( + khepri_topic_permission_path(Username, ?STAR, ?STAR))), + list_topic_permissions( + user_topic_perms_info_keys(), MnesiaThunk, KhepriThunk). list_vhost_topic_permissions(VHost) -> - list_topic_permissions(vhost_topic_perms_info_keys(), - rabbit_vhost:with(VHost, match_user_vhost_topic_permission('_', VHost))). + MnesiaThunk = rabbit_vhost:with_in_mnesia( + VHost, match_user_vhost_topic_permission('_', VHost)), + KhepriThunk = rabbit_vhost:with_in_khepri( + VHost, + match_path_in_khepri( + khepri_topic_permission_path(?STAR, VHost, ?STAR))), + list_topic_permissions( + vhost_topic_perms_info_keys(), MnesiaThunk, KhepriThunk). + +list_vhost_topic_permissions_in_khepri_tx_fun(VHost) -> + Fun = rabbit_vhost:with_in_khepri( + VHost, + match_path_in_khepri( + khepri_topic_permission_path(?STAR, VHost, ?STAR))), + case Fun() of + {ok, TopicPermissions} -> + [extract_topic_permission_params(vhost_topic_perms_info_keys(), U) + || U <- maps:values(TopicPermissions)]; + _ -> + [] + end. list_user_vhost_topic_permissions(Username, VHost) -> - list_topic_permissions(user_vhost_topic_perms_info_keys(), - rabbit_vhost:with_user_and_vhost(Username, VHost, match_user_vhost_topic_permission(Username, VHost))). - -list_topic_permissions(Keys, QueryThunk) -> - [extract_topic_permission_params(Keys, U) || - U <- rabbit_misc:execute_mnesia_transaction(QueryThunk)]. + MnesiaThunk = rabbit_vhost:with_user_and_vhost_in_mnesia( + Username, VHost, + match_user_vhost_topic_permission(Username, VHost)), + KhepriThunk = rabbit_vhost:with_user_and_vhost_in_khepri( + Username, VHost, + match_path_in_khepri( + khepri_topic_permission_path(Username, VHost, ?STAR))), + list_topic_permissions( + user_vhost_topic_perms_info_keys(), MnesiaThunk, KhepriThunk). + +list_topic_permissions(Keys, MnesiaThunk, KhepriThunk) -> + TopicPermissions = rabbit_khepri:try_mnesia_or_khepri( + fun() -> + list_topic_permissions_in_mnesia(MnesiaThunk) + end, + fun() -> + list_topic_permissions_in_khepri(KhepriThunk) + end), + [extract_topic_permission_params(Keys, U) || U <- TopicPermissions]. + +list_topic_permissions_in_mnesia(QueryThunk) when is_function(QueryThunk) -> + rabbit_misc:execute_mnesia_transaction(QueryThunk). + +list_topic_permissions_in_khepri(QueryThunk) when is_function(QueryThunk) -> + case rabbit_khepri:transaction(QueryThunk, ro) of + {ok, TopicPermissions} -> maps:values(TopicPermissions); + _ -> [] + end. match_user_vhost_topic_permission(Username, VHostPath) -> match_user_vhost_topic_permission(Username, VHostPath, '_'). @@ -1160,3 +1662,98 @@ notify_limit_clear(Username, ActingUser) -> rabbit_event:notify(user_limits_cleared, [{name, <<"limits">>}, {user_who_performed_action, ActingUser}, {username, Username}]). + +clear_data_in_khepri() -> + Path = khepri_users_path(), + case rabbit_khepri:delete(Path) of + {ok, _} -> ok; + Error -> throw(Error) + end. + +mnesia_write_to_khepri(User) when ?is_internal_user(User) -> + Username = internal_user:get_username(User), + Path = khepri_user_path(Username), + case rabbit_khepri:put(Path, User) of + {ok, _} -> ok; + Error -> throw(Error) + end; +mnesia_write_to_khepri( + #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHost}} = UserPermission) -> + Path = khepri_user_permission_path( + #if_all{conditions = + [Username, + #if_node_exists{exists = true}]}, + VHost), + Extra = #{keep_while => + #{rabbit_vhost:khepri_vhost_path(VHost) => + #if_node_exists{exists = true}}}, + case rabbit_khepri:put(Path, UserPermission, Extra) of + {ok, _} -> ok; + Error -> throw(Error) + end; +mnesia_write_to_khepri( + #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHost}, + exchange = Exchange}} = TopicPermission) -> + Path = khepri_topic_permission_path( + #if_all{conditions = + [Username, + #if_node_exists{exists = true}]}, + VHost, + Exchange), + Extra = #{keep_while => + #{rabbit_vhost:khepri_vhost_path(VHost) => + #if_node_exists{exists = true}}}, + case rabbit_khepri:put(Path, TopicPermission, Extra) of + {ok, _} -> ok; + Error -> throw(Error) + end. + +mnesia_delete_to_khepri(User) when ?is_internal_user(User) -> + Username = internal_user:get_username(User), + Path = khepri_user_path(Username), + case rabbit_khepri:delete(Path) of + {ok, _} -> ok; + Error -> throw(Error) + end; +mnesia_delete_to_khepri( + #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHost}}) -> + Path = khepri_user_permission_path(Username, VHost), + case rabbit_khepri:delete(Path) of + {ok, _} -> ok; + Error -> throw(Error) + end; +mnesia_delete_to_khepri( + #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHost}, + exchange = Exchange}}) -> + Path = khepri_topic_permission_path(Username, VHost, Exchange), + case rabbit_khepri:delete(Path) of + {ok, _} -> ok; + Error -> throw(Error) + end. + +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]. + +khepri_path_to_user([?MODULE, users, Username | _]) -> Username. diff --git a/deps/rabbit/src/rabbit_core_ff.erl b/deps/rabbit/src/rabbit_core_ff.erl index 60075c99bab0..6e6a74cab1b5 100644 --- a/deps/rabbit/src/rabbit_core_ff.erl +++ b/deps/rabbit/src/rabbit_core_ff.erl @@ -7,12 +7,23 @@ -module(rabbit_core_ff). +-include_lib("kernel/include/logger.hrl"). +-include_lib("stdlib/include/assert.hrl"). + +-include_lib("rabbit_common/include/rabbit.hrl"). +-include_lib("rabbit_common/include/logging.hrl"). + +-include("feature_flags.hrl"). +-include("internal_user.hrl"). +-include("vhost.hrl"). + -export([quorum_queue_migration/3, implicit_default_bindings_migration/3, virtual_host_metadata_migration/3, maintenance_mode_status_migration/3, user_limits_migration/3, - direct_exchange_routing_v2_migration/3]). + direct_exchange_routing_v2_migration/3, + mds_phase1_migration/1]). -rabbit_feature_flag( {classic_mirrored_queue_version, @@ -87,6 +98,18 @@ stability => stable }}). +-rabbit_feature_flag( + {raft_based_metadata_store_phase1, + #{desc => "Use the new Raft-based metadata store [phase 1]", + doc_url => "", %% TODO + stability => experimental, + depends_on => [feature_flags_v2, + maintenance_mode_status, + user_limits, + virtual_host_metadata], + migration_fun => {?MODULE, mds_phase1_migration} + }}). + %% ------------------------------------------------------------------- %% Quorum queues. %% ------------------------------------------------------------------- @@ -163,7 +186,15 @@ virtual_host_metadata_migration(_FeatureName, _FeatureProps, enable) -> {aborted, Reason} -> {error, Reason} end; virtual_host_metadata_migration(_FeatureName, _FeatureProps, is_enabled) -> - mnesia:table_info(rabbit_vhost, attributes) =:= vhost:fields(vhost_v2). + try + mnesia:table_info(rabbit_vhost, attributes) =:= vhost:fields(vhost_v2) + catch + exit:{aborted, {no_exists, rabbit_vhost, _}} -> + %% If the `rabbit_vhost' table is gone, it means the + %% `raft_based_metadata_store_phase1' feature is enabled and this + %% flag too. + true + end. %% ------------------------------------------------------------------- %% Maintenance mode. @@ -200,7 +231,16 @@ user_limits_migration(_FeatureName, _FeatureProps, enable) -> {aborted, Reason} -> {error, Reason} end; user_limits_migration(_FeatureName, _FeatureProps, is_enabled) -> - mnesia:table_info(rabbit_user, attributes) =:= internal_user:fields(internal_user_v2). + try + mnesia:table_info(rabbit_user, attributes) =:= + internal_user:fields(internal_user_v2) + catch + exit:{aborted, {no_exists, rabbit_user, _}} -> + %% If the `rabbit_user' table is gone, it means the + %% `raft_based_metadata_store_phase1' feature is enabled and this + %% flag too. + true + end. %% ------------------------------------------------------------------- %% Direct exchange routing v2. @@ -244,3 +284,559 @@ direct_exchange_routing_v2_migration(_FeatureName, _FeatureProps, is_enabled) -> ok end, Enabled. + +%% ------------------------------------------------------------------- +%% Raft-based metadata store (phase 1). +%% ------------------------------------------------------------------- + +%% Phase 1 covers the migration of the following data: +%% * virtual hosts +%% * users and their permissions +%% * runtime parameters +%% They all depend on each others in Mnesia transactions. That's why they must +%% be migrated atomically. + +%% This table order is important. For instance, user permissions depend on +%% both vhosts and users to exist in the metadata store. +-define(MDS_PHASE1_TABLES, [rabbit_vhost, + rabbit_user, + rabbit_user_permission, + rabbit_topic_permission, + rabbit_runtime_parameters]). + +mds_phase1_migration(#ffcommand{name = FeatureName, command = enable}) -> + case ensure_khepri_cluster_matches_mnesia(FeatureName) of + ok -> + Tables = ?MDS_PHASE1_TABLES, + case is_mds_migration_done(FeatureName) of + false -> migrate_tables_to_khepri(FeatureName, Tables); + true -> ok + end; + Error -> + Error + end; +mds_phase1_migration(#ffcommand{name = FeatureName, command = post_enable}) -> + ?assert(rabbit_khepri:is_enabled(non_blocking)), + Tables = ?MDS_PHASE1_TABLES, + empty_unused_mnesia_tables(FeatureName, Tables). + +ensure_khepri_cluster_matches_mnesia(FeatureName) -> + %% Initialize Khepri cluster based on Mnesia running nodes. Verify that + %% all Mnesia nodes are running (all == running). It would be more + %% difficult to add them later to the node when they start. + ?LOG_DEBUG( + "Feature flag `~s`: ensure Khepri Ra system is running", + [FeatureName]), + ok = rabbit_khepri:setup(), + ?LOG_DEBUG( + "Feature flag `~s`: making sure all Mnesia nodes are running", + [FeatureName]), + AllMnesiaNodes = lists:sort(rabbit_mnesia:cluster_nodes(all)), + RunningMnesiaNodes = lists:sort(rabbit_mnesia:cluster_nodes(running)), + MissingMnesiaNodes = AllMnesiaNodes -- RunningMnesiaNodes, + case MissingMnesiaNodes of + [] -> + %% This is the first time Khepri will be used for real. Therefore + %% we need to make sure the Khepri cluster matches the Mnesia + %% cluster. + ?LOG_DEBUG( + "Feature flag `~s`: updating the Khepri cluster to match " + "the Mnesia cluster", + [FeatureName]), + case expand_khepri_cluster(FeatureName, AllMnesiaNodes) of + ok -> + ok; + Error -> + ?LOG_ERROR( + "Feature flag `~s`: failed to migrate from Mnesia " + "to Khepri: failed to create Khepri cluster: ~p", + [FeatureName, Error]), + Error + end; + _ -> + ?LOG_ERROR( + "Feature flag `~s`: failed to migrate from Mnesia to Khepri: " + "all Mnesia nodes must run; the following nodes are missing: " + "~p", + [FeatureName, MissingMnesiaNodes]), + {error, all_mnesia_nodes_must_run} + end. + +expand_khepri_cluster(FeatureName, AllMnesiaNodes) -> + %% All Mnesia nodes are running (this is a requirement to enable this + %% feature flag). We use this unique list of nodes to find the largest + %% Khepri clusters among all of them. + %% + %% The idea is that at the beginning, each Mnesia node will also be an + %% unclustered Khepri node. Therefore, the first node in the sorted list + %% of Mnesia nodes will be picked (a "cluster" with 1 member, but the + %% "largest" at the beginning). + %% + %% After the first nodes join that single node, its cluster will grow and + %% will continue to be the largest. + %% + %% This function is executed on the node enabling the feature flag. It will + %% take care of adding all nodes in the Mnesia cluster to a Khepri cluster + %% (except those which are already part of it). + %% + %% This should avoid the situation where a large established cluster is + %% reset and joins a single new/empty node. + %% + %% Also, we only consider Khepri clusters which are in use (i.e. the + %% feature flag is enabled). Here is an example: + %% - Node2 is the only node in the Mnesia cluster at the time the + %% feature flag is enabled. It joins no other node and runs its own + %% one-node Khepri cluster. + %% - Node1 joins the Mnesia cluster which is now Node1 + Node2. Given + %% the sorting, Khepri clusters will be [[Node1], [Node2]] when + %% sorted by name and size. With this order, Node1 should "join" + %% itself. But the feature is not enabled yet on this node, + %% therefore, we skip this cluster to consider the following one, + %% [Node2]. + KhepriCluster = find_largest_khepri_cluster(FeatureName), + NodesToAdd = AllMnesiaNodes -- KhepriCluster, + ?LOG_DEBUG( + "Feature flags `~s`: selected Khepri cluster: ~p", + [FeatureName, KhepriCluster]), + ?LOG_DEBUG( + "Feature flags `~s`: Mnesia nodes to add to the Khepri cluster " + "above: ~p", + [FeatureName, NodesToAdd]), + add_nodes_to_khepri_cluster(FeatureName, KhepriCluster, NodesToAdd). + +add_nodes_to_khepri_cluster(FeatureName, KhepriCluster, [Node | Rest]) -> + add_node_to_khepri_cluster(FeatureName, KhepriCluster, Node), + add_nodes_to_khepri_cluster(FeatureName, KhepriCluster, Rest); +add_nodes_to_khepri_cluster(_FeatureName, _KhepriCluster, []) -> + ok. + +add_node_to_khepri_cluster(FeatureName, KhepriCluster, Node) -> + ?assertNotEqual([], KhepriCluster), + case lists:member(Node, KhepriCluster) of + true -> + ?LOG_DEBUG( + "Feature flag `~s`: node ~p is already a member of " + "the largest cluster: ~p", + [FeatureName, Node, KhepriCluster]), + ok; + false -> + ?LOG_DEBUG( + "Feature flag `~s`: adding node ~p to the largest " + "Khepri cluster found among Mnesia nodes: ~p", + [FeatureName, Node, KhepriCluster]), + case rabbit_khepri:add_member(Node, KhepriCluster) of + ok -> ok; + {ok, already_member} -> ok + end + end. + +find_largest_khepri_cluster(FeatureName) -> + case list_all_khepri_clusters(FeatureName) of + [] -> + [node()]; + KhepriClusters -> + KhepriClustersBySize = sort_khepri_clusters_by_size( + KhepriClusters), + ?LOG_DEBUG( + "Feature flag `~s`: existing Khepri clusters (sorted by " + "size): ~p", + [FeatureName, KhepriClustersBySize]), + LargestKhepriCluster = hd(KhepriClustersBySize), + LargestKhepriCluster + end. + +list_all_khepri_clusters(FeatureName) -> + MnesiaNodes = lists:sort(rabbit_mnesia:cluster_nodes(all)), + ?LOG_DEBUG( + "Feature flag `~s`: querying the following Mnesia nodes to learn " + "their Khepri cluster membership: ~p", + [FeatureName, MnesiaNodes]), + KhepriClusters = lists:foldl( + fun(MnesiaNode, Acc) -> + case khepri_cluster_on_node(MnesiaNode) of + [] -> Acc; + Cluster -> Acc#{Cluster => true} + end + end, #{}, MnesiaNodes), + lists:sort(maps:keys(KhepriClusters)). + +sort_khepri_clusters_by_size(KhepriCluster) -> + lists:sort( + fun(A, B) -> length(A) >= length(B) end, + KhepriCluster). + +khepri_cluster_on_node(Node) -> + lists:sort( + rabbit_misc:rpc_call(Node, rabbit_khepri, nodes_if_khepri_enabled, [])). + +migrate_tables_to_khepri(FeatureName, Tables) -> + rabbit_table:wait(Tables, _Retry = true), + ?LOG_NOTICE( + "Feature flag `~s`: starting migration from Mnesia " + "to Khepri; expect decrease in performance and " + "increase in memory footprint", + [FeatureName]), + Pid = spawn( + fun() -> + migrate_tables_to_khepri_run(FeatureName, Tables) + end), + MonitorRef = erlang:monitor(process, Pid), + receive + {'DOWN', MonitorRef, process, Pid, normal} -> + ?LOG_NOTICE( + "Feature flag `~s`: migration from Mnesia to Khepri " + "finished", + [FeatureName]), + ok; + {'DOWN', MonitorRef, process, Pid, Info} -> + ?LOG_ERROR( + "Feature flag `~s`: " + "failed to migrate Mnesia tables to Khepri:~n ~p", + [FeatureName, Info]), + {error, {migration_failure, Info}} + end. + +migrate_tables_to_khepri_run(FeatureName, Tables) -> + %% Clear data in Khepri which could come from a previously aborted copy + %% attempt. The table list order is important so we need to reverse that + %% order to clear the data. + ?LOG_DEBUG( + "Feature flag `~s`: clear data from any aborted migration attempts " + "(if any)", + [FeatureName]), + ok = clear_data_from_previous_attempt(FeatureName, lists:reverse(Tables)), + + %% Subscribe to Mnesia events: we want to know about all writes and + %% deletions happening in parallel to the copy we are about to start. + ?LOG_DEBUG( + "Feature flag `~s`: subscribe to Mnesia writes", + [FeatureName]), + ok = subscribe_to_mnesia_changes(FeatureName, Tables), + + %% Copy from Mnesia to Khepri. Tables are copied in a specific order to + %% make sure that if term A depends on term B, term B was copied before. + ?LOG_DEBUG( + "Feature flag `~s`: copy records from Mnesia to Khepri", + [FeatureName]), + ok = copy_from_mnesia_to_khepri(FeatureName, Tables), + + %% Mnesia transaction to handle received Mnesia events and tables removal. + ?LOG_DEBUG( + "Feature flag `~s`: final sync and Mnesia table removal", + [FeatureName]), + ok = final_sync_from_mnesia_to_khepri(FeatureName, Tables), + + %% Unsubscribe to Mnesia events. All Mnesia tables are synchronized and + %% read-only at this point. + ?LOG_DEBUG( + "Feature flag `~s`: subscribe to Mnesia writes", + [FeatureName]), + ok = unsubscribe_to_mnesia_changes(FeatureName, Tables). + +clear_data_from_previous_attempt( + FeatureName, [rabbit_vhost | Rest]) -> + ok = rabbit_vhost:clear_data_in_khepri(), + clear_data_from_previous_attempt(FeatureName, Rest); +clear_data_from_previous_attempt( + FeatureName, [rabbit_user | Rest]) -> + ok = rabbit_auth_backend_internal:clear_data_in_khepri(), + clear_data_from_previous_attempt(FeatureName, Rest); +clear_data_from_previous_attempt( + FeatureName, [rabbit_user_permission | Rest]) -> + clear_data_from_previous_attempt(FeatureName, Rest); +clear_data_from_previous_attempt( + FeatureName, [rabbit_topic_permission | Rest]) -> + clear_data_from_previous_attempt(FeatureName, Rest); +clear_data_from_previous_attempt( + FeatureName, [rabbit_runtime_parameters | Rest]) -> + ok = rabbit_runtime_parameters:clear_data_in_khepri(), + clear_data_from_previous_attempt(FeatureName, Rest); +clear_data_from_previous_attempt(_, []) -> + ok. + +subscribe_to_mnesia_changes(FeatureName, [Table | Rest]) -> + ?LOG_DEBUG( + "Feature flag `~s`: subscribe to writes to ~s", + [FeatureName, Table]), + case mnesia:subscribe({table, Table, simple}) of + {ok, _} -> subscribe_to_mnesia_changes(FeatureName, Rest); + Error -> Error + end; +subscribe_to_mnesia_changes(_, []) -> + ok. + +unsubscribe_to_mnesia_changes(FeatureName, [Table | Rest]) -> + ?LOG_DEBUG( + "Feature flag `~s`: subscribe to writes to ~s", + [FeatureName, Table]), + case mnesia:unsubscribe({table, Table, simple}) of + {ok, _} -> unsubscribe_to_mnesia_changes(FeatureName, Rest); + Error -> Error + end; +unsubscribe_to_mnesia_changes(_, []) -> + ok. + +copy_from_mnesia_to_khepri( + FeatureName, [rabbit_vhost = Table | Rest]) -> + Fun = fun rabbit_vhost:mnesia_write_to_khepri/1, + do_copy_from_mnesia_to_khepri(FeatureName, Table, Fun), + copy_from_mnesia_to_khepri(FeatureName, Rest); +copy_from_mnesia_to_khepri( + FeatureName, [rabbit_user = Table | Rest]) -> + Fun = fun rabbit_auth_backend_internal:mnesia_write_to_khepri/1, + do_copy_from_mnesia_to_khepri(FeatureName, Table, Fun), + copy_from_mnesia_to_khepri(FeatureName, Rest); +copy_from_mnesia_to_khepri( + FeatureName, [rabbit_user_permission = Table | Rest]) -> + Fun = fun rabbit_auth_backend_internal:mnesia_write_to_khepri/1, + do_copy_from_mnesia_to_khepri(FeatureName, Table, Fun), + copy_from_mnesia_to_khepri(FeatureName, Rest); +copy_from_mnesia_to_khepri( + FeatureName, [rabbit_topic_permission = Table | Rest]) -> + Fun = fun rabbit_auth_backend_internal:mnesia_write_to_khepri/1, + do_copy_from_mnesia_to_khepri(FeatureName, Table, Fun), + copy_from_mnesia_to_khepri(FeatureName, Rest); +copy_from_mnesia_to_khepri( + FeatureName, [rabbit_runtime_parameters = Table | Rest]) -> + Fun = fun rabbit_runtime_parameters:mnesia_write_to_khepri/1, + do_copy_from_mnesia_to_khepri(FeatureName, Table, Fun), + copy_from_mnesia_to_khepri(FeatureName, Rest); +copy_from_mnesia_to_khepri(_, []) -> + ok. + +do_copy_from_mnesia_to_khepri(FeatureName, Table, Fun) -> + Count = mnesia:table_info(Table, size), + ?LOG_DEBUG( + "Feature flag `~s`: table ~s: about ~b record(s) to copy", + [FeatureName, Table, Count]), + FirstKey = mnesia:dirty_first(Table), + do_copy_from_mnesia_to_khepri( + FeatureName, Table, FirstKey, Fun, Count, 0). + +do_copy_from_mnesia_to_khepri( + FeatureName, Table, '$end_of_table', _, Count, Copied) -> + ?LOG_DEBUG( + "Feature flag `~s`: table ~s: copy of ~b record(s) (out of ~b " + "initially) finished", + [FeatureName, Table, Copied, Count]), + ok; +do_copy_from_mnesia_to_khepri( + FeatureName, Table, Key, Fun, Count, Copied) -> + %% TODO: Batch several records in a single Khepri insert. + %% TODO: Can/should we parallelize? + case Copied rem 100 of + 0 -> + ?LOG_DEBUG( + "Feature flag `~s`: table ~s: copying record ~b/~b", + [FeatureName, Table, Copied, Count]); + _ -> + ok + end, + case mnesia:dirty_read(Table, Key) of + [Record] -> ok = Fun(Record); + [] -> ok + end, + NextKey = mnesia:dirty_next(Table, Key), + do_copy_from_mnesia_to_khepri( + FeatureName, Table, NextKey, Fun, Count, Copied + 1). + +final_sync_from_mnesia_to_khepri(FeatureName, Tables) -> + %% Switch all tables to read-only. All concurrent and future Mnesia + %% transaction involving a write to one of them will fail with the + %% `{no_exists, Table}` exception. + lists:foreach( + fun(Table) -> + ?LOG_DEBUG( + "Feature flag `~s`: switch table ~s to read-only", + [FeatureName, Table]), + {atomic, ok} = mnesia:change_table_access_mode(Table, read_only) + end, Tables), + + %% During the first round of copy, we received all write events as + %% messages (parallel writes were authorized). Now, we want to consume + %% those messages to record the writes we probably missed. + ok = consume_mnesia_events(FeatureName), + + ok. + +consume_mnesia_events(FeatureName) -> + {_, Count} = erlang:process_info(self(), message_queue_len), + ?LOG_DEBUG( + "Feature flag `~s`: handling queued Mnesia events " + "(about ~b events)", + [FeatureName, Count]), + consume_mnesia_events(FeatureName, Count, 0). + +consume_mnesia_events(FeatureName, Count, Handled) -> + %% TODO: Batch several events in a single Khepri command. + Handled1 = Handled + 1, + receive + {mnesia_table_event, {write, NewRecord, _}} -> + ?LOG_DEBUG( + "Feature flag `~s`: handling event ~b/~b (write)", + [FeatureName, Handled1, Count]), + handle_mnesia_write(NewRecord), + consume_mnesia_events(FeatureName, Count, Handled1); + {mnesia_table_event, {delete_object, OldRecord, _}} -> + ?LOG_DEBUG( + "Feature flag `~s`: handling event ~b/~b (delete)", + [FeatureName, Handled1, Count]), + handle_mnesia_delete(OldRecord), + consume_mnesia_events(FeatureName, Count, Handled1); + {mnesia_table_event, {delete, {Table, Key}, _}} -> + ?LOG_DEBUG( + "Feature flag `~s`: handling event ~b/~b (delete)", + [FeatureName, Handled1, Count]), + handle_mnesia_delete(Table, Key), + consume_mnesia_events(FeatureName, Count, Handled1) + after 0 -> + {_, MsgCount} = erlang:process_info(self(), message_queue_len), + ?LOG_DEBUG( + "Feature flag `~s`: ~b messages remaining", + [FeatureName, MsgCount]), + %% TODO: Wait for confirmation from Khepri. + ok + end. + +handle_mnesia_write(NewRecord) when ?is_vhost(NewRecord) -> + rabbit_vhost:mnesia_write_to_khepri(NewRecord); +handle_mnesia_write(NewRecord) when is_record(NewRecord, user_permission) -> + rabbit_auth_backend_internal:mnesia_write_to_khepri(NewRecord); +handle_mnesia_write(NewRecord) when is_record(NewRecord, topic_permission) -> + rabbit_auth_backend_internal:mnesia_write_to_khepri(NewRecord); +handle_mnesia_write(NewRecord) -> + %% The record and the Mnesia table have different names. + NewRecord1 = erlang:setelement(1, NewRecord, internal_user), + true = ?is_internal_user(NewRecord1), + rabbit_auth_backend_internal:mnesia_write_to_khepri(NewRecord1). + +handle_mnesia_delete(OldRecord) when ?is_vhost(OldRecord) -> + rabbit_vhost:mnesia_delete_to_khepri(OldRecord); +handle_mnesia_delete(OldRecord) when ?is_internal_user(OldRecord) -> + rabbit_auth_backend_internal:mnesia_delete_to_khepri(OldRecord); +handle_mnesia_delete(OldRecord) when is_record(OldRecord, user_permission) -> + rabbit_auth_backend_internal:mnesia_delete_to_khepri(OldRecord); +handle_mnesia_delete(OldRecord) when is_record(OldRecord, topic_permission) -> + rabbit_auth_backend_internal:mnesia_delete_to_khepri(OldRecord). + +handle_mnesia_delete(rabbit_vhost, VHost) -> + rabbit_vhost:mnesia_delete_to_khepri(VHost); +handle_mnesia_delete(rabbit_user, Username) -> + rabbit_auth_backend_internal:mnesia_delete_to_khepri(Username); +handle_mnesia_delete(rabbit_user_permission, UserVHost) -> + rabbit_auth_backend_internal:mnesia_delete_to_khepri(UserVHost); +handle_mnesia_delete(rabbit_topic_permission, TopicPermissionKey) -> + rabbit_auth_backend_internal:mnesia_delete_to_khepri(TopicPermissionKey); +handle_mnesia_delete(rabbit_runtime_parameters, RuntimeParamKey) -> + rabbit_runtime_parameters:mnesia_delete_to_khepri(RuntimeParamKey). + +%% We can't remove unused tables at this point yet. The reason is that tables +%% are synchronized before feature flags in `rabbit_mnesia`. So if a node is +%% already using Khepri and another node wants to join him, but is using Mnesia +%% only, it will hang while trying to sync the dropped tables. +%% +%% We can't simply reverse the two steps (i.e. synchronize feature flags before +%% tables) because some feature flags like `quorum_queue` need tables to modify +%% their schema. +%% +%% Another solution would be to have two groups of feature flags, depending on +%% whether a feature flag should be synchronized before or after Mnesia +%% tables. +%% +%% But for now, let's just empty the tables, add a forged record to mark them +%% as migrated and leave them around. + +empty_unused_mnesia_tables(FeatureName, [Table | Rest]) -> + %% The feature flag is enabled at this point. It means there should be no + %% code trying to read or write the Mnesia tables. + case mnesia:change_table_access_mode(Table, read_write) of + {atomic, ok} -> + ?LOG_DEBUG( + "Feature flag `~s`: dropping content from unused Mnesia " + "table ~s", + [FeatureName, Table]), + ok = empty_unused_mnesia_table(Table); + {aborted, {already_exists, Table, _}} -> + %% Another node is already taking care of this table. + ?LOG_DEBUG( + "Feature flag `~s`: Mnesia table ~s already emptied", + [FeatureName, Table]), + ok + end, + empty_unused_mnesia_tables(FeatureName, Rest); +empty_unused_mnesia_tables(FeatureName, []) -> + ?LOG_DEBUG( + "Feature flag `~s`: done with emptying unused Mnesia tables", + [FeatureName]), + ok. + +empty_unused_mnesia_table(Table) -> + FirstKey = mnesia:dirty_first(Table), + empty_unused_mnesia_table(Table, FirstKey). + +empty_unused_mnesia_table(_Table, '$end_of_table') -> + ok; +empty_unused_mnesia_table(Table, Key) -> + NextKey = mnesia:dirty_next(Table, Key), + ok = mnesia:dirty_delete(Table, Key), + empty_unused_mnesia_table(Table, NextKey). + +is_mds_migration_done(FeatureName) -> + %% To determine if the migration to Khepri was finished, we look at the + %% state of the feature flag on another node, if any. + ThisNode = node(), + KhepriNodes = rabbit_khepri:nodes(), + case KhepriNodes -- [ThisNode] of + [] -> + %% There are no other nodes. It means the node is unclustered + %% and the migration function is called for the first time. This + %% function returns `false'. + ?LOG_DEBUG( + "Feature flag `~s`: migration done? false, the node is " + "unclustered", + [FeatureName]), + false; + [RemoteKhepriNode | _] -> + %% This node is clustered already, either because of peer discovery + %% or because of the `expand_khepri_cluster()' function. + %% + %% We need to distinguish two situations: + %% + %% - The first time the feature flag is enabled in a cluster, we + %% want to migrate records from Mnesia to Khepri. In this case, + %% the state of the feature flag will be `state_changing' on all + %% nodes in the cluster. That's why we can pick any node to query + %% its state. + %% + %% - When a new node is joining an existing cluster which is + %% already using Khepri, we DO NOT want to migrate anything + %% (Mnesia tables are empty, or about to be if the + %% `post_enabled_locally' code is still running). To determine + %% this, we query a remote node (but not this local node) to see + %% the feature flag state. If it's `true' (enabled), it means the + %% migration is either in progress or done. Otherwise, we are in + %% the first situation described above. + ?LOG_DEBUG( + "Feature flag `~s`: migration done? unknown, querying node ~p", + [FeatureName, RemoteKhepriNode]), + IsEnabledRemotely = rabbit_misc:rpc_call( + RemoteKhepriNode, + rabbit_feature_flags, + is_enabled, + [FeatureName, non_blocking]), + ?LOG_DEBUG( + "Feature flag `~s`: feature flag state on node ~p: ~p", + [FeatureName, RemoteKhepriNode, IsEnabledRemotely]), + + %% If the RPC call fails (i.e. returns `{badrpc, ...}'), we throw + %% an exception because we want the migration function to abort. + Ret = case IsEnabledRemotely of + true -> true; + state_changing -> false; + {badrpc, Error} -> throw(Error) + end, + ?LOG_DEBUG( + "Feature flag `~s`: migration done? ~s", + [FeatureName, Ret]), + Ret + end. diff --git a/deps/rabbit/src/rabbit_definitions.erl b/deps/rabbit/src/rabbit_definitions.erl index f5022eb8b979..746115eddc98 100644 --- a/deps/rabbit/src/rabbit_definitions.erl +++ b/deps/rabbit/src/rabbit_definitions.erl @@ -419,8 +419,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)} end. -spec apply_defs(Map :: #{atom() => any()}, @@ -588,6 +588,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("~p", [E])). diff --git a/deps/rabbit/src/rabbit_khepri.erl b/deps/rabbit/src/rabbit_khepri.erl new file mode 100644 index 000000000000..46831925e52a --- /dev/null +++ b/deps/rabbit/src/rabbit_khepri.erl @@ -0,0 +1,509 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(rabbit_khepri). + +-include_lib("kernel/include/logger.hrl"). +-include_lib("stdlib/include/assert.hrl"). + +-include_lib("khepri/include/khepri.hrl"). +-include_lib("rabbit_common/include/logging.hrl"). + +-export([setup/0, + setup/1, + add_member/2, + remove_member/1, + members/0, + locally_known_members/0, + nodes/0, + locally_known_nodes/0, + get_store_id/0, + + create/2, + update/2, + cas/3, + + get/1, + get_data/1, + match/1, + match_and_get_data/1, + tx_match_and_get_data/1, + exists/1, + find/2, + list/1, + list_child_nodes/1, + list_child_data/1, + + put/2, put/3, + clear_payload/1, + delete/1, + delete_or_fail/1, + + transaction/1, + transaction/2, + + clear_store/0, + + dir/0, + info/0, + is_enabled/0, + is_enabled/1, + nodes_if_khepri_enabled/0, + try_mnesia_or_khepri/2]). +-export([do_join/1]). + +-ifdef(TEST). +-export([force_metadata_store/1, + clear_forced_metadata_store/0]). +-endif. + +-compile({no_auto_import, [get/1, get/2, nodes/0]}). + +-define(RA_SYSTEM, coordination). +-define(RA_CLUSTER_NAME, metadata_store). +-define(RA_FRIENDLY_NAME, "RabbitMQ metadata store"). +-define(STORE_ID, ?RA_CLUSTER_NAME). +-define(MDSTORE_SARTUP_LOCK, {?MODULE, self()}). +-define(PT_KEY, ?MODULE). + +%% ------------------------------------------------------------------- +%% API wrapping Khepri. +%% ------------------------------------------------------------------- + +-spec setup() -> ok | no_return(). + +setup() -> + setup(rabbit_prelaunch:get_context()). + +-spec setup(map()) -> ok | no_return(). + +setup(_) -> + ?LOG_DEBUG("Starting Khepri-based " ?RA_FRIENDLY_NAME), + ok = ensure_ra_system_started(), + ok = application:set_env( + khepri, default_timeout, 5000, [{persistent, true}]), + RaServerConfig = #{cluster_name => ?RA_CLUSTER_NAME, + friendly_name => ?RA_FRIENDLY_NAME}, + case khepri:start(?RA_SYSTEM, RaServerConfig) of + {ok, ?STORE_ID} -> + ?LOG_DEBUG( + "Khepri-based " ?RA_FRIENDLY_NAME " ready", + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + ok; + {error, _} = Error -> + exit(Error) + end. + +add_member(JoiningNode, JoinedNode) + when JoiningNode =:= node() andalso is_atom(JoinedNode) -> + Ret = do_join(JoinedNode), + post_add_member(JoiningNode, JoinedNode, Ret); +add_member(JoiningNode, JoinedNode) when is_atom(JoinedNode) -> + Ret = rabbit_misc:rpc_call( + JoiningNode, rabbit_khepri, do_join, [JoinedNode]), + post_add_member(JoiningNode, JoinedNode, Ret); +add_member(JoiningNode, [_ | _] = Cluster) -> + case lists:member(JoiningNode, Cluster) of + false -> + JoinedNode = pick_node_in_cluster(Cluster), + ?LOG_INFO( + "Khepri clustering: Attempt to add node ~p to cluster ~0p " + "through node ~p", + [JoiningNode, Cluster, JoinedNode], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + %% Recurse with a single node taken in the `Cluster' list. + add_member(JoiningNode, JoinedNode); + true -> + ?LOG_DEBUG( + "Khepri clustering: Node ~p is already a member of cluster ~p", + [JoiningNode, Cluster]), + {ok, already_member} + end. + +pick_node_in_cluster(Cluster) when is_list(Cluster) -> + ?assertNotEqual([], Cluster), + ThisNode = node(), + case lists:member(ThisNode, Cluster) of + true -> ThisNode; + false -> hd(Cluster) + end. + +do_join(RemoteNode) when RemoteNode =/= node() -> + ThisNode = node(), + + ?LOG_DEBUG( + "Khepri clustering: Trying to add this node (~p) to cluster \"~s\" " + "through node ~p", + [ThisNode, ?RA_CLUSTER_NAME, RemoteNode], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + + %% Ensure the local Khepri store is running before we can reset it. It + %% could be stopped if RabbitMQ is not running for instance. + ok = setup(), + khepri:info(?RA_CLUSTER_NAME), + + %% Ensure the remote node is reachable before we add it. + pong = net_adm:ping(RemoteNode), + + %% We verify the cluster membership before adding `ThisNode' to + %% `RemoteNode''s cluster. We do it mostly to keep the same behavior as + %% what we do with Mnesia. Otherwise, the interest is limited given the + %% check and the actual join are not atomic. + + ClusteredNodes = rabbit_misc:rpc_call( + RemoteNode, rabbit_khepri, locally_known_nodes, []), + case lists:member(ThisNode, ClusteredNodes) of + false -> + ?LOG_DEBUG( + "Adding this node (~p) to Khepri cluster \"~s\" through " + "node ~p", + [ThisNode, ?RA_CLUSTER_NAME, RemoteNode], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + + %% If the remote node to add is running RabbitMQ, we need to put it + %% in maintenance mode at least. We remember that state to revive + %% the node only if it was fully running before this code. + IsRunning = rabbit:is_running(ThisNode), + AlreadyBeingDrained = + rabbit_maintenance:is_being_drained_consistent_read(ThisNode), + NeedToRevive = IsRunning andalso not AlreadyBeingDrained, + maybe_drain_node(IsRunning), + + %% Joining a cluster includes a reset of the local Khepri store. + Ret = khepri_cluster:join(?RA_CLUSTER_NAME, RemoteNode), + + %% Revive the remote node if it was running and not under + %% maintenance before we changed the cluster membership. + maybe_revive_node(NeedToRevive), + + Ret; + true -> + ?LOG_DEBUG( + "This node (~p) is already part of the Khepri cluster \"~s\" " + "like node ~p", + [ThisNode, ?RA_CLUSTER_NAME, RemoteNode], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + {ok, already_member} + end. + +maybe_drain_node(true) -> + ok = rabbit_maintenance:drain(); +maybe_drain_node(false) -> + ok. + +maybe_revive_node(true) -> + ok = rabbit_maintenance:revive(); +maybe_revive_node(false) -> + ok. + +post_add_member(JoiningNode, JoinedNode, ok) -> + ?LOG_INFO( + "Khepri clustering: Node ~p successfully added to cluster \"~s\" " + "through node ~p", + [JoiningNode, ?RA_CLUSTER_NAME, JoinedNode], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + ok; +post_add_member( + JoiningNode, _JoinedNode, {error, {already_member, Cluster}}) -> + ?LOG_INFO( + "Khepri clustering: Asked to add node ~p to cluster \"~s\" " + "but already a member of it: ~p", + [JoiningNode, ?RA_CLUSTER_NAME, lists:sort(Cluster)], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + {ok, already_member}; +post_add_member( + JoiningNode, JoinedNode, + {badrpc, {'EXIT', {undef, [{rabbit_khepri, do_join, _, _}]}}} = Error) -> + ?LOG_INFO( + "Khepri clustering: Can't add node ~p to cluster \"~s\"; " + "Khepri unavailable on node ~p: ~p", + [JoiningNode, ?RA_CLUSTER_NAME, JoinedNode, Error], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + %% TODO: Should we return an error and let the caller decide? + ok; +post_add_member(JoiningNode, JoinedNode, Error) -> + ?LOG_INFO( + "Khepri clustering: Failed to add node ~p to cluster \"~s\" " + "through ~p: ~p", + [JoiningNode, ?RA_CLUSTER_NAME, JoinedNode, Error], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + Error. + +remove_member(NodeToRemove) when NodeToRemove =/= node() -> + ?LOG_DEBUG( + "Trying to remove node ~s from Khepri cluster \"~s\" on node ~s", + [NodeToRemove, ?RA_CLUSTER_NAME, node()], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + + %% Check if the node is part of the cluster. We query the local Ra server + %% only, in case the cluster can't elect a leader right now. + CurrentNodes = locally_known_nodes(), + case lists:member(NodeToRemove, CurrentNodes) of + true -> + %% Ensure the remote node is reachable before we remove it. + pong = net_adm:ping(NodeToRemove), + + ?LOG_DEBUG( + "Removing remote node ~s from Khepri cluster \"~s\"", + [NodeToRemove, ?RA_CLUSTER_NAME], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + + %% We need the Khepri store to run on the node to remove, to be + %% able to reset it. + ok = rabbit_misc:rpc_call( + NodeToRemove, ?MODULE, setup, []), + + Ret = rabbit_misc:rpc_call( + NodeToRemove, khepri_cluster, reset, [?RA_CLUSTER_NAME]), + case Ret of + ok -> + ?LOG_DEBUG( + "Node ~s removed from Khepri cluster \"~s\"", + [NodeToRemove, ?RA_CLUSTER_NAME], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + ok; + Error -> + ?LOG_ERROR( + "Failed to remove remote node ~s from Khepri " + "cluster \"~s\": ~p", + [NodeToRemove, ?RA_CLUSTER_NAME, Error], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + Error + end; + false -> + ?LOG_INFO( + "Asked to remove node ~s from Khepri cluster \"~s\" but not " + "member of it: ~p", + [NodeToRemove, ?RA_CLUSTER_NAME, lists:sort(CurrentNodes)], + #{domain => ?RMQLOG_DOMAIN_GLOBAL}), + ok + end. + +ensure_ra_system_started() -> + {ok, _} = application:ensure_all_started(khepri), + ok = rabbit_ra_systems:ensure_ra_system_started(?RA_SYSTEM). + +members() -> + khepri_cluster:members(?RA_CLUSTER_NAME). + +locally_known_members() -> + khepri_cluster:locally_known_members(?RA_CLUSTER_NAME). + +nodes() -> + khepri_cluster:nodes(?RA_CLUSTER_NAME). + +locally_known_nodes() -> + khepri_cluster:locally_known_nodes(?RA_CLUSTER_NAME). + +get_store_id() -> + ?STORE_ID. + +dir() -> + filename:join(rabbit_mnesia:dir(), atom_to_list(?STORE_ID)). + +%% ------------------------------------------------------------------- +%% "Proxy" functions to Khepri API. +%% ------------------------------------------------------------------- + +%% They just add the store ID to every calls. +%% +%% The only exceptions are get() and match() which both call khepri:get() +%% behind the scene with different options. +%% +%% They are some additional functions too, because they are useful in +%% RabbitMQ. They might be moved to Khepri in the future. + +create(Path, Data) -> khepri:create(?STORE_ID, Path, Data). +update(Path, Data) -> khepri:update(?STORE_ID, Path, Data). +cas(Path, Pattern, Data) -> + khepri:compare_and_swap(?STORE_ID, Path, Pattern, Data). + +get(Path) -> + case khepri:get(?STORE_ID, Path, #{expect_specific_node => true}) of + {ok, Result} -> + [PropsAndData] = maps:values(Result), + {ok, PropsAndData}; + Error -> + Error + end. + +get_data(Path) -> + case get(Path) of + {ok, #{data := Data}} -> {ok, Data}; + {ok, Result} -> {error, {no_data, Result}}; + Error -> Error + end. + +match(Path) -> khepri:get(?STORE_ID, Path). + +match_and_get_data(Path) -> + Ret = match(Path), + keep_data_only_in_result(Ret). + +tx_match_and_get_data(Path) -> + Ret = khepri_tx:get(Path), + keep_data_only_in_result(Ret). + +exists(Path) -> khepri:exists(?STORE_ID, Path). +find(Path, Condition) -> khepri:find(?STORE_ID, Path, Condition). + +list(Path) -> khepri:list(?STORE_ID, Path). + +list_child_nodes(Path) -> + Options = #{expect_specific_node => true, + include_child_names => true}, + case khepri:get(?STORE_ID, Path, Options) of + {ok, Result} -> + [#{child_names := ChildNames}] = maps:values(Result), + {ok, ChildNames}; + Error -> + Error + end. + +list_child_data(Path) -> + Ret = list(Path), + keep_data_only_in_result(Ret). + +keep_data_only(Result) -> + maps:fold( + fun + (Path, #{data := Data}, Acc) -> Acc#{Path => Data}; + (_, _, Acc) -> Acc + end, #{}, Result). + +keep_data_only_in_result({ok, Result}) -> + Result1 = keep_data_only(Result), + {ok, Result1}; +keep_data_only_in_result(Error) -> + Error. + +clear_payload(Path) -> khepri:clear_payload(?STORE_ID, Path). +delete(Path) -> khepri:delete(?STORE_ID, Path). + +delete_or_fail(Path) -> + case khepri:delete(?STORE_ID, Path) of + {ok, Result} -> + case maps:size(Result) of + 0 -> {error, {node_not_found, #{}}}; + _ -> ok + end; + Error -> + Error + end. + +put(PathPattern, Data) -> + khepri:put( + ?STORE_ID, PathPattern, Data). + +put(PathPattern, Data, Extra) -> + khepri:put( + ?STORE_ID, PathPattern, Data, Extra). + +transaction(Fun) -> + transaction(Fun, auto). + +transaction(Fun, ReadWrite) -> + case khepri:transaction(?STORE_ID, Fun, ReadWrite) of + {atomic, Result} -> Result; + {aborted, Reason} -> throw({error, Reason}) + end. + +clear_store() -> + khepri:clear_store(?STORE_ID). + +info() -> + ok = setup(), + khepri:info(?STORE_ID). + +%% ------------------------------------------------------------------- +%% Raft-based metadata store (phase 1). +%% ------------------------------------------------------------------- + +is_enabled() -> + rabbit_feature_flags:is_enabled(raft_based_metadata_store_phase1). + +is_enabled(Blocking) -> + rabbit_feature_flags:is_enabled( + raft_based_metadata_store_phase1, Blocking) =:= true. + +nodes_if_khepri_enabled() -> + case is_enabled(non_blocking) of + true -> nodes(); + false -> [] + end. + +-ifdef(TEST). +-define(FORCED_MDS_KEY, {?MODULE, forced_metadata_store}). + +force_metadata_store(Backend) -> + persistent_term:put(?FORCED_MDS_KEY, Backend). + +get_forced_metadata_store() -> + persistent_term:get(?FORCED_MDS_KEY, undefined). + +clear_forced_metadata_store() -> + _ = persistent_term:erase(?FORCED_MDS_KEY), + ok. +-endif. + +try_mnesia_or_khepri(MnesiaFun, KhepriFun) -> + case use_khepri() of + true -> + KhepriFun(); + false -> + try + MnesiaFun() + catch + Class:{Type, {no_exists, Table}} = Reason:Stacktrace + when Type =:= aborted orelse Type =:= error -> + case is_mnesia_table_covered_by_feature_flag(Table) of + true -> + %% We wait for the feature flag(s) to be enabled + %% or disabled (this is a blocking call) and + %% retry. + ?LOG_DEBUG( + "Mnesia function failed because table ~s " + "is gone or read-only; checking if the new " + "metadata store was enabled in parallel and " + "retry", + [Table]), + _ = is_enabled(), + try_mnesia_or_khepri(MnesiaFun, KhepriFun); + false -> + erlang:raise(Class, Reason, Stacktrace) + end + end + end. + +-ifdef(TEST). +use_khepri() -> + Ret = case get_forced_metadata_store() of + khepri -> + %% We use ?MODULE:is_enabled() to make sure the call goes through + %% the possibly mocked module. + ?assert(?MODULE:is_enabled(non_blocking)), + true; + mnesia -> + ?assertNot(?MODULE:is_enabled(non_blocking)), + false; + undefined -> + ?MODULE:is_enabled(non_blocking) + end, + %rabbit_log:notice("~s: ~p [TEST]", [?FUNCTION_NAME, Ret]), + Ret. +-else. +use_khepri() -> + Ret = is_enabled(non_blocking), + %rabbit_log:notice("~s: ~p", [?FUNCTION_NAME, Ret]), + Ret. +-endif. + +is_mnesia_table_covered_by_feature_flag(rabbit_vhost) -> true; +is_mnesia_table_covered_by_feature_flag(rabbit_user) -> true; +is_mnesia_table_covered_by_feature_flag(rabbit_user_permission) -> true; +is_mnesia_table_covered_by_feature_flag(rabbit_topic_permission) -> true; +is_mnesia_table_covered_by_feature_flag(_) -> false. diff --git a/deps/rabbit/src/rabbit_mnesia.erl b/deps/rabbit/src/rabbit_mnesia.erl index 464f9e526bf3..001c322de1d5 100644 --- a/deps/rabbit/src/rabbit_mnesia.erl +++ b/deps/rabbit/src/rabbit_mnesia.erl @@ -212,9 +212,16 @@ join_discovered_peers_with_retries(TryNodes, NodeType, RetriesLeft, DelayInterva %% we cluster to its cluster. -spec join_cluster(node(), node_type()) - -> ok | {ok, already_member} | {error, {inconsistent_cluster, string()}}. + -> ok | {ok, already_member} | {error, {inconsistent_cluster, string()}} | {badrpc, term()}. join_cluster(DiscoveryNode, NodeType) -> + case join_mnesia_cluster(DiscoveryNode, NodeType) of + ok -> join_khepri_cluster(DiscoveryNode); + {ok, already_member} -> join_khepri_cluster(DiscoveryNode); + Other -> Other + end. + +join_mnesia_cluster(DiscoveryNode, NodeType) -> ensure_mnesia_not_running(), ensure_mnesia_dir(), case is_only_clustered_disc_node() of @@ -258,6 +265,37 @@ join_cluster(DiscoveryNode, NodeType) -> end end. +join_khepri_cluster(DiscoveryNode) -> + ThisNode = node(), + retry_khepri_op(fun() -> rabbit_khepri:add_member(ThisNode, [DiscoveryNode]) end, 60). + +leave_khepri_cluster(Node) -> + retry_khepri_op(fun() -> rabbit_khepri:remove_member(Node) end, 60). + +retry_khepri_op(Fun, 0) -> + Fun(); +retry_khepri_op(Fun, N) -> + case Fun() of + {error, {no_more_servers_to_try, Reasons}} = Err -> + case lists:member({error,cluster_change_not_permitted}, Reasons) of + true -> + timer:sleep(1000), + retry_khepri_op(Fun, N - 1); + false -> + Err + end; + {no_more_servers_to_try, Reasons} = Err -> + case lists:member({error,cluster_change_not_permitted}, Reasons) of + true -> + timer:sleep(1000), + retry_khepri_op(Fun, N - 1); + false -> + Err + end; + Any -> + Any + end. + %% return node to its virgin state, where it is not member of any %% cluster, has no cluster configuration, no local database, and no %% persisted messages @@ -868,7 +906,7 @@ remove_node_if_mnesia_running(Node) -> {atomic, ok} -> rabbit_amqqueue:forget_all_durable(Node), rabbit_node_monitor:notify_left_cluster(Node), - ok; + leave_khepri_cluster(Node); {aborted, Reason} -> {error, {failed_to_remove_node, Node, Reason}} end @@ -911,7 +949,17 @@ stop_mnesia() -> stopped = mnesia:stop(), ensure_mnesia_not_running(). -change_extra_db_nodes(ClusterNodes0, CheckOtherNodes) -> +change_extra_db_nodes(ClusterNodes, CheckOtherNodes) -> + Nodes = change_extra_mnesia_nodes(ClusterNodes, CheckOtherNodes), + %% FIXME: Need to cluster Khepri at this point? I don't think so... I keep + %% the code but it does nothing (`false' condition). + case false andalso rabbit_khepri:is_enabled() of + true -> _ = change_extra_khepri_nodes(ClusterNodes, CheckOtherNodes); + false -> ok + end, + Nodes. + +change_extra_mnesia_nodes(ClusterNodes0, CheckOtherNodes) -> ClusterNodes = nodes_excl_me(ClusterNodes0), case {mnesia:change_config(extra_db_nodes, ClusterNodes), ClusterNodes} of {{ok, []}, [_|_]} when CheckOtherNodes -> @@ -921,6 +969,26 @@ change_extra_db_nodes(ClusterNodes0, CheckOtherNodes) -> Nodes end. +change_extra_khepri_nodes(ClusterNodes, CheckOtherNodes) -> + ThisNode = node(), + _ = rabbit_khepri:add_member(ThisNode, ClusterNodes), + ActualNodes = rabbit_khepri:locally_known_nodes(), + case CheckOtherNodes of + true -> + UnclusteredNodes = ClusterNodes -- ActualNodes, + case UnclusteredNodes of + [] -> + ActualNodes; + _ -> + rabbit_log:error("UnclusteredNodes = ~p~n", [UnclusteredNodes]), + throw({error, + {failed_to_cluster_with, ClusterNodes, + "Khepri could not connect to any nodes."}}) + end; + false -> + ActualNodes + end. + check_consistency(Node, OTP, Rabbit, ProtocolVersion) -> rabbit_misc:sequence_error( [check_mnesia_or_otp_consistency(Node, ProtocolVersion, OTP), @@ -1020,7 +1088,8 @@ is_virgin_node() -> rabbit_node_monitor:coordination_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: ~s, of them to be ignored: ~s", [string:join(lists:usort(List0), ", "), string:join(lists:usort(IgnoredFiles), ", ")]), diff --git a/deps/rabbit/src/rabbit_ra_registry.erl b/deps/rabbit/src/rabbit_ra_registry.erl index e528edf588c2..df6f7be19911 100644 --- a/deps/rabbit/src/rabbit_ra_registry.erl +++ b/deps/rabbit/src/rabbit_ra_registry.erl @@ -13,4 +13,4 @@ %% take them into account in operations such as memory calculation and data cleanup. %% Hardcoded atm list_not_quorum_clusters() -> - [rabbit_stream_coordinator]. + [rabbit_stream_coordinator, metadata_store]. diff --git a/deps/rabbit/src/rabbit_runtime_parameters.erl b/deps/rabbit/src/rabbit_runtime_parameters.erl index 428b05e5e239..d415eb65c0aa 100644 --- a/deps/rabbit/src/rabbit_runtime_parameters.erl +++ b/deps/rabbit/src/rabbit_runtime_parameters.erl @@ -40,6 +40,8 @@ %% * rabbit_registry %% * rabbit_event +-include_lib("khepri/include/khepri.hrl"). + -include_lib("rabbit_common/include/rabbit.hrl"). -export([parse_set/5, set/5, set_any/5, clear/4, clear_any/4, list/0, list/1, @@ -50,6 +52,10 @@ list_global/0, list_global_formatted/0, list_global_formatted/2, lookup_global/1, global_info_keys/0, clear_global/2]). +-export([clear_data_in_khepri/0, + mnesia_write_to_khepri/1, + mnesia_delete_to_khepri/1]). + %%---------------------------------------------------------------------------- -type ok_or_error_string() :: 'ok' | {'error_string', string()}. @@ -103,7 +109,7 @@ parse_set_global(Name, String, ActingUser) -> set_global(Name, Term, ActingUser) -> NameAsAtom = rabbit_data_coercion:to_atom(Name), rabbit_log:debug("Setting global parameter '~s' to ~p", [NameAsAtom, Term]), - mnesia_update(NameAsAtom, Term), + record_update(NameAsAtom, Term), event_notify(parameter_set, none, global, [{name, NameAsAtom}, {value, Term}, {user_who_performed_action, ActingUser}]), @@ -131,7 +137,7 @@ set_any0(VHost, Component, Name, Term, User) -> case flatten_errors( Mod:validate(VHost, Component, Name, Term, get_user(User))) of ok -> - case mnesia_update(VHost, Component, Name, Term) of + case record_update(VHost, Component, Name, Term) of {old, Term} -> ok; _ -> @@ -164,12 +170,23 @@ get_username(none) -> get_username(Any) -> Any. +record_update(Key, Term) -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> mnesia_update(Key, Term) end, + fun() -> khepri_update(Key, Term) end). + +record_update(VHost, Comp, Name, Term) -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> mnesia_update(VHost, Comp, Name, Term) end, + fun() -> khepri_update(VHost, Comp, Name, Term) end). + mnesia_update(Key, Term) -> rabbit_misc:execute_mnesia_transaction(mnesia_update_fun(Key, Term)). mnesia_update(VHost, Comp, Name, Term) -> rabbit_misc:execute_mnesia_transaction( - rabbit_vhost:with(VHost, mnesia_update_fun({VHost, Comp, Name}, Term))). + rabbit_vhost:with_in_mnesia( + VHost, mnesia_update_fun({VHost, Comp, Name}, Term))). mnesia_update_fun(Key, Term) -> fun () -> @@ -181,6 +198,25 @@ mnesia_update_fun(Key, Term) -> Res end. +khepri_update(Key, Term) -> + rabbit_khepri:transaction(khepri_update_fun(Key, Term)). + +khepri_update(VHost, Comp, Name, Term) -> + rabbit_khepri:transaction( + rabbit_vhost:with_in_khepri( + VHost, khepri_update_fun({VHost, Comp, Name}, Term))). + +khepri_update_fun(Key, Term) -> + Path = khepri_rp_path(Key), + fun () -> + case khepri_tx:put(Path, c(Key, Term)) of + {ok, #{Path := #{data := Params}}} -> + {old, Params#runtime_parameters.value}; + {ok, _} -> + new + end + end. + -spec clear(rabbit_types:vhost(), binary(), binary(), rabbit_types:username()) -> ok_thunk_or_error_string(). @@ -201,11 +237,8 @@ clear_global(Key, ActingUser) -> not_found -> {error_string, "Parameter does not exist"}; _ -> - F = fun () -> - ok = mnesia:delete(?TABLE, KeyAsAtom, write) - end, - ok = rabbit_misc:execute_mnesia_transaction(F), - case mnesia:is_transaction() of + record_clear(KeyAsAtom), + case is_transaction() of true -> Notify; false -> Notify() end @@ -239,18 +272,57 @@ clear_any(VHost, Component, Name, ActingUser) -> end, case lookup(VHost, Component, Name) of not_found -> {error_string, "Parameter does not exist"}; - _ -> mnesia_clear(VHost, Component, Name), - case mnesia:is_transaction() of + _ -> record_clear(VHost, Component, Name), + case is_transaction() of true -> Notify; false -> Notify() end end. +is_transaction() -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> mnesia:is_transaction() end, + fun() -> khepri_tx:is_transaction() end). + +record_clear(Key) -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> mnesia_clear(Key) end, + fun() -> khepri_clear(Key) end). + +record_clear(VHost, Comp, Name) -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> mnesia_clear(VHost, Comp, Name) end, + fun() -> khepri_clear(VHost, Comp, Name) end). + +mnesia_clear(Key) -> + F = fun () -> + ok = mnesia:delete(?TABLE, Key, write) + end, + ok = rabbit_misc:execute_mnesia_transaction(F). + mnesia_clear(VHost, Component, Name) -> F = fun () -> ok = mnesia:delete(?TABLE, {VHost, Component, Name}, write) end, - ok = rabbit_misc:execute_mnesia_transaction(rabbit_vhost:with(VHost, F)). + ok = rabbit_misc:execute_mnesia_transaction( + rabbit_vhost:with_in_mnesia(VHost, F)). + +khepri_clear(Key) -> + Path = khepri_rp_path(Key), + F = fun () -> + {ok, _} = khepri_tx:delete(Path), + ok + end, + ok = rabbit_khepri:transaction(F). + +khepri_clear(VHost, Component, Name) -> + Path = khepri_rp_path({VHost, Component, Name}), + F = fun () -> + {ok, _} = khepri_tx:delete(Path), + ok + end, + ok = rabbit_khepri:transaction( + rabbit_vhost:with_in_khepri(VHost, F)). event_notify(_Event, _VHost, <<"policy">>, _Props) -> ok; @@ -263,9 +335,20 @@ event_notify(Event, VHost, Component, Props) -> -spec list() -> [rabbit_types:infos()]. list() -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> list_in_mnesia() end, + fun() -> list_in_khepri() end). + +list_in_mnesia() -> [p(P) || #runtime_parameters{ key = {_VHost, Comp, _Name}} = P <- rabbit_misc:dirty_read_all(?TABLE), Comp /= <<"policy">>]. +list_in_khepri() -> + Path = khepri_vhost_rp_path(?STAR, ?STAR, ?STAR), + {ok, Map} = rabbit_khepri:match_and_get_data(Path), + [p(P) || #runtime_parameters{ key = {_VHost, Comp, _Name}} = P <- maps:values(Map), + Comp /= <<"policy">>]. + -spec list(rabbit_types:vhost() | '_') -> [rabbit_types:infos()]. list(VHost) -> list(VHost, '_'). @@ -280,6 +363,11 @@ list_component(Component) -> list('_', Component). -> [rabbit_types:infos()]. list(VHost, Component) -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> list_in_mnesia(VHost, Component) end, + fun() -> list_in_khepri(VHost, Component) end). + +list_in_mnesia(VHost, Component) -> mnesia:async_dirty( fun () -> case VHost of @@ -293,7 +381,38 @@ list(VHost, Component) -> Comp =/= <<"policy">> orelse Component =:= <<"policy">>] end). +list_in_khepri('_', Component) -> + list_in_khepri(?STAR, Component); +list_in_khepri(VHost, '_') -> + list_in_khepri(VHost, ?STAR); +list_in_khepri(VHost, Component) -> + Path = khepri_vhost_rp_path(VHost, Component, ?STAR), + rabbit_khepri:transaction( + fun() -> + case VHost of + ?STAR -> ok; + %% Inside of a transaction, using `rabbit_vhost:exists` will cause + %% a deadlock and timeout on the transaction, as it uses `rabbit_khepri:exists`. + %% The `with` function uses the `khepri_tx` API instead + _ -> rabbit_vhost:with_in_khepri(VHost, fun() -> ok end) + end, + case khepri_tx:get(Path) of + {ok, Result} -> + [p(P) || #{data := #runtime_parameters{key = {_VHost, Comp, _Name}}} = #{data := P} <- + maps:values(Result), + Comp =/= <<"policy">> orelse Component =:= <<"policy">>]; + _ -> + [] + end + end, ro). + list_global() -> + %% list only atom keys + rabbit_khepri:try_mnesia_or_khepri( + fun() -> list_global_in_mnesia() end, + fun() -> list_global_in_khepri() end). + +list_global_in_mnesia() -> %% list only atom keys mnesia:async_dirty( fun () -> @@ -302,6 +421,20 @@ list_global() -> is_atom(P#runtime_parameters.key)] end). +list_global_in_khepri() -> + %% list only atom keys + Path = khepri_global_rp_path(?STAR), + rabbit_khepri:transaction( + fun () -> + case khepri_tx:get(Path) of + {ok, Result} -> + [p(P) || #{data := P} <- maps:values(Result), + is_atom(P#runtime_parameters.key)]; + _ -> + [] + end + end, ro). + -spec list_formatted(rabbit_types:vhost()) -> [rabbit_types:infos()]. list_formatted(VHost) -> @@ -378,12 +511,29 @@ value0(Key, Default) -> Params#runtime_parameters.value. lookup0(Key, DefaultFun) -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> lookup0_in_mnesia(Key, DefaultFun) end, + fun() -> lookup0_in_khepri(Key, DefaultFun) end). + +lookup0_in_mnesia(Key, DefaultFun) -> case mnesia:dirty_read(?TABLE, Key) of [] -> DefaultFun(); [R] -> R end. +lookup0_in_khepri(Key, DefaultFun) -> + Path = khepri_rp_path(Key), + case rabbit_khepri:get(Path) of + {ok, #{data := R}} -> R; + {error, {node_not_found, _}} -> DefaultFun() + end. + lookup_missing(Key, Default) -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> lookup_missing_in_mnesia(Key, Default) end, + fun() -> lookup_missing_in_khepri(Key, Default) end). + +lookup_missing_in_mnesia(Key, Default) -> rabbit_misc:execute_mnesia_transaction( fun () -> case mnesia:read(?TABLE, Key, read) of @@ -394,6 +544,20 @@ lookup_missing(Key, Default) -> end end). +lookup_missing_in_khepri(Key, Default) -> + Path = khepri_rp_path(Key), + rabbit_khepri:transaction( + fun () -> + case khepri_tx:get(Path) of + {ok, #{Path := #{data := R}}} -> + R; + {ok, _} -> + Record = c(Key, Default), + khepri_tx:put(Path, Record), + Record + end + end). + c(Key, Default) -> #runtime_parameters{key = Key, value = Default}. @@ -408,6 +572,57 @@ p(#runtime_parameters{key = Key, value = Value}) when is_atom(Key) -> [{name, Key}, {value, Value}]. +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]. + +clear_data_in_khepri() -> + Path = khepri_rp_path(), + case rabbit_khepri:delete(Path) of + {ok, _} -> ok; + Error -> throw(Error) + end. + +mnesia_write_to_khepri( + #runtime_parameters{key = {VHost, Comp, Name}} = RuntimeParam) -> + Path = khepri_vhost_rp_path(VHost, Comp, Name), + case rabbit_khepri:put(Path, RuntimeParam) of + {ok, _} -> ok; + Error -> throw(Error) + end; +mnesia_write_to_khepri( + #runtime_parameters{key = Key} = RuntimeParam) -> + Path = khepri_global_rp_path(Key), + case rabbit_khepri:put(Path, RuntimeParam) of + {ok, _} -> ok; + Error -> throw(Error) + end. + +mnesia_delete_to_khepri( + #runtime_parameters{key = {VHost, Comp, Name}}) -> + Path = khepri_vhost_rp_path(VHost, Comp, Name), + case rabbit_khepri:delete(Path) of + {ok, _} -> ok; + Error -> throw(Error) + end; +mnesia_delete_to_khepri( + #runtime_parameters{key = Key}) -> + Path = khepri_global_rp_path(Key), + case rabbit_khepri:delete(Path) of + {ok, _} -> ok; + Error -> throw(Error) + end. + -spec info_keys() -> rabbit_types:info_keys(). info_keys() -> [component, name, value]. diff --git a/deps/rabbit/src/rabbit_table.erl b/deps/rabbit/src/rabbit_table.erl index c6a439eb1408..acc9a7e00111 100644 --- a/deps/rabbit/src/rabbit_table.erl +++ b/deps/rabbit/src/rabbit_table.erl @@ -14,11 +14,14 @@ check_schema_integrity/1, clear_ram_only_tables/0, retry_timeout/0, wait_for_replicated/0, exists/1]). -%% for testing purposes -export([definitions/0]). -include_lib("rabbit_common/include/rabbit.hrl"). +-ifdef(TEST). +-export([pre_khepri_definitions/0]). +-endif. + %%---------------------------------------------------------------------------- -type retry() :: boolean(). @@ -156,8 +159,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_vhost:khepri_vhosts_path(), + rabbit_auth_backend_internal: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, @@ -295,32 +318,8 @@ definitions(ram) -> {Tab, TabDef} <- definitions()]. definitions() -> - [{rabbit_user, - [{record_name, internal_user}, - {attributes, internal_user:fields()}, - {disc_copies, [node()]}, - {match, internal_user:pattern_match_all()}]}, - {rabbit_user_permission, - [{record_name, user_permission}, - {attributes, record_info(fields, user_permission)}, - {disc_copies, [node()]}, - {match, #user_permission{user_vhost = #user_vhost{_='_'}, - permission = #permission{_='_'}, - _='_'}}]}, - {rabbit_topic_permission, - [{record_name, topic_permission}, - {attributes, record_info(fields, topic_permission)}, - {disc_copies, [node()]}, - {match, #topic_permission{topic_permission_key = #topic_permission_key{_='_'}, - permission = #permission{_='_'}, - _='_'}}]}, - {rabbit_vhost, - [ - {record_name, vhost}, - {attributes, vhost:fields()}, - {disc_copies, [node()]}, - {match, vhost:pattern_match_all()}]}, - {rabbit_listener, + PreKhepriDefs = pre_khepri_definitions(), + [{rabbit_listener, [{record_name, listener}, {attributes, record_info(fields, listener)}, {type, bag}, @@ -375,11 +374,6 @@ definitions() -> [{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()}, @@ -390,9 +384,43 @@ definitions() -> {attributes, amqqueue:fields()}, {match, amqqueue:pattern_match_on_name(queue_name_match())}]} ] + ++ PreKhepriDefs ++ gm:table_definitions() ++ mirrored_supervisor:table_definitions(). +pre_khepri_definitions() -> + [{rabbit_user, + [{record_name, internal_user}, + {attributes, internal_user:fields()}, + {disc_copies, [node()]}, + {match, internal_user:pattern_match_all()}]}, + {rabbit_user_permission, + [{record_name, user_permission}, + {attributes, record_info(fields, user_permission)}, + {disc_copies, [node()]}, + {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)}, + {disc_copies, [node()]}, + {match, #topic_permission{topic_permission_key = #topic_permission_key{_='_'}, + permission = #permission{_='_'}, + _='_'}}]}, + {rabbit_vhost, + [ + {record_name, vhost}, + {attributes, vhost:fields()}, + {disc_copies, [node()]}, + {match, vhost:pattern_match_all()}]} + ]. + binding_match() -> #binding{source = exchange_name_match(), destination = binding_destination_match(), diff --git a/deps/rabbit/src/rabbit_vhost.erl b/deps/rabbit/src/rabbit_vhost.erl index ec8dc11cf0a5..bc084cb18e3d 100644 --- a/deps/rabbit/src/rabbit_vhost.erl +++ b/deps/rabbit/src/rabbit_vhost.erl @@ -8,10 +8,16 @@ -module(rabbit_vhost). -include_lib("rabbit_common/include/rabbit.hrl"). + +-include_lib("khepri/include/khepri.hrl"). + -include("vhost.hrl"). -export([recover/0, recover/1, read_config/1]). --export([add/2, add/4, delete/2, exists/1, with/2, with_user_and_vhost/3, assert/1, update/2, +-export([add/2, add/4, delete/2, exists/1, + with/2, with_in_mnesia/2, with_in_khepri/2, + with_user_and_vhost/3, with_user_and_vhost_in_mnesia/3, with_user_and_vhost_in_khepri/3, + assert/1, update/2, set_limits/2, vhost_cluster_state/1, is_running_on_all_nodes/1, await_running_on_all_nodes/2, list/0, count/0, list_names/0, all/0, all_tagged_with/1]). -export([parse_tags/1, update_metadata/2, tag_with/2, untag_from/2, update_tags/2, update_tags/3]). @@ -21,6 +27,34 @@ -export([delete_storage/1]). -export([vhost_down/1]). -export([put_vhost/5]). +-export([clear_data_in_khepri/0, + mnesia_write_to_khepri/1, + mnesia_delete_to_khepri/1]). +-export([khepri_vhosts_path/0, + khepri_vhost_path/1]). + +-ifdef(TEST). +-export([do_add_to_mnesia/3, + do_add_to_khepri/3, + lookup_in_mnesia/1, + lookup_in_khepri/1, + exists_in_mnesia/1, + exists_in_khepri/1, + list_names_in_mnesia/0, + list_names_in_khepri/0, + all_in_mnesia/0, + all_in_khepri/0, + update_in_mnesia/2, + update_in_khepri/2, + update_in_mnesia/3, + update_in_khepri/3, + info_in_mnesia/1, + info_in_khepri/1, + clear_permissions_in_mnesia/2, + clear_permissions_in_khepri/2, + internal_delete_in_mnesia/1, + internal_delete_in_khepri/1]). +-endif. %% %% API @@ -161,38 +195,10 @@ do_add(Name, Description, Tags, ActingUser) -> Value -> rabbit_log:info("Adding vhost '~s' (description: '~s', tags: ~p)", [Name, Value, Tags]) end, - VHost = rabbit_misc:execute_mnesia_transaction( - fun () -> - case mnesia:wread({rabbit_vhost, Name}) of - [] -> - Row = vhost:new(Name, [], #{description => Description, tags => Tags}), - rabbit_log:debug("Inserting a virtual host record ~p", [Row]), - ok = mnesia:write(rabbit_vhost, Row, write), - Row; - %% the vhost already exists - [Row] -> - Row - end - end, - fun (VHost1, true) -> - VHost1; - (VHost1, false) -> - [begin - Resource = rabbit_misc:r(Name, exchange, ExchangeName), - rabbit_log:debug("Will declare an exchange ~p", [Resource]), - _ = rabbit_exchange:declare(Resource, Type, true, false, Internal, [], ActingUser) - end || {ExchangeName, Type, Internal} <- - [{<<"">>, direct, false}, - {<<"amq.direct">>, direct, false}, - {<<"amq.topic">>, topic, false}, - %% per 0-9-1 pdf - {<<"amq.match">>, headers, false}, - %% per 0-9-1 xml - {<<"amq.headers">>, headers, false}, - {<<"amq.fanout">>, fanout, false}, - {<<"amq.rabbitmq.trace">>, topic, true}]], - VHost1 - end), + VHost = rabbit_khepri:try_mnesia_or_khepri( + fun() -> do_add_to_mnesia(Name, Description, Tags) end, + fun() -> do_add_to_khepri(Name, Description, Tags) end), + declare_default_exchanges(Name, ActingUser), case rabbit_vhost_sup_sup:start_on_all_nodes(Name) of ok -> rabbit_event:notify(vhost_created, info(VHost) @@ -206,8 +212,71 @@ do_add(Name, Description, Tags, ActingUser) -> {error, Msg} end. +do_add_to_mnesia(Name, Description, Tags) -> + rabbit_misc:execute_mnesia_transaction( + fun () -> + case mnesia:wread({rabbit_vhost, Name}) of + [] -> + Row = vhost:new(Name, [], #{description => Description, tags => Tags}), + rabbit_log:debug("Inserting a virtual host record ~p", [Row]), + ok = mnesia:write(rabbit_vhost, Row, write), + Row; + %% the vhost already exists + [Row] -> + Row + end + end). + +do_add_to_khepri(Name, Description, Tags) -> + Path = khepri_vhost_path(Name), + NewVHost = vhost:new( + Name, [], #{description => Description, tags => Tags}), + case rabbit_khepri:create(Path, NewVHost) of + {ok, _} -> + NewVHost; + {error, {mismatching_node, + #{node_path := Path, + node_props := #{data := ExistingVHost}}}} -> + ExistingVHost; + Error -> + throw(Error) + end. + +declare_default_exchanges(Name, ActingUser) -> + DefaultExchanges = [{<<"">>, direct, false}, + {<<"amq.direct">>, direct, false}, + {<<"amq.topic">>, topic, false}, + %% per 0-9-1 pdf + {<<"amq.match">>, headers, false}, + %% per 0-9-1 xml + {<<"amq.headers">>, headers, false}, + {<<"amq.fanout">>, fanout, false}, + {<<"amq.rabbitmq.trace">>, topic, true}], + lists:foreach( + fun({ExchangeName, Type, Internal}) -> + Resource = rabbit_misc:r(Name, exchange, ExchangeName), + rabbit_log:debug("Will declare an exchange ~p", [Resource]), + _ = rabbit_exchange:declare( + Resource, Type, true, false, Internal, [], ActingUser) + end, DefaultExchanges). + -spec update(vhost:name(), binary(), [atom()], rabbit_types:username()) -> rabbit_types:ok_or_error(any()). update(Name, Description, Tags, ActingUser) -> + Ret = rabbit_khepri:try_mnesia_or_khepri( + fun() -> update_in_mnesia(Name, Description, Tags) end, + fun() -> update_in_khepri(Name, Description, Tags) end), + case Ret of + {ok, VHost} -> + rabbit_event:notify(vhost_updated, info(VHost) + ++ [{user_who_performed_action, ActingUser}, + {description, Description}, + {tags, Tags}]), + ok; + _ -> + Ret + end. + +update_in_mnesia(Name, Description, Tags) -> rabbit_misc:execute_mnesia_transaction( fun () -> case mnesia:wread({rabbit_vhost, Name}) of @@ -217,14 +286,34 @@ update(Name, Description, Tags, ActingUser) -> VHost = vhost:merge_metadata(VHost0, #{description => Description, tags => Tags}), rabbit_log:debug("Updating a virtual host record ~p", [VHost]), ok = mnesia:write(rabbit_vhost, VHost, write), - rabbit_event:notify(vhost_updated, info(VHost) - ++ [{user_who_performed_action, ActingUser}, - {description, Description}, - {tags, Tags}]), - ok + {ok, VHost} end end). +update_in_khepri(Name, Description, Tags) -> + Path = khepri_vhost_path(Name), + Ret1 = rabbit_khepri:get(Path), + case Ret1 of + {ok, #{data := VHost0, payload_version := DVersion}} -> + VHost = vhost:merge_metadata( + VHost0, #{description => Description, tags => Tags}), + 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, {mismatching_node, #{node_path := Path}}} -> + update_in_khepri(Name, Description, Tags); + {error, _} = Error -> + Error + end; + {error, {node_not_found, _}} -> + {error, {no_such_vhost, Name}}; + {error, _} = Error -> + Error + end. -spec delete(vhost:name(), rabbit_types:username()) -> rabbit_types:ok_or_error(any()). @@ -236,8 +325,16 @@ delete(VHost, ActingUser) -> %% notifications which must be sent outside the TX rabbit_log:info("Deleting vhost '~s'", [VHost]), %% Clear the permissions first to prohibit new incoming connections when deleting a vhost - rabbit_misc:execute_mnesia_transaction( - with(VHost, fun () -> clear_permissions(VHost, ActingUser) end)), + rabbit_khepri:try_mnesia_or_khepri( + fun() -> + rabbit_misc:execute_mnesia_transaction( + with_in_mnesia( + VHost, + fun() -> clear_permissions_in_mnesia(VHost, ActingUser) end)) + end, + fun() -> + ok = clear_permissions_in_khepri(VHost, ActingUser) + end), QDelFun = fun (Q) -> rabbit_amqqueue:delete(Q, false, false, ActingUser) end, [begin Name = amqqueue:get_name(Q), @@ -245,8 +342,13 @@ delete(VHost, ActingUser) -> end || Q <- rabbit_amqqueue:list(VHost)], [assert_benign(rabbit_exchange:delete(Name, false, ActingUser), ActingUser) || #exchange{name = Name} <- rabbit_exchange:list(VHost)], - Funs = rabbit_misc:execute_mnesia_transaction( - with(VHost, fun () -> internal_delete(VHost, ActingUser) end)), + With = with(VHost, fun () -> internal_delete(VHost, ActingUser) end), + Funs = rabbit_khepri:try_mnesia_or_khepri( + fun() -> rabbit_misc:execute_mnesia_transaction(With) end, + %% FIXME: Do we need the atomicity? Currently we can't use a + %% transaction because of the many side effects here and there in + %% other modules. + fun() -> internal_delete(VHost, ActingUser) end), ok = rabbit_event:notify(vhost_deleted, [{name, VHost}, {user_who_performed_action, ActingUser}]), [case Fun() of @@ -382,23 +484,76 @@ internal_delete(VHost, ActingUser) -> || Info <- rabbit_runtime_parameters:list(VHost)], Fs2 = [rabbit_policy:delete(VHost, proplists:get_value(name, Info), ActingUser) || Info <- rabbit_policy:list(VHost)], - ok = mnesia:delete({rabbit_vhost, VHost}), + _ = rabbit_khepri:try_mnesia_or_khepri( + fun() -> internal_delete_in_mnesia(VHost) end, + fun() -> internal_delete_in_khepri(VHost) end), Fs1 ++ Fs2. +internal_delete_in_mnesia(VHost) -> + ok = mnesia:delete({rabbit_vhost, VHost}), + ok. + +internal_delete_in_khepri(VHost) -> + Path = khepri_vhost_path(VHost), + {ok, Result} = rabbit_khepri:delete(Path), + %% We reproduce the behavior of `with(...)' here without using it directly + %% (because it expects to run inside a transaction). + %% + %% So if the vhost didn't exist before deletion, we throw an exception. + case Result =:= #{} of + false -> ok; + true -> throw({error, {no_such_vhost, VHost}}) + end. + -spec exists(vhost:name()) -> boolean(). exists(VHost) -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> exists_in_mnesia(VHost) end, + fun() -> exists_in_khepri(VHost) end). + +exists_in_mnesia(VHost) -> mnesia:dirty_read({rabbit_vhost, VHost}) /= []. +exists_in_khepri(VHost) -> + Path = khepri_vhost_path(VHost), + rabbit_khepri:exists(Path). + -spec list_names() -> [vhost:name()]. -list_names() -> mnesia:dirty_all_keys(rabbit_vhost). +list_names() -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> list_names_in_mnesia() end, + fun() -> list_names_in_khepri() end). + +list_names_in_mnesia() -> + mnesia:dirty_all_keys(rabbit_vhost). + +list_names_in_khepri() -> + Path = khepri_vhosts_path(), + case rabbit_khepri:list_child_nodes(Path) of + {ok, Result} -> Result; + _ -> [] + end. %% Exists for backwards compatibility, prefer list_names/0. -spec list() -> [vhost:name()]. list() -> list_names(). -spec all() -> [vhost:vhost()]. -all() -> mnesia:dirty_match_object(rabbit_vhost, vhost:pattern_match_all()). +all() -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> all_in_mnesia() end, + fun() -> all_in_khepri() end). + +all_in_mnesia() -> + mnesia:dirty_match_object(rabbit_vhost, vhost:pattern_match_all()). + +all_in_khepri() -> + Path = khepri_vhosts_path(), + case rabbit_khepri:list_child_data(Path) of + {ok, VHosts} -> maps:values(VHosts); + _ -> [] + end. -spec all_tagged_with(atom()) -> [vhost:vhost()]. all_tagged_with(TagName) -> @@ -418,23 +573,65 @@ count() -> -spec lookup(vhost:name()) -> vhost:vhost() | rabbit_types:ok_or_error(any()). lookup(VHostName) -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> lookup_in_mnesia(VHostName) end, + fun() -> lookup_in_khepri(VHostName) end). + +lookup_in_mnesia(VHostName) -> case rabbit_misc:dirty_read({rabbit_vhost, VHostName}) of {error, not_found} -> {error, {no_such_vhost, VHostName}}; {ok, Record} -> Record end. +lookup_in_khepri(VHostName) -> + Path = khepri_vhost_path(VHostName), + case rabbit_khepri:get_data(Path) of + {ok, Record} -> Record; + _ -> {error, {no_such_vhost, VHostName}} + end. + -spec with(vhost:name(), rabbit_misc:thunk(A)) -> A. with(VHostName, Thunk) -> - fun () -> - case mnesia:read({rabbit_vhost, VHostName}) of - [] -> mnesia:abort({no_such_vhost, VHostName}); - [_V] -> Thunk() - end + fun() -> + rabbit_khepri:try_mnesia_or_khepri( + with_in_mnesia(VHostName, Thunk), + with_in_khepri(VHostName, Thunk)) + end. + +with_in_mnesia(VHostName, Thunk) -> + fun() -> + case mnesia:read({rabbit_vhost, VHostName}) of + [] -> mnesia:abort({no_such_vhost, VHostName}); + [_V] -> Thunk() + end + end. + +with_in_khepri(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. -spec with_user_and_vhost(rabbit_types:username(), vhost:name(), rabbit_misc:thunk(A)) -> A. with_user_and_vhost(Username, VHostName, Thunk) -> - rabbit_misc:with_user(Username, with(VHostName, Thunk)). + fun() -> + rabbit_khepri:try_mnesia_or_khepri( + rabbit_auth_backend_internal:with_user_in_mnesia( + Username, with_in_mnesia(VHostName, Thunk)), + rabbit_auth_backend_internal:with_user_in_khepri( + Username, with_in_khepri(VHostName, Thunk))) + end. + +with_user_and_vhost_in_mnesia(Username, VHostName, Thunk) -> + rabbit_auth_backend_internal:with_user_in_mnesia( + Username, with_in_mnesia(VHostName, Thunk)). + +with_user_and_vhost_in_khepri(Username, VHostName, Thunk) -> + rabbit_auth_backend_internal:with_user_in_khepri( + Username, with_in_khepri(VHostName, Thunk)). %% Like with/2 but outside an Mnesia tx @@ -447,6 +644,11 @@ assert(VHostName) -> -spec update(vhost:name(), fun((vhost:vhost()) -> vhost:vhost())) -> vhost:vhost(). update(VHostName, Fun) -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> update_in_mnesia(VHostName, Fun) end, + fun() -> update_in_khepri(VHostName, Fun) end). + +update_in_mnesia(VHostName, Fun) -> case mnesia:read({rabbit_vhost, VHostName}) of [] -> mnesia:abort({no_such_vhost, VHostName}); @@ -456,6 +658,27 @@ update(VHostName, Fun) -> V1 end. +update_in_khepri(VHostName, Fun) -> + Path = khepri_vhost_path(VHostName), + case rabbit_khepri:get(Path) of + {ok, #{data := V, payload_version := DVersion}} -> + V1 = Fun(V), + Path1 = khepri_path:combine_with_conditions( + Path, [#if_payload_version{version = DVersion}]), + case rabbit_khepri:put(Path1, V1) of + {ok, _} -> + V1; + {error, {mismatching_node, _}} -> + update(VHostName, Fun); + Error -> + throw(Error) + end; + {error, {node_not_found, _}} -> + throw({error, {no_such_vhost, VHostName}}); + Error -> + throw(Error) + end. + -spec update_metadata(vhost:name(), fun((map())-> map())) -> vhost:vhost() | rabbit_types:ok_or_error(any()). update_metadata(VHostName, Fun) -> update(VHostName, fun(Record) -> @@ -466,10 +689,11 @@ update_metadata(VHostName, Fun) -> -spec update_tags(vhost:name(), [vhost_tag()], rabbit_types:username()) -> vhost:vhost() | rabbit_types:ok_or_error(any()). update_tags(VHostName, Tags, ActingUser) -> ConvertedTags = [rabbit_data_coercion:to_atom(I) || I <- Tags], + Fun = fun() -> update_tags(VHostName, ConvertedTags) end, try - R = rabbit_misc:execute_mnesia_transaction(fun() -> - update_tags(VHostName, ConvertedTags) - end), + R = rabbit_khepri:try_mnesia_or_khepri( + fun() -> rabbit_misc:execute_mnesia_transaction(Fun) end, + fun() -> Fun() end), rabbit_log:info("Successfully set tags for virtual host '~s' to ~p", [VHostName, ConvertedTags]), rabbit_event:notify(vhost_tags_set, [{name, VHostName}, {tags, ConvertedTags}, @@ -556,11 +780,23 @@ i(Item, VHost) -> info(VHost) when ?is_vhost(VHost) -> infos(?INFO_KEYS, VHost); info(Key) -> + rabbit_khepri:try_mnesia_or_khepri( + fun() -> info_in_mnesia(Key) end, + fun() -> info_in_khepri(Key) end). + +info_in_mnesia(Key) -> case mnesia:dirty_read({rabbit_vhost, Key}) of [] -> []; [VHost] -> infos(?INFO_KEYS, VHost) end. +info_in_khepri(Key) -> + Path = khepri_vhost_path(Key), + case rabbit_khepri:get_data(Path) of + {ok, VHost} -> infos(?INFO_KEYS, VHost); + _ -> [] + end. + -spec info(vhost:vhost(), rabbit_types:info_keys()) -> rabbit_types:infos(). info(VHost, Items) -> infos(Items, VHost). @@ -577,12 +813,45 @@ info_all(Items, Ref, AggregatorPid) -> rabbit_control_misc:emitting_map( AggregatorPid, Ref, fun(VHost) -> info(VHost, Items) end, all()). - -clear_permissions(VHost, ActingUser) -> +clear_permissions_in_mnesia(VHost, ActingUser) -> [ok = rabbit_auth_backend_internal:clear_permissions( proplists:get_value(user, Info), VHost, ActingUser) - || Info <- rabbit_auth_backend_internal:list_vhost_permissions(VHost)], - TopicPermissions = rabbit_auth_backend_internal:list_vhost_topic_permissions(VHost), + || Info <- + rabbit_auth_backend_internal:list_vhost_permissions(VHost)], + TopicPermissions = + rabbit_auth_backend_internal:list_vhost_topic_permissions(VHost), [ok = rabbit_auth_backend_internal:clear_topic_permissions( proplists:get_value(user, TopicPermission), VHost, ActingUser) || TopicPermission <- TopicPermissions]. + +clear_permissions_in_khepri(VHost, ActingUser) -> + ok = rabbit_auth_backend_internal:clear_vhost_permissions_in_khepri( + VHost, ActingUser), + ok = rabbit_auth_backend_internal:clear_vhost_topic_permissions_in_khepri( + VHost, ActingUser). + +clear_data_in_khepri() -> + Path = khepri_vhosts_path(), + case rabbit_khepri:delete(Path) of + {ok, _} -> ok; + Error -> throw(Error) + end. + +mnesia_write_to_khepri(VHost) when ?is_vhost(VHost) -> + Name = vhost:get_name(VHost), + Path = khepri_vhost_path(Name), + case rabbit_khepri:put(Path, VHost) of + {ok, _} -> ok; + Error -> throw(Error) + end. + +mnesia_delete_to_khepri(VHost) when ?is_vhost(VHost) -> + Name = vhost:get_name(VHost), + Path = khepri_vhost_path(Name), + case rabbit_khepri:delete(Path) of + {ok, _} -> ok; + Error -> throw(Error) + end. + +khepri_vhosts_path() -> [?MODULE]. +khepri_vhost_path(VHost) -> [?MODULE, VHost]. diff --git a/deps/rabbit/test/metadata_store_phase1_SUITE.erl b/deps/rabbit/test/metadata_store_phase1_SUITE.erl new file mode 100644 index 000000000000..0cf891e82012 --- /dev/null +++ b/deps/rabbit/test/metadata_store_phase1_SUITE.erl @@ -0,0 +1,2806 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved. +%% + +-module(metadata_store_phase1_SUITE). + +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-include_lib("khepri/include/khepri.hrl"). + +-include_lib("rabbit_common/include/rabbit.hrl"). + +-export([suite/0, + all/0, + groups/0, + init_per_suite/1, end_per_suite/1, + init_per_group/2, end_per_group/2, + init_per_testcase/2, end_per_testcase/2, + + write_non_existing_vhost/1, + write_existing_vhost/1, + check_vhost_exists/1, + get_existing_vhost_info/1, + list_vhost_names/1, + list_vhost_objects/1, + update_non_existing_vhost/1, + update_existing_vhost/1, + update_non_existing_vhost_desc_and_tags/1, + update_existing_vhost_desc_and_tags/1, + delete_non_existing_vhost/1, + delete_existing_vhost/1, + + write_non_existing_user/1, + write_existing_user/1, + list_users/1, + update_non_existing_user/1, + update_existing_user/1, + delete_non_existing_user/1, + delete_existing_user/1, + + write_user_permission_for_non_existing_vhost/1, + write_user_permission_for_non_existing_user/1, + write_user_permission_for_existing_user/1, + check_resource_access/1, + list_user_permissions_on_non_existing_vhost/1, + list_user_permissions_for_non_existing_user/1, + list_user_permissions/1, + clear_user_permission_for_non_existing_vhost/1, + clear_user_permission_for_non_existing_user/1, + clear_user_permission/1, + delete_user_and_check_resource_access/1, + delete_vhost_and_check_resource_access/1, + + write_topic_permission_for_non_existing_vhost/1, + write_topic_permission_for_non_existing_user/1, + write_topic_permission_for_existing_user/1, + list_topic_permissions_on_non_existing_vhost/1, + list_topic_permissions_for_non_existing_user/1, + list_topic_permissions/1, + clear_specific_topic_permission_for_non_existing_vhost/1, + clear_specific_topic_permission_for_non_existing_user/1, + clear_specific_topic_permission/1, + clear_all_topic_permission_for_non_existing_vhost/1, + clear_all_topic_permission_for_non_existing_user/1, + clear_all_topic_permissions/1, + delete_user_and_check_topic_access/1, + delete_vhost_and_check_topic_access/1 + ]). + +suite() -> + [{timetrap, {minutes, 1}}]. + +all() -> + [ + {group, vhosts}, + {group, internal_users} + ]. + +groups() -> + [ + {vhosts, [], + [ + write_non_existing_vhost, + write_existing_vhost, + check_vhost_exists, + get_existing_vhost_info, + list_vhost_names, + list_vhost_objects, + update_non_existing_vhost, + update_existing_vhost, + update_non_existing_vhost_desc_and_tags, + update_existing_vhost_desc_and_tags, + delete_non_existing_vhost, + delete_existing_vhost + ] + }, + {internal_users, [], + [ + {users, [], + [ + write_non_existing_user, + write_existing_user, + list_users, + update_non_existing_user, + update_existing_user, + delete_non_existing_user, + delete_existing_user + ] + }, + {user_permissions, [], + [ + write_user_permission_for_non_existing_vhost, + write_user_permission_for_non_existing_user, + write_user_permission_for_existing_user, + check_resource_access, + list_user_permissions_on_non_existing_vhost, + list_user_permissions_for_non_existing_user, + list_user_permissions, + clear_user_permission_for_non_existing_vhost, + clear_user_permission_for_non_existing_user, + clear_user_permission, + delete_user_and_check_resource_access, + delete_vhost_and_check_resource_access + ] + }, + {topic_permissions, [], + [ + write_topic_permission_for_non_existing_vhost, + write_topic_permission_for_non_existing_user, + write_topic_permission_for_existing_user, + list_topic_permissions_on_non_existing_vhost, + list_topic_permissions_for_non_existing_user, + list_topic_permissions, + clear_specific_topic_permission_for_non_existing_vhost, + clear_specific_topic_permission_for_non_existing_user, + clear_specific_topic_permission, + clear_all_topic_permission_for_non_existing_vhost, + clear_all_topic_permission_for_non_existing_user, + clear_all_topic_permissions, + delete_user_and_check_topic_access, + delete_vhost_and_check_topic_access + ] + } + ] + } + ]. + +%% ------------------------------------------------------------------- +%% Testsuite setup/teardown. +%% ------------------------------------------------------------------- + +init_per_suite(Config) -> + rabbit_ct_helpers:run_setup_steps( + Config, + [ + fun init_feature_flags/1, + fun setup_code_mocking/1, + fun setup_mnesia/1, + fun setup_khepri/1 + ]). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps( + Config, + [ + fun remove_code_mocking/1 + ]). + +setup_mnesia(Config) -> + %% Configure Mnesia directory in the common_test priv_dir and start it. + MnesiaDir = filename:join( + ?config(priv_dir, Config), + "mnesia"), + ct:pal("Mnesia directory: ~ts", [MnesiaDir]), + ok = file:make_dir(MnesiaDir), + ok = application:load(mnesia), + ok = application:set_env(mnesia, dir, MnesiaDir), + ok = mnesia:create_schema([node()]), + {ok, _} = application:ensure_all_started(mnesia), + + ct:pal("Mnesia info below:"), + mnesia:info(), + Config. + +setup_khepri(Config) -> + %% Start Khepri. + {ok, _} = application:ensure_all_started(khepri), + + %% Configure Khepri. It takes care of configuring Ra system & cluster. It + %% uses the Mnesia directory to store files. + ok = rabbit_khepri:setup(undefined), + + ct:pal("Khepri info below:"), + rabbit_khepri:info(), + Config. + +setup_code_mocking(Config) -> + %% Bypass rabbit_misc:execute_mnesia_transaction/1 (no worker_pool + %% configured in particular) but keep the behavior of throwing the error. + meck:new(rabbit_misc, [passthrough, no_link]), + meck:expect( + rabbit_misc, execute_mnesia_transaction, + fun(Fun) -> + case mnesia:sync_transaction(Fun) of + {atomic, Result} -> Result; + {aborted, Reason} -> throw({error, Reason}) + end + end), + ?assert(meck:validate(rabbit_misc)), + + %% Bypass calls inside rabbit_vhost:vhost_cluster_state/1 because these + %% are unit testcases without any sort of clustering. + meck:new(rabbit_nodes, [passthrough, no_link]), + meck:expect( + rabbit_nodes, all_running, + fun() -> [node()] end), + + meck:new(rabbit_vhost_sup_sup, [passthrough, no_link]), + meck:expect( + rabbit_vhost_sup_sup, is_vhost_alive, + fun(_) -> true end), + + %% We ensure that we use the `vhost_v2` #vhost{} record so we can play + %% with the description and tags. + meck:new(rabbit_feature_flags, [passthrough, no_link]), + meck:expect( + rabbit_feature_flags, is_enabled, + fun + (virtual_host_metadata) -> true; + (FeatureNames) -> meck:passthrough([FeatureNames]) + end), + + ct:pal("Mocked: ~p", [meck:mocked()]), + Config. + +remove_code_mocking(Config) -> + lists:foreach( + fun(Mod) -> meck:unload(Mod) end, + meck:mocked()), + Config. + +init_per_group(_, Config) -> + Config. + +end_per_group(_, Config) -> + Config. + +init_per_testcase(Testcase, Config) -> + rabbit_ct_helpers:testcase_started(Config, Testcase), + + rabbit_khepri:clear_forced_metadata_store(), + + %% Create Mnesia tables. + TableDefs = rabbit_table:pre_khepri_definitions(), + lists:foreach( + fun ({Table, Def}) -> ok = rabbit_table:create(Table, Def) end, + TableDefs), + + Config. + +end_per_testcase(Testcase, Config) -> + rabbit_khepri:clear_forced_metadata_store(), + + %% Delete Mnesia tables to clear any data. + TableDefs = rabbit_table:pre_khepri_definitions(), + lists:foreach( + fun ({Table, _}) -> {atomic, ok} = mnesia:delete_table(Table) end, + TableDefs), + + %% Clear all data in Khepri. + {ok, _} = rabbit_khepri:clear_store(), + + rabbit_ct_helpers:testcase_finished(Config, Testcase). + +init_feature_flags(Config) -> + FFFile = filename:join( + ?config(priv_dir, Config), + "feature_flags"), + ct:pal("Feature flags file: ~ts", [FFFile]), + ok = application:load(rabbit), + ok = application:set_env(rabbit, feature_flags_file, FFFile), + Config. + +%% ------------------------------------------------------------------- +%% Testcases. +%% ------------------------------------------------------------------- + +%% We use `_With' (with the leading underscore) on purpose: we don't know if +%% the code in `T' will use it. That code can still use `_With' of course. +%% This simply avoids compiler warnings. +-define(with(T), fun(_With) -> T end). + +-define(vhost_path(V), + [rabbit_vhost, V]). +-define(user_path(U), + [rabbit_auth_backend_internal, users, U]). +-define(user_perm_path(U, V), + [rabbit_auth_backend_internal, users, U, user_permissions, V]). +-define(topic_perm_path(U, V, E), + [rabbit_auth_backend_internal, users, U, topic_permissions, V, E]). + +%% +%% Virtual hosts. +%% + +write_non_existing_vhost(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + + Tests = + [ + ?with(?assertEqual( + {error, {no_such_vhost, VHostName}}, + lookup_vhost(_With, VHostName))), + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + VHost, + lookup_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostName) => VHost}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +write_existing_vhost(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + + Tests = + [ + ?with(?assertEqual( + {error, {no_such_vhost, VHostName}}, + lookup_vhost(_With, VHostName))), + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + VHost, + lookup_vhost(_With, VHostName))), + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + VHost, + lookup_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostName) => VHost}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +check_vhost_exists(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assert( + vhost_exists(_With, VHostName))), + ?with(?assertNot( + vhost_exists(_With, <<"non-existing-vhost">>))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostName) => VHost}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +get_existing_vhost_info(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + [{name, VHostName}, + {description, VHostDesc}, + {tags, VHostTags}, + {metadata, #{description => VHostDesc, + tags => VHostTags}}, + {tracing, false}, + {cluster_state, [{node(), running}]}], + vhost_info(_With, VHostName))) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_vhost_names(_) -> + VHostNameA = <<"vhost-a">>, + VHostDescA = <<>>, + VHostTagsA = [], + VHostA = vhost:new( + VHostNameA, + VHostTagsA, + #{description => VHostDescA, + tags => VHostTagsA}), + VHostNameB = <<"vhost-b">>, + VHostDescB = <<>>, + VHostTagsB = [], + VHostB = vhost:new( + VHostNameB, + VHostTagsB, + #{description => VHostDescB, + tags => VHostTagsB}), + + Tests = + [ + ?with(?assertEqual( + VHostA, + add_vhost(_With, VHostNameA, VHostDescA, VHostTagsA))), + ?with(?assertEqual( + VHostB, + add_vhost(_With, VHostNameB, VHostDescB, VHostTagsB))), + ?with(?assertEqual( + [VHostNameA, VHostNameB], + list_vhosts(_With))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHostA, VHostB]}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostNameA) => VHostA, + ?vhost_path(VHostNameB) => VHostB}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_vhost_objects(_) -> + VHostNameA = <<"vhost-a">>, + VHostDescA = <<>>, + VHostTagsA = [], + VHostA = vhost:new( + VHostNameA, + VHostTagsA, + #{description => VHostDescA, + tags => VHostTagsA}), + VHostNameB = <<"vhost-b">>, + VHostDescB = <<>>, + VHostTagsB = [], + VHostB = vhost:new( + VHostNameB, + VHostTagsB, + #{description => VHostDescB, + tags => VHostTagsB}), + + Tests = + [ + ?with(?assertEqual( + VHostA, + add_vhost(_With, VHostNameA, VHostDescA, VHostTagsA))), + ?with(?assertEqual( + VHostB, + add_vhost(_With, VHostNameB, VHostDescB, VHostTagsB))), + ?with(?assertEqual( + [VHostA, VHostB], + list_vhost_records(_With))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHostA, VHostB]}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostNameA) => VHostA, + ?vhost_path(VHostNameB) => VHostB}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +update_non_existing_vhost(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + UpdatedVHost = vhost:set_limits(VHost, [limits]), + Fun = fun(_) -> UpdatedVHost end, + ?assertNotEqual(VHost, UpdatedVHost), + + Tests = + [ + ?with(?assertEqual( + {error, {no_such_vhost, VHostName}}, + lookup_vhost(_With, VHostName))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + update_vhost(_With, VHostName, Fun))), + ?with(?assertEqual( + {error, {no_such_vhost, VHostName}}, + lookup_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {khepri, [rabbit_vhost], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +update_existing_vhost(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + UpdatedVHost = vhost:set_limits(VHost, [limits]), + Fun = fun(_) -> UpdatedVHost end, + ?assertNotEqual(VHost, UpdatedVHost), + + Tests = + [ + ?with(?assertEqual( + {error, {no_such_vhost, VHostName}}, + lookup_vhost(_With, VHostName))), + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + UpdatedVHost, + update_vhost(_With, VHostName, Fun))), + ?with(?assertEqual( + UpdatedVHost, + lookup_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [UpdatedVHost]}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostName) => UpdatedVHost}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +update_non_existing_vhost_desc_and_tags(_) -> + VHostName = <<"vhost">>, + NewVHostDesc = <<"New desc">>, + NewVHostTags = [new_tag], + + Tests = + [ + ?with(?assertEqual( + {error, {no_such_vhost, VHostName}}, + lookup_vhost(_With, VHostName))), + ?with(?assertEqual( + {error, {no_such_vhost, VHostName}}, + update_vhost(_With, VHostName, NewVHostDesc, NewVHostTags))), + ?with(?assertEqual( + {error, {no_such_vhost, VHostName}}, + lookup_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {khepri, [rabbit_vhost], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +update_existing_vhost_desc_and_tags(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + NewVHostDesc = <<"New desc">>, + NewVHostTags = [new_tag], + UpdatedVHost = vhost:set_metadata( + VHost, + #{description => NewVHostDesc, + tags => NewVHostTags}), + ct:pal("VHost: ~p~nUpdatedVHost: ~p", [VHost, UpdatedVHost]), + ?assertNotEqual(VHost, UpdatedVHost), + + Tests = + [ + ?with(?assertEqual( + {error, {no_such_vhost, VHostName}}, + lookup_vhost(_With, VHostName))), + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + {ok, UpdatedVHost}, + update_vhost(_With, VHostName, NewVHostDesc, NewVHostTags))), + ?with(?assertEqual( + UpdatedVHost, + lookup_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [UpdatedVHost]}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostName) => UpdatedVHost}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +delete_non_existing_vhost(_) -> + VHostName = <<"vhost">>, + + Tests = + [ + ?with(?assertEqual( + {error, {no_such_vhost, VHostName}}, + lookup_vhost(_With, VHostName))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + delete_vhost(_With, VHostName))), + ?with(?assertEqual( + {error, {no_such_vhost, VHostName}}, + lookup_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {khepri, [rabbit_vhost], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +delete_existing_vhost(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + VHost, + lookup_vhost(_With, VHostName))), + ?with(?assertEqual( + ok, + delete_vhost(_With, VHostName))), + ?with(?assertEqual( + {error, {no_such_vhost, VHostName}}, + lookup_vhost(_With, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {khepri, [rabbit_vhost], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +%% +%% Users. +%% + +write_non_existing_user(_) -> + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + + Tests = + [ + ?with(?assertEqual( + {error, not_found}, + lookup_user(_With, Username))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + {ok, User}, + lookup_user(_With, Username))), + ?with(check_storage( + _With, + [{mnesia, rabbit_user, [User]}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +write_existing_user(_) -> + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + + Tests = + [ + ?with(?assertEqual( + {error, not_found}, + lookup_user(_With, Username))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + {ok, User}, + lookup_user(_With, Username))), + ?with(?assertThrow( + {error, {user_already_exists, Username}}, + add_user(_With, Username, User))), + ?with(?assertEqual( + {ok, User}, + lookup_user(_With, Username))), + ?with(check_storage( + _With, + [{mnesia, rabbit_user, [User]}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_users(_) -> + UsernameA = <<"alice">>, + UserA = internal_user:create_user(UsernameA, <<"password">>, undefined), + UsernameB = <<"bob">>, + UserB = internal_user:create_user(UsernameB, <<"password">>, undefined), + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, UsernameA, UserA))), + ?with(?assertEqual( + ok, + add_user(_With, UsernameB, UserB))), + ?with(?assertEqual( + [UserA, UserB], + list_user_records(_With))), + ?with(check_storage( + _With, + [{mnesia, rabbit_user, [UserA, UserB]}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(UsernameA) => UserA, + ?user_path(UsernameB) => UserB}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +update_non_existing_user(_) -> + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + UpdatedUser = internal_user:set_password_hash( + User, <<"updated-pw">>, undefined), + Fun = fun(_) -> UpdatedUser end, + ?assertNotEqual(User, UpdatedUser), + + Tests = + [ + ?with(?assertEqual( + {error, not_found}, + lookup_user(_With, Username))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + update_user(_With, Username, Fun))), + ?with(?assertEqual( + {error, not_found}, + lookup_user(_With, Username))), + ?with(check_storage( + _With, + [{mnesia, rabbit_user, []}, + {khepri, [rabbit_auth_backend_internal], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +update_existing_user(_) -> + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + UpdatedUser = internal_user:set_password_hash( + User, <<"updated-pw">>, undefined), + Fun = fun(_) -> UpdatedUser end, + ?assertNotEqual(User, UpdatedUser), + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + {ok, User}, + lookup_user(_With, Username))), + ?with(?assertEqual( + ok, + update_user(_With, Username, Fun))), + ?with(?assertEqual( + {ok, UpdatedUser}, + lookup_user(_With, Username))), + ?with(check_storage( + _With, + [{mnesia, rabbit_user, [UpdatedUser]}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(Username) => UpdatedUser}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +delete_non_existing_user(_) -> + Username = <<"alice">>, + + Tests = + [ + ?with(?assertEqual( + {error, not_found}, + lookup_user(_With, Username))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + delete_user(_With, Username))), + ?with(?assertEqual( + {error, not_found}, + lookup_user(_With, Username))), + ?with(check_storage( + _With, + [{mnesia, rabbit_user, []}, + {khepri, [rabbit_auth_backend_internal], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +delete_existing_user(_) -> + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + {ok, User}, + lookup_user(_With, Username))), + ?with(?assertEqual( + ok, + delete_user(_With, Username))), + ?with(?assertEqual( + {error, not_found}, + lookup_user(_With, Username))), + ?with(check_storage( + _With, + [{mnesia, rabbit_user, []}, + {khepri, [rabbit_auth_backend_internal], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +%% +%% User permissions. +%% + +write_user_permission_for_non_existing_vhost(_) -> + VHostName = <<"vhost">>, + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + UserPermission = #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + permission = #permission{ + configure = <<>>, + write = <<>>, + read = <<>>}}, + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + set_permissions(_With, Username, VHostName, UserPermission))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_vhost], + #{}}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +write_user_permission_for_non_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + UserPermission = #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + permission = #permission{ + configure = <<>>, + write = <<>>, + read = <<>>}}, + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + set_permissions(_With, Username, VHostName, UserPermission))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_auth_backend_internal], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +write_user_permission_for_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + UserPermission = #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + permission = #permission{ + configure = <<>>, + write = <<>>, + read = <<>>}}, + + Tests = + [ + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(?assertEqual( + ok, + set_permissions(_With, Username, VHostName, UserPermission))), + ?with(?assert( + check_vhost_access(_With, Username, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_user_permission, [UserPermission]}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(Username) => User, + ?user_perm_path(Username, VHostName) => UserPermission}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +check_resource_access(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + UserPermission = #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + permission = #permission{ + configure = <<"my-resource">>, + write = <<>>, + read = <<>>}}, + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + ok, + set_permissions(_With, Username, VHostName, UserPermission))), + ?with(?assert( + check_resource_access( + _With, Username, VHostName, "my-resource", configure))), + ?with(?assertNot( + check_resource_access( + _With, Username, VHostName, "my-resource", write))), + ?with(?assertNot( + check_resource_access( + _With, Username, VHostName, "other-resource", configure))) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_user_permissions_on_non_existing_vhost(_) -> + VHostName = <<"non-existing-vhost">>, + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + rabbit_auth_backend_internal:list_vhost_permissions( + VHostName))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + rabbit_auth_backend_internal:list_user_vhost_permissions( + Username, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_vhost], #{}}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_user_permissions_for_non_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"non-existing-user">>, + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + [], + rabbit_auth_backend_internal:list_vhost_permissions( + VHostName))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + rabbit_auth_backend_internal:list_user_permissions( + Username))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + rabbit_auth_backend_internal:list_user_vhost_permissions( + Username, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_vhost], #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_auth_backend_internal], #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_user_permissions(_) -> + VHostNameA = <<"vhost-a">>, + VHostDescA = <<>>, + VHostTagsA = [], + VHostA = vhost:new( + VHostNameA, + VHostTagsA, + #{description => VHostDescA, + tags => VHostTagsA}), + VHostNameB = <<"vhost-b">>, + VHostDescB = <<>>, + VHostTagsB = [], + VHostB = vhost:new( + VHostNameB, + VHostTagsB, + #{description => VHostDescB, + tags => VHostTagsB}), + UsernameA = <<"alice">>, + UserA = internal_user:create_user(UsernameA, <<"password">>, undefined), + UsernameB = <<"bob">>, + UserB = internal_user:create_user(UsernameB, <<"password">>, undefined), + + UserPermissionA1 = #user_permission{ + user_vhost = #user_vhost{ + username = UsernameA, + virtual_host = VHostNameA}, + permission = #permission{ + configure = <<"my-resource">>, + write = <<>>, + read = <<>>}}, + UserPermissionA2 = #user_permission{ + user_vhost = #user_vhost{ + username = UsernameA, + virtual_host = VHostNameB}, + permission = #permission{ + configure = <<"my-resource">>, + write = <<>>, + read = <<>>}}, + UserPermissionB1 = #user_permission{ + user_vhost = #user_vhost{ + username = UsernameB, + virtual_host = VHostNameA}, + permission = #permission{ + configure = <<"my-resource">>, + write = <<>>, + read = <<>>}}, + + Tests = + [ + ?with(?assertEqual( + VHostA, + add_vhost(_With, VHostNameA, VHostDescA, VHostTagsA))), + ?with(?assertEqual( + VHostB, + add_vhost(_With, VHostNameB, VHostDescB, VHostTagsB))), + ?with(?assertEqual( + ok, + add_user(_With, UsernameA, UserA))), + ?with(?assertEqual( + ok, + set_permissions( + _With, UsernameA, VHostNameA, UserPermissionA1))), + ?with(?assertEqual( + ok, + set_permissions( + _With, UsernameA, VHostNameB, UserPermissionA2))), + ?with(?assertEqual( + ok, + add_user(_With, UsernameB, UserB))), + ?with(?assertEqual( + ok, + set_permissions( + _With, UsernameB, VHostNameA, UserPermissionB1))), + ?with(?assertEqual( + [UserPermissionA1, UserPermissionA2, UserPermissionB1], + list_permissions( + _With, + rabbit_auth_backend_internal:match_user_vhost('_', '_'), + rabbit_auth_backend_internal:match_path_in_khepri( + ?user_perm_path(?STAR, ?STAR))))), + ?with(?assertEqual( + [rabbit_auth_backend_internal:extract_user_permission_params( + rabbit_auth_backend_internal:vhost_perms_info_keys(), + UserPermissionA1), + rabbit_auth_backend_internal:extract_user_permission_params( + rabbit_auth_backend_internal:vhost_perms_info_keys(), + UserPermissionB1)], + lists:sort( + rabbit_auth_backend_internal:list_vhost_permissions( + VHostNameA)))), + ?with(?assertEqual( + [rabbit_auth_backend_internal:extract_user_permission_params( + rabbit_auth_backend_internal:user_perms_info_keys(), + UserPermissionA1), + rabbit_auth_backend_internal:extract_user_permission_params( + rabbit_auth_backend_internal:user_perms_info_keys(), + UserPermissionA2)], + lists:sort( + rabbit_auth_backend_internal:list_user_permissions( + UsernameA)))), + ?with(?assertEqual( + [rabbit_auth_backend_internal:extract_user_permission_params( + rabbit_auth_backend_internal:user_vhost_perms_info_keys(), + UserPermissionA1)], + lists:sort( + rabbit_auth_backend_internal:list_user_vhost_permissions( + UsernameA, VHostNameA)))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHostA, VHostB]}, + {mnesia, rabbit_user, [UserA, UserB]}, + {mnesia, rabbit_user_permission, [UserPermissionA1, + UserPermissionA2, + UserPermissionB1]}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostNameA) => VHostA, + ?vhost_path(VHostNameB) => VHostB}}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(UsernameA) => UserA, + ?user_path(UsernameB) => UserB, + ?user_perm_path(UsernameA, VHostNameA) => + UserPermissionA1, + ?user_perm_path(UsernameA, VHostNameB) => + UserPermissionA2, + ?user_perm_path(UsernameB, VHostNameA) => + UserPermissionB1}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_user_permission_for_non_existing_vhost(_) -> + VHostName = <<"vhost">>, + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + clear_permissions(_With, Username, VHostName))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_vhost], + #{}}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_user_permission_for_non_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + clear_permissions(_With, Username, VHostName))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_auth_backend_internal], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_user_permission(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + UserPermission = #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + permission = #permission{ + configure = <<"my-resource">>, + write = <<>>, + read = <<>>}}, + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + ok, + set_permissions(_With, Username, VHostName, UserPermission))), + ?with(?assert( + check_resource_access( + _With, Username, VHostName, "my-resource", configure))), + ?with(?assertEqual( + ok, + clear_permissions(_With, Username, VHostName))), + ?with(?assertNot( + check_resource_access( + _With, Username, VHostName, "my-resource", configure))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +delete_user_and_check_resource_access(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + UserPermission = #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + permission = #permission{ + configure = <<"my-resource">>, + write = <<>>, + read = <<>>}}, + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + ok, + set_permissions(_With, Username, VHostName, UserPermission))), + ?with(?assert( + check_vhost_access(_With, Username, VHostName))), + ?with(?assert( + check_resource_access( + _With, Username, VHostName, "my-resource", configure))), + ?with(?assertEqual( + ok, + delete_user(_With, Username))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(?assertNot( + check_resource_access( + _With, Username, VHostName, "my-resource", configure))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_auth_backend_internal], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +delete_vhost_and_check_resource_access(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + UserPermission = #user_permission{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + permission = #permission{ + configure = <<"my-resource">>, + write = <<>>, + read = <<>>}}, + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + ok, + set_permissions(_With, Username, VHostName, UserPermission))), + ?with(?assert( + check_vhost_access(_With, Username, VHostName))), + ?with(?assert( + check_resource_access( + _With, Username, VHostName, "my-resource", configure))), + ?with(?assertEqual( + ok, + delete_vhost(_With, VHostName))), + ?with(?assertNot( + check_vhost_access(_With, Username, VHostName))), + ?with(?assertNot( + check_resource_access( + _With, Username, VHostName, "my-resource", configure))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_vhost], + #{}}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +%% +%% Topic permissions. +%% + +write_topic_permission_for_non_existing_vhost(_) -> + VHostName = <<"vhost">>, + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + Exchange = <<"exchange">>, + TopicPermission = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = Exchange}, + permission = #permission{ + write = <<>>, + read = <<>>} + }, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + %% Unset permissions equals to permissions granted. + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + set_topic_permissions( + _With, Username, VHostName, Exchange, TopicPermission))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_topic_permission, []}, + {khepri, [rabbit_vhost], + #{}}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +write_topic_permission_for_non_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + Exchange = <<"exchange">>, + TopicPermission = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = Exchange}, + permission = #permission{ + write = <<>>, + read = <<>>} + }, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + set_topic_permissions( + _With, Username, VHostName, Exchange, TopicPermission))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_topic_permission, []}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_auth_backend_internal], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +write_topic_permission_for_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + Exchange = <<"exchange">>, + TopicPermission = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = Exchange}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, Username, VHostName, Exchange, TopicPermission))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertNot( + check_topic_access( + _With, Username, VHostName, Exchange, read, + Context#{routing_key => <<"something-else">>}))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_topic_permission, [TopicPermission]}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(Username) => User, + ?topic_perm_path(Username, VHostName, Exchange) => + TopicPermission}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_topic_permissions_on_non_existing_vhost(_) -> + VHostName = <<"non-existing-vhost">>, + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + rabbit_auth_backend_internal:list_vhost_topic_permissions( + VHostName))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + rabbit_auth_backend_internal:list_user_vhost_topic_permissions( + Username, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_topic_permission, []}, + {khepri, [rabbit_vhost], + #{}}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_topic_permissions_for_non_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"non-existing-user">>, + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + [], + rabbit_auth_backend_internal:list_vhost_topic_permissions( + VHostName))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + rabbit_auth_backend_internal:list_user_topic_permissions( + Username))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + rabbit_auth_backend_internal:list_user_vhost_topic_permissions( + Username, VHostName))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_topic_permission, []}, + {khepri, [rabbit_vhost], #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_auth_backend_internal], #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +list_topic_permissions(_) -> + VHostNameA = <<"vhost-a">>, + VHostDescA = <<>>, + VHostTagsA = [], + VHostA = vhost:new( + VHostNameA, + VHostTagsA, + #{description => VHostDescA, + tags => VHostTagsA}), + VHostNameB = <<"vhost-b">>, + VHostDescB = <<>>, + VHostTagsB = [], + VHostB = vhost:new( + VHostNameB, + VHostTagsB, + #{description => VHostDescB, + tags => VHostTagsB}), + UsernameA = <<"alice">>, + UserA = internal_user:create_user(UsernameA, <<"password">>, undefined), + UsernameB = <<"bob">>, + UserB = internal_user:create_user(UsernameB, <<"password">>, undefined), + + ExchangeA = <<"exchange-a">>, + ExchangeB = <<"exchange-b">>, + TopicPermissionA1 = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = UsernameA, + virtual_host = VHostNameA}, + exchange = ExchangeA}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + TopicPermissionA2 = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = UsernameA, + virtual_host = VHostNameB}, + exchange = ExchangeB}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + TopicPermissionB1 = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = UsernameB, + virtual_host = VHostNameA}, + exchange = ExchangeA}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + + Tests = + [ + ?with(?assertEqual( + VHostA, + add_vhost(_With, VHostNameA, VHostDescA, VHostTagsA))), + ?with(?assertEqual( + VHostB, + add_vhost(_With, VHostNameB, VHostDescB, VHostTagsB))), + ?with(?assertEqual( + ok, + add_user(_With, UsernameA, UserA))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, UsernameA, VHostNameA, ExchangeA, TopicPermissionA1))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, UsernameA, VHostNameB, ExchangeB, TopicPermissionA2))), + ?with(?assertEqual( + ok, + add_user(_With, UsernameB, UserB))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, UsernameB, VHostNameA, ExchangeA, TopicPermissionB1))), + ?with(?assertEqual( + [TopicPermissionA1, TopicPermissionA2, TopicPermissionB1], + list_topic_permissions( + _With, + rabbit_auth_backend_internal: + match_user_vhost_topic_permission('_', '_', '_'), + rabbit_auth_backend_internal: + match_path_in_khepri(?topic_perm_path(?STAR, ?STAR, ?STAR))))), + ?with(?assertEqual( + [rabbit_auth_backend_internal:extract_topic_permission_params( + rabbit_auth_backend_internal:vhost_topic_perms_info_keys(), + TopicPermissionA1), + rabbit_auth_backend_internal:extract_topic_permission_params( + rabbit_auth_backend_internal:vhost_topic_perms_info_keys(), + TopicPermissionB1)], + lists:sort( + rabbit_auth_backend_internal:list_vhost_topic_permissions( + VHostNameA)))), + ?with(?assertEqual( + [rabbit_auth_backend_internal:extract_topic_permission_params( + rabbit_auth_backend_internal:user_topic_perms_info_keys(), + TopicPermissionA1), + rabbit_auth_backend_internal:extract_topic_permission_params( + rabbit_auth_backend_internal:user_topic_perms_info_keys(), + TopicPermissionA2)], + lists:sort( + rabbit_auth_backend_internal:list_user_topic_permissions( + UsernameA)))), + ?with(?assertEqual( + [rabbit_auth_backend_internal:extract_topic_permission_params( + rabbit_auth_backend_internal: + user_vhost_topic_perms_info_keys(), + TopicPermissionA1)], + lists:sort( + rabbit_auth_backend_internal:list_user_vhost_topic_permissions( + UsernameA, VHostNameA)))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHostA, VHostB]}, + {mnesia, rabbit_user, [UserA, UserB]}, + {mnesia, rabbit_topic_permission, [TopicPermissionA1, + TopicPermissionA2, + TopicPermissionB1]}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostNameA) => VHostA, + ?vhost_path(VHostNameB) => VHostB}}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(UsernameA) => UserA, + ?user_path(UsernameB) => UserB, + ?topic_perm_path(UsernameA, VHostNameA, ExchangeA) => + TopicPermissionA1, + ?topic_perm_path(UsernameA, VHostNameB, ExchangeB) => + TopicPermissionA2, + ?topic_perm_path(UsernameB, VHostNameA, ExchangeA) => + TopicPermissionB1}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_specific_topic_permission_for_non_existing_vhost(_) -> + VHostName = <<"vhost">>, + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + Exchange = <<"exchange">>, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + clear_topic_permissions(_With, Username, VHostName, Exchange))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_vhost], + #{}}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_specific_topic_permission_for_non_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + Exchange = <<"exchange">>, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + clear_topic_permissions(_With, Username, VHostName, Exchange))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_auth_backend_internal], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_specific_topic_permission(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + ExchangeA = <<"exchange-a">>, + ExchangeB = <<"exchange-b">>, + TopicPermissionA = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = ExchangeA}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + TopicPermissionB = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = ExchangeB}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, Username, VHostName, ExchangeA, TopicPermissionA))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, Username, VHostName, ExchangeB, TopicPermissionB))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, ExchangeA, read, Context))), + ?with(?assertNot( + check_topic_access( + _With, Username, VHostName, ExchangeA, read, + Context#{routing_key => <<"something-else">>}))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, ExchangeB, read, Context))), + ?with(?assertNot( + check_topic_access( + _With, Username, VHostName, ExchangeB, read, + Context#{routing_key => <<"something-else">>}))), + ?with(?assertEqual( + ok, + clear_topic_permissions(_With, Username, VHostName, ExchangeA))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, ExchangeA, read, Context))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, ExchangeA, read, + Context#{routing_key => <<"something-else">>}))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, ExchangeB, read, Context))), + ?with(?assertNot( + check_topic_access( + _With, Username, VHostName, ExchangeB, read, + Context#{routing_key => <<"something-else">>}))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_topic_permission, [TopicPermissionB]}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(Username) => User, + ?topic_perm_path(Username, VHostName, ExchangeB) => + TopicPermissionB}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_all_topic_permission_for_non_existing_vhost(_) -> + VHostName = <<"vhost">>, + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + Exchange = <<"exchange">>, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertThrow( + {error, {no_such_vhost, VHostName}}, + clear_topic_permissions(_With, Username, VHostName))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_vhost], + #{}}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_all_topic_permission_for_non_existing_user(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + Exchange = <<"exchange">>, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertThrow( + {error, {no_such_user, Username}}, + clear_topic_permissions(_With, Username, VHostName))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_user_permission, []}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_auth_backend_internal], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +clear_all_topic_permissions(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + ExchangeA = <<"exchange-a">>, + ExchangeB = <<"exchange-b">>, + TopicPermissionA = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = ExchangeA}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + TopicPermissionB = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = ExchangeB}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, Username, VHostName, ExchangeA, TopicPermissionA))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, Username, VHostName, ExchangeB, TopicPermissionB))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, ExchangeA, read, Context))), + ?with(?assertNot( + check_topic_access( + _With, Username, VHostName, ExchangeA, read, + Context#{routing_key => <<"something-else">>}))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, ExchangeB, read, Context))), + ?with(?assertNot( + check_topic_access( + _With, Username, VHostName, ExchangeB, read, + Context#{routing_key => <<"something-else">>}))), + ?with(?assertEqual( + ok, + clear_topic_permissions(_With, Username, VHostName))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, ExchangeA, read, Context))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, ExchangeA, read, + Context#{routing_key => <<"something-else">>}))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, ExchangeB, read, Context))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, ExchangeB, read, + Context#{routing_key => <<"something-else">>}))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_topic_permission, []}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +delete_user_and_check_topic_access(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + Exchange = <<"exchange">>, + TopicPermission = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = Exchange}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, Username, VHostName, Exchange, TopicPermission))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertNot( + check_topic_access( + _With, Username, VHostName, Exchange, read, + Context#{routing_key => <<"something-else">>}))), + ?with(?assertEqual( + ok, + delete_user(_With, Username))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, + Context#{routing_key => <<"something-else">>}))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, [VHost]}, + {mnesia, rabbit_user, []}, + {mnesia, rabbit_topic_permission, []}, + {khepri, [rabbit_vhost], + #{?vhost_path(VHostName) => VHost}}, + {khepri, [rabbit_auth_backend_internal], + #{}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +delete_vhost_and_check_topic_access(_) -> + VHostName = <<"vhost">>, + VHostDesc = <<>>, + VHostTags = [], + VHost = vhost:new( + VHostName, + VHostTags, + #{description => VHostDesc, + tags => VHostTags}), + Username = <<"alice">>, + User = internal_user:create_user(Username, <<"password">>, undefined), + Exchange = <<"exchange">>, + TopicPermission = #topic_permission{ + topic_permission_key = + #topic_permission_key{ + user_vhost = #user_vhost{ + username = Username, + virtual_host = VHostName}, + exchange = Exchange}, + permission = #permission{ + write = <<>>, + read = <<"^key$">>} + }, + Context = #{routing_key => <<"key">>, + variable_map => #{<<"vhost">> => VHostName, + <<"username">> => Username}}, + + Tests = + [ + ?with(?assertEqual( + VHost, + add_vhost(_With, VHostName, VHostDesc, VHostTags))), + ?with(?assertEqual( + ok, + add_user(_With, Username, User))), + ?with(?assertEqual( + ok, + set_topic_permissions( + _With, Username, VHostName, Exchange, TopicPermission))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assertNot( + check_topic_access( + _With, Username, VHostName, Exchange, read, + Context#{routing_key => <<"something-else">>}))), + ?with(?assertEqual( + ok, + delete_vhost(_With, VHostName))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, Context))), + ?with(?assert( + check_topic_access( + _With, Username, VHostName, Exchange, read, + Context#{routing_key => <<"something-else">>}))), + ?with(check_storage( + _With, + [{mnesia, rabbit_vhost, []}, + {mnesia, rabbit_user, [User]}, + {mnesia, rabbit_topic_permission, []}, + {khepri, [rabbit_vhost], + #{}}, + {khepri, [rabbit_auth_backend_internal], + #{?user_path(Username) => User}}])) + ], + + ?assertEqual( + ok, + eunit:test( + [{setup, fun force_mnesia_use/0, [{with, mnesia, Tests}]}, + {setup, fun force_khepri_use/0, [{with, khepri, Tests}]}], + [verbose])). + +%% ------------------------------------------------------------------- +%% Helpers. +%% ------------------------------------------------------------------- + +force_mnesia_use() -> + ct:pal(?LOW_IMPORTANCE, "Using Mnesia (disabling feature flag)", []), + rabbit_khepri:force_metadata_store(mnesia), + mock_feature_flag_state(false). + +force_khepri_use() -> + ct:pal(?LOW_IMPORTANCE, "Using Khepri (enabling feature flag)", []), + rabbit_khepri:force_metadata_store(khepri), + mock_feature_flag_state(true). + +mock_feature_flag_state(State) -> + _ = (catch meck:unload(rabbit_khepri)), + meck:new(rabbit_khepri, [passthrough, no_link]), + meck:expect(rabbit_khepri, is_enabled, fun(_) -> State end). + +add_vhost(mnesia, VHostName, VHostDesc, VHostTags) -> + rabbit_vhost:do_add_to_mnesia(VHostName, VHostDesc, VHostTags); +add_vhost(khepri, VHostName, VHostDesc, VHostTags) -> + rabbit_vhost:do_add_to_khepri(VHostName, VHostDesc, VHostTags). + +lookup_vhost(mnesia, VHostName) -> + rabbit_vhost:lookup_in_mnesia(VHostName); +lookup_vhost(khepri, VHostName) -> + rabbit_vhost:lookup_in_khepri(VHostName). + +vhost_exists(mnesia, VHostName) -> + rabbit_vhost:exists_in_mnesia(VHostName); +vhost_exists(khepri, VHostName) -> + rabbit_vhost:exists_in_khepri(VHostName). + +list_vhosts(mnesia) -> + lists:sort(rabbit_vhost:list_names_in_mnesia()); +list_vhosts(khepri) -> + lists:sort(rabbit_vhost:list_names_in_khepri()). + +list_vhost_records(mnesia) -> + lists:sort(rabbit_vhost:all_in_mnesia()); +list_vhost_records(khepri) -> + lists:sort(rabbit_vhost:all_in_khepri()). + +update_vhost(mnesia, VHostName, Fun) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + rabbit_vhost:update_in_mnesia(VHostName, Fun) + end); +update_vhost(khepri, VHostName, Fun) -> + rabbit_vhost:update_in_khepri(VHostName, Fun). + +update_vhost(mnesia, VHostName, Description, Tags) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + rabbit_vhost:update_in_mnesia(VHostName, Description, Tags) + end); +update_vhost(khepri, VHostName, Description, Tags) -> + rabbit_vhost:update_in_khepri(VHostName, Description, Tags). + +vhost_info(mnesia, VHostName) -> + rabbit_vhost:info_in_mnesia(VHostName); +vhost_info(khepri, VHostName) -> + rabbit_vhost:info_in_khepri(VHostName). + +delete_vhost(mnesia, VHostName) -> + rabbit_misc:execute_mnesia_transaction( + fun() -> + Fun = rabbit_vhost:with_in_mnesia( + VHostName, + fun() -> + rabbit_vhost:clear_permissions_in_mnesia( + VHostName, undefined) + end), + Fun(), + rabbit_vhost:internal_delete_in_mnesia(VHostName) + end); +delete_vhost(khepri, VHostName) -> + rabbit_vhost:clear_permissions_in_khepri(VHostName, undefined), + rabbit_vhost:internal_delete_in_khepri(VHostName). + +add_user(mnesia, Username, User) -> + rabbit_auth_backend_internal:add_user_sans_validation_in_mnesia( + Username, User); +add_user(khepri, Username, User) -> + rabbit_auth_backend_internal:add_user_sans_validation_in_khepri( + Username, User). + +lookup_user(mnesia, Username) -> + rabbit_auth_backend_internal:lookup_user_in_mnesia(Username); +lookup_user(khepri, Username) -> + rabbit_auth_backend_internal:lookup_user_in_khepri(Username). + +list_user_records(mnesia) -> + lists:sort(rabbit_auth_backend_internal:all_users_in_mnesia()); +list_user_records(khepri) -> + lists:sort(rabbit_auth_backend_internal:all_users_in_khepri()). + +update_user(mnesia, Username, Fun) -> + rabbit_auth_backend_internal:update_user_in_mnesia(Username, Fun); +update_user(khepri, Username, Fun) -> + rabbit_auth_backend_internal:update_user_in_khepri(Username, Fun). + +delete_user(mnesia, Username) -> + rabbit_auth_backend_internal:delete_user_in_mnesia(Username); +delete_user(khepri, Username) -> + rabbit_auth_backend_internal:delete_user_in_khepri(Username). + +set_permissions(mnesia, Username, VHostName, UserPermission) -> + rabbit_auth_backend_internal:set_permissions_in_mnesia( + Username, VHostName, UserPermission); +set_permissions(khepri, Username, VHostName, UserPermission) -> + rabbit_auth_backend_internal:set_permissions_in_khepri( + Username, VHostName, UserPermission). + +list_permissions(mnesia, MnesiaThunk, _) -> + lists:sort( + rabbit_auth_backend_internal:list_permissions_in_mnesia(MnesiaThunk)); +list_permissions(khepri, _, KhepriThunk) -> + lists:sort( + rabbit_auth_backend_internal:list_permissions_in_khepri(KhepriThunk)). + +check_vhost_access(mnesia, Username, VHostName) -> + rabbit_auth_backend_internal:check_vhost_access_in_mnesia( + Username, VHostName); +check_vhost_access(khepri, Username, VHostName) -> + rabbit_auth_backend_internal:check_vhost_access_in_khepri( + Username, VHostName). + +set_topic_permissions( + mnesia, Username, VHostName, Exchange, TopicPermission) -> + rabbit_auth_backend_internal:set_topic_permissions_in_mnesia( + Username, VHostName, Exchange, TopicPermission); +set_topic_permissions( + khepri, Username, VHostName, Exchange, TopicPermission) -> + rabbit_auth_backend_internal:set_topic_permissions_in_khepri( + Username, VHostName, Exchange, TopicPermission). + +check_topic_access(mnesia, Username, VHostName, Exchange, Perm, Context) -> + rabbit_auth_backend_internal:check_topic_access_in_mnesia( + Username, VHostName, Exchange, Perm, Context); +check_topic_access(khepri, Username, VHostName, Exchange, Perm, Context) -> + rabbit_auth_backend_internal:check_topic_access_in_khepri( + Username, VHostName, Exchange, Perm, Context). + +list_topic_permissions(mnesia, QueryThunk, _) -> + lists:sort( + rabbit_auth_backend_internal:list_topic_permissions_in_mnesia( + QueryThunk)); +list_topic_permissions(khepri, _, Path) -> + lists:sort( + rabbit_auth_backend_internal:list_topic_permissions_in_khepri( + Path)). + +clear_permissions(mnesia, Username, VHostName) -> + rabbit_auth_backend_internal:clear_permissions_in_mnesia( + Username, VHostName); +clear_permissions(khepri, Username, VHostName) -> + rabbit_auth_backend_internal:clear_permissions_in_khepri( + Username, VHostName). + +check_resource_access(mnesia, Username, VHostName, Resource, Perm) -> + rabbit_auth_backend_internal:check_resource_access_in_mnesia( + Username, VHostName, Resource, Perm); +check_resource_access(khepri, Username, VHostName, Resource, Perm) -> + rabbit_auth_backend_internal:check_resource_access_in_khepri( + Username, VHostName, Resource, Perm). + +clear_topic_permissions(mnesia, Username, VHostName) -> + rabbit_auth_backend_internal:clear_topic_permissions_in_mnesia( + Username, VHostName); +clear_topic_permissions(khepri, Username, VHostName) -> + rabbit_auth_backend_internal:clear_topic_permissions_in_khepri( + Username, VHostName). + +clear_topic_permissions(mnesia, Username, VHostName, Exchange) -> + rabbit_auth_backend_internal:clear_topic_permissions_in_mnesia( + Username, VHostName, Exchange); +clear_topic_permissions(khepri, Username, VHostName, Exchange) -> + rabbit_auth_backend_internal:clear_topic_permissions_in_khepri( + Username, VHostName, Exchange). + +check_storage(With, [{With, Source, Content} | Rest]) -> + check_storage(With, Source, Content), + check_storage(With, Rest); +check_storage(With, [_ | Rest]) -> + check_storage(With, Rest); +check_storage(_, []) -> + ok. + +check_storage(mnesia, Table, Content) -> + ?assertEqual(Content, lists:sort(ets:tab2list(Table))); +check_storage(khepri, Path, Content) -> + rabbit_khepri:info(), + Path1 = Path ++ [?STAR_STAR], + ?assertEqual({ok, Content}, rabbit_khepri:match_and_get_data(Path1)). diff --git a/deps/rabbit_common/src/rabbit_misc.erl b/deps/rabbit_common/src/rabbit_misc.erl index 811dcfbe7356..f6041a2abeaf 100644 --- a/deps/rabbit_common/src/rabbit_misc.erl +++ b/deps/rabbit_common/src/rabbit_misc.erl @@ -31,7 +31,6 @@ -export([start_cover/1]). -export([throw_on_error/2, with_exit_handler/2, is_abnormal_exit/1, filter_exit_map/2]). --export([with_user/2]). -export([execute_mnesia_transaction/1]). -export([execute_mnesia_transaction/2]). -export([execute_mnesia_tx_with_tail/1]). @@ -171,7 +170,6 @@ -spec with_exit_handler(thunk(A), thunk(A)) -> A. -spec is_abnormal_exit(any()) -> boolean(). -spec filter_exit_map(fun ((A) -> B), [A]) -> [B]. --spec with_user(rabbit_types:username(), thunk(A)) -> A. -spec execute_mnesia_transaction(thunk(A)) -> A. -spec execute_mnesia_transaction(thunk(A), fun ((A, boolean()) -> B)) -> B. -spec execute_mnesia_tx_with_tail @@ -544,17 +542,6 @@ filter_exit_map(F, L) -> fun () -> Ref end, fun () -> F(I) end) || I <- L]). - -with_user(Username, Thunk) -> - fun () -> - case mnesia:read({rabbit_user, Username}) of - [] -> - mnesia:abort({no_such_user, Username}); - [_U] -> - Thunk() - end - end. - execute_mnesia_transaction(TxFun) -> %% Making this a sync_transaction allows us to use dirty_read %% elsewhere and get a consistent result even when that read diff --git a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl index b22e508e000e..44102e783b63 100644 --- a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl +++ b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl @@ -785,10 +785,12 @@ query_node(Config, NodeConfig) -> [rabbit, plugins_dir]), {ok, EnabledPluginsFile} = rpc(Config, Nodename, application, get_env, [rabbit, enabled_plugins_file]), + LogLocations = rpc(Config, Nodename, rabbit, log_locations, []), Vars0 = [{pid_file, PidFile}, {mnesia_dir, MnesiaDir}, {plugins_dir, PluginsDir}, - {enabled_plugins_file, EnabledPluginsFile}], + {enabled_plugins_file, EnabledPluginsFile}, + {log_locations, LogLocations}], Vars = try EnabledFeatureFlagsFile = rpc(Config, Nodename, rabbit_feature_flags, @@ -999,6 +1001,8 @@ stop_rabbitmq_nodes(Config) -> fun(NodeConfig) -> stop_rabbitmq_node(Config, NodeConfig) end), + IgnoredCrashes = ["** force_vhost_failure"], + find_crashes_in_logs(NodeConfigs, IgnoredCrashes), proplists:delete(rmq_nodes, Config). stop_rabbitmq_node(Config, NodeConfig) -> @@ -1019,6 +1023,84 @@ stop_rabbitmq_node(Config, NodeConfig) -> end, NodeConfig. +find_crashes_in_logs(NodeConfigs, IgnoredCrashes) -> + ct:pal( + "Looking up any crash reports in the nodes' log files. If we find " + "some, they will appear below:"), + CrashesCount = lists:foldl( + fun(NodeConfig, Total) -> + Count = count_crashes_in_logs( + NodeConfig, IgnoredCrashes), + Total + Count + end, 0, NodeConfigs), + ct:pal("Found ~b crash report(s)", [CrashesCount]), + ?assertEqual(0, CrashesCount). + +count_crashes_in_logs(NodeConfig, IgnoredCrashes) -> + LogLocations = ?config(log_locations, NodeConfig), + lists:foldl( + fun(LogLocation, Total) -> + Count = count_crashes_in_log(LogLocation, IgnoredCrashes), + Total + Count + end, 0, LogLocations). + +count_crashes_in_log(LogLocation, IgnoredCrashes) -> + case file:read_file(LogLocation) of + {ok, Content} -> count_crashes_in_content(Content, IgnoredCrashes); + _ -> 0 + end. + +count_crashes_in_content(Content, IgnoredCrashes) -> + ReOpts = [multiline], + Lines = re:split(Content, "^", ReOpts), + count_gen_server_terminations(Lines, IgnoredCrashes). + +count_gen_server_terminations(Lines, IgnoredCrashes) -> + count_gen_server_terminations(Lines, 0, IgnoredCrashes). + +count_gen_server_terminations([Line | Rest], Count, IgnoredCrashes) -> + ReOpts = [{capture, all_but_first, list}], + Ret = re:run( + Line, + "(<[0-9.]+> )[*]{2} Generic server .+ terminating$", + ReOpts), + case Ret of + {match, [Prefix]} -> + capture_gen_server_termination( + Rest, Prefix, [Line], Count, IgnoredCrashes); + nomatch -> + count_gen_server_terminations(Rest, Count, IgnoredCrashes) + end; +count_gen_server_terminations([], Count, _IgnoredCrashes) -> + Count. + +capture_gen_server_termination( + [Line | Rest] = Lines, Prefix, Acc, Count, IgnoredCrashes) -> + ReOpts = [{capture, all_but_first, list}], + Ret = re:run(Line, Prefix ++ "( .*|\\*.*|)$", ReOpts), + case Ret of + {match, [Suffix]} -> + case lists:member(Suffix, IgnoredCrashes) of + false -> + capture_gen_server_termination( + Rest, Prefix, [Line | Acc], Count, IgnoredCrashes); + true -> + count_gen_server_terminations( + Lines, Count, IgnoredCrashes) + end; + nomatch -> + found_gen_server_termiation( + lists:reverse(Acc), Lines, Count, IgnoredCrashes) + end; +capture_gen_server_termination( + [] = Rest, _Prefix, Acc, Count, IgnoredCrashes) -> + found_gen_server_termiation( + lists:reverse(Acc), Rest, Count, IgnoredCrashes). + +found_gen_server_termiation(Message, Lines, Count, IgnoredCrashes) -> + ct:pal("gen_server termination:~n~n~s", [Message]), + count_gen_server_terminations(Lines, Count + 1, IgnoredCrashes). + %% ------------------------------------------------------------------- %% Helpers for partition simulation %% ------------------------------------------------------------------- @@ -1266,6 +1348,8 @@ delete_vhost(Config, Node, VHost) -> delete_vhost(Config, Node, VHost, Username) -> catch rpc(Config, Node, rabbit_vhost, delete, [VHost, Username]). +-define(FORCE_VHOST_FAILURE_REASON, force_vhost_failure). + force_vhost_failure(Config, VHost) -> force_vhost_failure(Config, 0, VHost). force_vhost_failure(Config, Node, VHost) -> @@ -1279,7 +1363,8 @@ force_vhost_failure(Config, Node, VHost, Attempts) -> try MessageStorePid = get_message_store_pid(Config, Node, VHost), rpc(Config, Node, - erlang, exit, [MessageStorePid, force_vhost_failure]), + erlang, exit, + [MessageStorePid, ?FORCE_VHOST_FAILURE_REASON]), %% Give it a time to fail timer:sleep(300), force_vhost_failure(Config, Node, VHost, Attempts - 1) diff --git a/workspace_helpers.bzl b/workspace_helpers.bzl index 10a3efd4b9aa..195454ee9e4c 100644 --- a/workspace_helpers.bzl +++ b/workspace_helpers.bzl @@ -140,6 +140,14 @@ erlang_app( sha256 = "0c5cc8fdc11b53cc25cf65ac6705ad39e54ecc56d1c22e4adb8f5a53fb9427f3", ) + github_erlang_app( + name = "khepri", + org = "rabbitmq", + ref = "15721dc37b84e14d9820b74b106cd56f763acd1c", + version = "15721dc37b84e14d9820b74b106cd56f763acd1c", + build_file = rabbitmq_workspace + "//:BUILD.khepri", + ) + github_erlang_app( name = "meck", org = "eproxus",