summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorMichael Klishin <mklishin@pivotal.io>2019-08-30 13:08:58 +0300
committerGitHub <noreply@github.com>2019-08-30 13:08:58 +0300
commit7c36cc35eee734e917b22526f8091b8b93f37694 (patch)
treee03981513a6a05e0f6886922750f09e9a89bc63e
parent501334a29225336080100df83f34623214337b4f (diff)
parentdbe1b022685cf404fc90710803360c7db42a435e (diff)
downloadrabbitmq-server-git-7c36cc35eee734e917b22526f8091b8b93f37694.tar.gz
Merge pull request #2080 from rabbitmq/vhost-description
Add virtual host description and tags
-rw-r--r--include/vhost.hrl6
-rw-r--r--include/vhost_v1.hrl4
-rw-r--r--include/vhost_v2.hrl5
-rw-r--r--src/rabbit.erl4
-rw-r--r--src/rabbit_core_ff.erl26
-rw-r--r--src/rabbit_fhc_helpers.erl2
-rw-r--r--src/rabbit_health_check.erl2
-rw-r--r--src/rabbit_table.erl11
-rw-r--r--src/rabbit_upgrade_functions.erl2
-rw-r--r--src/rabbit_variable_queue.erl2
-rw-r--r--src/rabbit_vhost.erl251
-rw-r--r--src/rabbit_vhost_limit.erl10
-rw-r--r--src/rabbit_vhost_sup_sup.erl2
-rw-r--r--src/vhost.erl181
-rw-r--r--src/vhost_v1.erl115
-rw-r--r--test/topic_permission_SUITE.erl4
16 files changed, 497 insertions, 130 deletions
diff --git a/include/vhost.hrl b/include/vhost.hrl
new file mode 100644
index 0000000000..d3abc0dd2a
--- /dev/null
+++ b/include/vhost.hrl
@@ -0,0 +1,6 @@
+-include("vhost_v1.hrl").
+-include("vhost_v2.hrl").
+
+-define(is_vhost(V),
+ (?is_vhost_v2(V) orelse
+ ?is_vhost_v1(V))).
diff --git a/include/vhost_v1.hrl b/include/vhost_v1.hrl
new file mode 100644
index 0000000000..185739c6be
--- /dev/null
+++ b/include/vhost_v1.hrl
@@ -0,0 +1,4 @@
+-define(is_vhost_v1(V), is_record(V, vhost, 3)).
+
+-define(vhost_v1_field_name(V), element(2, V)).
+-define(vhost_v1_field_limits(V), element(3, V)).
diff --git a/include/vhost_v2.hrl b/include/vhost_v2.hrl
new file mode 100644
index 0000000000..9345e8b206
--- /dev/null
+++ b/include/vhost_v2.hrl
@@ -0,0 +1,5 @@
+-define(is_vhost_v2(V), is_record(V, vhost, 4)).
+
+-define(vhost_v2_field_name(Q), element(2, Q)).
+-define(vhost_v2_field_limits(Q), element(3, Q)).
+-define(vhost_v2_field_metadata(Q), element(4, Q)).
diff --git a/src/rabbit.erl b/src/rabbit.erl
index 224b9e4908..29cd145330 100644
--- a/src/rabbit.erl
+++ b/src/rabbit.erl
@@ -894,7 +894,7 @@ total_queue_count() ->
lists:foldl(fun (VirtualHost, Acc) ->
Acc + rabbit_amqqueue:count(VirtualHost)
end,
- 0, rabbit_vhost:list()).
+ 0, rabbit_vhost:list_names()).
%% TODO this only determines if the rabbit application has started,
%% not if it is running, never mind plugins. It would be nice to have
@@ -1098,7 +1098,7 @@ insert_default_data() ->
DefaultWritePermBin = rabbit_data_coercion:to_binary(DefaultWritePerm),
DefaultReadPermBin = rabbit_data_coercion:to_binary(DefaultReadPerm),
- ok = rabbit_vhost:add(DefaultVHostBin, ?INTERNAL_USER),
+ ok = rabbit_vhost:add(DefaultVHostBin, <<"Default virtual host">>, [], ?INTERNAL_USER),
ok = lager_exchange_backend:maybe_init_exchange(),
ok = rabbit_auth_backend_internal:add_user(
DefaultUserBin,
diff --git a/src/rabbit_core_ff.erl b/src/rabbit_core_ff.erl
index 24d2163f16..03f686fb33 100644
--- a/src/rabbit_core_ff.erl
+++ b/src/rabbit_core_ff.erl
@@ -17,7 +17,8 @@
-module(rabbit_core_ff).
-export([quorum_queue_migration/3,
- implicit_default_bindings_migration/3]).
+ implicit_default_bindings_migration/3,
+ virtual_host_metadata_migration/3]).
-rabbit_feature_flag(
{quorum_queue,
@@ -35,6 +36,13 @@
migration_fun => {?MODULE, implicit_default_bindings_migration}
}}).
+-rabbit_feature_flag(
+ {virtual_host_metadata,
+ #{desc => "Virtual host metadata (description, tags, etc)",
+ stability => stable,
+ migration_fun => {?MODULE, virtual_host_metadata_migration}
+ }}).
+
%% -------------------------------------------------------------------
%% Quorum queues.
%% -------------------------------------------------------------------
@@ -95,3 +103,19 @@ remove_explicit_default_bindings(FeatureName, Queues) ->
[rabbit_binding:remove_default_exchange_binding_rows_of(Q)
|| Q <- Queues],
ok.
+
+
+%% -------------------------------------------------------------------
+%% Virtual host metadata.
+%% -------------------------------------------------------------------
+
+virtual_host_metadata_migration(_FeatureName, _FeatureProps, enable) ->
+ Tab = rabbit_vhost,
+ rabbit_table:wait([Tab], _Retry = true),
+ Fun = fun(Row) -> vhost:upgrade_to(vhost_v2, Row) end,
+ case mnesia:transform_table(Tab, Fun, vhost:fields(vhost_v2)) of
+ {atomic, ok} -> ok;
+ {aborted, Reason} -> {error, Reason}
+ end;
+virtual_host_metadata_migration(_FeatureName, _FeatureProps, is_enabled) ->
+ mnesia:table_info(rabbit_vhost, attributes) =:= vhost:fields(vhost_v2).
diff --git a/src/rabbit_fhc_helpers.erl b/src/rabbit_fhc_helpers.erl
index adf81f463a..9c66bfe956 100644
--- a/src/rabbit_fhc_helpers.erl
+++ b/src/rabbit_fhc_helpers.erl
@@ -24,7 +24,7 @@ clear_read_cache() ->
case application:get_env(rabbit, fhc_read_buffering) of
{ok, true} ->
file_handle_cache:clear_read_cache(),
- clear_vhost_read_cache(rabbit_vhost:list());
+ clear_vhost_read_cache(rabbit_vhost:list_names());
_ -> %% undefined or {ok, false}
ok
end.
diff --git a/src/rabbit_health_check.erl b/src/rabbit_health_check.erl
index ba081fac5d..7a127f9bf6 100644
--- a/src/rabbit_health_check.erl
+++ b/src/rabbit_health_check.erl
@@ -58,7 +58,7 @@ node_health_check(list_channels) ->
end;
node_health_check(list_queues) ->
- health_check_queues(rabbit_vhost:list());
+ health_check_queues(rabbit_vhost:list_names());
node_health_check(rabbit_node_monitor) ->
case rabbit_node_monitor:partitions() of
diff --git a/src/rabbit_table.erl b/src/rabbit_table.erl
index 7dfc5fb573..8326666edd 100644
--- a/src/rabbit_table.erl
+++ b/src/rabbit_table.erl
@@ -296,10 +296,11 @@ definitions() ->
permission = #permission{_='_'},
_='_'}}]},
{rabbit_vhost,
- [{record_name, vhost},
- {attributes, record_info(fields, vhost)},
+ [
+ {record_name, vhost},
+ {attributes, vhost:fields()},
{disc_copies, [node()]},
- {match, #vhost{_='_'}}]},
+ {match, vhost:pattern_match_all()}]},
{rabbit_listener,
[{record_name, listener},
{attributes, record_info(fields, listener)},
@@ -383,9 +384,9 @@ reverse_binding_match() ->
binding_destination_match() ->
resource_match('_').
trie_node_match() ->
- #trie_node{ exchange_name = exchange_name_match(), _='_'}.
+ #trie_node{exchange_name = exchange_name_match(), _='_'}.
trie_edge_match() ->
- #trie_edge{ exchange_name = exchange_name_match(), _='_'}.
+ #trie_edge{exchange_name = exchange_name_match(), _='_'}.
trie_binding_match() ->
#trie_binding{exchange_name = exchange_name_match(), _='_'}.
exchange_name_match() ->
diff --git a/src/rabbit_upgrade_functions.erl b/src/rabbit_upgrade_functions.erl
index d3cc363cbc..a75c8dae1a 100644
--- a/src/rabbit_upgrade_functions.erl
+++ b/src/rabbit_upgrade_functions.erl
@@ -181,7 +181,7 @@ exchange_event_serial() ->
trace_exchanges() ->
[declare_exchange(
rabbit_misc:r(VHost, exchange, <<"amq.rabbitmq.trace">>), topic) ||
- VHost <- rabbit_vhost:list()],
+ VHost <- rabbit_vhost:list_names()],
ok.
-spec user_admin_to_tags() -> 'ok'.
diff --git a/src/rabbit_variable_queue.erl b/src/rabbit_variable_queue.erl
index dfd55be1d0..4149a94932 100644
--- a/src/rabbit_variable_queue.erl
+++ b/src/rabbit_variable_queue.erl
@@ -2841,7 +2841,7 @@ move_messages_to_vhost_store(Queues) ->
OldStore = run_old_persistent_store(RecoveryRefs, StartFunState),
- VHosts = rabbit_vhost:list(),
+ VHosts = rabbit_vhost:list_names(),
%% New store should not be recovered.
NewMsgStore = start_new_store(VHosts),
diff --git a/src/rabbit_vhost.erl b/src/rabbit_vhost.erl
index ebf0c442a9..95758cec7f 100644
--- a/src/rabbit_vhost.erl
+++ b/src/rabbit_vhost.erl
@@ -17,17 +17,21 @@
-module(rabbit_vhost).
-include_lib("rabbit_common/include/rabbit.hrl").
-
-%%----------------------------------------------------------------------------
+-include("vhost.hrl").
-export([recover/0, recover/1]).
--export([add/2, delete/2, exists/1, list/0, count/0, with/2, with_user_and_vhost/3, assert/1, update/2,
- set_limits/2, limits_of/1, vhost_cluster_state/1, is_running_on_all_nodes/1, await_running_on_all_nodes/2]).
+-export([add/2, add/4, delete/2, exists/1, with/2, with_user_and_vhost/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, parse_tags/1]).
-export([info/1, info/2, info_all/0, info_all/1, info_all/2, info_all/3]).
-export([dir/1, msg_store_dir_path/1, msg_store_dir_wildcard/0]).
-export([delete_storage/1]).
-export([vhost_down/1]).
+%%
+%% API
+%%
+
recover() ->
%% Clear out remnants of old incarnation, in case we restarted
%% faster than other nodes handled DOWN messages from us.
@@ -42,11 +46,11 @@ recover() ->
%% So recovery will be run every time a vhost supervisor is restarted.
ok = rabbit_vhost_sup_sup:start(),
- [ok = rabbit_vhost_sup_sup:init_vhost(VHost) || VHost <- rabbit_vhost:list()],
+ [ok = rabbit_vhost_sup_sup:init_vhost(VHost) || VHost <- list_names()],
ok.
recover(VHost) ->
- VHostDir = rabbit_vhost:msg_store_dir_path(VHost),
+ VHostDir = msg_store_dir_path(VHost),
rabbit_log:info("Making sure data directory '~ts' for vhost '~s' exists~n",
[VHostDir, VHost]),
VHostStubFile = filename:join(VHostDir, ".vhost"),
@@ -61,37 +65,59 @@ recover(VHost) ->
ok = rabbit_mirror_queue_misc:on_vhost_up(VHost),
ok.
-%%----------------------------------------------------------------------------
+-define(INFO_KEYS, vhost:info_keys()).
--define(INFO_KEYS, [name, tracing, cluster_state]).
+-spec parse_tags(binary() | string() | atom()) -> [atom()].
+parse_tags(undefined) ->
+ [];
+parse_tags("") ->
+ [];
+parse_tags(<<"">>) ->
+ [];
+parse_tags(Val) when is_binary(Val) ->
+ parse_tags(rabbit_data_coercion:to_list(Val));
+parse_tags(Val) when is_list(Val) ->
+ [trim_tag(Tag) || Tag <- re:split(Val, ",", [{return, list}])].
--spec add(rabbit_types:vhost(), rabbit_types:username()) -> rabbit_types:ok_or_error(any()).
+-spec add(vhost:name(), rabbit_types:username()) -> rabbit_types:ok_or_error(any()).
add(VHost, ActingUser) ->
case exists(VHost) of
true -> ok;
- false -> do_add(VHost, ActingUser)
+ false -> do_add(VHost, <<"">>, [], ActingUser)
end.
-do_add(VHostPath, ActingUser) ->
- rabbit_log:info("Adding vhost '~s'~n", [VHostPath]),
- R = rabbit_misc:execute_mnesia_transaction(
+-spec add(vhost:name(), binary(), [atom()], rabbit_types:username()) -> rabbit_types:ok_or_error(any()).
+
+add(Name, Description, Tags, ActingUser) ->
+ case exists(Name) of
+ true -> ok;
+ false -> do_add(Name, Description, Tags, ActingUser)
+ end.
+
+do_add(Name, Description, Tags, ActingUser) ->
+ rabbit_log:info("Adding vhost '~s' (description: '~s')", [Name, Description]),
+ VHost = rabbit_misc:execute_mnesia_transaction(
fun () ->
- case mnesia:wread({rabbit_vhost, VHostPath}) of
- [] -> ok = mnesia:write(rabbit_vhost,
- #vhost{virtual_host = VHostPath},
- write);
+ 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
- [_] -> ok
+ [Row] ->
+ Row
end
end,
- fun (ok, true) ->
- ok;
- (ok, false) ->
- [_ = rabbit_exchange:declare(
- rabbit_misc:r(VHostPath, exchange, Name),
- Type, true, false, Internal, [], ActingUser) ||
- {Name, Type, Internal} <-
+ 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},
@@ -101,52 +127,54 @@ do_add(VHostPath, ActingUser) ->
{<<"amq.headers">>, headers, false},
{<<"amq.fanout">>, fanout, false},
{<<"amq.rabbitmq.trace">>, topic, true}]],
- ok
+ VHost1
end),
- case rabbit_vhost_sup_sup:start_on_all_nodes(VHostPath) of
+ case rabbit_vhost_sup_sup:start_on_all_nodes(Name) of
ok ->
- rabbit_event:notify(vhost_created, info(VHostPath)
- ++ [{user_who_performed_action, ActingUser}]),
- R;
+ rabbit_event:notify(vhost_created, info(VHost)
+ ++ [{user_who_performed_action, ActingUser},
+ {description, Description},
+ {tags, Tags}]),
+ ok;
{error, Reason} ->
Msg = rabbit_misc:format("failed to set up vhost '~s': ~p",
- [VHostPath, Reason]),
+ [Name, Reason]),
{error, Msg}
end.
--spec delete(rabbit_types:vhost(), rabbit_types:username()) -> rabbit_types:ok_or_error(any()).
+-spec delete(vhost:name(), rabbit_types:username()) -> rabbit_types:ok_or_error(any()).
-delete(VHostPath, ActingUser) ->
+delete(VHost, ActingUser) ->
%% FIXME: We are forced to delete the queues and exchanges outside
%% the TX below. Queue deletion involves sending messages to the queue
%% process, which in turn results in further mnesia actions and
%% eventually the termination of that process. Exchange deletion causes
%% notifications which must be sent outside the TX
- rabbit_log:info("Deleting vhost '~s'~n", [VHostPath]),
+ rabbit_log:info("Deleting vhost '~s'~n", [VHost]),
QDelFun = fun (Q) -> rabbit_amqqueue:delete(Q, false, false, ActingUser) end,
[begin
Name = amqqueue:get_name(Q),
assert_benign(rabbit_amqqueue:with(Name, QDelFun), ActingUser)
- end || Q <- rabbit_amqqueue:list(VHostPath)],
+ end || Q <- rabbit_amqqueue:list(VHost)],
[assert_benign(rabbit_exchange:delete(Name, false, ActingUser), ActingUser) ||
- #exchange{name = Name} <- rabbit_exchange:list(VHostPath)],
+ #exchange{name = Name} <- rabbit_exchange:list(VHost)],
Funs = rabbit_misc:execute_mnesia_transaction(
- with(VHostPath, fun () -> internal_delete(VHostPath, ActingUser) end)),
- ok = rabbit_event:notify(vhost_deleted, [{name, VHostPath},
+ with(VHost, fun () -> internal_delete(VHost, ActingUser) end)),
+ ok = rabbit_event:notify(vhost_deleted, [{name, VHost},
{user_who_performed_action, ActingUser}]),
[case Fun() of
ok -> ok;
- {error, {no_such_vhost, VHostPath}} -> ok
+ {error, {no_such_vhost, VHost}} -> ok
end || Fun <- Funs],
%% After vhost was deleted from mnesia DB, we try to stop vhost supervisors
%% on all the nodes.
- rabbit_vhost_sup_sup:delete_on_all_nodes(VHostPath),
+ rabbit_vhost_sup_sup:delete_on_all_nodes(VHost),
ok.
%% 50 ms
-define(AWAIT_SAMPLE_INTERVAL, 50).
--spec await_running_on_all_nodes(rabbit_types:vhost(), integer()) -> ok | {error, timeout}.
+-spec await_running_on_all_nodes(vhost:name(), integer()) -> ok | {error, timeout}.
await_running_on_all_nodes(VHost, Timeout) ->
Attempts = round(Timeout / ?AWAIT_SAMPLE_INTERVAL),
await_running_on_all_nodes0(VHost, Attempts).
@@ -161,13 +189,13 @@ await_running_on_all_nodes0(VHost, Attempts) ->
await_running_on_all_nodes0(VHost, Attempts - 1)
end.
--spec is_running_on_all_nodes(rabbit_types:vhost()) -> boolean().
+-spec is_running_on_all_nodes(vhost:name()) -> boolean().
is_running_on_all_nodes(VHost) ->
States = vhost_cluster_state(VHost),
lists:all(fun ({_Node, State}) -> State =:= running end,
States).
--spec vhost_cluster_state(rabbit_types:vhost()) -> [{atom(), atom()}].
+-spec vhost_cluster_state(vhost:name()) -> [{atom(), atom()}].
vhost_cluster_state(VHost) ->
Nodes = rabbit_nodes:all_running(),
lists:map(fun(Node) ->
@@ -182,9 +210,9 @@ vhost_cluster_state(VHost) ->
end,
Nodes).
-vhost_down(VHostPath) ->
+vhost_down(VHost) ->
ok = rabbit_event:notify(vhost_down,
- [{name, VHostPath},
+ [{name, VHost},
{node, node()},
{user_who_performed_action, ?INTERNAL_USER}]).
@@ -213,92 +241,86 @@ assert_benign({error, {absent, Q, _}}, ActingUser) ->
QName = amqqueue:get_name(Q),
rabbit_amqqueue:internal_delete(QName, ActingUser).
-internal_delete(VHostPath, ActingUser) ->
+internal_delete(VHost, ActingUser) ->
[ok = rabbit_auth_backend_internal:clear_permissions(
- proplists:get_value(user, Info), VHostPath, ActingUser)
- || Info <- rabbit_auth_backend_internal:list_vhost_permissions(VHostPath)],
- TopicPermissions = rabbit_auth_backend_internal:list_vhost_topic_permissions(VHostPath),
+ 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),
[ok = rabbit_auth_backend_internal:clear_topic_permissions(
- proplists:get_value(user, TopicPermission), VHostPath, ActingUser)
+ proplists:get_value(user, TopicPermission), VHost, ActingUser)
|| TopicPermission <- TopicPermissions],
- Fs1 = [rabbit_runtime_parameters:clear(VHostPath,
+ Fs1 = [rabbit_runtime_parameters:clear(VHost,
proplists:get_value(component, Info),
proplists:get_value(name, Info),
ActingUser)
- || Info <- rabbit_runtime_parameters:list(VHostPath)],
- Fs2 = [rabbit_policy:delete(VHostPath, proplists:get_value(name, Info), ActingUser)
- || Info <- rabbit_policy:list(VHostPath)],
- ok = mnesia:delete({rabbit_vhost, VHostPath}),
+ || 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}),
Fs1 ++ Fs2.
--spec exists(rabbit_types:vhost()) -> boolean().
+-spec exists(vhost:name()) -> boolean().
-exists(VHostPath) ->
- mnesia:dirty_read({rabbit_vhost, VHostPath}) /= [].
+exists(VHost) ->
+ mnesia:dirty_read({rabbit_vhost, VHost}) /= [].
--spec list() -> [rabbit_types:vhost()].
+-spec list_names() -> [vhost:name()].
+list_names() -> mnesia:dirty_all_keys(rabbit_vhost).
-list() ->
- mnesia:dirty_all_keys(rabbit_vhost).
+%% 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()).
-spec count() -> non_neg_integer().
count() ->
length(list()).
--spec with(rabbit_types:vhost(), rabbit_misc:thunk(A)) -> A.
+-spec with(vhost:name(), rabbit_misc:thunk(A)) -> A.
-with(VHostPath, Thunk) ->
+with(VHost, Thunk) ->
fun () ->
- case mnesia:read({rabbit_vhost, VHostPath}) of
+ case mnesia:read({rabbit_vhost, VHost}) of
[] ->
- mnesia:abort({no_such_vhost, VHostPath});
+ mnesia:abort({no_such_vhost, VHost});
[_V] ->
Thunk()
end
end.
-spec with_user_and_vhost
- (rabbit_types:username(), rabbit_types:vhost(), rabbit_misc:thunk(A)) -> A.
+ (rabbit_types:username(), vhost:name(), rabbit_misc:thunk(A)) -> A.
-with_user_and_vhost(Username, VHostPath, Thunk) ->
- rabbit_misc:with_user(Username, with(VHostPath, Thunk)).
+with_user_and_vhost(Username, VHost, Thunk) ->
+ rabbit_misc:with_user(Username, with(VHost, Thunk)).
%% Like with/2 but outside an Mnesia tx
--spec assert(rabbit_types:vhost()) -> 'ok'.
+-spec assert(vhost:name()) -> 'ok'.
-assert(VHostPath) -> case exists(VHostPath) of
+assert(VHost) -> case exists(VHost) of
true -> ok;
- false -> throw({error, {no_such_vhost, VHostPath}})
+ false -> throw({error, {no_such_vhost, VHost}})
end.
--spec update(rabbit_types:vhost(), fun((#vhost{}) -> #vhost{})) -> #vhost{}.
+-spec update(vhost:name(), fun((vhost:vhost()) -> vhost:vhost())) -> vhost:vhost().
-update(VHostPath, Fun) ->
- case mnesia:read({rabbit_vhost, VHostPath}) of
+update(VHost, Fun) ->
+ case mnesia:read({rabbit_vhost, VHost}) of
[] ->
- mnesia:abort({no_such_vhost, VHostPath});
+ mnesia:abort({no_such_vhost, VHost});
[V] ->
V1 = Fun(V),
ok = mnesia:write(rabbit_vhost, V1, write),
V1
end.
-limits_of(VHostPath) when is_binary(VHostPath) ->
- assert(VHostPath),
- case mnesia:dirty_read({rabbit_vhost, VHostPath}) of
- [] ->
- mnesia:abort({no_such_vhost, VHostPath});
- [#vhost{limits = Limits}] ->
- Limits
- end;
-limits_of(#vhost{virtual_host = Name}) ->
- limits_of(Name).
-
-set_limits(VHost = #vhost{}, undefined) ->
- VHost#vhost{limits = undefined};
-set_limits(VHost = #vhost{}, Limits) ->
- VHost#vhost{limits = Limits}.
+set_limits(VHost, undefined) ->
+ vhost:set_limits(VHost, undefined);
+set_limits(VHost, Limits) ->
+ vhost:set_limits(VHost, Limits).
dir(Vhost) ->
@@ -307,8 +329,7 @@ dir(Vhost) ->
msg_store_dir_path(VHost) ->
EncodedName = dir(VHost),
- rabbit_data_coercion:to_list(filename:join([msg_store_dir_base(),
- EncodedName])).
+ rabbit_data_coercion:to_list(filename:join([msg_store_dir_base(), EncodedName])).
msg_store_dir_wildcard() ->
rabbit_data_coercion:to_list(filename:join([msg_store_dir_base(), "*"])).
@@ -317,36 +338,46 @@ msg_store_dir_base() ->
Dir = rabbit_mnesia:dir(),
filename:join([Dir, "msg_stores", "vhosts"]).
+-spec trim_tag(list() | binary() | atom()) -> atom().
+trim_tag(Val) ->
+ rabbit_data_coercion:to_atom(string:trim(rabbit_data_coercion:to_list(Val))).
+
%%----------------------------------------------------------------------------
infos(Items, X) -> [{Item, i(Item, X)} || Item <- Items].
-i(name, VHost) -> VHost;
-i(tracing, VHost) -> rabbit_trace:enabled(VHost);
-i(cluster_state, VHost) -> vhost_cluster_state(VHost);
-i(Item, _) -> throw({bad_argument, Item}).
-
--spec info(rabbit_types:vhost()) -> rabbit_types:infos().
-
-info(VHost) -> infos(?INFO_KEYS, VHost).
-
--spec info(rabbit_types:vhost(), rabbit_types:info_keys())
- -> rabbit_types:infos().
+i(name, VHost) -> vhost:get_name(VHost);
+i(tracing, VHost) -> rabbit_trace:enabled(vhost:get_name(VHost));
+i(cluster_state, VHost) -> vhost_cluster_state(vhost:get_name(VHost));
+i(description, VHost) -> vhost:get_description(VHost);
+i(tags, VHost) -> vhost:get_tags(VHost);
+i(metadata, VHost) -> vhost:get_metadata(VHost);
+i(Item, VHost) ->
+ rabbit_log:error("Don't know how to compute a virtual host info item '~s' for virtual host '~p'", [Item, VHost]),
+ throw({bad_argument, Item}).
+
+-spec info(vhost:vhost() | vhost:name()) -> rabbit_types:infos().
+
+info(VHost) when ?is_vhost(VHost) ->
+ infos(?INFO_KEYS, VHost);
+info(Key) ->
+ case mnesia:dirty_read({rabbit_vhost, Key}) of
+ [] -> [];
+ [VHost] -> infos(?INFO_KEYS, VHost)
+ end.
+-spec info(vhost:vhost(), rabbit_types:info_keys()) -> rabbit_types:infos().
info(VHost, Items) -> infos(Items, VHost).
-spec info_all() -> [rabbit_types:infos()].
-
-info_all() -> info_all(?INFO_KEYS).
+info_all() -> info_all(?INFO_KEYS).
-spec info_all(rabbit_types:info_keys()) -> [rabbit_types:infos()].
-
-info_all(Items) -> [info(VHost, Items) || VHost <- list()].
+info_all(Items) -> [info(VHost, Items) || VHost <- all()].
info_all(Ref, AggregatorPid) -> info_all(?INFO_KEYS, Ref, AggregatorPid).
--spec info_all(rabbit_types:info_keys(), reference(), pid()) ->
- 'ok'.
+-spec info_all(rabbit_types:info_keys(), reference(), pid()) -> 'ok'.
info_all(Items, Ref, AggregatorPid) ->
rabbit_control_misc:emitting_map(
- AggregatorPid, Ref, fun(VHost) -> info(VHost, Items) end, list()).
+ AggregatorPid, Ref, fun(VHost) -> info(VHost, Items) end, all()).
diff --git a/src/rabbit_vhost_limit.erl b/src/rabbit_vhost_limit.erl
index f1ae63cfa0..6eefae349a 100644
--- a/src/rabbit_vhost_limit.erl
+++ b/src/rabbit_vhost_limit.erl
@@ -79,18 +79,18 @@ query_limits(VHost) ->
end.
--spec list() -> [{rabbit_types:vhost(), rabbit_types:infos()}].
+-spec list() -> [{vhost:name(), rabbit_types:infos()}].
list() ->
query_limits('_').
--spec list(rabbit_types:vhost()) -> rabbit_types:infos().
+-spec list(vhost:name()) -> rabbit_types:infos().
list(VHost) ->
case query_limits(VHost) of
[] -> [];
[{VHost, Value}] -> Value
end.
--spec is_over_connection_limit(rabbit_types:vhost()) -> {true, non_neg_integer()} | false.
+-spec is_over_connection_limit(vhost:name()) -> {true, non_neg_integer()} | false.
is_over_connection_limit(VirtualHost) ->
case rabbit_vhost_limit:connection_limit(VirtualHost) of
@@ -112,7 +112,7 @@ is_over_connection_limit(VirtualHost) ->
{ok, _Limit} -> false
end.
--spec would_exceed_queue_limit(non_neg_integer(), rabbit_types:vhost()) ->
+-spec would_exceed_queue_limit(non_neg_integer(), vhost:name()) ->
{true, non_neg_integer(), non_neg_integer()} | false.
would_exceed_queue_limit(AdditionalCount, VirtualHost) ->
@@ -140,7 +140,7 @@ would_exceed_queue_limit(AdditionalCount, VirtualHost) ->
false
end.
--spec is_over_queue_limit(rabbit_types:vhost()) -> {true, non_neg_integer()} | false.
+-spec is_over_queue_limit(vhost:name()) -> {true, non_neg_integer()} | false.
is_over_queue_limit(VirtualHost) ->
case would_exceed_queue_limit(1, VirtualHost) of
diff --git a/src/rabbit_vhost_sup_sup.erl b/src/rabbit_vhost_sup_sup.erl
index d9d6091c39..ceb00026f3 100644
--- a/src/rabbit_vhost_sup_sup.erl
+++ b/src/rabbit_vhost_sup_sup.erl
@@ -263,7 +263,7 @@ vhost_restart_strategy() ->
end.
check() ->
- VHosts = rabbit_vhost:list(),
+ VHosts = rabbit_vhost:list_names(),
lists:filter(
fun(V) ->
case rabbit_vhost_sup_sup:get_vhost_sup(V) of
diff --git a/src/vhost.erl b/src/vhost.erl
new file mode 100644
index 0000000000..7bcb6d18e5
--- /dev/null
+++ b/src/vhost.erl
@@ -0,0 +1,181 @@
+%% The contents of this file are subject to the Mozilla Public License
+%% Version 1.1 (the "License"); you may not use this file except in
+%% compliance with the License. You may obtain a copy of the License
+%% at https://www.mozilla.org/MPL/
+%%
+%% Software distributed under the License is distributed on an "AS IS"
+%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See
+%% the License for the specific language governing rights and
+%% limitations under the License.
+%%
+%% The Original Code is RabbitMQ.
+%%
+%% The Initial Developer of the Original Code is GoPivotal, Inc.
+%% Copyright (c) 2018-2019 Pivotal Software, Inc. All rights reserved.
+%%
+
+-module(vhost).
+
+-include_lib("rabbit_common/include/rabbit.hrl").
+-include("vhost.hrl").
+
+-export([
+ new/2,
+ new/3,
+ fields/0,
+ fields/1,
+ info_keys/0,
+ record_version_to_use/0,
+ upgrade/1,
+ upgrade_to/2,
+ pattern_match_all/0,
+ get_name/1,
+ get_limits/1,
+ get_metadata/1,
+ get_description/1,
+ get_tags/1,
+ set_limits/2
+]).
+
+-define(record_version, vhost_v2).
+
+-type(name() :: binary()).
+
+-type(metadata_key() :: atom()).
+
+-type(metadata() :: #{description => binary(),
+ tags => [atom()],
+ metadata_key() => any()} | undefined).
+
+-type vhost() :: vhost_v1:vhost_v1() | vhost_v2().
+
+-record(vhost, {
+ %% name as a binary
+ virtual_host :: name() | '_',
+ %% proplist of limits configured, if any
+ limits :: list() | '_',
+ metadata :: metadata() | '_'
+}).
+
+-type vhost_v2() :: #vhost{
+ virtual_host :: name(),
+ limits :: list(),
+ metadata :: metadata()
+ }.
+
+-type vhost_pattern() :: vhost_v1:vhost_v1_pattern() |
+ vhost_v2_pattern().
+-type vhost_v2_pattern() :: #vhost{
+ virtual_host :: name() | '_',
+ limits :: '_',
+ metadata :: '_'
+ }.
+
+-export_type([name/0,
+ metadata_key/0,
+ metadata/0,
+ vhost/0,
+ vhost_v2/0,
+ vhost_pattern/0,
+ vhost_v2_pattern/0]).
+
+-spec new(name(), list()) -> vhost().
+new(Name, Limits) ->
+ case record_version_to_use() of
+ ?record_version ->
+ #vhost{virtual_host = Name, limits = Limits};
+ _ ->
+ vhost_v1:new(Name, Limits)
+ end.
+
+-spec new(rabbit_vhost:name(), list(), map()) -> vhost().
+new(Name, Limits, Metadata) ->
+ case record_version_to_use() of
+ ?record_version ->
+ #vhost{virtual_host = Name, limits = Limits, metadata = Metadata};
+ _ ->
+ vhost_v1:new(Name, Limits)
+ end.
+
+-spec record_version_to_use() -> vhost_v1 | vhost_v2.
+
+record_version_to_use() ->
+ case rabbit_feature_flags:is_enabled(virtual_host_metadata) of
+ true -> ?record_version;
+ false -> vhost_v1:record_version_to_use()
+ end.
+
+-spec upgrade(vhost()) -> vhost().
+
+upgrade(#vhost{} = VHost) -> VHost;
+upgrade(OldVHost) -> upgrade_to(record_version_to_use(), OldVHost).
+
+-spec upgrade_to
+(vhost_v2, vhost()) -> vhost_v2();
+(vhost_v1, vhost_v1:vhost_v1()) -> vhost_v1:vhost_v1().
+
+upgrade_to(?record_version, #vhost{} = VHost) ->
+ VHost;
+upgrade_to(?record_version, OldVHost) ->
+ Fields = erlang:tuple_to_list(OldVHost) ++ [#{}],
+ #vhost{} = erlang:list_to_tuple(Fields);
+upgrade_to(Version, OldVHost) ->
+ vhost_v1:upgrade_to(Version, OldVHost).
+
+
+fields() ->
+ case record_version_to_use() of
+ ?record_version -> fields(?record_version);
+ _ -> vhost_v1:fields()
+ end.
+
+fields(?record_version) -> record_info(fields, vhost);
+fields(Version) -> vhost_v1:fields(Version).
+
+info_keys() ->
+ case record_version_to_use() of
+ %% note: this reports description and tags separately even though
+ %% they are stored in the metadata map. MK.
+ ?record_version -> [name, description, tags, metadata, tracing, cluster_state];
+ _ -> vhost_v1:info_keys()
+ end.
+
+-spec pattern_match_all() -> vhost_pattern().
+
+pattern_match_all() ->
+ case record_version_to_use() of
+ ?record_version -> #vhost{_ = '_'};
+ _ -> vhost_v1:pattern_match_all()
+ end.
+
+-spec get_name(vhost()) -> name().
+get_name(#vhost{virtual_host = Value}) -> Value;
+get_name(VHost) -> vhost_v1:get_name(VHost).
+
+-spec get_limits(vhost()) -> list().
+get_limits(#vhost{limits = Value}) -> Value;
+get_limits(VHost) -> vhost_v1:get_limits(VHost).
+
+-spec get_metadata(vhost()) -> metadata().
+get_metadata(#vhost{metadata = Value}) -> Value;
+get_metadata(VHost) -> vhost_v1:get_limits(VHost).
+
+-spec get_description(vhost()) -> binary().
+get_description(#vhost{} = VHost) ->
+ maps:get(description, get_metadata(VHost), undefined);
+get_description(VHost) ->
+ vhost_v1:get_description(VHost).
+
+-spec get_tags(vhost()) -> [atom()].
+get_tags(#vhost{} = VHost) ->
+ maps:get(tags, get_metadata(VHost), undefined);
+get_tags(VHost) ->
+ vhost_v1:get_tags(VHost).
+
+set_limits(VHost, Value) ->
+ case record_version_to_use() of
+ ?record_version ->
+ VHost#vhost{limits = Value};
+ _ ->
+ vhost_v1:set_limits(VHost, Value)
+ end.
diff --git a/src/vhost_v1.erl b/src/vhost_v1.erl
new file mode 100644
index 0000000000..0dec8d3373
--- /dev/null
+++ b/src/vhost_v1.erl
@@ -0,0 +1,115 @@
+%% The contents of this file are subject to the Mozilla Public License
+%% Version 1.1 (the "License"); you may not use this file except in
+%% compliance with the License. You may obtain a copy of the License
+%% at https://www.mozilla.org/MPL/
+%%
+%% Software distributed under the License is distributed on an "AS IS"
+%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See
+%% the License for the specific language governing rights and
+%% limitations under the License.
+%%
+%% The Original Code is RabbitMQ.
+%%
+%% The Initial Developer of the Original Code is GoPivotal, Inc.
+%% Copyright (c) 2018-2019 Pivotal Software, Inc. All rights reserved.
+%%
+
+-module(vhost_v1).
+
+-include("vhost.hrl").
+
+-export([new/2,
+ new/3,
+ upgrade/1,
+ upgrade_to/2,
+ fields/0,
+ fields/1,
+ info_keys/0,
+ field_name/0,
+ record_version_to_use/0,
+ pattern_match_all/0,
+ get_name/1,
+ get_limits/1,
+ get_metadata/1,
+ get_description/1,
+ get_tags/1,
+ set_limits/2
+]).
+
+-define(record_version, ?MODULE).
+
+%% Represents a vhost.
+%%
+%% Historically this record had 2 arguments although the 2nd
+%% was never used (`dummy`, always undefined). This is because
+%% single field records were/are illegal in OTP.
+%%
+%% As of 3.6.x, the second argument is vhost limits,
+%% which is actually used and has the same default.
+%% Nonetheless, this required a migration, see rabbit_upgrade_functions.
+
+-record(vhost, {
+ %% name as a binary
+ virtual_host :: vhost:name() | '_',
+ %% proplist of limits configured, if any
+ limits :: list() | '_'}).
+
+-type vhost() :: vhost_v1().
+-type vhost_v1() :: #vhost{
+ virtual_host :: vhost:name(),
+ limits :: list()
+ }.
+
+-export_type([vhost/0,
+ vhost_v1/0,
+ vhost_pattern/0,
+ vhost_v1_pattern/0]).
+
+
+-spec new(rabbit_vhost:name(), list()) -> vhost().
+new(Name, Limits) ->
+ #vhost{virtual_host = Name, limits = Limits}.
+
+-spec new(rabbit_vhost:name(), list(), map()) -> vhost().
+new(Name, Limits, _Metadata) ->
+ #vhost{virtual_host = Name, limits = Limits}.
+
+
+-spec record_version_to_use() -> vhost_v1.
+record_version_to_use() ->
+ ?record_version.
+
+-spec upgrade(vhost()) -> vhost().
+upgrade(#vhost{} = VHost) -> VHost.
+
+-spec upgrade_to(vhost_v1, vhost()) -> vhost().
+upgrade_to(?record_version, #vhost{} = VHost) ->
+ VHost.
+
+fields() -> fields(?record_version).
+
+fields(?record_version) -> record_info(fields, vhost).
+
+field_name() -> #vhost.virtual_host.
+
+info_keys() -> [name, tracing, cluster_state].
+
+-type vhost_pattern() :: vhost_v1_pattern().
+-type vhost_v1_pattern() :: #vhost{
+ virtual_host :: vhost:name() | '_',
+ limits :: '_'
+ }.
+
+-spec pattern_match_all() -> vhost_pattern().
+
+pattern_match_all() -> #vhost{_ = '_'}.
+
+get_name(#vhost{virtual_host = Value}) -> Value.
+get_limits(#vhost{limits = Value}) -> Value.
+
+get_metadata(_VHost) -> undefined.
+get_description(_VHost) -> undefined.
+get_tags(_VHost) -> undefined.
+
+set_limits(VHost, Value) ->
+ VHost#vhost{limits = Value}.
diff --git a/test/topic_permission_SUITE.erl b/test/topic_permission_SUITE.erl
index 3960766ad6..ccbb884e6f 100644
--- a/test/topic_permission_SUITE.erl
+++ b/test/topic_permission_SUITE.erl
@@ -156,10 +156,10 @@ topic_permission_checks1(_Config) ->
0 = length(ets:tab2list(rabbit_topic_permission)),
rabbit_misc:execute_mnesia_transaction(fun() ->
ok = mnesia:write(rabbit_vhost,
- #vhost{virtual_host = <<"/">>},
+ vhost:new(<<"/">>, []),
write),
ok = mnesia:write(rabbit_vhost,
- #vhost{virtual_host = <<"other-vhost">>},
+ vhost:new(<<"other-vhost">>, []),
write)
end),
rabbit_auth_backend_internal:add_user(<<"guest">>, <<"guest">>, <<"acting-user">>),