From 2482c9fe4b540d9ef9a66cd6ca3534015eeb88bc Mon Sep 17 00:00:00 2001 From: Alexey Lebedeff Date: Mon, 8 Aug 2016 19:31:27 +0300 Subject: [PATCH] Add support for listing only local queues Partially implements https://github.com/rabbitmq/rabbitmq-server/issues/851 - Made old `--online`/`--offline` options mutually exclusive between themselves and the new `--local` option - Added documentation both for the old and the new option - Fixed some ugly indentation in generated usage (only `set_policy` wrapped line remains unfixed) - Added integration test suite for `rabbitmqctl list_queues` --- docs/rabbitmqctl.1.xml | 42 ++++++- docs/usage.xsl | 10 +- include/rabbit_cli.hrl | 2 + src/rabbit_cli.erl | 35 +++--- src/rabbit_control_main.erl | 21 ++-- test/rabbitmqctl_integration_SUITE.erl | 146 +++++++++++++++++++++++++ test/unit_SUITE.erl | 45 +++++--- 7 files changed, 257 insertions(+), 44 deletions(-) create mode 100644 test/rabbitmqctl_integration_SUITE.erl diff --git a/docs/rabbitmqctl.1.xml b/docs/rabbitmqctl.1.xml index ec864af6cf9f..7dd926eba886 100644 --- a/docs/rabbitmqctl.1.xml +++ b/docs/rabbitmqctl.1.xml @@ -749,9 +749,7 @@ - - authenticate_user username password - + authenticate_user username password @@ -1237,13 +1235,49 @@ - list_queues -p vhost queueinfoitem ... + + list_queues -p vhost --offline--online--local queueinfoitem ... + Returns queue details. Queue details of the / virtual host are returned if the "-p" flag is absent. The "-p" flag can be used to override this default. + + Displayed queues can be filtered by their status or + location using one of the following mutually exclusive + options: + + + --offline + + + List only those durable queues that are not + currently running - i.e. they are located on + inaccessible nodes. + + + + + --online + + + List queues that are currently live. + + + + + --local + + + List only those queues whose master process is + located on the current node. + + + + + The queueinfoitem parameter is used to indicate which queue information items to include in the results. The column order in the diff --git a/docs/usage.xsl b/docs/usage.xsl index 586f830327e2..0d3cb6f235c4 100644 --- a/docs/usage.xsl +++ b/docs/usage.xsl @@ -8,7 +8,7 @@ encoding="UTF-8" indent="no"/> - + %% Generated, do not edit! @@ -68,6 +68,14 @@ usage() -> %QUOTE%Usage: + + [ + + + | + + ] + [] <> diff --git a/include/rabbit_cli.hrl b/include/rabbit_cli.hrl index a0d1ecfdd519..b1cf41261f10 100644 --- a/include/rabbit_cli.hrl +++ b/include/rabbit_cli.hrl @@ -29,6 +29,7 @@ -define(RAM_OPT, "--ram"). -define(OFFLINE_OPT, "--offline"). -define(ONLINE_OPT, "--online"). +-define(LOCAL_OPT, "--local"). -define(NODE_DEF(Node), {?NODE_OPT, {option, Node}}). @@ -46,6 +47,7 @@ -define(RAM_DEF, {?RAM_OPT, flag}). -define(OFFLINE_DEF, {?OFFLINE_OPT, flag}). -define(ONLINE_DEF, {?ONLINE_OPT, flag}). +-define(LOCAL_DEF, {?LOCAL_OPT, flag}). %% Subset of standartized exit codes from sysexits.h, see %% https://github.com/rabbitmq/rabbitmq-server/issues/396 for discussion. diff --git a/src/rabbit_cli.erl b/src/rabbit_cli.erl index 6b3548221747..ca00226ba6ec 100644 --- a/src/rabbit_cli.erl +++ b/src/rabbit_cli.erl @@ -18,7 +18,7 @@ -include("rabbit_cli.hrl"). -export([main/3, start_distribution/0, start_distribution/1, - parse_arguments/4, filter_opts/2, + parse_arguments/4, mutually_exclusive_flags/3, rpc_call/4, rpc_call/5, rpc_call/7]). %%---------------------------------------------------------------------------- @@ -42,8 +42,7 @@ [{string(), optdef()}], string(), [string()]) -> parse_result(). --spec filter_opts([{option_name(), option_value()}], [option_name()]) -> - [boolean()]. +-spec mutually_exclusive_flags([{option_name(), option_value()}], term(), [{option_name(), term()}]) -> {ok, term()} | {error, string()}. -spec rpc_call(node(), atom(), atom(), [any()]) -> any(). -spec rpc_call(node(), atom(), atom(), [any()], number()) -> any(). @@ -250,20 +249,22 @@ process_opts(Defs, C, [A | As], Found, KVs, Outs) -> {none, _, _} -> no_command end. -%% When we have a set of flags that are used for filtering, we want by -%% default to include every such option in our output. But if a user -%% explicitly specified any such flag, we want to include only items -%% which he has requested. -filter_opts(CurrentOptionValues, AllOptionNames) -> - Explicit = lists:map(fun(OptName) -> - proplists:get_bool(OptName, CurrentOptionValues) - end, - AllOptionNames), - case lists:member(true, Explicit) of - true -> - Explicit; - false -> - lists:duplicate(length(AllOptionNames), true) +mutually_exclusive_flags(CurrentOptionValues, Default, FlagsAndValues) -> + PresentFlags = lists:filtermap(fun({OptName, _} = _O) -> + proplists:get_bool(OptName, CurrentOptionValues) + end, + FlagsAndValues), + case PresentFlags of + [] -> + {ok, Default}; + [{_, Value}] -> + {ok, Value}; + _ -> + Names = [ [$', N, $'] || {N, _} <- PresentFlags ], + CommaSeparated = string:join(lists:droplast(Names), ", "), + AndOneMore = lists:last(Names), + Msg = io_lib:format("Options ~s and ~s are mutually exclusive", [CommaSeparated, AndOneMore]), + {error, lists:flatten(Msg)} end. %%---------------------------------------------------------------------------- diff --git a/src/rabbit_control_main.erl b/src/rabbit_control_main.erl index ea9d6a2030dc..92898c2a2cd7 100644 --- a/src/rabbit_control_main.erl +++ b/src/rabbit_control_main.erl @@ -74,7 +74,7 @@ {clear_policy, [?VHOST_DEF]}, {list_policies, [?VHOST_DEF]}, - {list_queues, [?VHOST_DEF, ?OFFLINE_DEF, ?ONLINE_DEF]}, + {list_queues, [?VHOST_DEF, ?OFFLINE_DEF, ?ONLINE_DEF, ?LOCAL_DEF]}, {list_exchanges, [?VHOST_DEF]}, {list_bindings, [?VHOST_DEF]}, {list_connections, [?VHOST_DEF]}, @@ -632,12 +632,19 @@ action(list_user_permissions, Node, Args = [_Username], _Opts, Inform, Timeout) true); action(list_queues, Node, Args, Opts, Inform, Timeout) -> - [Online, Offline] = rabbit_cli:filter_opts(Opts, [?ONLINE_OPT, ?OFFLINE_OPT]), - Inform("Listing queues", []), - VHostArg = list_to_binary(proplists:get_value(?VHOST_OPT, Opts)), - ArgAtoms = default_if_empty(Args, [name, messages]), - call(Node, {rabbit_amqqueue, info_all, [VHostArg, ArgAtoms, Online, Offline]}, - ArgAtoms, Timeout); + case rabbit_cli:mutually_exclusive_flags( + Opts, all, [{?ONLINE_OPT, online} + ,{?OFFLINE_OPT, offline} + ,{?LOCAL_OPT, local}]) of + {ok, Filter} -> + Inform("Listing queues", []), + VHostArg = list_to_binary(proplists:get_value(?VHOST_OPT, Opts)), + ArgAtoms = default_if_empty(Args, [name, messages]), + call(Node, {rabbit_amqqueue, info_all, [VHostArg, ArgAtoms, Filter]}, + ArgAtoms, Timeout); + {error, ErrStr} -> + {error_string, ErrStr} + end; action(list_exchanges, Node, Args, Opts, Inform, Timeout) -> Inform("Listing exchanges", []), diff --git a/test/rabbitmqctl_integration_SUITE.erl b/test/rabbitmqctl_integration_SUITE.erl new file mode 100644 index 000000000000..9305781bda74 --- /dev/null +++ b/test/rabbitmqctl_integration_SUITE.erl @@ -0,0 +1,146 @@ +%% 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 http://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) 2016 Pivotal Software, Inc. All rights reserved. +%% +-module(rabbitmqctl_integration_SUITE). + +-include_lib("common_test/include/ct.hrl"). +-include_lib("amqp_client/include/amqp_client.hrl"). + +-export([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 + ]). + +-export([list_queues_local/1 + ,list_queues_offline/1 + ,list_queues_online/1 + ]). + +all() -> + [{group, list_queues}]. + +groups() -> + [{list_queues, [], + [list_queues_local + ,list_queues_online + ,list_queues_offline + ]}]. + +init_per_suite(Config) -> + rabbit_ct_helpers:log_environment(), + rabbit_ct_helpers:run_setup_steps(Config). + +end_per_suite(Config) -> + rabbit_ct_helpers:run_teardown_steps(Config). + +init_per_group(list_queues, Config0) -> + NumNodes = 3, + Config = create_n_node_cluster(Config0, NumNodes), + Config1 = declare_some_queues(Config), + rabbit_ct_broker_helpers:stop_node(Config1, NumNodes - 1), + Config1; +init_per_group(_, Config) -> + Config. + +create_n_node_cluster(Config0, NumNodes) -> + Config1 = rabbit_ct_helpers:set_config( + Config0, [{rmq_nodes_count, NumNodes}, + {rmq_nodes_clustered, true}]), + rabbit_ct_helpers:run_steps(Config1, + rabbit_ct_broker_helpers:setup_steps() ++ + rabbit_ct_client_helpers:setup_steps()). + +declare_some_queues(Config) -> + Nodes = rabbit_ct_helpers:get_config(Config, rmq_nodes), + PerNodeQueues = [ declare_some_queues(Config, NodeNum) + || NodeNum <- lists:seq(0, length(Nodes)-1) ], + rabbit_ct_helpers:set_config(Config, {per_node_queues, PerNodeQueues}). + +declare_some_queues(Config, NodeNum) -> + {Conn, Chan} = rabbit_ct_client_helpers:open_connection_and_channel(Config, NodeNum), + NumQueues = 5, + Queues = [ list_to_binary(io_lib:format("queue-~b-on-node-~b", [QueueNum, NodeNum])) + || QueueNum <- lists:seq(1, NumQueues) ], + lists:foreach(fun (QueueName) -> + #'queue.declare_ok'{} = amqp_channel:call(Chan, #'queue.declare'{queue = QueueName, durable = true}) + end, Queues), + rabbit_ct_client_helpers:close_connection_and_channel(Conn, Chan), + Queues. + +end_per_group(list_queues, Config0) -> + Config1 = case rabbit_ct_helpers:get_config(Config0, save_config) of + undefined -> Config0; + C -> C + end, + rabbit_ct_helpers:run_steps(Config1, + rabbit_ct_client_helpers:teardown_steps() ++ + rabbit_ct_broker_helpers:teardown_steps()); +end_per_group(_, Config) -> + Config. + +init_per_testcase(Testcase, Config0) -> + rabbit_ct_helpers:testcase_started(Config0, Testcase). + +end_per_testcase(Testcase, Config0) -> + rabbit_ct_helpers:testcase_finished(Config0, Testcase). + +%%---------------------------------------------------------------------------- +%% Test cases +%%---------------------------------------------------------------------------- +list_queues_local(Config) -> + Node1Queues = lists:sort(lists:nth(1, ?config(per_node_queues, Config))), + Node2Queues = lists:sort(lists:nth(2, ?config(per_node_queues, Config))), + assert_ctl_queues(Config, 0, ["--local"], Node1Queues), + assert_ctl_queues(Config, 1, ["--local"], Node2Queues), + ok. + +list_queues_online(Config) -> + Node1Queues = lists:sort(lists:nth(1, ?config(per_node_queues, Config))), + Node2Queues = lists:sort(lists:nth(2, ?config(per_node_queues, Config))), + OnlineQueues = Node1Queues ++ Node2Queues, + assert_ctl_queues(Config, 0, ["--online"], OnlineQueues), + assert_ctl_queues(Config, 1, ["--online"], OnlineQueues), + ok. + +list_queues_offline(Config) -> + Node3Queues = lists:sort(lists:nth(3, ?config(per_node_queues, Config))), + OfflineQueues = Node3Queues, + assert_ctl_queues(Config, 0, ["--offline"], OfflineQueues), + assert_ctl_queues(Config, 1, ["--offline"], OfflineQueues), + ok. + +%%---------------------------------------------------------------------------- +%% Helpers +%%---------------------------------------------------------------------------- +assert_ctl_queues(Config, Node, Args, Expected0) -> + Expected = lists:sort(Expected0), + Got0 = run_list_queues(Config, Node, Args), + Got = lists:sort(lists:map(fun hd/1, Got0)), + case Got of + Expected -> + ok; + _ -> + ct:pal(error, "Listing queues on node ~p failed. Expected:~n~p~n~nGot:~n~p~n~n", + [Node, Expected, Got]), + exit({list_queues_unexpected_on, Node, Expected, Got}) + end. + +run_list_queues(Config, Node, Args) -> + rabbit_ct_broker_helpers:rabbitmqctl_list(Config, Node, ["list_queues"] ++ Args ++ ["name"]). diff --git a/test/unit_SUITE.erl b/test/unit_SUITE.erl index ba0f43f11e34..43e812fa3dd0 100644 --- a/test/unit_SUITE.erl +++ b/test/unit_SUITE.erl @@ -31,7 +31,7 @@ groups() -> [ {parallel_tests, [parallel], [ arguments_parser, - filtering_flags_parsing, + mutually_exclusive_flags_parsing, {basic_header_handling, [parallel], [ write_table_with_invalid_existing_type, invalid_existing_headers, @@ -135,26 +135,41 @@ check_parse_arguments(ExpRes, Fun, As) -> true = SortRes(ExpRes) =:= SortRes(Fun(As)). -filtering_flags_parsing(_Config) -> - Cases = [{[], [], []} - ,{[{"--online", true}], ["--offline", "--online", "--third-option"], [false, true, false]} - ,{[{"--online", true}, {"--third-option", true}, {"--offline", true}], ["--offline", "--online", "--third-option"], [true, true, true]} - ,{[], ["--offline", "--online", "--third-option"], [true, true, true]} - ], - lists:foreach(fun({Vals, Opts, Expect}) -> - case rabbit_cli:filter_opts(Vals, Opts) of - Expect -> +mutually_exclusive_flags_parsing(_Config) -> + Matcher = fun ({ok, Value}, {ok, Value}) -> true; + ({error, Value}, {error, Pattern}) -> + case re:run(Value, Pattern) of + {match, _} -> true; + _ -> false + end; + (_, _) -> false + end, + Spec = [{"--online", online} + ,{"--offline", offline} + ,{"--local", local}], + Default = all, + Cases =[{["--online"], {ok, online}} + ,{[], {ok, Default}} + ,{["--offline"], {ok, offline}} + ,{["--local"], {ok, local}} + ,{["--offline", "--local"], {error, "mutually exclusive"}} + ,{["--offline", "--online"], {error, "mutually exclusive"}} + ,{["--offline", "--local", "--online"], {error, "mutually exclusive"}} + ], + lists:foreach(fun({Opts, Expected}) -> + ExpandedOpts = [ {Opt, true} || Opt <- Opts ], + Got = rabbit_cli:mutually_exclusive_flags(ExpandedOpts, all, Spec), + case Matcher(Got, Expected) of + true -> ok; - Got -> - exit({no_match, Got, Expect, {args, Vals, Opts}}) + false -> + exit({no_match, Got, Expected, {opts, Opts}}) end - end, - Cases). + end, Cases). %% ------------------------------------------------------------------- %% basic_header_handling. %% ------------------------------------------------------------------- - -define(XDEATH_TABLE, [{<<"reason">>, longstr, <<"blah">>}, {<<"queue">>, longstr, <<"foo.bar.baz">>},