Enable adding queues with plugins, core
This commit is contained in:
		
							parent
							
								
									dc0d4735c2
								
							
						
					
					
						commit
						3c1a890a0a
					
				|  | @ -48,6 +48,10 @@ | ||||||
|         (?is_amqqueue_v2(Q) andalso |         (?is_amqqueue_v2(Q) andalso | ||||||
|          ?amqqueue_v2_field_type(Q) =:= Type)). |          ?amqqueue_v2_field_type(Q) =:= Type)). | ||||||
| 
 | 
 | ||||||
|  | -define(amqqueue_type(Q), | ||||||
|  |         (?is_amqqueue_v2(Q) andalso | ||||||
|  |          ?amqqueue_v2_field_type(Q))). | ||||||
|  | 
 | ||||||
| -define(amqqueue_has_valid_pid(Q), | -define(amqqueue_has_valid_pid(Q), | ||||||
|         (?is_amqqueue_v2(Q) andalso |         (?is_amqqueue_v2(Q) andalso | ||||||
|          is_pid(?amqqueue_v2_field_pid(Q)))). |          is_pid(?amqqueue_v2_field_pid(Q)))). | ||||||
|  |  | ||||||
|  | @ -368,6 +368,7 @@ get_exclusive_owner(#amqqueue{exclusive_owner = Owner}) -> | ||||||
| 
 | 
 | ||||||
| -spec get_leader(amqqueue_v2()) -> node(). | -spec get_leader(amqqueue_v2()) -> node(). | ||||||
| 
 | 
 | ||||||
|  | %% TODO: not only qqs can have leaders, dispatch via queue type | ||||||
| get_leader(#amqqueue{type = rabbit_quorum_queue, pid = {_, Leader}}) -> Leader. | get_leader(#amqqueue{type = rabbit_quorum_queue, pid = {_, Leader}}) -> Leader. | ||||||
| 
 | 
 | ||||||
| % operator_policy | % operator_policy | ||||||
|  |  | ||||||
|  | @ -461,32 +461,8 @@ encode_queue(Q, NumMsgs, NumConsumers) -> | ||||||
| -spec queue_topology(amqqueue:amqqueue()) -> | -spec queue_topology(amqqueue:amqqueue()) -> | ||||||
|     {Leader :: undefined | node(), Replicas :: undefined | [node(),...]}. |     {Leader :: undefined | node(), Replicas :: undefined | [node(),...]}. | ||||||
| queue_topology(Q) -> | queue_topology(Q) -> | ||||||
|     case amqqueue:get_type(Q) of |     Type = amqqueue:get_type(Q), | ||||||
|         rabbit_quorum_queue -> |     Type:queue_topology(Q). | ||||||
|             [{leader, Leader0}, |  | ||||||
|              {members, Members}] = rabbit_queue_type:info(Q, [leader, members]), |  | ||||||
|             Leader = case Leader0 of |  | ||||||
|                          '' -> undefined; |  | ||||||
|                          _ -> Leader0 |  | ||||||
|                      end, |  | ||||||
|             {Leader, Members}; |  | ||||||
|         rabbit_stream_queue -> |  | ||||||
|             #{name := StreamId} = amqqueue:get_type_state(Q), |  | ||||||
|             case rabbit_stream_coordinator:members(StreamId) of |  | ||||||
|                 {ok, Members} -> |  | ||||||
|                     maps:fold(fun(Node, {_Pid, writer}, {_, Replicas}) -> |  | ||||||
|                                       {Node, [Node | Replicas]}; |  | ||||||
|                                  (Node, {_Pid, replica}, {Writer, Replicas}) -> |  | ||||||
|                                       {Writer, [Node | Replicas]} |  | ||||||
|                               end, {undefined, []}, Members); |  | ||||||
|                 {error, _} -> |  | ||||||
|                     {undefined, undefined} |  | ||||||
|             end; |  | ||||||
|         _ -> |  | ||||||
|             Pid = amqqueue:get_pid(Q), |  | ||||||
|             Node = node(Pid), |  | ||||||
|             {Node, [Node]} |  | ||||||
|     end. |  | ||||||
| 
 | 
 | ||||||
| decode_exchange({map, KVList}) -> | decode_exchange({map, KVList}) -> | ||||||
|     M = lists:foldl( |     M = lists:foldl( | ||||||
|  |  | ||||||
|  | @ -150,11 +150,7 @@ filter_pid_per_type(QPids) -> | ||||||
| 
 | 
 | ||||||
| -spec stop(rabbit_types:vhost()) -> 'ok'. | -spec stop(rabbit_types:vhost()) -> 'ok'. | ||||||
| stop(VHost) -> | stop(VHost) -> | ||||||
|     %% Classic queues |     rabbit_queue_type:stop(VHost). | ||||||
|     ok = rabbit_amqqueue_sup_sup:stop_for_vhost(VHost), |  | ||||||
|     {ok, BQ} = application:get_env(rabbit, backing_queue_module), |  | ||||||
|     ok = BQ:stop(VHost), |  | ||||||
|     rabbit_quorum_queue:stop(VHost). |  | ||||||
| 
 | 
 | ||||||
| -spec start([amqqueue:amqqueue()]) -> 'ok'. | -spec start([amqqueue:amqqueue()]) -> 'ok'. | ||||||
| 
 | 
 | ||||||
|  | @ -424,6 +420,8 @@ rebalance(Type, VhostSpec, QueueSpec) -> | ||||||
|     %% We have not yet acquired the rebalance_queues global lock. |     %% We have not yet acquired the rebalance_queues global lock. | ||||||
|     maybe_rebalance(get_rebalance_lock(self()), Type, VhostSpec, QueueSpec). |     maybe_rebalance(get_rebalance_lock(self()), Type, VhostSpec, QueueSpec). | ||||||
| 
 | 
 | ||||||
|  | %% TODO: classic queues do not support rebalancing, it looks like they are simply | ||||||
|  | %% filtered out with is_replicated(Q). Maybe error instead? | ||||||
| maybe_rebalance({true, Id}, Type, VhostSpec, QueueSpec) -> | maybe_rebalance({true, Id}, Type, VhostSpec, QueueSpec) -> | ||||||
|     rabbit_log:info("Starting queue rebalance operation: '~ts' for vhosts matching '~ts' and queues matching '~ts'", |     rabbit_log:info("Starting queue rebalance operation: '~ts' for vhosts matching '~ts' and queues matching '~ts'", | ||||||
|                     [Type, VhostSpec, QueueSpec]), |                     [Type, VhostSpec, QueueSpec]), | ||||||
|  | @ -459,10 +457,15 @@ filter_per_type(stream, Q) -> | ||||||
| filter_per_type(classic, Q) -> | filter_per_type(classic, Q) -> | ||||||
|     ?amqqueue_is_classic(Q). |     ?amqqueue_is_classic(Q). | ||||||
| 
 | 
 | ||||||
| rebalance_module(Q) when ?amqqueue_is_quorum(Q) -> | %% TODO: note that it can return {error, not_supported}. | ||||||
|     rabbit_quorum_queue; | %% this will result in a badmatch. However that's fine | ||||||
| rebalance_module(Q) when ?amqqueue_is_stream(Q) -> | %% for now because the original function will fail with | ||||||
|     rabbit_stream_queue. | %% bad clause if called with classical queue. | ||||||
|  | %% The assumption is all non-replicated queues | ||||||
|  | %% are filtered before calling this with is_replicated/0 | ||||||
|  | rebalance_module(Q) -> | ||||||
|  |     TypeModule = ?amqqueue_type(Q), | ||||||
|  |     TypeModule:rebalance_module(). | ||||||
| 
 | 
 | ||||||
| get_resource_name(#resource{name = Name}) -> | get_resource_name(#resource{name = Name}) -> | ||||||
|     Name. |     Name. | ||||||
|  | @ -487,13 +490,19 @@ iterative_rebalance(ByNode, MaxQueuesDesired) -> | ||||||
| maybe_migrate(ByNode, MaxQueuesDesired) -> | maybe_migrate(ByNode, MaxQueuesDesired) -> | ||||||
|     maybe_migrate(ByNode, MaxQueuesDesired, maps:keys(ByNode)). |     maybe_migrate(ByNode, MaxQueuesDesired, maps:keys(ByNode)). | ||||||
| 
 | 
 | ||||||
|  | %% TODO: unfortunate part - UI bits mixed deep inside logic. | ||||||
|  | %% I will not be moving this inside queue type. Instead | ||||||
|  | %% an attempt to generate something more readable than | ||||||
|  | %% Other made. | ||||||
| column_name(rabbit_classic_queue) -> <<"Number of replicated classic queues">>; | column_name(rabbit_classic_queue) -> <<"Number of replicated classic queues">>; | ||||||
| column_name(rabbit_quorum_queue) -> <<"Number of quorum queues">>; | column_name(rabbit_quorum_queue) -> <<"Number of quorum queues">>; | ||||||
| column_name(rabbit_stream_queue) -> <<"Number of streams">>; | column_name(rabbit_stream_queue) -> <<"Number of streams">>; | ||||||
| column_name(Other) -> Other. | column_name(TypeModule) -> | ||||||
|  |     Alias = rabbit_queue_type:short_alias_of(TypeModule), | ||||||
|  |     <<"Number of \"", Alias/binary, "\" queues">>. | ||||||
| 
 | 
 | ||||||
| maybe_migrate(ByNode, _, []) -> | maybe_migrate(ByNode, _, []) -> | ||||||
|     ByNodeAndType = maps:map(fun(_Node, Queues) -> maps:groups_from_list(fun({_, Q, _}) -> column_name(?amqqueue_v2_field_type(Q)) end, Queues) end, ByNode), |     ByNodeAndType = maps:map(fun(_Node, Queues) -> maps:groups_from_list(fun({_, Q, _}) -> column_name(?amqqueue_type(Q)) end, Queues) end, ByNode), | ||||||
|     CountByNodeAndType = maps:map(fun(_Node, Type) -> maps:map(fun (_, Qs)-> length(Qs) end, Type) end, ByNodeAndType), |     CountByNodeAndType = maps:map(fun(_Node, Type) -> maps:map(fun (_, Qs)-> length(Qs) end, Type) end, ByNodeAndType), | ||||||
|     {ok, maps:values(maps:map(fun(Node,Counts) -> [{<<"Node name">>, Node} | maps:to_list(Counts)] end, CountByNodeAndType))}; |     {ok, maps:values(maps:map(fun(Node,Counts) -> [{<<"Node name">>, Node} | maps:to_list(Counts)] end, CountByNodeAndType))}; | ||||||
| maybe_migrate(ByNode, MaxQueuesDesired, [N | Nodes]) -> | maybe_migrate(ByNode, MaxQueuesDesired, [N | Nodes]) -> | ||||||
|  | @ -1252,14 +1261,12 @@ list_durable() -> | ||||||
| 
 | 
 | ||||||
| -spec list_by_type(atom()) -> [amqqueue:amqqueue()]. | -spec list_by_type(atom()) -> [amqqueue:amqqueue()]. | ||||||
| 
 | 
 | ||||||
| list_by_type(classic) -> list_by_type(rabbit_classic_queue); | list_by_type(TypeDescriptor) -> | ||||||
| list_by_type(quorum)  -> list_by_type(rabbit_quorum_queue); |     TypeModule = rabbit_queue_type:discover(TypeDescriptor), | ||||||
| list_by_type(stream)  -> list_by_type(rabbit_stream_queue); |     rabbit_db_queue:get_all_durable_by_type(TypeModule). | ||||||
| list_by_type(Type) -> |  | ||||||
|     rabbit_db_queue:get_all_durable_by_type(Type). |  | ||||||
| 
 | 
 | ||||||
|  | %% TODO: looks unused | ||||||
| -spec list_local_quorum_queue_names() -> [name()]. | -spec list_local_quorum_queue_names() -> [name()]. | ||||||
| 
 |  | ||||||
| list_local_quorum_queue_names() -> | list_local_quorum_queue_names() -> | ||||||
|     [ amqqueue:get_name(Q) || Q <- list_by_type(quorum), |     [ amqqueue:get_name(Q) || Q <- list_by_type(quorum), | ||||||
|            amqqueue:get_state(Q) =/= crashed, |            amqqueue:get_state(Q) =/= crashed, | ||||||
|  | @ -1296,6 +1303,7 @@ list_local_followers() -> | ||||||
|          rabbit_quorum_queue:is_recoverable(Q) |          rabbit_quorum_queue:is_recoverable(Q) | ||||||
|          ]. |          ]. | ||||||
| 
 | 
 | ||||||
|  | %% TODO: looks unused | ||||||
| -spec list_local_quorum_queues_with_name_matching(binary()) -> [amqqueue:amqqueue()]. | -spec list_local_quorum_queues_with_name_matching(binary()) -> [amqqueue:amqqueue()]. | ||||||
| list_local_quorum_queues_with_name_matching(Pattern) -> | list_local_quorum_queues_with_name_matching(Pattern) -> | ||||||
|     [ Q || Q <- list_by_type(quorum), |     [ Q || Q <- list_by_type(quorum), | ||||||
|  | @ -1882,11 +1890,9 @@ run_backing_queue(QPid, Mod, Fun) -> | ||||||
| 
 | 
 | ||||||
| -spec is_replicated(amqqueue:amqqueue()) -> boolean(). | -spec is_replicated(amqqueue:amqqueue()) -> boolean(). | ||||||
| 
 | 
 | ||||||
| is_replicated(Q) when ?amqqueue_is_classic(Q) -> | is_replicated(Q) -> | ||||||
|     false; |     TypeModule = ?amqqueue_type(Q), | ||||||
| is_replicated(_Q) -> |     TypeModule:is_replicated(). | ||||||
|     %% streams and quorum queues are all replicated |  | ||||||
|     true. |  | ||||||
| 
 | 
 | ||||||
| is_exclusive(Q) when ?amqqueue_exclusive_owner_is(Q, none) -> | is_exclusive(Q) when ?amqqueue_exclusive_owner_is(Q, none) -> | ||||||
|     false; |     false; | ||||||
|  |  | ||||||
|  | @ -4,6 +4,8 @@ | ||||||
| %% | %% | ||||||
| %% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. | %% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved. | ||||||
| %% | %% | ||||||
|  | %% README: https://github.com/rabbitmq/internals/blob/master/rabbit_boot_process.md | ||||||
|  | %% | ||||||
| 
 | 
 | ||||||
| -module(rabbit_boot_steps). | -module(rabbit_boot_steps). | ||||||
| 
 | 
 | ||||||
|  |  | ||||||
|  | @ -1265,11 +1265,7 @@ handle_method(#'basic.get'{queue = QueueNameBin, no_ack = NoAck}, | ||||||
|             ?INCR_STATS(queue_stats, QueueName, 1, get_empty, State), |             ?INCR_STATS(queue_stats, QueueName, 1, get_empty, State), | ||||||
|             {reply, #'basic.get_empty'{}, State#ch{queue_states = QueueStates}}; |             {reply, #'basic.get_empty'{}, State#ch{queue_states = QueueStates}}; | ||||||
|         {error, {unsupported, single_active_consumer}} -> |         {error, {unsupported, single_active_consumer}} -> | ||||||
|             rabbit_misc:protocol_error( |              rabbit_amqqueue:with_or_die(QueueName, fun unsupported_single_active_consumer_error/1); | ||||||
|               resource_locked, |  | ||||||
|               "cannot obtain access to locked ~ts. basic.get operations " |  | ||||||
|               "are not supported by quorum queues with single active consumer", |  | ||||||
|               [rabbit_misc:rs(QueueName)]); |  | ||||||
|         {error, Reason} -> |         {error, Reason} -> | ||||||
|             %% TODO add queue type to error message |             %% TODO add queue type to error message | ||||||
|             rabbit_misc:protocol_error(internal_error, |             rabbit_misc:protocol_error(internal_error, | ||||||
|  | @ -2005,6 +2001,7 @@ foreach_per_queue(_F, [], Acc) -> | ||||||
| foreach_per_queue(F, [#pending_ack{tag = CTag, | foreach_per_queue(F, [#pending_ack{tag = CTag, | ||||||
|                                    queue = QName, |                                    queue = QName, | ||||||
|                                    msg_id = MsgId}], Acc) -> |                                    msg_id = MsgId}], Acc) -> | ||||||
|  |     %% TODO: fix this abstraction leak | ||||||
|     %% quorum queue, needs the consumer tag |     %% quorum queue, needs the consumer tag | ||||||
|     F({QName, CTag}, [MsgId], Acc); |     F({QName, CTag}, [MsgId], Acc); | ||||||
| foreach_per_queue(F, UAL, Acc) -> | foreach_per_queue(F, UAL, Acc) -> | ||||||
|  | @ -2032,6 +2029,7 @@ notify_limiter(Limiter, Acked) -> | ||||||
|      case rabbit_limiter:is_active(Limiter) of |      case rabbit_limiter:is_active(Limiter) of | ||||||
|         false -> ok; |         false -> ok; | ||||||
|         true  -> case lists:foldl(fun (#pending_ack{tag = CTag}, Acc) when is_integer(CTag) -> |         true  -> case lists:foldl(fun (#pending_ack{tag = CTag}, Acc) when is_integer(CTag) -> | ||||||
|  |                                           %% TODO: fix absctraction leak | ||||||
|                                           %% Quorum queues use integer CTags |                                           %% Quorum queues use integer CTags | ||||||
|                                           %% classic queues use binaries |                                           %% classic queues use binaries | ||||||
|                                           %% Quorum queues do not interact |                                           %% Quorum queues do not interact | ||||||
|  | @ -2792,3 +2790,12 @@ maybe_decrease_global_publishers(#ch{publishing_mode = true}) -> | ||||||
| 
 | 
 | ||||||
| is_global_qos_permitted() -> | is_global_qos_permitted() -> | ||||||
|     rabbit_deprecated_features:is_permitted(global_qos). |     rabbit_deprecated_features:is_permitted(global_qos). | ||||||
|  | 
 | ||||||
|  | -spec unsupported_single_active_consumer_error(amqqueue:amqqueue()) -> no_return(). | ||||||
|  | unsupported_single_active_consumer_error(Q) -> | ||||||
|  |     rabbit_misc:protocol_error( | ||||||
|  |       resource_locked, | ||||||
|  |       "cannot obtain access to locked ~ts. basic.get operations " | ||||||
|  |       "are not supported by ~p queues with single active consumer", | ||||||
|  |       [rabbit_misc:rs(amqqueue:get_name(Q)), | ||||||
|  |        rabbit_queue_type:short_alias_of(amqqueue:get_type(Q))]). | ||||||
|  |  | ||||||
|  | @ -64,8 +64,32 @@ | ||||||
|          send_drained_credit_api_v1/4, |          send_drained_credit_api_v1/4, | ||||||
|          send_credit_reply/7]). |          send_credit_reply/7]). | ||||||
| 
 | 
 | ||||||
|  | -export([queue_topology/1, | ||||||
|  |          feature_flag_name/0, | ||||||
|  |          policy_apply_to_name/0, | ||||||
|  |          can_redeliver/0, | ||||||
|  |          stop/1, | ||||||
|  |          is_replicated/0, | ||||||
|  |          rebalance_module/0, | ||||||
|  |          list_with_minimum_quorum/0, | ||||||
|  |          drain/1, | ||||||
|  |          revive/0, | ||||||
|  |          queue_vm_stats_sups/0, | ||||||
|  |          queue_vm_ets/0, | ||||||
|  |          dir_base/0]). | ||||||
|  | 
 | ||||||
| -export([validate_policy/1]). | -export([validate_policy/1]). | ||||||
| 
 | 
 | ||||||
|  | -rabbit_boot_step( | ||||||
|  |    {rabbit_classic_queue_type, | ||||||
|  |     [{description, "Classic queue: queue type"}, | ||||||
|  |      {mfa,      {rabbit_registry, register, | ||||||
|  |                     [queue, <<"classic">>, ?MODULE]}}, | ||||||
|  |      {cleanup,  {rabbit_registry, unregister, | ||||||
|  |                  [queue, <<"classic">>]}}, | ||||||
|  |      {requires, rabbit_registry}, | ||||||
|  |      {enables,     ?MODULE}]}). | ||||||
|  | 
 | ||||||
| -rabbit_boot_step( | -rabbit_boot_step( | ||||||
|    {?MODULE, |    {?MODULE, | ||||||
|     [{description, "Deprecated queue-master-locator support." |     [{description, "Deprecated queue-master-locator support." | ||||||
|  | @ -74,7 +98,7 @@ | ||||||
|             [policy_validator, <<"queue-master-locator">>, ?MODULE]}}, |             [policy_validator, <<"queue-master-locator">>, ?MODULE]}}, | ||||||
|      {mfa, {rabbit_registry, register, |      {mfa, {rabbit_registry, register, | ||||||
|             [operator_policy_validator, <<"queue-master-locator">>, ?MODULE]}}, |             [operator_policy_validator, <<"queue-master-locator">>, ?MODULE]}}, | ||||||
|      {requires, rabbit_registry}, |      {requires, [rabbit_classic_queue_type]}, | ||||||
|      {enables, recovery}]}). |      {enables, recovery}]}). | ||||||
| 
 | 
 | ||||||
| validate_policy(Args) -> | validate_policy(Args) -> | ||||||
|  | @ -674,3 +698,56 @@ send_credit_reply(Pid, QName, Ctag, DeliveryCount, Credit, Available, Drain) -> | ||||||
| 
 | 
 | ||||||
| send_queue_event(Pid, QName, Event) -> | send_queue_event(Pid, QName, Event) -> | ||||||
|     gen_server:cast(Pid, {queue_event, QName, Event}). |     gen_server:cast(Pid, {queue_event, QName, Event}). | ||||||
|  | 
 | ||||||
|  | -spec queue_topology(amqqueue:amqqueue()) -> | ||||||
|  |     {Leader :: undefined | node(), Replicas :: undefined | [node(),...]}. | ||||||
|  | queue_topology(Q) -> | ||||||
|  |     Pid = amqqueue:get_pid(Q), | ||||||
|  |     Node = node(Pid), | ||||||
|  |     {Node, [Node]}. | ||||||
|  | 
 | ||||||
|  | feature_flag_name() -> | ||||||
|  |     undefined. | ||||||
|  | 
 | ||||||
|  | policy_apply_to_name() -> | ||||||
|  |     <<"classic_queues">>. | ||||||
|  | 
 | ||||||
|  | can_redeliver() -> | ||||||
|  |     true. | ||||||
|  | 
 | ||||||
|  | stop(VHost) -> | ||||||
|  |     ok = rabbit_amqqueue_sup_sup:stop_for_vhost(VHost), | ||||||
|  |     {ok, BQ} = application:get_env(rabbit, backing_queue_module), | ||||||
|  |     ok = BQ:stop(VHost). | ||||||
|  | 
 | ||||||
|  | is_replicated() -> | ||||||
|  |     false. | ||||||
|  | 
 | ||||||
|  | rebalance_module() -> | ||||||
|  |     {error, not_supported}. | ||||||
|  | 
 | ||||||
|  | list_with_minimum_quorum() -> | ||||||
|  |     []. | ||||||
|  | 
 | ||||||
|  | drain(_TransferCandidates) -> | ||||||
|  |     ok. | ||||||
|  | 
 | ||||||
|  | revive() -> | ||||||
|  |     ok. | ||||||
|  | 
 | ||||||
|  | queue_vm_stats_sups() -> | ||||||
|  |     {[queue_procs], [rabbit_vm:all_vhosts_children(rabbit_amqqueue_sup_sup)]}. | ||||||
|  | 
 | ||||||
|  | %% return nothing because of this line in rabbit_vm: | ||||||
|  | %% {msg_index,            MsgIndexETS + MsgIndexProc}, | ||||||
|  | %% it mixes procs and ets, | ||||||
|  | %% TODO: maybe instead of separating sups and ets | ||||||
|  | %% I need vm_memory callback that just | ||||||
|  | %% returns proplist? And rabbit_vm calculates | ||||||
|  | %% Other as usual by substraction. | ||||||
|  | queue_vm_ets() -> | ||||||
|  |     {[], | ||||||
|  |      []}. | ||||||
|  | 
 | ||||||
|  | dir_base() -> | ||||||
|  |     [rabbit_vhost:msg_store_dir_base()]. | ||||||
|  |  | ||||||
|  | @ -74,22 +74,15 @@ gc_local_queues() -> | ||||||
|     GbSetDown = gb_sets:from_list(QueuesDown), |     GbSetDown = gb_sets:from_list(QueuesDown), | ||||||
|     gc_queue_metrics(GbSet, GbSetDown), |     gc_queue_metrics(GbSet, GbSetDown), | ||||||
|     gc_entity(queue_coarse_metrics, GbSet), |     gc_entity(queue_coarse_metrics, GbSet), | ||||||
|     Followers = gb_sets:from_list([amqqueue:get_name(Q) || Q <- rabbit_amqqueue:list_local_followers() ]), |     %% remove coarse metrics for quorum queues without local leader | ||||||
|     gc_leader_data(Followers). |     gc_leader_data(). | ||||||
| 
 | 
 | ||||||
| gc_leader_data(Followers) -> | gc_leader_data() -> | ||||||
|     ets:foldl(fun({Id, _, _, _, _}, none) -> |     _ = [begin | ||||||
|                       gc_leader_data(Id, queue_coarse_metrics, Followers) |              QName = amqqueue:get_name(Q), | ||||||
|               end, none, queue_coarse_metrics). |              rabbit_core_metrics:delete_queue_coarse_metrics(QName) | ||||||
| 
 |          end || Q <- rabbit_amqqueue:list_local_followers()], | ||||||
| gc_leader_data(Id, Table, GbSet) -> |     ok. | ||||||
|     case gb_sets:is_member(Id, GbSet) of |  | ||||||
|         true -> |  | ||||||
|             ets:delete(Table, Id), |  | ||||||
|             none; |  | ||||||
|         false -> |  | ||||||
|             none |  | ||||||
|     end. |  | ||||||
| 
 | 
 | ||||||
| gc_global_queues() -> | gc_global_queues() -> | ||||||
|     GbSet = gb_sets:from_list(rabbit_amqqueue:list_names()), |     GbSet = gb_sets:from_list(rabbit_amqqueue:list_names()), | ||||||
|  |  | ||||||
|  | @ -1045,16 +1045,11 @@ list_queues() -> | ||||||
| 
 | 
 | ||||||
| queue_definition(Q) -> | queue_definition(Q) -> | ||||||
|     #resource{virtual_host = VHost, name = Name} = amqqueue:get_name(Q), |     #resource{virtual_host = VHost, name = Name} = amqqueue:get_name(Q), | ||||||
|     Type = case amqqueue:get_type(Q) of |     TypeModule =  amqqueue:get_type(Q), | ||||||
|                rabbit_classic_queue -> classic; |  | ||||||
|                rabbit_quorum_queue -> quorum; |  | ||||||
|                rabbit_stream_queue -> stream; |  | ||||||
|                T -> T |  | ||||||
|            end, |  | ||||||
|     #{ |     #{ | ||||||
|         <<"vhost">> => VHost, |         <<"vhost">> => VHost, | ||||||
|         <<"name">> => Name, |         <<"name">> => Name, | ||||||
|         <<"type">> => Type, |         <<"type">> => rabbit_registry:lookup_type_name(queue, TypeModule), | ||||||
|         <<"durable">> => amqqueue:is_durable(Q), |         <<"durable">> => amqqueue:is_durable(Q), | ||||||
|         <<"auto_delete">> => amqqueue:is_auto_delete(Q), |         <<"auto_delete">> => amqqueue:is_auto_delete(Q), | ||||||
|         <<"arguments">> => rabbit_misc:amqp_table(amqqueue:get_arguments(Q)) |         <<"arguments">> => rabbit_misc:amqp_table(amqqueue:get_arguments(Q)) | ||||||
|  |  | ||||||
|  | @ -538,14 +538,8 @@ redeliver0(#pending{delivery = Msg0, | ||||||
| clients_redeliver(Qs, QTypeState) -> | clients_redeliver(Qs, QTypeState) -> | ||||||
|     lists:filter(fun(Q) -> |     lists:filter(fun(Q) -> | ||||||
|                          case rabbit_queue_type:module(Q, QTypeState) of |                          case rabbit_queue_type:module(Q, QTypeState) of | ||||||
|                              {ok, rabbit_quorum_queue} -> |                              {ok, TypeModule} -> TypeModule:can_redeliver(); | ||||||
|                                  % If #enqueue{} Raft command does not get applied |                              _ -> false | ||||||
|                                  % rabbit_fifo_client will resend. |  | ||||||
|                                  true; |  | ||||||
|                              {ok, rabbit_stream_queue} -> |  | ||||||
|                                  true; |  | ||||||
|                              _ -> |  | ||||||
|                                  false |  | ||||||
|                          end |                          end | ||||||
|                  end, Qs). |                  end, Qs). | ||||||
| 
 | 
 | ||||||
|  |  | ||||||
|  | @ -266,8 +266,8 @@ messages_dead_lettered(Reason, QueueType, DeadLetterStrategy, Num) -> | ||||||
|             end, |             end, | ||||||
|     counters:add(fetch(QueueType, DeadLetterStrategy), Index, Num). |     counters:add(fetch(QueueType, DeadLetterStrategy), Index, Num). | ||||||
| 
 | 
 | ||||||
| messages_dead_lettered_confirmed(rabbit_quorum_queue, at_least_once, Num) -> | messages_dead_lettered_confirmed(QTypeModule, at_least_once, Num) -> | ||||||
|     counters:add(fetch(rabbit_quorum_queue, at_least_once), ?MESSAGES_DEAD_LETTERED_CONFIRMED, Num). |     counters:add(fetch(QTypeModule, at_least_once), ?MESSAGES_DEAD_LETTERED_CONFIRMED, Num). | ||||||
| 
 | 
 | ||||||
| fetch(Protocol) -> | fetch(Protocol) -> | ||||||
|     persistent_term:get({?MODULE, Protocol}). |     persistent_term:get({?MODULE, Protocol}). | ||||||
|  |  | ||||||
|  | @ -33,7 +33,6 @@ | ||||||
|     close_all_client_connections/0, |     close_all_client_connections/0, | ||||||
|     primary_replica_transfer_candidate_nodes/0, |     primary_replica_transfer_candidate_nodes/0, | ||||||
|     random_primary_replica_transfer_candidate_node/2, |     random_primary_replica_transfer_candidate_node/2, | ||||||
|     transfer_leadership_of_quorum_queues/1, |  | ||||||
|     table_definitions/0 |     table_definitions/0 | ||||||
| ]). | ]). | ||||||
| 
 | 
 | ||||||
|  | @ -78,13 +77,7 @@ drain() -> | ||||||
|     TransferCandidates = primary_replica_transfer_candidate_nodes(), |     TransferCandidates = primary_replica_transfer_candidate_nodes(), | ||||||
|     %% Note: only QQ leadership is transferred because it is a reasonably quick thing to do a lot of queues |     %% Note: only QQ leadership is transferred because it is a reasonably quick thing to do a lot of queues | ||||||
|     %% in the cluster, unlike with CMQs. |     %% in the cluster, unlike with CMQs. | ||||||
|     transfer_leadership_of_quorum_queues(TransferCandidates), |     rabbit_queue_type:drain(TransferCandidates), | ||||||
|     stop_local_quorum_queue_followers(), |  | ||||||
| 
 |  | ||||||
|     case whereis(rabbit_stream_coordinator) of |  | ||||||
|         undefined -> ok; |  | ||||||
|         _Pid -> transfer_leadership_of_stream_coordinator(TransferCandidates) |  | ||||||
|     end, |  | ||||||
| 
 | 
 | ||||||
|     transfer_leadership_of_metadata_store(TransferCandidates), |     transfer_leadership_of_metadata_store(TransferCandidates), | ||||||
| 
 | 
 | ||||||
|  | @ -99,7 +92,7 @@ drain() -> | ||||||
| -spec revive() -> ok. | -spec revive() -> ok. | ||||||
| revive() -> | revive() -> | ||||||
|     rabbit_log:info("This node is being revived from maintenance (drain) mode"), |     rabbit_log:info("This node is being revived from maintenance (drain) mode"), | ||||||
|     revive_local_quorum_queue_replicas(), |     rabbit_queue_type:revive(), | ||||||
|     rabbit_log:info("Resumed all listeners and will accept client connections again"), |     rabbit_log:info("Resumed all listeners and will accept client connections again"), | ||||||
|     _ = resume_all_client_listeners(), |     _ = resume_all_client_listeners(), | ||||||
|     rabbit_log:info("Resumed all listeners and will accept client connections again"), |     rabbit_log:info("Resumed all listeners and will accept client connections again"), | ||||||
|  | @ -186,32 +179,6 @@ close_all_client_connections() -> | ||||||
|     rabbit_networking:close_connections(Pids, "Node was put into maintenance mode"), |     rabbit_networking:close_connections(Pids, "Node was put into maintenance mode"), | ||||||
|     {ok, length(Pids)}. |     {ok, length(Pids)}. | ||||||
| 
 | 
 | ||||||
| -spec transfer_leadership_of_quorum_queues([node()]) -> ok. |  | ||||||
| transfer_leadership_of_quorum_queues([]) -> |  | ||||||
|     rabbit_log:warning("Skipping leadership transfer of quorum queues: no candidate " |  | ||||||
|                        "(online, not under maintenance) nodes to transfer to!"); |  | ||||||
| transfer_leadership_of_quorum_queues(_TransferCandidates) -> |  | ||||||
|     %% we only transfer leadership for QQs that have local leaders |  | ||||||
|     Queues = rabbit_amqqueue:list_local_leaders(), |  | ||||||
|     rabbit_log:info("Will transfer leadership of ~b quorum queues with current leader on this node", |  | ||||||
|                     [length(Queues)]), |  | ||||||
|     [begin |  | ||||||
|         Name = amqqueue:get_name(Q), |  | ||||||
|         rabbit_log:debug("Will trigger a leader election for local quorum queue ~ts", |  | ||||||
|                          [rabbit_misc:rs(Name)]), |  | ||||||
|         %% we trigger an election and exclude this node from the list of candidates |  | ||||||
|         %% by simply shutting its local QQ replica (Ra server) |  | ||||||
|         RaLeader = amqqueue:get_pid(Q), |  | ||||||
|         rabbit_log:debug("Will stop Ra server ~tp", [RaLeader]), |  | ||||||
|         case rabbit_quorum_queue:stop_server(RaLeader) of |  | ||||||
|             ok     -> |  | ||||||
|                 rabbit_log:debug("Successfully stopped Ra server ~tp", [RaLeader]); |  | ||||||
|             {error, nodedown} -> |  | ||||||
|                 rabbit_log:error("Failed to stop Ra server ~tp: target node was reported as down") |  | ||||||
|         end |  | ||||||
|      end || Q <- Queues], |  | ||||||
|     rabbit_log:info("Leadership transfer for quorum queues hosted on this node has been initiated"). |  | ||||||
| 
 |  | ||||||
| transfer_leadership_of_metadata_store(TransferCandidates) -> | transfer_leadership_of_metadata_store(TransferCandidates) -> | ||||||
|     rabbit_log:info("Will transfer leadership of metadata store with current leader on this node", |     rabbit_log:info("Will transfer leadership of metadata store with current leader on this node", | ||||||
|                     []), |                     []), | ||||||
|  | @ -224,47 +191,6 @@ transfer_leadership_of_metadata_store(TransferCandidates) -> | ||||||
|             rabbit_log:warning("Skipping leadership transfer of metadata store: ~p", [Error]) |             rabbit_log:warning("Skipping leadership transfer of metadata store: ~p", [Error]) | ||||||
|     end. |     end. | ||||||
| 
 | 
 | ||||||
| -spec transfer_leadership_of_stream_coordinator([node()]) -> ok. |  | ||||||
| transfer_leadership_of_stream_coordinator([]) -> |  | ||||||
|     rabbit_log:warning("Skipping leadership transfer of stream coordinator: no candidate " |  | ||||||
|                        "(online, not under maintenance) nodes to transfer to!"); |  | ||||||
| transfer_leadership_of_stream_coordinator(TransferCandidates) -> |  | ||||||
|     % try to transfer to the node with the lowest uptime; the assumption is that |  | ||||||
|     % nodes are usually restarted in a rolling fashion, in a consistent order; |  | ||||||
|     % therefore, the youngest node has already been restarted  or (if we are draining the first node) |  | ||||||
|     % that it will be restarted last. either way, this way we limit the number of transfers |  | ||||||
|     Uptimes = rabbit_misc:append_rpc_all_nodes(TransferCandidates, erlang, statistics, [wall_clock]), |  | ||||||
|     Candidates = lists:zipwith(fun(N, {U, _}) -> {N, U}  end, TransferCandidates, Uptimes), |  | ||||||
|     BestCandidate = element(1, hd(lists:keysort(2, Candidates))), |  | ||||||
|     case rabbit_stream_coordinator:transfer_leadership([BestCandidate]) of |  | ||||||
|         {ok, Node} -> |  | ||||||
|             rabbit_log:info("Leadership transfer for stream coordinator completed. The new leader is ~p", [Node]); |  | ||||||
|         Error -> |  | ||||||
|             rabbit_log:warning("Skipping leadership transfer of stream coordinator: ~p", [Error]) |  | ||||||
|     end. |  | ||||||
| 
 |  | ||||||
| -spec stop_local_quorum_queue_followers() -> ok. |  | ||||||
| stop_local_quorum_queue_followers() -> |  | ||||||
|     Queues = rabbit_amqqueue:list_local_followers(), |  | ||||||
|     rabbit_log:info("Will stop local follower replicas of ~b quorum queues on this node", |  | ||||||
|                     [length(Queues)]), |  | ||||||
|     [begin |  | ||||||
|         Name = amqqueue:get_name(Q), |  | ||||||
|         rabbit_log:debug("Will stop a local follower replica of quorum queue ~ts", |  | ||||||
|                          [rabbit_misc:rs(Name)]), |  | ||||||
|         %% shut down Ra nodes so that they are not considered for leader election |  | ||||||
|         {RegisteredName, _LeaderNode} = amqqueue:get_pid(Q), |  | ||||||
|         RaNode = {RegisteredName, node()}, |  | ||||||
|         rabbit_log:debug("Will stop Ra server ~tp", [RaNode]), |  | ||||||
|         case rabbit_quorum_queue:stop_server(RaNode) of |  | ||||||
|             ok     -> |  | ||||||
|                 rabbit_log:debug("Successfully stopped Ra server ~tp", [RaNode]); |  | ||||||
|             {error, nodedown} -> |  | ||||||
|                 rabbit_log:error("Failed to stop Ra server ~tp: target node was reported as down") |  | ||||||
|         end |  | ||||||
|      end || Q <- Queues], |  | ||||||
|     rabbit_log:info("Stopped all local replicas of quorum queues hosted on this node"). |  | ||||||
| 
 |  | ||||||
| -spec primary_replica_transfer_candidate_nodes() -> [node()]. | -spec primary_replica_transfer_candidate_nodes() -> [node()]. | ||||||
| primary_replica_transfer_candidate_nodes() -> | primary_replica_transfer_candidate_nodes() -> | ||||||
|     filter_out_drained_nodes_consistent_read(rabbit_nodes:list_running() -- [node()]). |     filter_out_drained_nodes_consistent_read(rabbit_nodes:list_running() -- [node()]). | ||||||
|  | @ -289,24 +215,6 @@ random_nth(Nodes) -> | ||||||
|     Nth = erlang:phash2(erlang:monotonic_time(), length(Nodes)), |     Nth = erlang:phash2(erlang:monotonic_time(), length(Nodes)), | ||||||
|     lists:nth(Nth + 1, Nodes). |     lists:nth(Nth + 1, Nodes). | ||||||
| 
 | 
 | ||||||
| revive_local_quorum_queue_replicas() -> |  | ||||||
|     Queues = rabbit_amqqueue:list_local_followers(), |  | ||||||
|     %% NB: this function ignores the first argument so we can just pass the |  | ||||||
|     %% empty binary as the vhost name. |  | ||||||
|     {Recovered, Failed} = rabbit_quorum_queue:recover(<<>>, Queues), |  | ||||||
|     rabbit_log:debug("Successfully revived ~b quorum queue replicas", |  | ||||||
|                      [length(Recovered)]), |  | ||||||
|     case length(Failed) of |  | ||||||
|         0 -> |  | ||||||
|             ok; |  | ||||||
|         NumFailed -> |  | ||||||
|             rabbit_log:error("Failed to revive ~b quorum queue replicas", |  | ||||||
|                              [NumFailed]) |  | ||||||
|     end, |  | ||||||
| 
 |  | ||||||
|     rabbit_log:info("Restart of local quorum queue replicas is complete"), |  | ||||||
|     ok. |  | ||||||
| 
 |  | ||||||
| %% | %% | ||||||
| %% Implementation | %% Implementation | ||||||
| %% | %% | ||||||
|  |  | ||||||
|  | @ -7,10 +7,21 @@ | ||||||
| 
 | 
 | ||||||
| -module(rabbit_observer_cli). | -module(rabbit_observer_cli). | ||||||
| 
 | 
 | ||||||
| -export([init/0]). | -export([init/0, add_plugin/1]). | ||||||
| 
 | 
 | ||||||
| init() -> | init() -> | ||||||
|     application:set_env(observer_cli, plugins, [ |     application:set_env(observer_cli, plugins, [ | ||||||
|         rabbit_observer_cli_classic_queues:plugin_info(), |         rabbit_observer_cli_classic_queues:plugin_info(), | ||||||
|         rabbit_observer_cli_quorum_queues:plugin_info() |         rabbit_observer_cli_quorum_queues:plugin_info() | ||||||
|     ]). |     ]). | ||||||
|  | 
 | ||||||
|  | %% must be executed after observer_cli boot_step | ||||||
|  | add_plugin(PluginInfo) -> | ||||||
|  |     case application:get_env(observer_cli, plugins, undefined) of | ||||||
|  |         undefined -> %% shouldn't be there, die | ||||||
|  |             exit({rabbit_observer_cli_step_not_there, "Can't add observer_cli plugin, required boot_step wasn't executed"}); | ||||||
|  |         Plugins when is_list(Plugins) -> | ||||||
|  |             application:set_env(observer_cli, plugins, Plugins ++ [PluginInfo]); | ||||||
|  |         _ -> | ||||||
|  |             exit({rabbit_observer_cli_plugins_error, "Can't add observer_cli plugin, existing entry is not a list"}) | ||||||
|  |     end. | ||||||
|  |  | ||||||
|  | @ -493,10 +493,13 @@ matches_type(_,        _)               -> false. | ||||||
| 
 | 
 | ||||||
| matches_queue_type(queue, _, <<"all">>)    -> true; | matches_queue_type(queue, _, <<"all">>)    -> true; | ||||||
| matches_queue_type(queue, _, <<"queues">>) -> true; | matches_queue_type(queue, _, <<"queues">>) -> true; | ||||||
| matches_queue_type(queue, rabbit_classic_queue, <<"classic_queues">>) -> true; | matches_queue_type(queue, TypeModule, Term) -> | ||||||
| matches_queue_type(queue, rabbit_quorum_queue,  <<"quorum_queues">>)  -> true; |     %% we assume here TypeModule comes from queue struct, | ||||||
| matches_queue_type(queue, rabbit_stream_queue,  <<"streams">>)        -> true; |     %% therefore it is used and loaded - no need to check | ||||||
| matches_queue_type(queue, _, _) -> false. |     %% with registry. | ||||||
|  |     %% we also assume here and elsewhere that queue type | ||||||
|  |     %% module developer implemented all needed callbacks | ||||||
|  |     TypeModule:policy_apply_to_name() == Term. | ||||||
| 
 | 
 | ||||||
| priority_comparator(A, B) -> pget(priority, A) >= pget(priority, B). | priority_comparator(A, B) -> pget(priority, A) >= pget(priority, B). | ||||||
| 
 | 
 | ||||||
|  | @ -578,9 +581,20 @@ is_proplist(L) -> length(L) =:= length([I || I = {_, _} <- L]). | ||||||
| apply_to_validation(_Name, <<"all">>)       -> ok; | apply_to_validation(_Name, <<"all">>)       -> ok; | ||||||
| apply_to_validation(_Name, <<"exchanges">>) -> ok; | apply_to_validation(_Name, <<"exchanges">>) -> ok; | ||||||
| apply_to_validation(_Name, <<"queues">>)    -> ok; | apply_to_validation(_Name, <<"queues">>)    -> ok; | ||||||
| apply_to_validation(_Name, <<"classic_queues">>)    -> ok; |  | ||||||
| apply_to_validation(_Name, <<"quorum_queues">>)    -> ok; |  | ||||||
| apply_to_validation(_Name, <<"streams">>)    -> ok; | apply_to_validation(_Name, <<"streams">>)    -> ok; | ||||||
| apply_to_validation(_Name, Term) -> | apply_to_validation(_Name, Term) -> | ||||||
|     {error, "apply-to '~ts' unrecognised; should be one of: 'queues', 'classic_queues', " |     %% as a last restort go to queue types registry | ||||||
|      " 'quorum_queues', 'streams', 'exchanges', or 'all'", [Term]}. |     %% and try to find something here | ||||||
|  |     case maybe_apply_to_queue_type(Term) of | ||||||
|  |         true -> ok; | ||||||
|  |         false -> | ||||||
|  |             %% TODO: get recognized queue terms from queue types from queue type. | ||||||
|  |             {error, "apply-to '~ts' unrecognised; should be one of: 'queues', 'classic_queues', " | ||||||
|  |              " 'quorum_queues', 'streams', 'exchanges', or 'all'", [Term]} | ||||||
|  |     end. | ||||||
|  | 
 | ||||||
|  | maybe_apply_to_queue_type(Term) -> | ||||||
|  |     [] =/= lists:filter(fun({_TypeName, TypeModule}) -> | ||||||
|  |                                TypeModule:policy_apply_to_name() == Term | ||||||
|  |                        end, | ||||||
|  |                        rabbit_registry:lookup_all(queue)). | ||||||
|  |  | ||||||
|  | @ -45,7 +45,7 @@ queue_leader_locators() -> | ||||||
| -spec select_leader_and_followers(amqqueue:amqqueue(), pos_integer()) -> | -spec select_leader_and_followers(amqqueue:amqqueue(), pos_integer()) -> | ||||||
|     {Leader :: node(), Followers :: [node()]}. |     {Leader :: node(), Followers :: [node()]}. | ||||||
| select_leader_and_followers(Q, Size) | select_leader_and_followers(Q, Size) | ||||||
|   when (?amqqueue_is_quorum(Q) orelse ?amqqueue_is_stream(Q) orelse ?amqqueue_is_classic(Q)) andalso is_integer(Size) -> |   when (?is_amqqueue_v2(Q)) andalso is_integer(Size) -> | ||||||
|     LeaderLocator = leader_locator(Q), |     LeaderLocator = leader_locator(Q), | ||||||
|     QueueType = amqqueue:get_type(Q), |     QueueType = amqqueue:get_type(Q), | ||||||
|     do_select_leader_and_followers(Size, QueueType, LeaderLocator). |     do_select_leader_and_followers(Size, QueueType, LeaderLocator). | ||||||
|  | @ -109,6 +109,7 @@ leader_locator0(_) -> | ||||||
|     %% default |     %% default | ||||||
|     <<"client-local">>. |     <<"client-local">>. | ||||||
| 
 | 
 | ||||||
|  | %% TODO: allow dispatching by queue type | ||||||
| -spec select_members(pos_integer(), rabbit_queue_type:queue_type(), [node(),...], [node(),...], | -spec select_members(pos_integer(), rabbit_queue_type:queue_type(), [node(),...], [node(),...], | ||||||
|                       non_neg_integer(), non_neg_integer(), function()) -> |                       non_neg_integer(), non_neg_integer(), function()) -> | ||||||
|     {[node(),...], function()}. |     {[node(),...], function()}. | ||||||
|  |  | ||||||
|  | @ -62,7 +62,14 @@ | ||||||
|          arguments/1, |          arguments/1, | ||||||
|          arguments/2, |          arguments/2, | ||||||
|          notify_decorators/1, |          notify_decorators/1, | ||||||
|          publish_at_most_once/2 |          publish_at_most_once/2, | ||||||
|  |          can_redeliver/2, | ||||||
|  |          stop/1, | ||||||
|  |          endangered_queues/0, | ||||||
|  |          drain/1, | ||||||
|  |          revive/0, | ||||||
|  |          queue_vm_stats_sups/0, | ||||||
|  |          queue_vm_ets/0 | ||||||
|          ]). |          ]). | ||||||
| 
 | 
 | ||||||
