Skip to content

Commit fff6490

Browse files
Merge pull request #7916 from rabbitmq/mergify/bp/v3.11.x/pr-7915
Introduce a way to update virtual host metadata using CLI tools (backport #7914) (backport #7915)
2 parents c5cede1 + 2b8ea66 commit fff6490

File tree

8 files changed

+286
-35
lines changed

8 files changed

+286
-35
lines changed

deps/rabbit/src/rabbit_vhost.erl

Lines changed: 15 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@
1515
set_limits/2, vhost_cluster_state/1, is_running_on_all_nodes/1, await_running_on_all_nodes/2,
1616
list/0, count/0, list_names/0, all/0, all_tagged_with/1]).
1717
-export([parse_tags/1, update_metadata/2, tag_with/2, untag_from/2, update_tags/2, update_tags/3]).
18+
-export([update_metadata/3]).
1819
-export([lookup/1, default_name/0]).
1920
-export([info/1, info/2, info_all/0, info_all/1, info_all/2, info_all/3]).
2021
-export([dir/1, msg_store_dir_path/1, msg_store_dir_wildcard/0, config_file_path/1, ensure_config_file/1]).
@@ -27,8 +28,8 @@
2728
%% API
2829
%%
2930

31+
%% this module deals with user inputs, so accepts more than just atoms
3032
-type vhost_tag() :: atom() | string() | binary().
31-
-export_type([vhost_tag/0]).
3233

3334
recover() ->
3435
%% Clear out remnants of old incarnation, in case we restarted
@@ -235,28 +236,14 @@ do_add(Name, Metadata, ActingUser) ->
235236
{error, Msg}
236237
end.
237238

239+
238240
-spec update(vhost:name(), binary(), [atom()], rabbit_types:username()) -> rabbit_types:ok_or_error(any()).
239241
update(Name, Description, Tags, ActingUser) ->
240-
rabbit_misc:execute_mnesia_transaction(
241-
fun () ->
242-
case mnesia:wread({rabbit_vhost, Name}) of
243-
[] ->
244-
{error, {no_such_vhost, Name}};
245-
[VHost0] ->
246-
VHost = vhost:merge_metadata(VHost0, #{description => Description, tags => Tags}),
247-
rabbit_log:debug("Updating a virtual host record ~tp", [VHost]),
248-
ok = mnesia:write(rabbit_vhost, VHost, write),
249-
rabbit_event:notify(vhost_updated, info(VHost)
250-
++ [{user_who_performed_action, ActingUser},
251-
{description, Description},
252-
{tags, Tags}]),
253-
ok
254-
end
255-
end).
242+
Metadata = #{description => Description, tags => Tags},
243+
update_metadata(Name, Metadata, ActingUser).
256244

257245

258246
-spec delete(vhost:name(), rabbit_types:username()) -> rabbit_types:ok_or_error(any()).
259-
260247
delete(VHost, ActingUser) ->
261248
%% FIXME: We are forced to delete the queues and exchanges outside
262249
%% the TX below. Queue deletion involves sending messages to the queue
@@ -548,6 +535,16 @@ update_metadata(VHostName, Fun) ->
548535
vhost:set_metadata(Record, Meta)
549536
end).
550537

538+
-spec update_metadata(vhost:name(), vhost:metadata(), rabbit_types:username()) -> rabbit_types:ok_or_error(any()).
539+
update_metadata(Name, Metadata0, _ActingUser) ->
540+
Metadata = maps:with([description, tags, default_queue_type], Metadata0),
541+
rabbit_misc:execute_mnesia_transaction(fun() ->
542+
update_metadata(Name, fun(CurrentMetadata) ->
543+
maps:merge(CurrentMetadata, Metadata)
544+
end)
545+
end),
546+
ok.
547+
551548
are_different0([], []) ->
552549
false;
553550
are_different0([], [_ | _]) ->
Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,18 @@
1+
## This Source Code Form is subject to the terms of the Mozilla Public
2+
## License, v. 2.0. If a copy of the MPL was not distributed with this
3+
## file, You can obtain one at https://mozilla.org/MPL/2.0/.
4+
##
5+
## Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
6+
defmodule RabbitMQ.CLI.Core.VirtualHosts do
7+
def parse_tags(tags) do
8+
case tags do
9+
nil ->
10+
nil
11+
12+
val ->
13+
String.split(val, ",", trim: true)
14+
|> Enum.map(&String.trim/1)
15+
|> Enum.map(&String.to_atom/1)
16+
end
17+
end
18+
end