| -export([ | -export([ | ||||||
|  | @ -77,7 +84,7 @@ | ||||||
| %% sequence number typically | %% sequence number typically | ||||||
| -type correlation() :: term(). | -type correlation() :: term(). | ||||||
| -type arguments() :: queue_arguments | consumer_arguments. | -type arguments() :: queue_arguments | consumer_arguments. | ||||||
| -type queue_type() :: rabbit_classic_queue | rabbit_quorum_queue | rabbit_stream_queue | module(). | -type queue_type() ::  module(). | ||||||
| %% see AMQP 1.0 §2.6.7 | %% see AMQP 1.0 §2.6.7 | ||||||
| -type delivery_count() :: sequence_no(). | -type delivery_count() :: sequence_no(). | ||||||
| -type credit() :: uint(). | -type credit() :: uint(). | ||||||
|  | @ -86,10 +93,6 @@ | ||||||
| 
 | 
 | ||||||
| -define(DOWN_KEYS, [name, durable, auto_delete, arguments, pid, type, state]). | -define(DOWN_KEYS, [name, durable, auto_delete, arguments, pid, type, state]). | ||||||
| 
 | 
 | ||||||
| %% TODO resolve all registered queue types from registry |  | ||||||
| -define(QUEUE_MODULES, [rabbit_classic_queue, rabbit_quorum_queue, rabbit_stream_queue]). |  | ||||||
| -define(KNOWN_QUEUE_TYPES, [<<"classic">>, <<"quorum">>, <<"stream">>]). |  | ||||||
| 
 |  | ||||||
| -type credit_reply_action() :: {credit_reply, rabbit_types:ctag(), delivery_count(), credit(), | -type credit_reply_action() :: {credit_reply, rabbit_types:ctag(), delivery_count(), credit(), | ||||||
|                                 Available :: non_neg_integer(), Drain :: boolean()}. |                                 Available :: non_neg_integer(), Drain :: boolean()}. | ||||||
| 
 | 
 | ||||||
|  | @ -274,75 +277,59 @@ | ||||||
| -callback notify_decorators(amqqueue:amqqueue()) -> | -callback notify_decorators(amqqueue:amqqueue()) -> | ||||||
|     ok. |     ok. | ||||||
| 
 | 
 | ||||||
|  | -callback queue_topology(amqqueue:amqqueue()) -> | ||||||
|  |     {Leader :: undefined | node(), Replicas :: undefined | [node(),...]}. | ||||||
|  | 
 | ||||||
|  | -callback feature_flag_name() -> atom(). | ||||||
|  | 
 | ||||||
|  | -callback policy_apply_to_name() -> binary(). | ||||||
|  | 
 | ||||||
|  | %% -callback on_node_up(node()) -> ok. | ||||||
|  | 
 | ||||||
|  | %% -callback on_node_down(node()) -> ok. | ||||||
|  | 
 | ||||||
|  | -callback can_redeliver() -> boolean(). | ||||||
|  | 
 | ||||||
|  | -callback stop(rabbit_types:vhost()) -> ok. | ||||||
|  | 
 | ||||||
|  | -callback is_replicated() -> boolean(). | ||||||
|  | 
 | ||||||
|  | -callback rebalance_module() -> module() | {error, not_supported}. | ||||||
|  | 
 | ||||||
|  | -callback list_with_minimum_quorum() -> [amqqueue:amqqueue()]. | ||||||
|  | 
 | ||||||
|  | -callback drain([node()]) -> ok. | ||||||
|  | 
 | ||||||
|  | -callback revive() -> ok. | ||||||
|  | 
 | ||||||
|  | %% used by rabbit_vm to emit queue process | ||||||
|  | %% (currently memory and binary) stats | ||||||
|  | -callback queue_vm_stats_sups() -> {StatsKeys :: [atom()], SupsNames:: [[atom()]]}. | ||||||
|  | 
 | ||||||
| -spec discover(binary() | atom()) -> queue_type(). | -spec discover(binary() | atom()) -> queue_type(). | ||||||
| discover(<<"undefined">>) -> | discover(<<"undefined">>) -> | ||||||
|     fallback(); |     fallback(); | ||||||
| discover(undefined) -> | discover(undefined) -> | ||||||
|     fallback(); |     fallback(); | ||||||
| %% TODO: should this use a registry that's populated on boot? | discover(TypeDescriptor) -> | ||||||
| discover(<<"quorum">>) -> |     {ok, TypeModule} = rabbit_registry:lookup_type_module(queue, TypeDescriptor), | ||||||
|     rabbit_quorum_queue; |     TypeModule. | ||||||
| discover(rabbit_quorum_queue) -> |  | ||||||
|     rabbit_quorum_queue; |  | ||||||
| discover(<<"classic">>) -> |  | ||||||
|     rabbit_classic_queue; |  | ||||||
| discover(rabbit_classic_queue) -> |  | ||||||
|     rabbit_classic_queue; |  | ||||||
| discover(rabbit_stream_queue) -> |  | ||||||
|     rabbit_stream_queue; |  | ||||||
| discover(<<"stream">>) -> |  | ||||||
|     rabbit_stream_queue; |  | ||||||
| discover(Other) when is_atom(Other) -> |  | ||||||
|     discover(rabbit_data_coercion:to_binary(Other)); |  | ||||||
| discover(Other) when is_binary(Other) -> |  | ||||||
|     T = rabbit_registry:binary_to_type(Other), |  | ||||||
|     rabbit_log:debug("Queue type discovery: will look up a module for type '~tp'", [T]), |  | ||||||
|     {ok, Mod} = rabbit_registry:lookup_module(queue, T), |  | ||||||
|     Mod. |  | ||||||
| 
 | 
 | ||||||
| -spec short_alias_of(queue_type()) -> binary(). | -spec short_alias_of(TypeDescriptor) -> Ret when | ||||||
| %% The opposite of discover/1: returns a short alias given a module name |       TypeDescriptor :: atom() | binary(), | ||||||
| short_alias_of(<<"rabbit_quorum_queue">>) -> |       Ret :: binary(). | ||||||
|     <<"quorum">>; | short_alias_of(TypeDescriptor) -> | ||||||
| short_alias_of(rabbit_quorum_queue) -> |     case rabbit_registry:lookup_type_name(queue, TypeDescriptor) of | ||||||
|     <<"quorum">>; |         {ok, TypeName} -> TypeName; | ||||||
| %% AMQP 1.0 management client |         _ -> undefined | ||||||
| short_alias_of({utf8, <<"quorum">>}) -> |     end. | ||||||
|     <<"quorum">>; |  | ||||||
| short_alias_of(<<"rabbit_classic_queue">>) -> |  | ||||||
|     <<"classic">>; |  | ||||||
| short_alias_of(rabbit_classic_queue) -> |  | ||||||
|     <<"classic">>; |  | ||||||
| %% AMQP 1.0 management client |  | ||||||
| short_alias_of({utf8, <<"classic">>}) -> |  | ||||||
|     <<"classic">>; |  | ||||||
| short_alias_of(<<"rabbit_stream_queue">>) -> |  | ||||||
|     <<"stream">>; |  | ||||||
| short_alias_of(rabbit_stream_queue) -> |  | ||||||
|     <<"stream">>; |  | ||||||
| %% AMQP 1.0 management client |  | ||||||
| short_alias_of({utf8, <<"stream">>}) -> |  | ||||||
|     <<"stream">>; |  | ||||||
| %% for cases where this function is used for |  | ||||||
| %% formatting of values that already might use these |  | ||||||
| %% short aliases |  | ||||||
| short_alias_of(<<"quorum">>) -> |  | ||||||
|     <<"quorum">>; |  | ||||||
| short_alias_of(<<"classic">>) -> |  | ||||||
|     <<"classic">>; |  | ||||||
| short_alias_of(<<"stream">>) -> |  | ||||||
|     <<"stream">>; |  | ||||||
| short_alias_of(_Other) -> |  | ||||||
|     undefined. |  | ||||||
| 
 | 
 | ||||||
| feature_flag_name(<<"quorum">>) -> | feature_flag_name(TypeDescriptor) -> | ||||||
|     quorum_queue; |     case rabbit_registry:lookup_type_module(queue, TypeDescriptor) of | ||||||
| feature_flag_name(<<"classic">>) -> |         {ok, TypeModule} -> | ||||||
|     undefined; |             TypeModule:feature_flag_name(); | ||||||
| feature_flag_name(<<"stream">>) -> |         _ -> undefined | ||||||
|     stream_queue; |     end. | ||||||
| feature_flag_name(_) -> |  | ||||||
|     undefined. |  | ||||||
| 
 | 
 | ||||||
| %% If the client does not specify the type, the virtual host does not have any | %% If the client does not specify the type, the virtual host does not have any | ||||||
| %% metadata default, and rabbit.default_queue_type is not set in the application env, | %% metadata default, and rabbit.default_queue_type is not set in the application env, | ||||||
|  | @ -362,15 +349,15 @@ default() -> | ||||||
| default_alias() -> | default_alias() -> | ||||||
|     short_alias_of(default()). |     short_alias_of(default()). | ||||||
| 
 | 
 | ||||||
|  | %% used for example like this | ||||||
|  | %% {{utf8, <<"type">>}, {utf8, rabbit_queue_type:to_binary(QType)}}, | ||||||
|  | %% so not just any binary but a type name | ||||||
| -spec to_binary(module()) -> binary(). | -spec to_binary(module()) -> binary(). | ||||||
| to_binary(rabbit_classic_queue) -> | to_binary(TypeModule) -> | ||||||
|     <<"classic">>; |     case rabbit_registry:lookup_type_name(queue, TypeModule) of | ||||||
| to_binary(rabbit_quorum_queue) -> |         {ok, TypeName} -> TypeName; | ||||||
|     <<"quorum">>; |         _ -> undefined | ||||||
| to_binary(rabbit_stream_queue) -> |     end. | ||||||
|     <<"stream">>; |  | ||||||
| to_binary(Other) -> |  | ||||||
|     atom_to_binary(Other). |  | ||||||
| 
 | 
 | ||||||
| %% is a specific queue type implementation enabled | %% is a specific queue type implementation enabled | ||||||
| -spec is_enabled(module()) -> boolean(). | -spec is_enabled(module()) -> boolean(). | ||||||
|  | @ -849,14 +836,13 @@ qref(Q) when ?is_amqqueue(Q) -> | ||||||
| known_queue_type_modules() -> | known_queue_type_modules() -> | ||||||
|     Registered = rabbit_registry:lookup_all(queue), |     Registered = rabbit_registry:lookup_all(queue), | ||||||
|     {_, Modules} = lists:unzip(Registered), |     {_, Modules} = lists:unzip(Registered), | ||||||
|     ?QUEUE_MODULES ++ Modules. |     Modules. | ||||||
| 
 | 
 | ||||||
| -spec known_queue_type_names() -> [binary()]. | -spec known_queue_type_names() -> [binary()]. | ||||||
| known_queue_type_names() -> | known_queue_type_names() -> | ||||||
|     Registered = rabbit_registry:lookup_all(queue), |     Registered = rabbit_registry:lookup_all(queue), | ||||||
|     {QueueTypes, _} = lists:unzip(Registered), |     {QueueTypes, _} = lists:unzip(Registered), | ||||||
|     QTypeBins = lists:map(fun(X) -> atom_to_binary(X) end, QueueTypes), |     lists:map(fun(X) -> atom_to_binary(X) end, QueueTypes). | ||||||
|     ?KNOWN_QUEUE_TYPES ++ QTypeBins. |  | ||||||
| 
 | 
 | ||||||
| inject_dqt(VHost) when ?is_vhost(VHost) -> | inject_dqt(VHost) when ?is_vhost(VHost) -> | ||||||
|     inject_dqt(vhost:to_map(VHost)); |     inject_dqt(vhost:to_map(VHost)); | ||||||
|  | @ -920,3 +906,46 @@ check_cluster_queue_limit(Q) -> | ||||||
| 
 | 
 | ||||||
| queue_limit_error(Reason, ReasonArgs) -> | queue_limit_error(Reason, ReasonArgs) -> | ||||||
|     {error, queue_limit_exceeded, Reason, ReasonArgs}. |     {error, queue_limit_exceeded, Reason, ReasonArgs}. | ||||||
|  | 
 | ||||||
|  | -spec can_redeliver(queue_name(), state()) -> | ||||||
|  |     {ok, module()} | {error, not_found}. | ||||||
|  | can_redeliver(Q, State) -> | ||||||
|  |     case module(Q, State) of | ||||||
|  |         {ok, TypeModule} -> | ||||||
|  |             TypeModule:can_redeliver(); | ||||||
|  |         _ -> false | ||||||
|  |     end. | ||||||
|  | 
 | ||||||
|  | -spec stop(rabbit_types:vhost()) -> ok. | ||||||
|  | stop(VHost) -> | ||||||
|  |     %% original rabbit_amqqueue:stop doesn't do any catches or try after | ||||||
|  |     _ = [TypeModule:stop(VHost) || {_Type, TypeModule} <- rabbit_registry:lookup_all(queue)], | ||||||
|  |     ok. | ||||||
|  | 
 | ||||||
|  | endangered_queues() -> | ||||||
|  |     lists:append([TypeModule:list_with_minimum_quorum() | ||||||
|  |                   || {_Type, TypeModule} <- rabbit_registry:lookup_all(queue)]). | ||||||
|  | 
 | ||||||
|  | drain(TransferCandidates) -> | ||||||
|  |     _ = [TypeModule:drain(TransferCandidates) || | ||||||
|  |             {_Type, TypeModule} <- rabbit_registry:lookup_all(queue)], | ||||||
|  |     ok. | ||||||
|  | 
 | ||||||
|  | revive() -> | ||||||
|  |     _ = [TypeModule:revive() || | ||||||
|  |             {_Type, TypeModule} <- rabbit_registry:lookup_all(queue)], | ||||||
|  |     ok. | ||||||
|  | 
 | ||||||
|  | queue_vm_stats_sups() -> | ||||||
|  |     lists:foldl(fun({_TypeName, TypeModule}, {KeysAcc, SupsAcc}) -> | ||||||
|  |                         {Keys, Sups} = TypeModule:queue_vm_stats_sups(), | ||||||
|  |                         {KeysAcc ++ Keys, SupsAcc ++ Sups} | ||||||
|  |                 end, | ||||||
|  |                 {[], []}, rabbit_registry:lookup_all(queue)). | ||||||
|  | 
 | ||||||
|  | queue_vm_ets() -> | ||||||
|  |     lists:foldl(fun({_TypeName, TypeModule}, {KeysAcc, SupsAcc}) -> | ||||||
|  |                         {Keys, Tables} = TypeModule:queue_vm_ets(), | ||||||
|  |                         {KeysAcc ++ Keys, SupsAcc ++ Tables} | ||||||
|  |                 end, | ||||||
|  |                 {[], []}, rabbit_registry:lookup_all(queue)). | ||||||
|  |  | ||||||
|  | @ -77,6 +77,17 @@ | ||||||
|          force_vhost_queues_shrink_member_to_current_member/1, |          force_vhost_queues_shrink_member_to_current_member/1, | ||||||
|          force_all_queues_shrink_member_to_current_member/0]). |          force_all_queues_shrink_member_to_current_member/0]). | ||||||
| 
 | 
 | ||||||
|  | -export([queue_topology/1, | ||||||
|  |          feature_flag_name/0, | ||||||
|  |          policy_apply_to_name/0, | ||||||
|  |          can_redeliver/0, | ||||||
|  |          is_replicated/0, | ||||||
|  |          rebalance_module/0, | ||||||
|  |          drain/1, | ||||||
|  |          revive/0, | ||||||
|  |          queue_vm_stats_sups/0, | ||||||
|  |          queue_vm_ets/0]). | ||||||
|  | 
 | ||||||
| %% for backwards compatibility | %% for backwards compatibility | ||||||
| -export([file_handle_leader_reservation/1, | -export([file_handle_leader_reservation/1, | ||||||
|          file_handle_other_reservation/0, |          file_handle_other_reservation/0, | ||||||
|  | @ -95,6 +106,15 @@ | ||||||
| -include_lib("rabbit_common/include/rabbit.hrl"). | -include_lib("rabbit_common/include/rabbit.hrl"). | ||||||
| -include("amqqueue.hrl"). | -include("amqqueue.hrl"). | ||||||
| 
 | 
 | ||||||
|  | -rabbit_boot_step( | ||||||
|  |    {rabbit_quorum_queue_type, | ||||||
|  |     [{description, "Quorum queue: queue type"}, | ||||||
|  |      {mfa,      {rabbit_registry, register, | ||||||
|  |                     [queue, <<"quorum">>, ?MODULE]}}, | ||||||
|  |      {cleanup,  {rabbit_registry, unregister, | ||||||
|  |                  [queue, <<"quorum">>]}}, | ||||||
|  |      {requires, rabbit_registry}]}). | ||||||
|  | 
 | ||||||
| -type msg_id() :: non_neg_integer(). | -type msg_id() :: non_neg_integer(). | ||||||
| -type qmsg() :: {rabbit_types:r('queue'), pid(), msg_id(), boolean(), | -type qmsg() :: {rabbit_types:r('queue'), pid(), msg_id(), boolean(), | ||||||
|                  mc:state()}. |                  mc:state()}. | ||||||
|  | @ -160,7 +180,7 @@ | ||||||
|             [operator_policy_validator, <<"target-group-size">>, ?MODULE]}}, |             [operator_policy_validator, <<"target-group-size">>, ?MODULE]}}, | ||||||
|      {mfa, {rabbit_registry, register, |      {mfa, {rabbit_registry, register, | ||||||
|             [policy_merge_strategy, <<"target-group-size">>, ?MODULE]}}, |             [policy_merge_strategy, <<"target-group-size">>, ?MODULE]}}, | ||||||
|      {requires, rabbit_registry}, |      {requires, [rabbit_registry]}, | ||||||
|      {enables, recovery}]}). |      {enables, recovery}]}). | ||||||
| 
 | 
 | ||||||