deps/rabbitmq_cli/lib/rabbitmq/cli/ctl/commands/add_vhost_command.ex

Lines changed: 8 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -5,7 +5,7 @@
55
## Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
66

77
defmodule RabbitMQ.CLI.Ctl.Commands.AddVhostCommand do
8-
alias RabbitMQ.CLI.Core.{DocGuide, ExitCodes, FeatureFlags, Helpers}
8+
alias RabbitMQ.CLI.Core.{DocGuide, ExitCodes, FeatureFlags, Helpers, VirtualHosts}
99

1010
@behaviour RabbitMQ.CLI.CommandBehaviour
1111

@@ -25,7 +25,12 @@ defmodule RabbitMQ.CLI.Ctl.Commands.AddVhostCommand do
2525
tags: tags,
2626
default_queue_type: default_qt
2727
}) do
28-
meta = %{description: desc, tags: parse_tags(tags), default_queue_type: default_qt}
28+
meta = %{
29+
description: desc,
30+
tags: VirtualHosts.parse_tags(tags),
31+
default_queue_type: default_qt
32+
}
33+
2934
# check if the respective feature flag is enabled
3035
case default_qt do
3136
"quorum" ->
@@ -59,7 +64,7 @@ defmodule RabbitMQ.CLI.Ctl.Commands.AddVhostCommand do
5964
:rabbit_misc.rpc_call(node_name, :rabbit_vhost, :add, [
6065
vhost,
6166
desc,
62-
parse_tags(tags),
67+
VirtualHosts.parse_tags(tags),
6368
Helpers.cli_acting_user()
6469
])
6570
end
@@ -105,14 +110,4 @@ defmodule RabbitMQ.CLI.Ctl.Commands.AddVhostCommand do
105110
def description(), do: "Creates a virtual host"
106111