| validate_policy(Args) -> | validate_policy(Args) -> | ||||||
|  | @ -2145,3 +2165,113 @@ file_handle_other_reservation() -> | ||||||
| file_handle_release_reservation() -> | file_handle_release_reservation() -> | ||||||
|     ok. |     ok. | ||||||
| 
 | 
 | ||||||
|  | -spec queue_topology(amqqueue:amqqueue()) -> | ||||||
|  |     {Leader :: undefined | node(), Replicas :: undefined | [node(),...]}. | ||||||
|  | queue_topology(Q) -> | ||||||
|  |     [{leader, Leader0}, | ||||||
|  |              {members, Members}] = rabbit_queue_type:info(Q, [leader, members]), | ||||||
|  |             Leader = case Leader0 of | ||||||
|  |                          '' -> undefined; | ||||||
|  |                          _ -> Leader0 | ||||||
|  |                      end, | ||||||
|  |             {Leader, Members}. | ||||||
|  | 
 | ||||||
|  | feature_flag_name() -> | ||||||
|  |     quorum_queue. | ||||||
|  | 
 | ||||||
|  | policy_apply_to_name() -> | ||||||
|  |     <<"quorum_queues">>. | ||||||
|  | 
 | ||||||
|  | can_redeliver() -> | ||||||
|  |     true. | ||||||
|  | 
 | ||||||
|  | is_replicated() -> | ||||||
|  |     true. | ||||||
|  | 
 | ||||||
|  | rebalance_module() -> | ||||||
|  |     ?MODULE. | ||||||
|  | 
 | ||||||
|  | -spec drain([node()]) -> ok. | ||||||
|  | drain(TransferCandidates) -> | ||||||
|  |     _ = transfer_leadership(TransferCandidates), | ||||||
|  |     _ = stop_local_quorum_queue_followers(), | ||||||
|  |     ok. | ||||||
|  | 
 | ||||||
|  | transfer_leadership([]) -> | ||||||
|  |     rabbit_log:warning("Skipping leadership transfer of quorum queues: no candidate " | ||||||
|  |                        "(online, not under maintenance) nodes to transfer to!"); | ||||||
|  | transfer_leadership(_TransferCandidates) -> | ||||||
|  |     %% we only transfer leadership for QQs that have local leaders | ||||||
|  |     Queues = rabbit_amqqueue:list_local_leaders(), | ||||||
|  |     rabbit_log:info("Will transfer leadership of ~b quorum queues with current leader on this node", | ||||||
|  |                     [length(Queues)]), | ||||||
|  |     [begin | ||||||
|  |         Name = amqqueue:get_name(Q), | ||||||
|  |         rabbit_log:debug("Will trigger a leader election for local quorum queue ~ts", | ||||||
|  |                          [rabbit_misc:rs(Name)]), | ||||||
|  |         %% we trigger an election and exclude this node from the list of candidates | ||||||
|  |         %% by simply shutting its local QQ replica (Ra server) | ||||||
|  |         RaLeader = amqqueue:get_pid(Q), | ||||||
|  |         rabbit_log:debug("Will stop Ra server ~tp", [RaLeader]), | ||||||
|  |         case rabbit_quorum_queue:stop_server(RaLeader) of | ||||||
|  |             ok     -> | ||||||
|  |                 rabbit_log:debug("Successfully stopped Ra server ~tp", [RaLeader]); | ||||||
|  |             {error, nodedown} -> | ||||||
|  |                 rabbit_log:error("Failed to stop Ra server ~tp: target node was reported as down") | ||||||
|  |         end | ||||||
|  |      end || Q <- Queues], | ||||||
|  |     rabbit_log:info("Leadership transfer for quorum queues hosted on this node has been initiated"). | ||||||
|  | 
 | ||||||
|  | %% TODO: I just copied it over, it looks like was always called inside maintenance so... | ||||||
|  | -spec stop_local_quorum_queue_followers() -> ok. | ||||||
|  | stop_local_quorum_queue_followers() -> | ||||||
|  |     Queues = rabbit_amqqueue:list_local_followers(), | ||||||
|  |     rabbit_log:info("Will stop local follower replicas of ~b quorum queues on this node", | ||||||
|  |                     [length(Queues)]), | ||||||
|  |     [begin | ||||||
|  |         Name = amqqueue:get_name(Q), | ||||||
|  |         rabbit_log:debug("Will stop a local follower replica of quorum queue ~ts", | ||||||
|  |                          [rabbit_misc:rs(Name)]), | ||||||
|  |         %% shut down Ra nodes so that they are not considered for leader election | ||||||
|  |         {RegisteredName, _LeaderNode} = amqqueue:get_pid(Q), | ||||||
|  |         RaNode = {RegisteredName, node()}, | ||||||
|  |         rabbit_log:debug("Will stop Ra server ~tp", [RaNode]), | ||||||
|  |         case rabbit_quorum_queue:stop_server(RaNode) of | ||||||
|  |             ok     -> | ||||||
|  |                 rabbit_log:debug("Successfully stopped Ra server ~tp", [RaNode]); | ||||||
|  |             {error, nodedown} -> | ||||||
|  |                 rabbit_log:error("Failed to stop Ra server ~tp: target node was reported as down") | ||||||
|  |         end | ||||||
|  |      end || Q <- Queues], | ||||||
|  |     rabbit_log:info("Stopped all local replicas of quorum queues hosted on this node"). | ||||||
|  | 
 | ||||||
|  | revive() -> | ||||||
|  |     revive_local_queue_replicas(). | ||||||
|  | 
 | ||||||
|  | revive_local_queue_replicas() -> | ||||||
|  |     Queues = rabbit_amqqueue:list_local_followers(), | ||||||
|  |     %% NB: this function ignores the first argument so we can just pass the | ||||||
|  |     %% empty binary as the vhost name. | ||||||
|  |     {Recovered, Failed} = rabbit_quorum_queue:recover(<<>>, Queues), | ||||||
|  |     rabbit_log:debug("Successfully revived ~b quorum queue replicas", | ||||||
|  |                      [length(Recovered)]), | ||||||
|  |     case length(Failed) of | ||||||
|  |         0 -> | ||||||
|  |             ok; | ||||||
|  |         NumFailed -> | ||||||
|  |             rabbit_log:error("Failed to revive ~b quorum queue replicas", | ||||||
|  |                              [NumFailed]) | ||||||
|  |     end, | ||||||
|  | 
 | ||||||
|  |     rabbit_log:info("Restart of local quorum queue replicas is complete"), | ||||||
|  |     ok. | ||||||
|  | 
 | ||||||
|  | queue_vm_stats_sups() -> | ||||||
|  |     {[quorum_queue_procs, | ||||||
|  |       quorum_queue_dlx_procs], | ||||||
|  |      [[ra_server_sup_sup], | ||||||
|  |       [rabbit_fifo_dlx_sup]]}. | ||||||
|  | 
 | ||||||
|  | queue_vm_ets() -> | ||||||
|  |     {[quorum_ets], | ||||||
|  |      [[ra_log_ets]]}. | ||||||
|  |  | ||||||
|  | @ -59,6 +59,18 @@ | ||||||
| 
 | 
 | ||||||
| -export([check_max_segment_size_bytes/1]). | -export([check_max_segment_size_bytes/1]). | ||||||
| 
 | 
 | ||||||
|  | -export([queue_topology/1, | ||||||
|  |          feature_flag_name/0, | ||||||
|  |          policy_apply_to_name/0, | ||||||
|  |          can_redeliver/0, | ||||||
|  |          stop/1, | ||||||
|  |          is_replicated/0, | ||||||
|  |          rebalance_module/0, | ||||||
|  |          drain/1, | ||||||
|  |          revive/0, | ||||||
|  |          queue_vm_stats_sups/0, | ||||||
|  |          queue_vm_ets/0]). | ||||||
|  | 
 | ||||||
| -include_lib("rabbit_common/include/rabbit.hrl"). | -include_lib("rabbit_common/include/rabbit.hrl"). | ||||||
| -include("amqqueue.hrl"). | -include("amqqueue.hrl"). | ||||||
| 
 | 
 | ||||||