107112
def banner([vhost], _), do: "Adding vhost \"#{vhost}\" ..."
108-
109-
#
110-
# Implementation
111-
#
112-
113-
def parse_tags(tags) do
114-
String.split(tags, ",", trim: true)
115-
|> Enum.map(&String.trim/1)
116-
|> Enum.map(&String.to_atom/1)
117-
end
118113
end
Lines changed: 114 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,114 @@
1+
## This Source Code Form is subject to the terms of the Mozilla Public
2+
## License, v. 2.0. If a copy of the MPL was not distributed with this
3+
## file, You can obtain one at https://mozilla.org/MPL/2.0/.
4+
##
5+
## Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
6+
7+
defmodule RabbitMQ.CLI.Ctl.Commands.UpdateVhostMetadataCommand do
8+
alias RabbitMQ.CLI.Core.{DocGuide, ExitCodes, Helpers, VirtualHosts}
9+
10+
@behaviour RabbitMQ.CLI.CommandBehaviour
11+
12+
@metadata_keys [:description, :tags, :default_queue_type]
13+
14+
def switches(), do: [description: :string, tags: :string, default_queue_type: :string]
15+
def aliases(), do: [d: :description]
16+
17+
def merge_defaults(args, opts) do
18+
{args, opts}
19+
end
20+
21+
use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
22+
23+
def validate(args, _) when length(args) == 0 do
24+
{:validation_failure, :not_enough_args}
25+
end
26+
27+
def validate(args, _) when length(args) > 1 do
28+
{:validation_failure, :too_many_args}
29+
end
30+
31+
def validate([_vhost], opts) do
32+
m = :maps.with(@metadata_keys, opts)
33+
34+
case map_size(m) do
35+
0 ->
36+
{:validation_failure, :not_enough_args}
37+
38+
_ ->
39+
# description and tags can be anything but default queue type must
40+
# be a value from a known set
41+
case m[:default_queue_type] do
42+
nil ->
43+
:ok
44+
45+
"quorum" ->
46+
:ok
47+
48+
"stream" ->
49+
:ok
50+
51+
"classic" ->
52+
:ok
53+
54+
other ->
55+
{:validation_failure,
56+
{:bad_argument,
57+
"Default queue type must be one of: quorum, stream, classic. Provided: #{other}"}}
58+
end
59+
end
60+
end
61+
62+
def validate(_, _), do: :ok
63+
64+
def run([vhost], %{node: node_name} = opts) do
65+
meta = :maps.with(@metadata_keys, opts)
66+
tags = meta[:tags]
67+
68+
meta =
69+
case tags do
70+
nil -> meta
71+
other -> %{meta | tags: VirtualHosts.parse_tags(other)}
72+
end
73+
74+
:rabbit_misc.rpc_call(node_name, :rabbit_vhost, :update_metadata, [
75+
vhost,
76+
meta,
77+
Helpers.cli_acting_user()
78+
])
79+
end
80+
81+
def output({:error, :invalid_queue_type}, _opts) do
82+
{:error, ExitCodes.exit_usage(), "Unsupported default queue type"}
83+
end
84+
85+
use RabbitMQ.CLI.DefaultOutput
86+
87+
def usage,
88+
do:
89+
"update_vhost_metadata <vhost> [--description <description>] [--tags \"<tag1>,<tag2>,<...>\"] [--default-queue-type <quorum|classic|stream>]"
90+
91+
def usage_additional() do
92+
[
93+
["<vhost>", "Virtual host name"],
94+
["--description <description>", "Virtual host description"],
95+
["--tags <tag1,tag2>", "Comma-separated list of tags"],
96+
[
97+
"--default-queue-type <quorum|classic|stream>",
98+
"Queue type to use if no type is explicitly provided by the client"
99+
]
100+
]
101+
end
102+
103+
def usage_doc_guides() do
104+
[
105+
DocGuide.virtual_hosts()
106+
]
107+
end
108+
109+
def help_section(), do: :virtual_hosts
110+
111+
def description(), do: "Updates metadata (tags, description, default queue type) a virtual host"
112+
113+
def banner([vhost], _), do: "Updating metadata of vhost \"#{vhost}\" ..."
114+
end

deps/rabbitmq_cli/test/ctl/add_vhost_command_test.exs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -77,7 +77,7 @@ defmodule AddVhostCommandTest do
7777
end
7878