|  | @ -103,6 +115,17 @@ | ||||||
| -import(rabbit_queue_type_util, [args_policy_lookup/3]). | -import(rabbit_queue_type_util, [args_policy_lookup/3]). | ||||||
| -import(rabbit_misc, [queue_resource/2]). | -import(rabbit_misc, [queue_resource/2]). | ||||||
| 
 | 
 | ||||||
|  | -rabbit_boot_step( | ||||||
|  |    {?MODULE, | ||||||
|  |     [{description, "Stream queue: queue type"}, | ||||||
|  |      {mfa,      {rabbit_registry, register, | ||||||
|  |                     [queue, <<"stream">>, ?MODULE]}}, | ||||||
|  |      %% {cleanup,  {rabbit_registry, unregister, | ||||||
|  |      %%             [queue, <<"stream">>]}}, | ||||||
|  |      {requires, rabbit_registry}%%, | ||||||
|  |      %% {enables,     rabbit_stream_queue_type} | ||||||
|  |     ]}). | ||||||
|  | 
 | ||||||
| -type client() :: #stream_client{}. | -type client() :: #stream_client{}. | ||||||
| 
 | 
 | ||||||
| -spec is_enabled() -> boolean(). | -spec is_enabled() -> boolean(). | ||||||
|  | @ -832,10 +855,6 @@ status(Vhost, QueueName) -> | ||||||
|     %% Handle not found queues |     %% Handle not found queues | ||||||
|     QName = #resource{virtual_host = Vhost, name = QueueName, kind = queue}, |     QName = #resource{virtual_host = Vhost, name = QueueName, kind = queue}, | ||||||
|     case rabbit_amqqueue:lookup(QName) of |     case rabbit_amqqueue:lookup(QName) of | ||||||
|         {ok, Q} when ?amqqueue_is_classic(Q) -> |  | ||||||
|             {error, classic_queue_not_supported}; |  | ||||||
|         {ok, Q} when ?amqqueue_is_quorum(Q) -> |  | ||||||
|             {error, quorum_queue_not_supported}; |  | ||||||
|         {ok, Q} when ?amqqueue_is_stream(Q) -> |         {ok, Q} when ?amqqueue_is_stream(Q) -> | ||||||
|             [begin |             [begin | ||||||
|                  [get_key(role, C), |                  [get_key(role, C), | ||||||
|  | @ -847,6 +866,8 @@ status(Vhost, QueueName) -> | ||||||
|                   get_key(readers, C), |                   get_key(readers, C), | ||||||
|                   get_key(segments, C)] |                   get_key(segments, C)] | ||||||
|              end || C <- get_counters(Q)]; |              end || C <- get_counters(Q)]; | ||||||
|  |         {ok, _Q} -> | ||||||
|  |             {error, not_supported}; | ||||||
|         {error, not_found} = E -> |         {error, not_found} = E -> | ||||||
|             E |             E | ||||||
|     end. |     end. | ||||||
|  | @ -905,10 +926,6 @@ tracking_status(Vhost, QueueName) -> | ||||||
|     %% Handle not found queues |     %% Handle not found queues | ||||||
|     QName = #resource{virtual_host = Vhost, name = QueueName, kind = queue}, |     QName = #resource{virtual_host = Vhost, name = QueueName, kind = queue}, | ||||||
|     case rabbit_amqqueue:lookup(QName) of |     case rabbit_amqqueue:lookup(QName) of | ||||||
|         {ok, Q} when ?amqqueue_is_classic(Q) -> |  | ||||||
|             {error, classic_queue_not_supported}; |  | ||||||
|         {ok, Q} when ?amqqueue_is_quorum(Q) -> |  | ||||||
|             {error, quorum_queue_not_supported}; |  | ||||||
|         {ok, Q} when ?amqqueue_is_stream(Q) -> |         {ok, Q} when ?amqqueue_is_stream(Q) -> | ||||||
|             Leader = amqqueue:get_pid(Q), |             Leader = amqqueue:get_pid(Q), | ||||||
|             Map = osiris:read_tracking(Leader), |             Map = osiris:read_tracking(Leader), | ||||||
|  | @ -921,6 +938,8 @@ tracking_status(Vhost, QueueName) -> | ||||||
|                                                   {value, TrkData}] | Acc0] |                                                   {value, TrkData}] | Acc0] | ||||||
|                                         end, [], Trackings) ++ Acc |                                         end, [], Trackings) ++ Acc | ||||||
|                       end, [], Map); |                       end, [], Map); | ||||||
|  |         {ok, Q} -> | ||||||
|  |             {error, {queue_not_supported, ?amqqueue_type(Q)}}; | ||||||
|         {error, not_found} = E-> |         {error, not_found} = E-> | ||||||
|             E |             E | ||||||
|     end. |     end. | ||||||
|  | @ -1021,10 +1040,6 @@ restart_stream(VHost, Queue, Options) | ||||||
| add_replica(VHost, Name, Node) -> | add_replica(VHost, Name, Node) -> | ||||||
|     QName = queue_resource(VHost, Name), |     QName = queue_resource(VHost, Name), | ||||||
|     case rabbit_amqqueue:lookup(QName) of |     case rabbit_amqqueue:lookup(QName) of | ||||||
|         {ok, Q} when ?amqqueue_is_classic(Q) -> |  | ||||||
|             {error, classic_queue_not_supported}; |  | ||||||
|         {ok, Q} when ?amqqueue_is_quorum(Q) -> |  | ||||||
|             {error, quorum_queue_not_supported}; |  | ||||||
|         {ok, Q} when ?amqqueue_is_stream(Q) -> |         {ok, Q} when ?amqqueue_is_stream(Q) -> | ||||||
|             case lists:member(Node, rabbit_nodes:list_running()) of |             case lists:member(Node, rabbit_nodes:list_running()) of | ||||||
|                 false -> |                 false -> | ||||||
|  | @ -1032,6 +1047,8 @@ add_replica(VHost, Name, Node) -> | ||||||
|                 true -> |                 true -> | ||||||
|                     rabbit_stream_coordinator:add_replica(Q, Node) |                     rabbit_stream_coordinator:add_replica(Q, Node) | ||||||
|             end; |             end; | ||||||
|  |         {ok, Q} -> | ||||||
|  |             {error, {queue_not_supported, ?amqqueue_type(Q)}}; | ||||||
|         E -> |         E -> | ||||||
|             E |             E | ||||||
|     end. |     end. | ||||||
|  | @ -1039,14 +1056,12 @@ add_replica(VHost, Name, Node) -> | ||||||
| delete_replica(VHost, Name, Node) -> | delete_replica(VHost, Name, Node) -> | ||||||
|     QName = queue_resource(VHost, Name), |     QName = queue_resource(VHost, Name), | ||||||
|     case rabbit_amqqueue:lookup(QName) of |     case rabbit_amqqueue:lookup(QName) of | ||||||
|         {ok, Q} when ?amqqueue_is_classic(Q) -> |  | ||||||
|             {error, classic_queue_not_supported}; |  | ||||||
|         {ok, Q} when ?amqqueue_is_quorum(Q) -> |  | ||||||
|             {error, quorum_queue_not_supported}; |  | ||||||
|         {ok, Q} when ?amqqueue_is_stream(Q) -> |         {ok, Q} when ?amqqueue_is_stream(Q) -> | ||||||
|             #{name := StreamId} = amqqueue:get_type_state(Q), |             #{name := StreamId} = amqqueue:get_type_state(Q), | ||||||
|             {ok, Reply, _} = rabbit_stream_coordinator:delete_replica(StreamId, Node), |             {ok, Reply, _} = rabbit_stream_coordinator:delete_replica(StreamId, Node), | ||||||
|             Reply; |             Reply; | ||||||
|  |         {ok, Q} -> | ||||||
|  |             {error, {queue_not_supported, ?amqqueue_type(Q)}}; | ||||||
|         E -> |         E -> | ||||||
|             E |             E | ||||||
|     end. |     end. | ||||||
|  | @ -1393,3 +1408,76 @@ delivery_count_add(none, _) -> | ||||||
|     none; |     none; | ||||||
| delivery_count_add(Count, N) -> | delivery_count_add(Count, N) -> | ||||||
|     serial_number:add(Count, N). |     serial_number:add(Count, N). | ||||||
|  | 
 | ||||||
|  | -spec queue_topology(amqqueue:amqqueue()) -> | ||||||
|  |     {Leader :: undefined | node(), Replicas :: undefined | [node(),...]}. | ||||||
|  | queue_topology(Q) -> | ||||||
|  |     #{name := StreamId} = amqqueue:get_type_state(Q), | ||||||
|  |     case rabbit_stream_coordinator:members(StreamId) of | ||||||
|  |         {ok, Members} -> | ||||||
|  |             maps:fold(fun(Node, {_Pid, writer}, {_, Replicas}) -> | ||||||
|  |                               {Node, [Node | Replicas]}; | ||||||
|  |                          (Node, {_Pid, replica}, {Writer, Replicas}) -> | ||||||
|  |                               {Writer, [Node | Replicas]} | ||||||
|  |                       end, {undefined, []}, Members); | ||||||
|  |                 {error, _} -> | ||||||
|  |             {undefined, undefined} | ||||||
|  |     end. | ||||||
|  | 
 | ||||||
|  | feature_flag_name() -> | ||||||
|  |     stream_queue. | ||||||
|  | 
 | ||||||
|  | policy_apply_to_name() -> | ||||||
|  |     <<"streams">>. | ||||||
|  | 
 | ||||||
|  | can_redeliver() -> | ||||||
|  |     true. | ||||||
|  | 
 | ||||||
|  | stop(_VHost) -> | ||||||
|  |     ok. | ||||||
|  | 
 | ||||||
|  | is_replicated() -> | ||||||
|  |     true. | ||||||
|  | 
 | ||||||
|  | rebalance_module() -> | ||||||
|  |     ?MODULE. | ||||||
|  | 
 | ||||||
|  | drain(TransferCandidates) -> | ||||||
|  |     case whereis(rabbit_stream_coordinator) of | ||||||
|  |         undefined -> ok; | ||||||
|  |         _Pid -> transfer_leadership_of_stream_coordinator(TransferCandidates) | ||||||
|  |     end. | ||||||
|  | 
 | ||||||
|  | revive() -> | ||||||
|  |     ok. | ||||||
|  | 
 | ||||||
|  | -spec transfer_leadership_of_stream_coordinator([node()]) -> ok. | ||||||
|  | transfer_leadership_of_stream_coordinator([]) -> | ||||||
|  |     rabbit_log:warning("Skipping leadership transfer of stream coordinator: no candidate " | ||||||
|  |                        "(online, not under maintenance) nodes to transfer to!"); | ||||||
|  | transfer_leadership_of_stream_coordinator(TransferCandidates) -> | ||||||
|  |     % try to transfer to the node with the lowest uptime; the assumption is that | ||||||
|  |     % nodes are usually restarted in a rolling fashion, in a consistent order; | ||||||
|  |     % therefore, the youngest node has already been restarted  or (if we are draining the first node) | ||||||
|  |     % that it will be restarted last. either way, this way we limit the number of transfers | ||||||
|  |     Uptimes = rabbit_misc:append_rpc_all_nodes(TransferCandidates, erlang, statistics, [wall_clock]), | ||||||
|  |     Candidates = lists:zipwith(fun(N, {U, _}) -> {N, U}  end, TransferCandidates, Uptimes), | ||||||
|  |     BestCandidate = element(1, hd(lists:keysort(2, Candidates))), | ||||||
|  |     case rabbit_stream_coordinator:transfer_leadership([BestCandidate]) of | ||||||
|  |         {ok, Node} -> | ||||||
|  |             rabbit_log:info("Leadership transfer for stream coordinator completed. The new leader is ~p", [Node]); | ||||||
|  |         Error -> | ||||||
|  |             rabbit_log:warning("Skipping leadership transfer of stream coordinator: ~p", [Error]) | ||||||
|  |     end. | ||||||
|  | 
 | ||||||
|  | queue_vm_stats_sups() -> | ||||||
|  |     {[stream_queue_procs, | ||||||
|  |       stream_queue_replica_reader_procs, | ||||||
|  |       stream_queue_coordinator_procs], | ||||||
|  |      [[osiris_server_sup], | ||||||
|  |       [osiris_replica_reader_sup], | ||||||
|  |       [rabbit_stream_coordinator]]}. | ||||||
|  | 
 | ||||||
|  | queue_vm_ets() -> | ||||||
|  |     {[], | ||||||
|  |      []}. | ||||||
|  |  | ||||||
|  | @ -56,9 +56,7 @@ endangered_critical_components() -> | ||||||
| do_await_safe_online_quorum(0) -> | do_await_safe_online_quorum(0) -> | ||||||
|     false; |     false; | ||||||
| do_await_safe_online_quorum(IterationsLeft) -> | do_await_safe_online_quorum(IterationsLeft) -> | ||||||
|     EndangeredQueues = lists:append( |     EndangeredQueues = rabbit_queue_type:endangered_queues(), | ||||||
|                          rabbit_quorum_queue:list_with_minimum_quorum(), |  | ||||||
|                          rabbit_stream_queue:list_with_minimum_quorum()), |  | ||||||
|     case EndangeredQueues =:= [] andalso endangered_critical_components() =:= [] of |     case EndangeredQueues =:= [] andalso endangered_critical_components() =:= [] of | ||||||
|         true -> true; |         true -> true; | ||||||
|         false -> |         false -> | ||||||
|  | @ -83,9 +81,7 @@ do_await_safe_online_quorum(IterationsLeft) -> | ||||||
| 
 | 
 | ||||||
| -spec list_with_minimum_quorum_for_cli() -> [#{binary() => term()}]. | -spec list_with_minimum_quorum_for_cli() -> [#{binary() => term()}]. | ||||||
| list_with_minimum_quorum_for_cli() -> | list_with_minimum_quorum_for_cli() -> | ||||||
|     EndangeredQueues = lists:append( |     EndangeredQueues = rabbit_queue_type:endangered_queues(), | ||||||
|                          rabbit_quorum_queue:list_with_minimum_quorum(), |  | ||||||
|                          rabbit_stream_queue:list_with_minimum_quorum()), |  | ||||||
|     [amqqueue:to_printable(Q) || Q <- EndangeredQueues] ++ |     [amqqueue:to_printable(Q) || Q <- EndangeredQueues] ++ | ||||||
|     [#{ |     [#{ | ||||||
|            <<"readable_name">> => C, |            <<"readable_name">> => C, | ||||||
|  |  | ||||||
|  | @ -7,7 +7,7 @@ | ||||||
| 
 | 
 | ||||||
| -module(rabbit_vm). | -module(rabbit_vm). | ||||||
| 
 | 
 | ||||||
| -export([memory/0, binary/0, ets_tables_memory/1]). | -export([memory/0, binary/0, ets_tables_memory/1, all_vhosts_children/1]). | ||||||
| 
 | 
 | ||||||
| -define(MAGIC_PLUGINS, ["cowboy", "ranch", "sockjs"]). | -define(MAGIC_PLUGINS, ["cowboy", "ranch", "sockjs"]). | ||||||
| 
 | 
 | ||||||
|  | @ -16,19 +16,37 @@ | ||||||
| -spec memory() -> rabbit_types:infos(). | -spec memory() -> rabbit_types:infos(). | ||||||
| 
 | 
 | ||||||
| memory() -> | memory() -> | ||||||
|     All = interesting_sups(), |     %% this whole aggregation pipeline preserves sups order | ||||||
|  |     %% [{info_key, [SupName...]}...] i.e. flattened list of | ||||||
|  |     %% info key, sups list pairs for each queue type | ||||||
|  |     %% example for existing info keys: | ||||||
|  |     %% [{queue_procs,          queue_sups()}, | ||||||
|  |     %%  {quorum_queue_procs,   [ra_server_sup_sup]}, | ||||||
|  |     %%  {quorum_queue_dlx_procs, [rabbit_fifo_dlx_sup]}, | ||||||
|  |     %%  {stream_queue_procs,   [osiris_server_sup]}, | ||||||
|  |     %%  {stream_queue_replica_reader_procs,  [osiris_replica_reader_sup]}, | ||||||
|  |     %%  {stream_queue_coordinator_procs, [rabbit_stream_coordinator]}] | ||||||
|  |     {QueueSupsStatsKeys, QueueStatsSups} = rabbit_queue_type:queue_vm_stats_sups(), | ||||||
|  | 
 | ||||||
|  |     %% we keep order and that means this variable queues part | ||||||
|  |     %% has to be matched somehow - | Rest is the best. | ||||||
|  |     All = interesting_sups() ++ QueueStatsSups, | ||||||
|     {Sums, _Other} = sum_processes( |     {Sums, _Other} = sum_processes( | ||||||
|                        lists:append(All), distinguishers(), [memory]), |                        lists:append(All), distinguishers(), [memory]), | ||||||
| 
 | 
 | ||||||
|     [Qs, Qqs, DlxWorkers, Ssqs, Srqs, SCoor, ConnsReader, ConnsWriter, ConnsChannel, |     [ConnsReader, ConnsWriter, ConnsChannel, | ||||||
|      ConnsOther, MsgIndexProc, MgmtDbProc, Plugins] = |      ConnsOther, MsgIndexProc, MgmtDbProc, Plugins | QueueSupsStats] = | ||||||
|         [aggregate(Names, Sums, memory, fun (X) -> X end) |         [aggregate(Names, Sums, memory, fun (X) -> X end) | ||||||
|          || Names <- distinguished_interesting_sups()], |          || Names <- distinguished_interesting_sups() ++ QueueStatsSups], | ||||||
|  | 
 | ||||||
|  | 
 | ||||||
|  |     {QueuesEtsStatsKeys, QueueStatsEtsNames} = rabbit_queue_type:queue_vm_ets(), | ||||||
|  | 
 | ||||||
|  |     QueuesEtsStats = lists:map(fun ets_memory/1, QueueStatsEtsNames), | ||||||
| 
 | 
 | ||||||
|     MnesiaETS           = mnesia_memory(), |     MnesiaETS           = mnesia_memory(), | ||||||
|     MsgIndexETS         = ets_memory(msg_stores()), |     MsgIndexETS         = ets_memory(msg_stores()), | ||||||
|     MetricsETS          = ets_memory([rabbit_metrics]), |     MetricsETS          = ets_memory([rabbit_metrics]), | ||||||
|     QuorumETS           = ets_memory([ra_log_ets]), |  | ||||||
|     MetricsProc  = try |     MetricsProc  = try | ||||||
|                        [{_, M}] = process_info(whereis(rabbit_metrics), [memory]), |                        [{_, M}] = process_info(whereis(rabbit_metrics), [memory]), | ||||||
|                        M |                        M | ||||||
|  | @ -63,23 +81,20 @@ memory() -> | ||||||
| 
 | 
 | ||||||
|     OtherProc = Processes |     OtherProc = Processes | ||||||
|         - ConnsReader - ConnsWriter - ConnsChannel - ConnsOther |         - ConnsReader - ConnsWriter - ConnsChannel - ConnsOther | ||||||
|         - Qs - Qqs - DlxWorkers - Ssqs - Srqs - SCoor - MsgIndexProc - Plugins |         - lists:sum(QueueSupsStats) - MsgIndexProc - Plugins | ||||||
|         - MgmtDbProc - MetricsProc - MetadataStoreProc, |         - MgmtDbProc - MetricsProc - MetadataStoreProc, | ||||||
|  | 
 | ||||||
|     [ |     [ | ||||||
|      %% Connections |      %% Connections | ||||||
|      {connection_readers,   ConnsReader}, |      {connection_readers,   ConnsReader}, | ||||||
|      {connection_writers,   ConnsWriter}, |      {connection_writers,   ConnsWriter}, | ||||||
|      {connection_channels,  ConnsChannel}, |      {connection_channels,  ConnsChannel}, | ||||||
|      {connection_other,     ConnsOther}, |      {connection_other,     ConnsOther}] ++ | ||||||
| 
 | 
 | ||||||
|      %% Queues |      %% Queues | ||||||
|      {queue_procs,          Qs}, |     lists:zip(QueueSupsStatsKeys, QueueSupsStats) ++ | ||||||
|      {quorum_queue_procs,   Qqs}, |  | ||||||
|      {quorum_queue_dlx_procs, DlxWorkers}, |  | ||||||
|      {stream_queue_procs,   Ssqs}, |  | ||||||
|      {stream_queue_replica_reader_procs,  Srqs}, |  | ||||||
|      {stream_queue_coordinator_procs, SCoor}, |  | ||||||
| 
 | 
 | ||||||
|  |     [ | ||||||
|      %% Processes |      %% Processes | ||||||
|      {plugins,              Plugins}, |      {plugins,              Plugins}, | ||||||
|      {metadata_store,       MetadataStoreProc}, |      {metadata_store,       MetadataStoreProc}, | ||||||
|  | @ -87,13 +102,16 @@ memory() -> | ||||||
| 
 | 
 | ||||||
|      %% Metrics |      %% Metrics | ||||||
|      {metrics,              MetricsETS + MetricsProc}, |      {metrics,              MetricsETS + MetricsProc}, | ||||||
|      {mgmt_db,              MgmtDbETS + MgmtDbProc}, |      {mgmt_db,              MgmtDbETS + MgmtDbProc}] ++ | ||||||
| 
 | 
 | ||||||
|      %% ETS |      %% ETS | ||||||
|  |      %% queues | ||||||
|  |     lists:zip(QueuesEtsStatsKeys, QueuesEtsStats) ++ | ||||||
|  | 
 | ||||||
|  |     [ | ||||||
|      {mnesia,               MnesiaETS}, |      {mnesia,               MnesiaETS}, | ||||||
|      {quorum_ets,           QuorumETS}, |  | ||||||
|      {metadata_store_ets,   MetadataStoreETS}, |      {metadata_store_ets,   MetadataStoreETS}, | ||||||
|      {other_ets,            ETS - MnesiaETS - MetricsETS - MgmtDbETS - MsgIndexETS - QuorumETS - MetadataStoreETS}, |      {other_ets,            ETS - MnesiaETS - MetricsETS - MgmtDbETS - MsgIndexETS  - MetadataStoreETS - lists:sum(QueuesEtsStats)}, | ||||||
| 
 | 
 | ||||||
|      %% Messages (mostly, some binaries are not messages) |      %% Messages (mostly, some binaries are not messages) | ||||||
|      {binary,               Bin}, |      {binary,               Bin}, | ||||||
|  | @ -110,6 +128,7 @@ memory() -> | ||||||
|                              {rss, Rss}, |                              {rss, Rss}, | ||||||
|                              {allocated, Allocated}]} |                              {allocated, Allocated}]} | ||||||
|     ]. |     ]. | ||||||
|  | 
 | ||||||
| %% [1] - erlang:memory(processes) can be less than the sum of its | %% [1] - erlang:memory(processes) can be less than the sum of its | ||||||
| %% parts. Rather than display something nonsensical, just silence any | %% parts. Rather than display something nonsensical, just silence any | ||||||
| %% claims about negative memory. See | %% claims about negative memory. See | ||||||
|  | @ -118,7 +137,9 @@ memory() -> | ||||||
| -spec binary() -> rabbit_types:infos(). | -spec binary() -> rabbit_types:infos(). | ||||||
| 
 | 
 | ||||||
| binary() -> | binary() -> | ||||||
|     All = interesting_sups(), |     {QueueSupsStatsKeys, QueueStatsSups} = rabbit_queue_type:queue_vm_stats_sups(), | ||||||
|  | 
 | ||||||