7979
@tag vhost: @vhost
80-
test "run: vhost tags are conformed to a list", context do
80+
test "run: vhost tags are coerced to a list", context do
8181
opts = Map.merge(context[:opts], %{description: "My vhost", tags: "my_tag"})
8282
assert @command.run([context[:vhost]], opts) == :ok
8383
record = list_vhosts() |> Enum.find(fn record -> record[:name] == context[:vhost] end)
Lines changed: 111 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,111 @@
1+
## This Source Code Form is subject to the terms of the Mozilla Public
2+
## License, v. 2.0. If a copy of the MPL was not distributed with this
3+
## file, You can obtain one at https://mozilla.org/MPL/2.0/.
4+
##
5+
## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
6+
7+
defmodule UpdateVhostMetadataCommandTest do
8+
use ExUnit.Case, async: false
9+
import TestHelper
10+
11+
@command RabbitMQ.CLI.Ctl.Commands.UpdateVhostMetadataCommand
12+
@vhost "update-metadata-test"
13+
14+
setup_all do
15+
RabbitMQ.CLI.Core.Distribution.start()
16+
{:ok, opts: %{node: get_rabbit_hostname()}}
17+
end
18+
19+
setup context do
20+
on_exit(context, fn -> delete_vhost(context[:vhost]) end)
21+
:ok
22+
end
23+
24+
test "validate: no arguments fails validation" do
25+
assert @command.validate([], %{}) == {:validation_failure, :not_enough_args}
26+
end
27+
28+
test "validate: too many arguments fails validation" do
29+
assert @command.validate(["test", "extra"], %{}) == {:validation_failure, :too_many_args}
30+
end
31+
32+
test "validate: virtual host name without options fails validation" do
33+
assert @command.validate(["a-vhost"], %{}) == {:validation_failure, :not_enough_args}
34+
end
35+
36+
test "validate: virtual host name and one or more metadata options succeeds" do
37+
assert @command.validate(["a-vhost"], %{description: "Used by team A"}) == :ok
38+
39+
assert @command.validate(["a-vhost"], %{
40+
description: "Used by team A for QA purposes",
41+
tags: "qa,team-a"
42+
}) == :ok
43+
44+
assert @command.validate(["a-vhost"], %{
45+
description: "Used by team A for QA purposes",
46+
tags: "qa,team-a",
47+
default_queue_type: "quorum"
48+
}) == :ok
49+
end
50+
51+
test "validate: unknown default queue type fails validation" do
52+
assert @command.validate(["a-vhost"], %{
53+
description: "Used by team A for QA purposes",
54+
tags: "qa,team-a",
55+
default_queue_type: "unknown"
56+
}) ==
57+
{:validation_failure,
58+
{:bad_argument,
59+
"Default queue type must be one of: quorum, stream, classic. Provided: unknown"}}
60+
end
61+
62+
test "run: passing a valid vhost name and description succeeds", context do
63+
add_vhost(@vhost)
64+
desc = "desc 2"
65+
66+
assert @command.run([@vhost], Map.merge(context[:opts], %{description: desc})) == :ok
67+
vh = find_vhost(@vhost)
68+
69+
assert vh
70+
assert vh[:description] == desc
71+
end
72+
73+
test "run: passing a valid vhost name and a set of tags succeeds", context do
74+
add_vhost(@vhost)
75+
tags = "a1,b2,c3"
76+
77+
assert @command.run([@vhost], Map.merge(context[:opts], %{tags: tags})) == :ok
78+
vh = find_vhost(@vhost)
79+
80+
assert vh
81+
assert vh[:tags] == [:a1, :b2, :c3]
82+
end
83+
84+
test "run: attempt to use a non-existent virtual host fails", context do
85+
vh = "a-non-existent-3882-vhost"
86+
87+
assert match?(
88+
{:error, {:no_such_vhost, _}},
89+
@command.run([vh], Map.merge(context[:opts], %{description: "irrelevant"}))
90+
)
91+
end
92+
93+
test "run: attempt to use an unreachable node returns a nodedown" do
94+
opts = %{node: :jake@thedog, timeout: 200, description: "does not matter"}
95+
assert match?({:badrpc, _}, @command.run(["na"], opts))
96+
end
97+
98+
test "run: vhost tags are coerced to a list", context do
99+
add_vhost(@vhost)
100+
101+
opts = Map.merge(context[:opts], %{description: "My vhost", tags: "my_tag"})
102+
assert @command.run([@vhost], opts) == :ok
103+
vh = find_vhost(@vhost)
104+
assert vh[:tags] == [:my_tag]
105+
end
106+
107+
test "banner", context do
108+
assert @command.banner([@vhost], context[:opts]) =~
109+
~r/Updating metadata of vhost/
110+
end
111+
end

deps/rabbitmq_cli/test/test_helper.exs

Lines changed: 13 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -41,8 +41,19 @@ defmodule TestHelper do
4141
:rpc.call(get_rabbit_hostname(), :rabbit_nodes, :cluster_name, [])
4242
end
4343

44-
def add_vhost(name) do
45-
:rpc.call(get_rabbit_hostname(), :rabbit_vhost, :add, [name, "acting-user"])
44+
def add_vhost(name, meta \\ %{}) do
45+
:rpc.call(get_rabbit_hostname(), :rabbit_vhost, :add, [name, meta, "acting-user"])
46+
end
47+
48+
def find_vhost(name) do
49+
case :rpc.call(get_rabbit_hostname(), :rabbit_vhost, :lookup, [name]) do
50+
{:error, _} = err ->
51+
err
52+
53+
vhost_rec ->
54+
{:vhost, _name, limits, meta} = vhost_rec
55+
Map.merge(meta, %{name: name, limits: limits})
56+
end
4657
end
4758

4859
def delete_vhost(name) do

0 commit comments

Comments
 (0)