|  |     All = interesting_sups() ++ QueueStatsSups, | ||||||
|     {Sums, Rest} = |     {Sums, Rest} = | ||||||
|         sum_processes( |         sum_processes( | ||||||
|           lists:append(All), |           lists:append(All), | ||||||
|  | @ -127,10 +148,10 @@ binary() -> | ||||||
|                                       sets:add_element({Ptr, Sz}, Acc0) |                                       sets:add_element({Ptr, Sz}, Acc0) | ||||||
|                               end, Acc, Info) |                               end, Acc, Info) | ||||||
|           end, distinguishers(), [{binary, sets:new()}]), |           end, distinguishers(), [{binary, sets:new()}]), | ||||||
|     [Other, Qs, Qqs, DlxWorkers, Ssqs, Srqs, Scoor, ConnsReader, ConnsWriter, |     [Other, ConnsReader, ConnsWriter, | ||||||
|      ConnsChannel, ConnsOther, MsgIndexProc, MgmtDbProc, Plugins] = |      ConnsChannel, ConnsOther, MsgIndexProc, MgmtDbProc, Plugins | QueueSupsStats] = | ||||||
|         [aggregate(Names, [{other, Rest} | Sums], binary, fun sum_binary/1) |         [aggregate(Names, [{other, Rest} | Sums], binary, fun sum_binary/1) | ||||||
|          || Names <- [[other] | distinguished_interesting_sups()]], |          || Names <- [[other] | distinguished_interesting_sups()] ++ QueueStatsSups], | ||||||
|     MetadataStoreProc = try |     MetadataStoreProc = try | ||||||
|                             [{_, B}] = process_info(whereis(rabbit_khepri:get_ra_cluster_name()), [binary]), |                             [{_, B}] = process_info(whereis(rabbit_khepri:get_ra_cluster_name()), [binary]), | ||||||
|                             lists:foldl(fun({_, Sz, _}, Acc) -> |                             lists:foldl(fun({_, Sz, _}, Acc) -> | ||||||
|  | @ -143,13 +164,10 @@ binary() -> | ||||||
|     [{connection_readers,  ConnsReader}, |     [{connection_readers,  ConnsReader}, | ||||||
|      {connection_writers,  ConnsWriter}, |      {connection_writers,  ConnsWriter}, | ||||||
|      {connection_channels, ConnsChannel}, |      {connection_channels, ConnsChannel}, | ||||||
|      {connection_other,    ConnsOther}, |      {connection_other,    ConnsOther}] ++ | ||||||
|      {queue_procs,         Qs}, |      %% Queues | ||||||
|      {quorum_queue_procs,  Qqs}, |     lists:zip(QueueSupsStatsKeys, QueueSupsStats) ++ | ||||||
|      {quorum_queue_dlx_procs, DlxWorkers}, |     [ | ||||||
|      {stream_queue_procs,  Ssqs}, |  | ||||||
|      {stream_queue_replica_reader_procs, Srqs}, |  | ||||||
|      {stream_queue_coordinator_procs, Scoor}, |  | ||||||
|      {metadata_store,      MetadataStoreProc}, |      {metadata_store,      MetadataStoreProc}, | ||||||
|      {plugins,             Plugins}, |      {plugins,             Plugins}, | ||||||
|      {mgmt_db,             MgmtDbProc}, |      {mgmt_db,             MgmtDbProc}, | ||||||
|  | @ -194,19 +212,7 @@ bytes(Words) ->  try | ||||||
|                  end. |                  end. | ||||||
| 
 | 
 | ||||||
| interesting_sups() -> | interesting_sups() -> | ||||||
|     [queue_sups(), quorum_sups(), dlx_sups(), |     [conn_sups() | interesting_sups0()]. | ||||||
|      stream_server_sups(), stream_reader_sups(), stream_coordinator(), |  | ||||||
|      conn_sups() | interesting_sups0()]. |  | ||||||
| 
 |  | ||||||
| queue_sups() -> |  | ||||||
|     all_vhosts_children(rabbit_amqqueue_sup_sup). |  | ||||||
| 
 |  | ||||||
| quorum_sups() -> [ra_server_sup_sup]. |  | ||||||
| 
 |  | ||||||
| dlx_sups() -> [rabbit_fifo_dlx_sup]. |  | ||||||
| stream_server_sups() -> [osiris_server_sup]. |  | ||||||
| stream_reader_sups() -> [osiris_replica_reader_sup]. |  | ||||||
| stream_coordinator() -> [rabbit_stream_coordinator]. |  | ||||||
| 
 | 
 | ||||||
| msg_stores() -> | msg_stores() -> | ||||||
|     all_vhosts_children(msg_store_transient) |     all_vhosts_children(msg_store_transient) | ||||||
|  | @ -256,12 +262,6 @@ distinguishers() -> with(conn_sups(), fun conn_type/1). | ||||||
| 
 | 
 | ||||||
| distinguished_interesting_sups() -> | distinguished_interesting_sups() -> | ||||||
|     [ |     [ | ||||||
|      queue_sups(), |  | ||||||
|      quorum_sups(), |  | ||||||
|      dlx_sups(), |  | ||||||
|      stream_server_sups(), |  | ||||||
|      stream_reader_sups(), |  | ||||||
|      stream_coordinator(), |  | ||||||
|      with(conn_sups(), reader), |      with(conn_sups(), reader), | ||||||
|      with(conn_sups(), writer), |      with(conn_sups(), writer), | ||||||
|      with(conn_sups(), channel), |      with(conn_sups(), channel), | ||||||
|  |  | ||||||
|  | @ -2,6 +2,8 @@ | ||||||
| 
 | 
 | ||||||
| -include_lib("eunit/include/eunit.hrl"). | -include_lib("eunit/include/eunit.hrl"). | ||||||
| 
 | 
 | ||||||
|  | -include_lib("rabbit_common/include/rabbit.hrl"). | ||||||
|  | 
 | ||||||
| -export([ | -export([ | ||||||
|          wait_for_messages_ready/3, |          wait_for_messages_ready/3, | ||||||
|          wait_for_messages_pending_ack/3, |          wait_for_messages_pending_ack/3, | ||||||
|  |  | ||||||
|  | @ -550,10 +550,10 @@ add_replica(Config) -> | ||||||
|     ?assertEqual({error, node_not_running}, |     ?assertEqual({error, node_not_running}, | ||||||
|                  rpc:call(Server0, rabbit_stream_queue, add_replica, |                  rpc:call(Server0, rabbit_stream_queue, add_replica, | ||||||
|                           [<<"/">>, Q, Server1])), |                           [<<"/">>, Q, Server1])), | ||||||
|     ?assertEqual({error, classic_queue_not_supported}, |     ?assertEqual({error, {queue_not_supported, rabbit_classic_queue}}, | ||||||
|                  rpc:call(Server0, rabbit_stream_queue, add_replica, |                  rpc:call(Server0, rabbit_stream_queue, add_replica, | ||||||
|                           [<<"/">>, QClassic, Server1])), |                           [<<"/">>, QClassic, Server1])), | ||||||
|     ?assertEqual({error, quorum_queue_not_supported}, |     ?assertEqual({error, {queue_not_supported, rabbit_quorum_queue}}, | ||||||
|                  rpc:call(Server0, rabbit_stream_queue, add_replica, |                  rpc:call(Server0, rabbit_stream_queue, add_replica, | ||||||
|                           [<<"/">>, QQuorum, Server1])), |                           [<<"/">>, QQuorum, Server1])), | ||||||
| 
 | 
 | ||||||
|  | @ -561,10 +561,10 @@ add_replica(Config) -> | ||||||
|     ok = rabbit_control_helper:command(join_cluster, Server1, [atom_to_list(Server0)], []), |     ok = rabbit_control_helper:command(join_cluster, Server1, [atom_to_list(Server0)], []), | ||||||
|     rabbit_control_helper:command(start_app, Server1), |     rabbit_control_helper:command(start_app, Server1), | ||||||
|     timer:sleep(1000), |     timer:sleep(1000), | ||||||
|     ?assertEqual({error, classic_queue_not_supported}, |     ?assertEqual({error, {queue_not_supported, rabbit_classic_queue}}, | ||||||
|                  rpc:call(Server0, rabbit_stream_queue, add_replica, |                  rpc:call(Server0, rabbit_stream_queue, add_replica, | ||||||
|                           [<<"/">>, QClassic, Server1])), |                           [<<"/">>, QClassic, Server1])), | ||||||
|     ?assertEqual({error, quorum_queue_not_supported}, |     ?assertEqual({error, {queue_not_supported, rabbit_quorum_queue}}, | ||||||
|                  rpc:call(Server0, rabbit_stream_queue, add_replica, |                  rpc:call(Server0, rabbit_stream_queue, add_replica, | ||||||
|                           [<<"/">>, QQuorum, Server1])), |                           [<<"/">>, QQuorum, Server1])), | ||||||
|     ?assertEqual(ok, |     ?assertEqual(ok, | ||||||
|  | @ -748,10 +748,10 @@ delete_classic_replica(Config) -> | ||||||
|     ?assertEqual({'queue.declare_ok', Q, 0, 0}, |     ?assertEqual({'queue.declare_ok', Q, 0, 0}, | ||||||
|                  declare(Config, Server0, Q, [{<<"x-queue-type">>, longstr, <<"classic">>}])), |                  declare(Config, Server0, Q, [{<<"x-queue-type">>, longstr, <<"classic">>}])), | ||||||
|     %% Not a member of the cluster, what would happen? |     %% Not a member of the cluster, what would happen? | ||||||
|     ?assertEqual({error, classic_queue_not_supported}, |     ?assertEqual({error, {queue_not_supported, rabbit_classic_queue}}, | ||||||
|                  rpc:call(Server0, rabbit_stream_queue, delete_replica, |                  rpc:call(Server0, rabbit_stream_queue, delete_replica, | ||||||
|                           [<<"/">>, Q, 'zen@rabbit'])), |                           [<<"/">>, Q, 'zen@rabbit'])), | ||||||
|     ?assertEqual({error, classic_queue_not_supported}, |     ?assertEqual({error, {queue_not_supported, rabbit_classic_queue}}, | ||||||
|                  rpc:call(Server0, rabbit_stream_queue, delete_replica, |                  rpc:call(Server0, rabbit_stream_queue, delete_replica, | ||||||
|                           [<<"/">>, Q, Server1])), |                           [<<"/">>, Q, Server1])), | ||||||
|     rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_testcase_queue, [Q]). |     rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_testcase_queue, [Q]). | ||||||
|  | @ -763,10 +763,10 @@ delete_quorum_replica(Config) -> | ||||||
|     ?assertEqual({'queue.declare_ok', Q, 0, 0}, |     ?assertEqual({'queue.declare_ok', Q, 0, 0}, | ||||||
|                  declare(Config, Server0, Q, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), |                  declare(Config, Server0, Q, [{<<"x-queue-type">>, longstr, <<"quorum">>}])), | ||||||
|     %% Not a member of the cluster, what would happen? |     %% Not a member of the cluster, what would happen? | ||||||
|     ?assertEqual({error, quorum_queue_not_supported}, |     ?assertEqual({error, {queue_not_supported, rabbit_quorum_queue}}, | ||||||
|                  rpc:call(Server0, rabbit_stream_queue, delete_replica, |                  rpc:call(Server0, rabbit_stream_queue, delete_replica, | ||||||
|                           [<<"/">>, Q, 'zen@rabbit'])), |                           [<<"/">>, Q, 'zen@rabbit'])), | ||||||
|     ?assertEqual({error, quorum_queue_not_supported}, |     ?assertEqual({error, {queue_not_supported, rabbit_quorum_queue}}, | ||||||
|                  rpc:call(Server0, rabbit_stream_queue, delete_replica, |                  rpc:call(Server0, rabbit_stream_queue, delete_replica, | ||||||
|                           [<<"/">>, Q, Server1])), |                           [<<"/">>, Q, Server1])), | ||||||
|     rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_testcase_queue, [Q]). |     rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_testcase_queue, [Q]). | ||||||
|  |  | ||||||
|  | @ -36,7 +36,10 @@ | ||||||
|          queue_declared/1, |          queue_declared/1, | ||||||
|          queue_created/1, |          queue_created/1, | ||||||
|          queue_deleted/1, |          queue_deleted/1, | ||||||
|          queues_deleted/1]). |          queues_deleted/1, | ||||||
|  |          %% used by ra-based queues to cleanup follower metrics, | ||||||
|  |          %% see rabbit_core_metrics_gc for an example | ||||||
|  |          delete_queue_coarse_metrics/1]). | ||||||
| 
 | 
 | ||||||
| -export([node_stats/2]). | -export([node_stats/2]). | ||||||
| 
 | 
 | ||||||
|  | @ -321,10 +324,14 @@ partition_queues(Queues) -> | ||||||
|     [Queues]. |     [Queues]. | ||||||
| 
 | 
 | ||||||
| delete_queue_metrics(Queue) -> | delete_queue_metrics(Queue) -> | ||||||
|     ets:delete(queue_coarse_metrics, Queue), |     delete_queue_coarse_metrics(Queue), | ||||||
|     ets:update_element(queue_metrics, Queue, {3, 1}), |     ets:update_element(queue_metrics, Queue, {3, 1}), | ||||||
|     ok. |     ok. | ||||||
| 
 | 
 | ||||||
|  | delete_queue_coarse_metrics(Queue) -> | ||||||
|  |     ets:delete(queue_coarse_metrics, Queue), | ||||||
|  |     ok. | ||||||
|  | 
 | ||||||
| delete_channel_queue_exchange_metrics(MatchSpecCondition) -> | delete_channel_queue_exchange_metrics(MatchSpecCondition) -> | ||||||
|     ChannelQueueExchangeMetricsToUpdate = ets:select( |     ChannelQueueExchangeMetricsToUpdate = ets:select( | ||||||
|         channel_queue_exchange_metrics, |         channel_queue_exchange_metrics, | ||||||
|  |  | ||||||
|  | @ -15,7 +15,7 @@ | ||||||
|          code_change/3]). |          code_change/3]). | ||||||
| 
 | 
 | ||||||
| -export([register/3, unregister/2, | -export([register/3, unregister/2, | ||||||
|          binary_to_type/1, lookup_module/2, lookup_all/1]). |          binary_to_type/1, lookup_module/2, lookup_type_module/2, lookup_type_name/2, lookup_all/1]). | ||||||
| 
 | 
 | ||||||
| -define(SERVER, ?MODULE). | -define(SERVER, ?MODULE). | ||||||
| -define(ETS_NAME, ?MODULE). | -define(ETS_NAME, ?MODULE). | ||||||
|  | @ -61,6 +61,61 @@ lookup_module(Class, T) when is_atom(T) -> | ||||||
|             {error, not_found} |             {error, not_found} | ||||||
|     end. |     end. | ||||||
| 
 | 
 | ||||||
|  | 
 | ||||||
|  | -spec lookup_type_module(Class, TypeDescriptor) -> | ||||||
|  |           Ret when | ||||||
|  |       Class :: atom(), | ||||||
|  |       TypeDescriptor :: atom() | %% can be TypeModule or Type | ||||||
|  |                         binary(), %% or whati currently called "alias" - a TypeName | ||||||
|  |       Ret :: {ok, TypeModule} | {error, not_found}, | ||||||
|  |       TypeModule :: atom(). | ||||||
|  | lookup_type_module(Class, TypeDescriptor) -> | ||||||
|  |     case lookup_type(Class, TypeDescriptor) of | ||||||
|  |             {error, _} = Error -> | ||||||
|  |             Error; | ||||||
|  |         {ok, {_TypeName, TypeModule}} -> | ||||||
|  |             {ok, TypeModule} | ||||||
|  |     end. | ||||||
|  | 
 | ||||||
|  | -spec lookup_type_name(Class, TypeDescriptor) -> | ||||||
|  |           Ret when | ||||||
|  |       Class :: atom(), | ||||||
|  |       TypeDescriptor :: atom() | %% either full typemodule or atomized typename | ||||||
|  |                         binary(), %% typename pr typemodule in binary | ||||||
|  |       Ret :: {ok, binary()} | {error, not_found}. | ||||||
|  | lookup_type_name(Class, TypeDescriptor) -> | ||||||
|  |     case lookup_type(Class, TypeDescriptor) of | ||||||
|  |             {error, _} = Error -> | ||||||
|  |             Error; | ||||||
|  |         {ok, {TypeName, _TypeModule}} -> | ||||||
|  |             {ok, atom_to_binary(TypeName)} | ||||||
|  |     end. | ||||||
|  | 
 | ||||||
|  | lookup_type(Class, TypeDescriptor) | ||||||
|  |     when is_atom(TypeDescriptor) -> | ||||||
|  |     case ets:lookup(?ETS_NAME, {Class, TypeDescriptor}) of | ||||||
|  |         [{_, Module}] -> | ||||||
|  |             {ok, {TypeDescriptor, Module}}; | ||||||
|  |         [] -> | ||||||
|  |              %% In principle it is enough to do the same sanity check | ||||||
|  |              %% we do when registring a type. | ||||||
|  |              %% This however will return false positives for loaded | ||||||
|  |              %% but unregistered modules. | ||||||
|  |              TMMatch = ets:match(?ETS_NAME, {{Class, '$1'}, TypeDescriptor}), | ||||||
|  |              case TMMatch of | ||||||
|  |                  [[TypeName]] -> {ok, {TypeName, TypeDescriptor}}; | ||||||
|  |                  [] -> | ||||||
|  |                      {error, not_found} | ||||||
|  |              end | ||||||
|  |     end; | ||||||
|  | lookup_type(Class, TypeDescriptor) | ||||||
|  |     when is_binary(TypeDescriptor) -> | ||||||
|  |     %% when we register a type we convert | ||||||
|  |     %% typename to atom so we can lookup | ||||||
|  |     %% only existing atoms. | ||||||
|  |     lookup_type(Class, binary_to_existing_atom(TypeDescriptor)). | ||||||
|  | 
 | ||||||
|  | 
 | ||||||
| lookup_all(Class) -> | lookup_all(Class) -> | ||||||
|     [{K, V} || [K, V] <- ets:match(?ETS_NAME, {{Class, '$1'}, '$2'})]. |     [{K, V} || [K, V] <- ets:match(?ETS_NAME, {{Class, '$1'}, '$2'})]. | ||||||
| 
 | 
 | ||||||
|  |  | ||||||
|  | @ -41,6 +41,19 @@ | ||||||
|          notify_decorators/1 |          notify_decorators/1 | ||||||
|         ]). |         ]). | ||||||
| 
 | 
 | ||||||
|  | -export([queue_topology/1, | ||||||
|  |          feature_flag_name/0, | ||||||
|  |          policy_apply_to_name/0, | ||||||
|  |          can_redeliver/0, | ||||||
|  |          stop/1, | ||||||
|  |          is_replicated/0, | ||||||
|  |          rebalance_module/0, | ||||||
|  |          list_with_minimum_quorum/0, | ||||||
|  |          drain/1, | ||||||
|  |          revive/0, | ||||||
|  |          queue_vm_stats_sups/0, | ||||||
|  |          dir_base/0]). | ||||||
|  | 
 | ||||||
| %% Stateful rabbit_queue_type callbacks are unsupported by this queue type. | %% Stateful rabbit_queue_type callbacks are unsupported by this queue type. | ||||||
| -define(STATEFUL_CALLBACKS, | -define(STATEFUL_CALLBACKS, | ||||||
|         [ |         [ | ||||||
|  | @ -301,3 +314,43 @@ dequeue(A1,A2,A3,A4,A5) -> | ||||||
| 
 | 
 | ||||||
| state_info(A1) -> | state_info(A1) -> | ||||||
|     ?UNSUPPORTED([A1]). |     ?UNSUPPORTED([A1]). | ||||||
|  | 
 | ||||||
|  | -spec queue_topology(amqqueue:amqqueue()) -> | ||||||
|  |     {Leader :: undefined | node(), Replicas :: undefined | [node(),...]}. | ||||||
|  | queue_topology(Q) -> | ||||||
|  |     Pid = amqqueue:get_pid(Q), | ||||||
|  |     Node = node(Pid), | ||||||
|  |     {Node, [Node]}. | ||||||
|  | 
 | ||||||
|  | feature_flag_name() -> | ||||||
|  |     undefined. | ||||||
|  | 
 | ||||||
|  | policy_apply_to_name() -> | ||||||
|  |     <<"qos0_queues">>. | ||||||
|  | 
 | ||||||
|  | can_redeliver() -> | ||||||
|  |     true. | ||||||
|  | 
 | ||||||
|  | stop(_VHost) -> | ||||||
|  |     ok. | ||||||
|  | 
 | ||||||
|  | is_replicated() -> | ||||||
|  |     false. | ||||||
|  | 
 | ||||||
|  | rebalance_module() -> | ||||||
|  |     {error, not_supported}. | ||||||
|  | 
 | ||||||
|  | list_with_minimum_quorum() -> | ||||||
|  |     []. | ||||||
|  | 
 | ||||||
|  | drain(_TransferCandidates) -> | ||||||
|  |     ok. | ||||||
|  | 
 | ||||||
|  | revive() -> | ||||||
|  |     ok. | ||||||
|  | 
 | ||||||
|  | queue_vm_stats_sups() -> | ||||||
|  |     {[], []}. | ||||||
|  | 
 | ||||||
|  | dir_base() -> | ||||||
|  |     []. | ||||||
|  |  | ||||||
		Loading…
	
		Reference in New Issue